From 5f0b116669b70b3a095c75a599a8d04a30db94e5 Mon Sep 17 00:00:00 2001 From: Shwetha N <97447566+ShwethaSNayak@users.noreply.github.com> Date: Wed, 18 Oct 2023 10:38:10 +0530 Subject: [PATCH 01/15] Improved error handling (#17) Improved error handling. Separate parallel tasks for File watch and processing files Checks for File reads likes valid directory path, file format etc Pre-condition checks before commit/abort the transactions. Added more logs for debugging. Signed-off-by: Shwetha N --- .../collector/file/LogFileIngestService.java | 46 +++++- .../collector/file/LogFileSequenceConfig.java | 5 +- .../file/LogFileSequenceProcessor.java | 156 ++++++++++++++---- .../parquet/ParquetFileIngestService.java | 36 +++- .../parquet/ParquetFileProcessor.java | 82 +++++++-- .../rawfile/RawFileIngestService.java | 36 +++- .../collector/rawfile/RawFileProcessor.java | 87 +++++++--- .../sensor/collector/util/EventWriter.java | 9 + .../util/NonTransactionalEventWriter.java | 11 ++ .../util/TransactionCoordinator.java | 6 +- .../util/TransactionalEventWriter.java | 26 ++- .../file/LogFileSequenceProcessorTests.java | 2 +- .../resources/LogFileIngestTest.properties | 3 +- 13 files changed, 406 insertions(+), 99 deletions(-) diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileIngestService.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileIngestService.java index 5bf5b70f..3fcb0e31 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileIngestService.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileIngestService.java @@ -26,6 +26,7 @@ public class LogFileIngestService extends DeviceDriver { private static final Logger log = LoggerFactory.getLogger(LogFileIngestService.class); private static final String FILE_SPEC_KEY = "FILE_SPEC"; + private static final String FILE_EXT= "FILE_EXTENSION"; private static final String DELETE_COMPLETED_FILES_KEY = "DELETE_COMPLETED_FILES"; private static final String DATABASE_FILE_KEY = "DATABASE_FILE"; private static final String EVENT_TEMPLATE_KEY = "EVENT_TEMPLATE"; @@ -41,13 +42,15 @@ public class LogFileIngestService extends DeviceDriver { private final LogFileSequenceProcessor processor; private final ScheduledExecutorService executor; - private ScheduledFuture task; + private ScheduledFuture watchFiletask; + private ScheduledFuture processFileTask; public LogFileIngestService(DeviceDriverConfig config) { super(config); final LogFileSequenceConfig logFileSequenceConfig = new LogFileSequenceConfig( getDatabaseFileName(), getFileSpec(), + getFileExtension(), getRoutingKey(), getStreamName(), getEventTemplate(), @@ -69,6 +72,9 @@ public LogFileIngestService(DeviceDriverConfig config) { String getFileSpec() { return getProperty(FILE_SPEC_KEY); } + String getFileExtension() { + return getProperty(FILE_EXT, ""); + } boolean getDeleteCompletedFiles() { return Boolean.parseBoolean(getProperty(DELETE_COMPLETED_FILES_KEY, Boolean.toString(true))); @@ -113,29 +119,51 @@ boolean getExactlyOnce() { return Double.parseDouble(getProperty(TRANSACTION_TIMEOUT_MINUTES_KEY, Double.toString(18.0 * 60.0))); } - protected void ingestLogFiles() { - log.info("ingestLogFiles: BEGIN"); + protected void watchLogFiles() { + log.info("watchLogFiles: BEGIN"); try { - processor.ingestLogFiles(); + processor.watchLogFiles(); } catch (Exception e) { - log.error("Error", e); + log.error("watchLogFiles: watch file error", e); // Continue on any errors. We will retry on the next iteration. } - log.info("ingestLogFiles: END"); + log.info("watchLogFiles: END"); + } + protected void processLogFiles() { + log.trace("processLogFiles: BEGIN"); + try { + processor.processLogFiles(); + } catch (Exception e) { + log.error("processLogFiles: Process file error", e); + // Continue on any errors. We will retry on the next iteration. + } + log.trace("processLogFiles: END"); } @Override protected void doStart() { - task = executor.scheduleAtFixedRate( - this::ingestLogFiles, + watchFiletask = executor.scheduleAtFixedRate( + this::watchLogFiles, 0, getIntervalMs(), TimeUnit.MILLISECONDS); + /* + Submits a periodic action that becomes enabled immediately for the first time, + and subsequently with the delay of 0 milliseconds between the termination of one execution and the commencement of the next + ie immediately after completion of first action. + */ + processFileTask = executor.scheduleWithFixedDelay( + this::processLogFiles, + 0, + 0, + TimeUnit.MILLISECONDS); notifyStarted(); } @Override protected void doStop() { - task.cancel(false); + log.info("doStop: Cancelling ingestion task and process file task"); + watchFiletask.cancel(false); + processFileTask.cancel(false); } } diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceConfig.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceConfig.java index d9693d0b..c63d1db5 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceConfig.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceConfig.java @@ -12,6 +12,7 @@ public class LogFileSequenceConfig { public final String stateDatabaseFileName; public final String fileSpec; + public final String fileExtension; public final String routingKey; public final String streamName; public final String eventTemplateStr; @@ -25,9 +26,10 @@ public class LogFileSequenceConfig { public final boolean exactlyOnce; public final double transactionTimeoutMinutes; - public LogFileSequenceConfig(String stateDatabaseFileName, String fileSpec, String routingKey, String streamName, String eventTemplateStr, int maxRecordsPerEvent, boolean enableDeleteCompletedFiles, boolean exactlyOnce, double transactionTimeoutMinutes) { + public LogFileSequenceConfig(String stateDatabaseFileName, String fileSpec, String fileExtension, String routingKey, String streamName, String eventTemplateStr, int maxRecordsPerEvent, boolean enableDeleteCompletedFiles, boolean exactlyOnce, double transactionTimeoutMinutes) { this.stateDatabaseFileName = stateDatabaseFileName; this.fileSpec = fileSpec; + this.fileExtension = fileExtension; this.routingKey = routingKey; this.streamName = streamName; this.eventTemplateStr = eventTemplateStr; @@ -42,6 +44,7 @@ public String toString() { return "LogFileSequenceConfig{" + "stateDatabaseFileName='" + stateDatabaseFileName + '\'' + ", fileSpec='" + fileSpec + '\'' + + ", fileExtension='" + fileExtension + '\'' + ", routingKey='" + routingKey + '\'' + ", streamName='" + streamName + '\'' + ", eventTemplateStr='" + eventTemplateStr + '\'' + diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceProcessor.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceProcessor.java index d99db339..4a03ca9e 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceProcessor.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/LogFileSequenceProcessor.java @@ -15,6 +15,7 @@ import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.Transaction; import io.pravega.client.stream.TxnFailedException; import io.pravega.client.stream.impl.ByteArraySerializer; import io.pravega.sensor.collector.util.EventWriter; @@ -27,18 +28,14 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.file.DirectoryStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.*; import java.sql.Connection; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; +import java.util.*; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -90,21 +87,25 @@ public static LogFileSequenceProcessor create( return new LogFileSequenceProcessor(config, state, writer, transactionCoordinator, eventGenerator); } - public void ingestLogFiles() throws Exception { - log.info("ingestLogFiles: BEGIN"); + public void watchLogFiles() throws Exception { findAndRecordNewFiles(); + } + public void processLogFiles() throws Exception { + log.info("processLogFiles: BEGIN"); processNewFiles(); if (config.enableDeleteCompletedFiles) { + log.debug("processLogFiles: Deleting completed files"); deleteCompletedFiles(); } - log.info("ingestLogFiles: END"); + log.info("processLogFiles: END"); } public void processNewFiles() throws Exception { for (;;) { + // If nextFile is null then check for new files to process is handled as part of scheduleWithDelay final Pair nextFile = state.getNextPendingFile(); if (nextFile == null) { - log.info("No more files to ingest"); + log.info("processNewFiles: No more files to watch"); break; } else { processFile(nextFile.getLeft(), nextFile.getRight()); @@ -124,21 +125,47 @@ protected void findAndRecordNewFiles() throws Exception { */ protected List getDirectoryListing() throws IOException { log.info("getDirectoryListing: fileSpec={}", config.fileSpec); - final List directoryListing = getDirectoryListing(config.fileSpec); + final List directoryListing = getDirectoryListing(config.fileSpec, config.fileExtension); log.trace("getDirectoryListing: directoryListing={}", directoryListing); return directoryListing; } /** * @return list of file name and file size in bytes + * Handle the below cases + * 1. If given file path does not exist then log the message and continue + * 2. If directory does not exist and no file with given extn like .csv then log the message and continue + * 3. check for empty file, log the message and continue with valid files + * */ - static protected List getDirectoryListing(String fileSpec) throws IOException { + static protected List getDirectoryListing(String fileSpec, String fileExtension) throws IOException { final Path pathSpec = Paths.get(fileSpec); - try (DirectoryStream dirStream = Files.newDirectoryStream(pathSpec.getParent(), pathSpec.getFileName().toString())) { - return StreamSupport.stream(dirStream.spliterator(), false) - .map(f -> new FileNameWithOffset(f.toAbsolutePath().toString(), f.toFile().length())) - .collect(Collectors.toList()); + if (!Files.isDirectory(pathSpec.toAbsolutePath())) { + log.error("getDirectoryListing: Directory does not exist or spec is not valid : {}", pathSpec.toAbsolutePath()); + throw new IOException("Directory does not exist or spec is not valid"); } + List directoryListing = new ArrayList<>(); + try (DirectoryStream dirStream = Files.newDirectoryStream(pathSpec)) { + for (Path path : dirStream) { + if (Files.isDirectory(path)) //traverse subdirectories + directoryListing.addAll(getDirectoryListing(path.toString(), fileExtension)); + else { + FileNameWithOffset fileEntry = new FileNameWithOffset(path.toAbsolutePath().toString(), path.toFile().length()); + if (isValidFile(fileEntry, fileExtension)) { + directoryListing.add(fileEntry); + } + } + } + }catch(Exception ex){ + if(ex instanceof IOException){ + log.error("getDirectoryListing: Directory does not exist or spec is not valid : {}", pathSpec.toAbsolutePath()); + throw new IOException("Directory does not exist or spec is not valid"); + }else{ + log.error("getDirectoryListing: Exception while listing files: {}", pathSpec.toAbsolutePath()); + throw new IOException(ex); + } + } + return directoryListing; } /** @@ -155,7 +182,7 @@ static protected List getNewFiles(List d newFiles.add(new FileNameWithOffset(dirFile.fileName, 0)); } }); - log.info("getNewFiles={}", newFiles); + log.info("getNewFiles: new file lists ={}", newFiles); return newFiles; } @@ -163,10 +190,18 @@ void processFile(FileNameWithOffset fileNameWithBeginOffset, long firstSequenceN log.info("processFile: Ingesting file {}; beginOffset={}, firstSequenceNumber={}", fileNameWithBeginOffset.fileName, fileNameWithBeginOffset.offset, firstSequenceNumber); + AtomicLong numofbytes = new AtomicLong(0); + long timestamp = System.nanoTime(); // In case a previous iteration encountered an error, we need to ensure that // previous flushed transactions are committed and any unflushed transactions as aborted. transactionCoordinator.performRecovery(); - writer.abort(); + /* Check if transactions can be aborted. + * Will fail with {@link TxnFailedException} if the transaction has already been committed or aborted. + */ + log.debug("processFile: Transaction status {} ", writer.getTransactionStatus()); + if(writer.getTransactionStatus() == Transaction.Status.OPEN){ + writer.abort(); + } try (final InputStream inputStream = new FileInputStream(fileNameWithBeginOffset.fileName)) { final CountingInputStream countingInputStream = new CountingInputStream(inputStream); @@ -177,36 +212,93 @@ void processFile(FileNameWithOffset fileNameWithBeginOffset, long firstSequenceN try { writer.writeEvent(e.routingKey, e.bytes); } catch (TxnFailedException ex) { + log.error("processFile: Write event to transaction failed with exception {} while processing file: {}, event: {}", ex, fileNameWithBeginOffset.fileName, e); + /* + TODO while writing event if we get Transaction failed exception then should we abort the trasaction and process again? + This will occur only if Transaction state is not open + */ throw new RuntimeException(ex); } }); final Optional txnId = writer.flush(); final long nextSequenceNumber = result.getLeft(); final long endOffset = result.getRight(); + log.debug("processFile: Adding completed file: {}", fileNameWithBeginOffset.fileName); state.addCompletedFile(fileNameWithBeginOffset.fileName, fileNameWithBeginOffset.offset, endOffset, nextSequenceNumber, txnId); // injectCommitFailure(); - writer.commit(); - state.deleteTransactionToCommit(txnId); + try { + // commit fails only if Transaction is not in open state. + log.info("processFile: Commit transaction for Id: {}; file: {}", txnId.orElse(null), fileNameWithBeginOffset.fileName); + writer.commit(); + } catch (TxnFailedException ex) { + log.error("processFile: Commit transaction for id: {}, file : {}, failed with exception: {}", txnId, fileNameWithBeginOffset.fileName, ex); + throw new RuntimeException(ex); + } + // Add to completed file list only if commit is successfull else it will be taken care as part of recovery + if(txnId.isPresent()){ + Transaction.Status status = writer.getTransactionStatus(txnId.get()); + if(status == Transaction.Status.COMMITTED || status == Transaction.Status.ABORTED) + state.deleteTransactionToCommit(txnId); + } + + double elapsedSec = (System.nanoTime() - timestamp) / 1_000_000_000.0; + double megabyteCount = numofbytes.getAndSet(0) / 1_000_000.0; + double megabytesPerSec = megabyteCount / elapsedSec; + log.info("Sent {} MB in {} sec. Transfer rate: {} MB/sec ", megabyteCount, elapsedSec, megabytesPerSec ); log.info("processFile: Finished ingesting file {}; endOffset={}, nextSequenceNumber={}", fileNameWithBeginOffset.fileName, endOffset, nextSequenceNumber); } + // Delete file right after ingesting + if (config.enableDeleteCompletedFiles) { + deleteCompletedFiles(); + } } void deleteCompletedFiles() throws Exception { final List completedFiles = state.getCompletedFiles(); completedFiles.forEach(file -> { - try { - Files.deleteIfExists(Paths.get(file.fileName)); - log.info("deleteCompletedFiles: Deleted file {}", file.fileName); - // Only remove from database if we could delete file. - state.deleteCompletedFile(file.fileName); + //Obtain a lock on file + try(FileChannel channel = FileChannel.open(Paths.get(file.fileName), StandardOpenOption.WRITE)){ + try(FileLock lock = channel.tryLock()) { + if(lock!=null){ + Files.deleteIfExists(Paths.get(file.fileName)); + log.info("deleteCompletedFiles: Deleted file {}", file.fileName); + lock.release(); + // Only remove from database if we could delete file. + state.deleteCompletedFile(file.fileName); + } + else{ + log.warn("Unable to obtain lock on file {}. File is locked by another process.", file.fileName); + throw new Exception(); + } + } } catch (Exception e) { - log.warn("Unable to delete ingested file {}", e); - // We can continue on this error. It will be retried on the next iteration. + log.warn("Unable to delete ingested file {}", e.getMessage()); + log.warn("Deletion will be retried on the next iteration."); + // We can continue on this error. Deletion will be retried on the next iteration. } }); } + /* + Check for below file validation + 1. Is File empty + 2. If extension is null or extension is valid ingest all file + */ + public static boolean isValidFile(FileNameWithOffset fileEntry, String fileExtension ){ + + if(fileEntry.offset<=0){ + log.warn("isValidFile: Empty file {} can not be processed ",fileEntry.fileName); + } + // If extension is null, ingest all files + else if(fileExtension.isEmpty() || fileExtension.equals(fileEntry.fileName.substring(fileEntry.fileName.lastIndexOf(".")+1))) + return true; + else + log.warn("isValidFile: File format {} is not supported ", fileEntry.fileName); + + return false; + } + /** * Inject a failure before commit for testing. */ diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileIngestService.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileIngestService.java index 97a8cd3e..ec0d9ecc 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileIngestService.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileIngestService.java @@ -47,7 +47,8 @@ public class ParquetFileIngestService extends DeviceDriver{ private final ParquetFileProcessor processor; private final ScheduledExecutorService executor; - private ScheduledFuture task; + private ScheduledFuture watchFileTask; + private ScheduledFuture processFileTask; public ParquetFileIngestService(DeviceDriverConfig config){ super(config); @@ -126,30 +127,47 @@ boolean getExactlyOnce() { return Double.parseDouble(getProperty(TRANSACTION_TIMEOUT_MINUTES_KEY, Double.toString(18.0 * 60.0))); } - protected void ingestParquetFiles() { - log.trace("ingestParquetFiles: BEGIN"); + protected void watchParquetFiles() { + log.trace("watchParquetFiles: BEGIN"); try { - processor.ingestParquetFiles(); + processor.watchParquetFiles(); } catch (Exception e) { - log.error("Error", e); + log.error("watchParquetFiles: watch file error", e); // Continue on any errors. We will retry on the next iteration. } - log.trace("ingestParquetFiles: END"); + log.trace("watchParquetFiles: END"); + } + protected void processParquetFiles() { + log.trace("processParquetFiles: BEGIN"); + try { + processor.processParquetFiles(); + } catch (Exception e) { + log.error("processParquetFiles: Process file error", e); + // Continue on any errors. We will retry on the next iteration. + } + log.trace("processParquetFiles: END"); } @Override protected void doStart() { - task = executor.scheduleAtFixedRate( - this::ingestParquetFiles, + watchFileTask = executor.scheduleAtFixedRate( + this::watchParquetFiles, 0, getIntervalMs(), TimeUnit.MILLISECONDS); + processFileTask = executor.scheduleWithFixedDelay( + this::processParquetFiles, + 0, + 0, + TimeUnit.MILLISECONDS); notifyStarted(); } @Override protected void doStop() { - task.cancel(false); + log.info("doStop: Cancelling ingestion task and process file task"); + watchFileTask.cancel(false); + processFileTask.cancel(false); } } diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileProcessor.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileProcessor.java index fa9a68c6..9ace9e5d 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileProcessor.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/ParquetFileProcessor.java @@ -28,9 +28,12 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import com.google.common.io.CountingInputStream; +import io.pravega.client.stream.Transaction; import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,22 +96,26 @@ public static ParquetFileProcessor create(ParquetFileConfig config, EventStreamC return new ParquetFileProcessor(config, state, writer, transactionCoordinator, eventGenerator); } - public void ingestParquetFiles() throws Exception { - log.trace("ingestParquetFiles: BEGIN"); + public void watchParquetFiles() throws Exception { + log.trace("watchParquetFiles: BEGIN"); + findAndRecordNewFiles(); + log.trace("watchParquetFiles: END"); + } + public void processParquetFiles() throws Exception { + log.trace("processParquetFiles: BEGIN"); // delete leftover completed files if (config.enableDeleteCompletedFiles) { deleteCompletedFiles(); } - findAndRecordNewFiles(); processNewFiles(); - log.trace("ingestParquetFiles: END"); + log.trace("processParquetFiles: END"); } public void processNewFiles() throws Exception { for (;;) { final Pair nextFile = state.getNextPendingFile(); if (nextFile == null) { - log.trace("No more files to ingest"); + log.trace("processNewFiles: No more files to watch"); break; } else { processFile(nextFile.getLeft(), nextFile.getRight()); @@ -138,6 +145,10 @@ protected List getDirectoryListing() throws IOException { */ static protected List getDirectoryListing(String fileSpec, String fileExtension) throws IOException { final Path pathSpec = Paths.get(fileSpec); + if (!Files.isDirectory(pathSpec.toAbsolutePath())) { + log.error("getDirectoryListing: Directory does not exist or spec is not valid : {}", pathSpec.toAbsolutePath()); + throw new IOException("Directory does not exist or spec is not valid"); + } List directoryListing = new ArrayList<>(); try(DirectoryStream dirStream=Files.newDirectoryStream(pathSpec)){ for(Path path: dirStream){ @@ -145,11 +156,19 @@ static protected List getDirectoryListing(String fileSpec, S directoryListing.addAll(getDirectoryListing(path.toString(), fileExtension)); else { FileNameWithOffset fileEntry = new FileNameWithOffset(path.toAbsolutePath().toString(), path.toFile().length()); - // If extension is null, ingest all files - if(fileExtension.isEmpty() || fileExtension.equals(fileEntry.fileName.substring(fileEntry.fileName.lastIndexOf(".")+1))) - directoryListing.add(fileEntry); + if(isValidFile(fileEntry, fileExtension)){ + directoryListing.add(fileEntry); + } } } + }catch(Exception ex){ + if(ex instanceof IOException){ + log.error("getDirectoryListing: Directory does not exist or spec is not valid : {}", pathSpec.toAbsolutePath()); + throw new IOException("Directory does not exist or spec is not valid"); + }else{ + log.error("getDirectoryListing: Exception while listing files: {}", pathSpec.toAbsolutePath()); + throw new IOException(ex); + } } return directoryListing; } @@ -186,7 +205,10 @@ void processFile(FileNameWithOffset fileNameWithBeginOffset, long firstSequenceN // In case a previous iteration encountered an error, we need to ensure that // previous flushed transactions are committed and any unflushed transactions as aborted. transactionCoordinator.performRecovery(); - writer.abort(); + log.debug("processFile: Transaction status {} ", writer.getTransactionStatus()); + if(writer.getTransactionStatus() == Transaction.Status.OPEN){ + writer.abort(); + } try (final InputStream inputStream = new FileInputStream(fileNameWithBeginOffset.fileName)) { try(final CountingInputStream countingInputStream = new CountingInputStream(inputStream)) { @@ -199,24 +221,37 @@ void processFile(FileNameWithOffset fileNameWithBeginOffset, long firstSequenceN numofbytes.addAndGet(e.bytes.length); } catch (TxnFailedException ex) { + log.error("processFile: Write event to transaction failed with exception {} while processing file: {}, event: {}", ex, fileNameWithBeginOffset.fileName, e); throw new RuntimeException(ex); } }); final Optional txnId = writer.flush(); final long nextSequenceNumber = result.getLeft(); final long endOffset = result.getRight(); + log.debug("processFile: Adding completed file: {}", fileNameWithBeginOffset.fileName); state.addCompletedFile(fileNameWithBeginOffset.fileName, fileNameWithBeginOffset.offset, endOffset, nextSequenceNumber, txnId); // injectCommitFailure(); - writer.commit(); - state.deleteTransactionToCommit(txnId); - + try { + // commit fails only if Transaction is not in open state. + log.info("processFile: Commit transaction for Id: {}; file: {}", txnId.orElse(null), fileNameWithBeginOffset.fileName); + writer.commit(); + } catch (TxnFailedException ex) { + log.error("processFile: Commit transaction for id: {}, file : {}, failed with exception: {}", txnId, fileNameWithBeginOffset.fileName, ex); + throw new RuntimeException(ex); + } + // Add to completed file list only if commit is successfull else it will be taken care as part of recovery + if(txnId.isPresent()){ + Transaction.Status status = writer.getTransactionStatus(txnId.get()); + if(status == Transaction.Status.COMMITTED || status == Transaction.Status.ABORTED) + state.deleteTransactionToCommit(txnId); + } + double elapsedSec = (System.nanoTime() - timestamp) / 1_000_000_000.0; double megabyteCount = numofbytes.getAndSet(0) / 1_000_000.0; double megabytesPerSec = megabyteCount / elapsedSec; log.info("processFile: Finished ingesting file {}; endOffset={}, nextSequenceNumber={}", fileNameWithBeginOffset.fileName, endOffset, nextSequenceNumber); - log.info("Sent {} MB in {} sec", megabyteCount, elapsedSec ); - log.info("Transfer rate: {} MB/sec", megabytesPerSec); + log.info("Sent {} MB in {} sec. Transfer rate: {} MB/sec ", megabyteCount, elapsedSec, megabytesPerSec ); } } @@ -253,6 +288,25 @@ void deleteCompletedFiles() throws Exception { }); } + /* + Check for below file validation + 1. Is File empty + 2. If extension is null or extension is valid ingest all file + */ + public static boolean isValidFile(FileNameWithOffset fileEntry, String fileExtension ){ + + if(fileEntry.offset<=0){ + log.warn("isValidFile: Empty file {} can not be processed ",fileEntry.fileName); + } + // If extension is null, ingest all files + else if(fileExtension.isEmpty() || fileExtension.equals(fileEntry.fileName.substring(fileEntry.fileName.lastIndexOf(".")+1))) + return true; + else + log.warn("isValidFile: File format {} is not supported ", fileEntry.fileName); + + return false; + } + /** * Inject a failure before commit for testing. */ diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileIngestService.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileIngestService.java index df2941e0..5d1f1138 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileIngestService.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileIngestService.java @@ -45,7 +45,8 @@ public class RawFileIngestService extends DeviceDriver{ private final RawFileProcessor processor; private final ScheduledExecutorService executor; - private ScheduledFuture task; + private ScheduledFuture watchFileTask; + private ScheduledFuture processFileTask; public RawFileIngestService(DeviceDriverConfig config){ super(config); @@ -119,30 +120,47 @@ boolean getExactlyOnce() { return Double.parseDouble(getProperty(TRANSACTION_TIMEOUT_MINUTES_KEY, Double.toString(18.0 * 60.0))); } - protected void ingestRawFiles() { - log.trace("ingestRawFiles: BEGIN"); + protected void watchRawFiles() { + log.trace("watchRawFiles: BEGIN"); try { - processor.ingestRawFiles(); + processor.watchRawFiles(); } catch (Exception e) { - log.error("Error", e); + log.error("watchRawFiles: ingest file error", e); // Continue on any errors. We will retry on the next iteration. } - log.trace("ingestRawFiles: END"); + log.trace("watchRawFiles: END"); + } + protected void processRawFiles() { + log.trace("processRawFiles: BEGIN"); + try { + processor.processRawFiles(); + } catch (Exception e) { + log.error("processRawFiles: Process file error", e); + // Continue on any errors. We will retry on the next iteration. + } + log.trace("processRawFiles: END"); } @Override protected void doStart() { - task = executor.scheduleAtFixedRate( - this::ingestRawFiles, + watchFileTask = executor.scheduleAtFixedRate( + this::watchRawFiles, 0, getIntervalMs(), TimeUnit.MILLISECONDS); + processFileTask = executor.scheduleWithFixedDelay( + this::processRawFiles, + 0, + 0, + TimeUnit.MILLISECONDS); notifyStarted(); } @Override protected void doStop() { - task.cancel(false); + log.info("doStop: Cancelling ingestion task and process file task"); + watchFileTask.cancel(false); + processFileTask.cancel(false); } } diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileProcessor.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileProcessor.java index e420493d..5cafb13f 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileProcessor.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileProcessor.java @@ -14,11 +14,7 @@ import java.io.InputStream; import java.nio.channels.FileChannel; import java.nio.channels.FileLock; -import java.nio.file.DirectoryStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; +import java.nio.file.*; import java.sql.Connection; import java.util.ArrayList; import java.util.Collections; @@ -28,9 +24,12 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import com.google.common.io.CountingInputStream; +import io.pravega.client.stream.Transaction; import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,22 +91,26 @@ public static RawFileProcessor create(RawFileConfig config, EventStreamClientFac return new RawFileProcessor(config, state, writer, transactionCoordinator, eventGenerator); } - public void ingestRawFiles() throws Exception { - log.trace("ingestRawFiles: BEGIN"); + public void watchRawFiles() throws Exception { + log.trace("watchRawFiles: BEGIN"); + findAndRecordNewFiles(); + log.trace("watchRawFiles: END"); + } + public void processRawFiles() throws Exception { + log.trace("processRawFiles: BEGIN"); // delete leftover completed files if (config.enableDeleteCompletedFiles) { deleteCompletedFiles(); } - findAndRecordNewFiles(); processNewFiles(); - log.trace("ingestRawFiles: END"); + log.trace("processRawFiles: END"); } public void processNewFiles() throws Exception { for (;;) { final Pair nextFile = state.getNextPendingFile(); if (nextFile == null) { - log.trace("No more files to ingest"); + log.trace("processNewFiles: No more files to ingest"); break; } else { processFile(nextFile.getLeft(), nextFile.getRight()); @@ -137,18 +140,27 @@ protected List getDirectoryListing() throws IOException { */ static protected List getDirectoryListing(String fileSpec, String fileExtension) throws IOException { final Path pathSpec = Paths.get(fileSpec); + List directoryListing = new ArrayList<>(); try(DirectoryStream dirStream=Files.newDirectoryStream(pathSpec)){ for(Path path: dirStream){ - if(Files.isDirectory(path)) + if(Files.isDirectory(path)) //traverse subdirectories directoryListing.addAll(getDirectoryListing(path.toString(), fileExtension)); else { FileNameWithOffset fileEntry = new FileNameWithOffset(path.toAbsolutePath().toString(), path.toFile().length()); - // If extension is null, ingest all files - if(fileExtension.isEmpty() || fileExtension.equals(fileEntry.fileName.substring(fileEntry.fileName.lastIndexOf(".")+1))) - directoryListing.add(fileEntry); + if(isValidFile(fileEntry, fileExtension)){ + directoryListing.add(fileEntry); + } } } + }catch(Exception ex){ + if(ex instanceof IOException){ + log.error("getDirectoryListing: Directory does not exist or spec is not valid : {}", pathSpec.toAbsolutePath()); + throw new IOException("Directory does not exist or spec is not valid"); + }else{ + log.error("getDirectoryListing: Exception while listing files: {}", pathSpec.toAbsolutePath()); + throw new IOException(ex); + } } return directoryListing; } @@ -183,7 +195,10 @@ void processFile(FileNameWithOffset fileNameWithBeginOffset, long firstSequenceN // In case a previous iteration encountered an error, we need to ensure that // previous flushed transactions are committed and any unflushed transactions as aborted. transactionCoordinator.performRecovery(); - writer.abort(); + log.debug("processFile: Transaction status {} ", writer.getTransactionStatus()); + if(writer.getTransactionStatus() == Transaction.Status.OPEN){ + writer.abort(); + } try (final InputStream inputStream = new FileInputStream(fileNameWithBeginOffset.fileName)) { try(final CountingInputStream countingInputStream = new CountingInputStream(inputStream)) { @@ -196,24 +211,37 @@ void processFile(FileNameWithOffset fileNameWithBeginOffset, long firstSequenceN numofbytes.addAndGet(e.bytes.length); } catch (TxnFailedException ex) { + log.error("processFile: Write event to transaction failed with exception {} while processing file: {}, event: {}", ex, fileNameWithBeginOffset.fileName, e); throw new RuntimeException(ex); } }); final Optional txnId = writer.flush(); final long nextSequenceNumber = result.getLeft(); final long endOffset = result.getRight(); + log.debug("processFile: Adding completed file: {}", fileNameWithBeginOffset.fileName); state.addCompletedFile(fileNameWithBeginOffset.fileName, fileNameWithBeginOffset.offset, endOffset, nextSequenceNumber, txnId); // injectCommitFailure(); - writer.commit(); - state.deleteTransactionToCommit(txnId); - + try { + // commit fails only if Transaction is not in open state. + log.info("processFile: Commit transaction for Id: {}; file: {}", txnId.orElse(null), fileNameWithBeginOffset.fileName); + writer.commit(); + } catch (TxnFailedException ex) { + log.error("processFile: Commit transaction for id: {}, file : {}, failed with exception: {}", txnId, fileNameWithBeginOffset.fileName, ex); + throw new RuntimeException(ex); + } + // Add to completed file list only if commit is successfull else it will be taken care as part of recovery + if(txnId.isPresent()){ + Transaction.Status status = writer.getTransactionStatus(txnId.get()); + if(status == Transaction.Status.COMMITTED || status == Transaction.Status.ABORTED) + state.deleteTransactionToCommit(txnId); + } + double elapsedSec = (System.nanoTime() - timestamp) / 1_000_000_000.0; double megabyteCount = numofbytes.getAndSet(0) / 1_000_000.0; double megabytesPerSec = megabyteCount / elapsedSec; log.info("processFile: Finished ingesting file {}; endOffset={}, nextSequenceNumber={}", fileNameWithBeginOffset.fileName, endOffset, nextSequenceNumber); - log.info("Sent {} MB in {} sec", megabyteCount, elapsedSec ); - log.info("Transfer rate: {} MB/sec", megabytesPerSec); + log.info("Sent {} MB in {} sec. Transfer rate: {} MB/sec ", megabyteCount, elapsedSec, megabytesPerSec ); } } @@ -249,6 +277,25 @@ void deleteCompletedFiles() throws Exception { }); } + /* + Check for below file validation + 1. Is File empty + 2. If extension is null or extension is valid ingest all file + */ + public static boolean isValidFile(FileNameWithOffset fileEntry, String fileExtension ){ + + if(fileEntry.offset<=0){ + log.warn("isValidFile: Empty file {} can not be processed",fileEntry.fileName); + } + // If extension is null, ingest all files + else if(fileExtension.isEmpty() || fileExtension.equals(fileEntry.fileName.substring(fileEntry.fileName.lastIndexOf(".")+1))) + return true; + else + log.warn("isValidFile: File format {} is not supported ", fileEntry.fileName); + + return false; + } + /** * Inject a failure before commit for testing. */ diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/EventWriter.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/EventWriter.java index 17cf5d65..742f5d42 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/EventWriter.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/EventWriter.java @@ -12,6 +12,7 @@ import io.pravega.client.EventStreamClientFactory; import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.Serializer; +import io.pravega.client.stream.Transaction; import io.pravega.client.stream.TxnFailedException; import java.util.Optional; @@ -71,5 +72,13 @@ static EventWriter create( */ void abort(); + /** + * This should called be prior to aborting any transactions to make sure it is not open. + */ + public Transaction.Status getTransactionStatus(UUID txnId); + + public Transaction.Status getTransactionStatus(); + void close(); + } diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/NonTransactionalEventWriter.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/NonTransactionalEventWriter.java index 6117193c..8c4c2682 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/NonTransactionalEventWriter.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/NonTransactionalEventWriter.java @@ -10,6 +10,7 @@ package io.pravega.sensor.collector.util; import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.Transaction; import io.pravega.client.stream.TxnFailedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,6 +54,16 @@ public void commit(UUID txnId) throws TxnFailedException { public void abort() { } + @Override + public Transaction.Status getTransactionStatus(UUID txnId) { + throw new UnsupportedOperationException("Non-transactional writer cannot commit transactions"); + } + + @Override + public Transaction.Status getTransactionStatus() { + throw new UnsupportedOperationException("Non-transactional writer do not have transaction status"); + } + public void close() { writer.close(); } diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionCoordinator.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionCoordinator.java index e06a610d..b0e14e28 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionCoordinator.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionCoordinator.java @@ -9,6 +9,7 @@ */ package io.pravega.sensor.collector.util; +import io.pravega.client.stream.Transaction; import io.pravega.client.stream.TxnFailedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -139,7 +140,7 @@ protected List getTransactionsToCommit() { public void performRecovery() { final List transactionsToCommit = getTransactionsToCommit(); if (transactionsToCommit.isEmpty()) { - log.debug("Transaction recovery not needed"); + log.info("performRecovery: No transactions to be recovered"); } else { log.warn("Transaction recovery needed on {} transactions", transactionsToCommit.size()); transactionsToCommit.forEach((txnId) -> { @@ -160,6 +161,9 @@ public void performRecovery() { txnId, e); // Continue recovery and run as normal. } else { + log.error( + "Unable to commit transaction {} from a previous process. Events may have been lost. " + + "Try increasing the transaction timeout.", txnId, e); throw e; } } diff --git a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionalEventWriter.java b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionalEventWriter.java index 7bae945a..d53c0c87 100644 --- a/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionalEventWriter.java +++ b/pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/TransactionalEventWriter.java @@ -9,6 +9,7 @@ */ package io.pravega.sensor.collector.util; +import com.google.common.base.Preconditions; import io.pravega.client.stream.Transaction; import io.pravega.client.stream.TransactionalEventStreamWriter; import io.pravega.client.stream.TxnFailedException; @@ -17,6 +18,7 @@ import java.util.Optional; import java.util.UUID; +import java.util.concurrent.CompletableFuture; public class TransactionalEventWriter implements EventWriter { private static final Logger log = LoggerFactory.getLogger(TransactionalEventWriter.class); @@ -65,10 +67,19 @@ public void commit(long timestamp) throws TxnFailedException { currentTxn = null; } } + private boolean canCommitTransaction(UUID txnId){ + Transaction.Status transactionStatus = writer.getTxn(txnId).checkStatus(); + log.info("canCommitTransaction: Status of Transaction id {} is {}", txnId, transactionStatus); + return transactionStatus == Transaction.Status.OPEN; + } public void commit(UUID txnId) throws TxnFailedException { - log.info("commit: committing transaction {}", txnId); - writer.getTxn(txnId).commit(); + /*Check the transaction status before committing transaction + Only transactions which rea in open status can be committed */ + if(canCommitTransaction(txnId)){ + log.info("commit: committing transaction {}", txnId); + writer.getTxn(txnId).commit(); + } } public void abort() { @@ -79,6 +90,17 @@ public void abort() { } } + public Transaction.Status getTransactionStatus() { + if (currentTxn != null){ + return currentTxn.checkStatus(); + } + return null; + } + public Transaction.Status getTransactionStatus(UUID txnId) { + return writer.getTxn(txnId).checkStatus(); + } + + public void close() { try { abort(); diff --git a/pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/file/LogFileSequenceProcessorTests.java b/pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/file/LogFileSequenceProcessorTests.java index 87c92354..ae2401f3 100644 --- a/pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/file/LogFileSequenceProcessorTests.java +++ b/pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/file/LogFileSequenceProcessorTests.java @@ -40,7 +40,7 @@ public void getNewFilesTest() { @Test public void getDirectoryListingTest() throws IOException { final List actual = LogFileSequenceProcessor.getDirectoryListing( - "../log-file-sample-data/*.csv"); + "../log-file-sample-data/","csv"); log.info("actual={}", actual); } } diff --git a/pravega-sensor-collector/src/test/resources/LogFileIngestTest.properties b/pravega-sensor-collector/src/test/resources/LogFileIngestTest.properties index b6f644fb..65431bca 100644 --- a/pravega-sensor-collector/src/test/resources/LogFileIngestTest.properties +++ b/pravega-sensor-collector/src/test/resources/LogFileIngestTest.properties @@ -11,7 +11,8 @@ # Run scripts/simulate-logs-accel.sh concurrently. PRAVEGA_SENSOR_COLLECTOR_ACCEL2_CLASS=io.pravega.sensor.collector.file.LogFileIngestService -PRAVEGA_SENSOR_COLLECTOR_ACCEL2_FILE_SPEC=/tmp/watch/Accelerometer.*.csv +PRAVEGA_SENSOR_COLLECTOR_ACCEL2_FILE_SPEC=/tmp/watch/files +PRAVEGA_SENSOR_COLLECTOR_ACCEL2_FILE_EXTENSION=csv PRAVEGA_SENSOR_COLLECTOR_ACCEL2_DELETE_COMPLETED_FILES=true PRAVEGA_SENSOR_COLLECTOR_ACCEL2_DATABASE_FILE=/tmp/accelerometer.db PRAVEGA_SENSOR_COLLECTOR_ACCEL2_EVENT_TEMPLATE={"RemoteAddr":"myaddr1","SensorType":"Accelerometer"} From d7aa4b7cd1fd2ea11e3ea379e2067d9a205de770 Mon Sep 17 00:00:00 2001 From: Apoorva Sh <51382176+apoorva918@users.noreply.github.com> Date: Fri, 27 Oct 2023 07:40:11 -0700 Subject: [PATCH 02/15] Support ingestion from multiple directories (#16) Support ingestion from multiple directories --- parquet-file-sample-data/sub1.parquet | Bin 0 -> 1827112 bytes parquet-file-sample-data/sub2.parquet | Bin 0 -> 1827368 bytes parquet-file-sample-data/sub3.parquet | Bin 0 -> 1827316 bytes .../sensor/collector/file/EventGenerator.java | 5 + .../collector/file/LogFileIngestService.java | 5 +- .../file/LogFileSequenceProcessor.java | 72 ++----------- .../file/LogFileSequenceProcessorState.java | 4 + .../collector/file/PravegaWriterEvent.java | 31 ------ .../collector/parquet/EventGenerator.java | 2 + .../collector/parquet/FileNameWithOffset.java | 58 ---------- .../parquet/ParquetFileIngestService.java | 2 +- .../parquet/ParquetFileProcessor.java | 60 +---------- .../collector/parquet/ParquetFileState.java | 2 +- .../collector/rawfile/EventGenerator.java | 10 +- .../collector/rawfile/FileNameWithOffset.java | 58 ---------- .../rawfile/RawFileIngestService.java | 2 +- .../collector/rawfile/RawFileProcessor.java | 55 +--------- .../collector/rawfile/RawFileState.java | 1 + .../collector/rawfile/RawFileWriterEvent.java | 34 ------ .../{file => util}/FileNameWithOffset.java | 3 +- .../sensor/collector/util/FileUtils.java | 86 +++++++++++++++ .../{parquet => util}/PravegaWriterEvent.java | 2 +- .../collector/file/EventGeneratorTests.java | 2 + .../LogFileSequenceProcessorStateTests.java | 2 + .../file/LogFileSequenceProcessorTests.java | 5 +- .../parquet/EventGeneratorTests.java | 37 +++++++ .../parquet/ParquetFileProcessorTests.java | 40 +++++++ .../parquet/ParquetFileStateTests.java | 101 ++++++++++++++++++ .../rawfile/EventGeneratorTests.java | 49 +++++++++ .../rawfile/RawFileProcessorTests.java | 42 ++++++++ .../collector/rawfile/RawFileStateTests.java | 101 ++++++++++++++++++ .../resources/ParquetFileIngest.properties | 2 +- .../test/resources/RawFileIngest.properties | 2 +- .../run-with-gradle-parquet-file-ingest.sh | 2 +- scripts/run-with-gradle-raw-file.sh | 2 +- 35 files changed, 509 insertions(+), 370 deletions(-) create mode 100644 parquet-file-sample-data/sub1.parquet create mode 100644 parquet-file-sample-data/sub2.parquet create mode 100644 parquet-file-sample-data/sub3.parquet delete mode 100644 pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/file/PravegaWriterEvent.java delete mode 100644 pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/parquet/FileNameWithOffset.java delete mode 100644 pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/FileNameWithOffset.java delete mode 100644 pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/rawfile/RawFileWriterEvent.java rename pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/{file => util}/FileNameWithOffset.java (95%) create mode 100644 pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/util/FileUtils.java rename pravega-sensor-collector/src/main/java/io/pravega/sensor/collector/{parquet => util}/PravegaWriterEvent.java (95%) create mode 100644 pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/parquet/EventGeneratorTests.java create mode 100644 pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/parquet/ParquetFileProcessorTests.java create mode 100644 pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/parquet/ParquetFileStateTests.java create mode 100644 pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/rawfile/EventGeneratorTests.java create mode 100644 pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/rawfile/RawFileProcessorTests.java create mode 100644 pravega-sensor-collector/src/test/java/io/pravega/sensor/collector/rawfile/RawFileStateTests.java diff --git a/parquet-file-sample-data/sub1.parquet b/parquet-file-sample-data/sub1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..05b51be661b75cc923ef93e69790a3a1ba871afa GIT binary patch literal 1827112 zcmW)Hdsq`^+V@N{6DCI{M*{>+fFKc2ARvdLCLsv{0wf$gV9fxLqyz}zBwf2E;UFj~ zAS$(X8x&gY>VBYTb=SU45EK+NC|Yf?1&gijYrler*6!NfZ~Xov*EPx9&;9&P_cMb| zo)gW5xO?8?Hm7O13luKE0)Un{4GaK4I#yCuOv3+=SC-n07|2FbOjwN(WOH(=Dltnb z$d+Xo>&=xu_$4VOqou}(L0}S6_4X2eCRST(GB$uK(Ok@inZP7eUV&AX!kMVnTv>|2 zD>>#86K1r4%ayrUMV--t;J;E}E~%}unK77$)*9=LmEdNY4y&_(2rUz{HG+j`I#yw< z0m(EiR%e4FIC6_|hq(q+2-1y}mFDe68yJOTG+N+pK3=KKf?t}CDvelWC2Z&97>kW1 zB`}qfhZUP^P4FjZrm+mG1SQ~2$Z7(OO1Z^itR&-AD61;WmFDU?BS;sjZN^fg&0JCr z2BUdcMMWdH7EP(LSj@1DudTvwY9xx%$N}JK$ZQ0U(DLxLf+>Ow3}3s=Tq!ORR{;$o zKYpIMqN>)^2(F^3@E-#Ws1mC;mx5BfPMf)=)=VDZC~SCps^Cga3TDBZDgp#}U$Dwr z69&>a3S;GaSOvKg%`@7JHSiCpvJR^>f(59yst#{}jibbDHlszn7nRqTEJpYSDz7ZV zKd<1(@n5K_6u*z=mQ>Z&f=}@)@uDr@I$A1LY@<{09$L&bRVqF|1Mi`+)nvt*5Dti9qpem%65>1Z z9=;A+6?m1VG-H)2JEE$nfDu}Hm8lZ+qvcjrHo|VsW(?na8+;c{uc|TC!CW*2-&mVu z9VgF*H>L*sE0ohH`2$te8!e5hKjDz6!KkVyqDHjJiZ>RV#H%)!m@%*;x*I;h!NE~d zR|77Ce-MR;g3&^gxwcYLASb2WeT$oN(EvXS2V0c|Q+ZLm&83ZC3`mCXHsN3}Z{Lnp z!vDijl{R3uQqYd#6_l98V~EaJDfx(2h&5Kil|T-*39uPIL$k2Tl19+TPsa&>?+kng zNR6bSa!X@Pjd8skfEq2LSS0-|V9*pBW-bF+PzFXHMDom)a3s2^-dI};|6fR1C*KQh zOJe{QxJ%Rt7(p&yVKdj(;)RT%sWxL-m81#H!CPd6D~$7CDWWQ_BCq5Zm@7++m9-)Q zztD_t8dd?WfN-pyb4Vd0rmB~Vpqd)-bxyjmrp8>N`j)0MmX*O)G~Kv;yG>$~`vR55 zEbJV(1<_%(*d?)sW~!>dj1bHO2+ioO;usG zz)nsM)>u(hX^eE3$?4!ja5aL9k+Bj`dh(2Y#{Ua%=?fdU=~)mRl_A|14=iavZA?-rIkkb4oy*2T#RppsNMN0kasw zA=pPNG}c(i?*N&g4IH9raSau1jnH_RJuvdW_sW z-~`O=qlgk~kURoPu>)WX2ZvIX4I8VLuLgg?8&Fa%evIUqjMIOhYFrh|`}*)Dl$0AQ z-#5e0X}LI0&Q!hKugXgDuaOOA|M_nqqY?JDJUHWZ@K2p&JV2ltoSPiH4>$==k>($R ziQo-foGZ+@m1JNxGx%>>N>!Z=OoM5t1($%TQ(s^gfhx%^@z+pJ70*s)lp*w?`8snJ z4w8EF#34r}c?FbdW&|KD5Sr46iz-h~#-VL1ZNJY-LR$9X-e|NnijR6=z-Vm1J6vP7 z14_n^Wbk7k!}AAVp4bLX1Gb8P=X1t7MYT8w%}*E50t`p;(@S_Sjplax+X}HX75IZH z3(SW1yk%H%+Fy`dBPPCsr6HKb0v3u6x42O)R$J8wC(?3^l}4i#$-^qiETj;3ue*Q- z^8lsbCEmZtl0B+^ywe@1!h)3>MSBr^qhSp6h%LA#@p33gj;U(71EJwYQVV7R1&n9} zF_*k`o3t8S=`B1V8L!K1%UpjdA3lPwsDgZpCNIS*-BS zIN@#nf2QClVjxuTUr>sfj;pWHQq{iEvjM8EQ(cPu5Y(VKb+}SWR^tGxHK~3d7zHra zx5!(}o8-Ab+Qesl``}<$0T^M0qX#gjiD)TxwpyNm!s{Wc_)w0yLdPk<3$CeY*+$Ba)twUH;lXBu z58@@5YtKxIJ^`v=H;($JDqJvYjbcAq4pwi(cUg&*SJkT6j5So%D~`_8_}27y0SZP%*f*^0|6>pPlQ z3&|}K6QBX}z_UCC9Sp$ zVt|(0XsMq_z-do`yhE$m16}6 zzJrY06ma+BV8DE80uzF}5oMK)aghdIQst}uNyAa!$e_}|9zYR2kEB{HM2l7O zUL$x0EXAWmr6>dE6s!U=J*_|kScwzZxE*{SfaFmKZrpGsYQz&zB`m>%Rb^!r>j)La z0cggI1!$(Z9BXW^0}2`G6mczX&BppFmOlg0)|Ioa(y>)RO>`*J$WYNiBfc<;_%tAw z;I;r}fPZ@{Pj&DY2%dw?5ApR@!b}<-o@@gkEfv?4#}$r8k$kAGENG4jH1bL);5o@z zNQwEosd&C=Z2t{Vf*C*w>pO&&ij(}*UO*F_%=m%_G6bn{YdP+@8#RH>}jswY7 zVP3}4Gg#kJIPzk18D?fN5S*N4=H;WlxPjSt8xYK>azI)mc@ys`11J$ukn}nXp2HOc zs}z3>ZLWgvwokDfbevQco|X(`;BwRmuK_Y07et>So2o2~fBDL}D6ZtSX7M&4#h*&U zjonsNp$dR98vXwu<3ZV0{xB<_2b+|Naz@@IzoYsKtu4u#IXs%Ty!{dP2d~4ps2&ICPc-F){ zB<*Y5nM+y!raG35l_GeggeTs5IM0s>Wbm5EU;=Kzm8E!`h+ynOXeH*F5^*AKfR=We zaSi-WM~VD7r@(A0GgnGx`<(Cvd5)xyk7v}XN^=#wm1e;jYkA`gMj?WSm3r1?U)ZUnz&oFjL!z+d12Fv+Z^Wf{#DumDn7 z7z#2-`TyNE!J(+UuC}fMK8EwjC_aRXRlU)Fgo0@5DyIaj5jyy1n#O1`vi?Sgudu%I z1^)|W)TJT#Z7R*n&$A$lMSmE=N;0oP%EmL@c-XMD_t5b8T0`!@6_q|Or9?t-I*L#eozThdE!d6$JlGBu!xkg20e9i*> zkyNZcEeBHASeIGgTP@=~MHs!*_HS`HL8#(4Knhp`nJdQz@VgsDHJnV`$4kw;QYJ3Y zSQ)uXKP#313i-F-D8BENrMwelJSQ~piWrCjGy8v!yiKR2WyV_3A3)}W68mW42cC;6 zMi9AWyZ<;<{%-(le*tJ(sT@y&NzLPsf#YyxTlfiKqN3IQ6YkV6?29N9|@@yP5#>c%tYW`vWI4Z-< z0*t(IDw1L`dxYQ`n$m_J8N9Xe?c2fQkjl*HV1V}+Lx}MVlxJhy;DBQADfnHx0>B>s z0w^UJjB*OgLID(ve^3D%OUp!5xR9;k6>%n-7<3linqql0RBmbiyXVrGTu6)gukl4U z8Ef#Q6fPQEp2uP6e0lL?yn5CM89a?Aq#Atg$*w8k1u!8s_Va15I{L@HXK>Wpwl=~a zfsYv&1^hRvz(=AI);aF-4LlVKQdEt72IW;%gaGomn}`N$^fxeZ=FRg!B@G{&-{Tb_ z2+k{UCzMt%xrizo@R^boOTitsLX?U7Ry~Zf9%}@z$WMxQAh&j?q-5b=c2man74rrx@~o0F81TpiDC$a&yH~@+m^*M*nmwq{HA2 z#`je4D;i!;DQ}i)d?|YNmiQ&cQeKuH?iTozfgk8tFtg2gK_mx11H%P)hKec5*!ZL+jAft zemsGTITzp_zwN{Q+ijl+u4^BOi*iX^l3*6kzo_d{!8M5rU{KiU5X$=$z!0 zu1+U;)oHhay81GZLR&K)MWn~ww-OnN7S|O<@`CpYW&IlzWW)kYB}GA#E15itVU(!} z=_1n8BRooWSc1dD)^%D_h&9saKm6(T2{MJ8wFm8u zy4@ChK*nkhIXEm&BOKajn;`aH>}n7F$a=edCA5>360ApJQiU!Px+e@u#;%4Tok61k z)Sl-M{ZrI-9X#28Jbk(Lba6U*dic@x@T!PG56L-fB}WWJF&)ISt|)4x`81Nd8X*#$TEi^g$_S3oy!akvgVEmvXZA$=+au^6zQWKFQ($9 z_c3L}Tm-64kVK$ajJP&w7DVY}11lc)2Jc?`Q!++RdWzU&gfg$|MpV^H*G=004%Up& zlPuo-umoVr(Nh=CdEQySXJL(Ys}egvC+LH?#67kjRW> zu)owDmq#$~54qog{c3gLh&Tk`+-tKsi9d(MUQyH-Ms(Z?olq&|#@1c~ExJL+2F4+# zo9(*}U_#&9yA{f=*mE*@izb#yCJ6^UF%qTA5lp{aJjGHc$n`4T)=^09HyzX}pt|f% zh;4qkPY7vTh$`S?yWLOlYDtx_>X?H`SfyK3gk?juG-&K@ z;+M@KIRs7zIx4K0?bNY0c&+<`rF_hz;dJ(7r#Xey9x?r4-^8+romjVD)-g^uC2YF$ z2K$V5T1$wV6=oB5x(rH9flSy81!XQdmE1O&EPd6fxvBJ)!I^F%=hCo@lZ%YB29hsx zWC8UF-nPIfGfS=w(?wkq1%s%a88a2Kn}~TAl<|t05Y39lPS)N`&g{};{OiFXd(d*d z;fA!VdFiHa{KHN=G2ugD^XemE!SXDvQrL!kk!)ATpAro1%L;hbPl0>{?mH~z%Y{1t zKWb?#I|_UZF)rL^&TNQ8Yt{6#wYIT{>^DTYjzNy&XtxOmodG1BtOp*7)#>$v7p-Kz zI*0X^ICLJ!Bay^d?7ZFS+Lslio?Hd;er5jvJWXd zTU(bxxF@W$qr2WcKj>y}a~x7^tZh5gMh|$6UgcMH9P-^XgLXT)g|CZ!Hc2`{3O1p= zb@$i-5BjbMre+=Y2x?cp&xZKf=xci(F5>&+9or0nN_ImP=mB+yVI^i-&m(_bXC(wMg9Q z?}V&ZNw(k7tMQWl=5gv_w2lY|supK_^U#MORP9cBhP_vGDKknLAb;LCB-NsiK%y_@?LGXk_y2Zlv5AG}E{0};#4ssLjM`!x3HAJP91QoHavKgo6?volW zT3eV0n1l4vRc{uvN|SUz3o}4tXd`9ww>SG2E(H0J4nCC4`13;KyU|eoPzJ>i(rHK6 za=fXLZ}`&imb0><13gVHG7SpR_J^M|3Ce$K%2Q0%4Kr|3>?pC8K>cdihi?i*=B zaxJZVc#E|lU-x1>MolRhwz+**Elg|5!Z>mnB-o01=#pU@NarP2!*E{R6-V%{U9urJ zan+1FbuFsZU#0ga{OA`@)8ArJa|r zrfvlH#116RT@P2#ayA1Jcbrcm>eXM#TM1n~Voo~N>Lz?Vk4+KQTkJ;zTKW&~p+}CL zV8(rZw=@MU&ck-Y=@C(?EuqX1D&^>TuJ(-8s)x4Rs#OB9xzQm7~2TEv>T;i9Du$(|k1=sOyf z(+>Z)_OYDK7~MWf#r?~(>}G$AW?M{MxJ>T7h`Q-f!je@p1?05zZvP{hw%a3pW*zc*>QK(}pKyNzaO zVK&ITdyvhD=PsCgxvw?ku}N+xxrQAd@tsyoC5yUD@T_Mfy04LVj}MP)UO7BA_74ry-{^A zqGunQ3KgQ92+%tNrY$_T^LPnZ3UO`eOa~5BJtIHuWmiV&2-lEjIRo{KNV%dYL1J^i z^;&EJ;#Bc_p0PZ<{r5exXkEl2soM7<)(o!5*Qxkpk3moPE?08jf$DKVLyBx1)TwN0 zZP$o@tSM&9uUZ^S-*7#$Y|NiO+H6P3QD19V91XUQUBwBe>LQ_;V+4i^NY9|o0G^FE z|Ilp~suEOo^wxB-AU#nd^@({umOaDuaT$C>u31OBlr1`PlfBM+B?UxbpZkQ(5et;s zqgxfSV{K60Vhfq`1Q-kVueDx54y9qxiq9W7J`R#Bgl6i032|#ES7u}Y zdmaC9;D&n$bALVjxYHTRxcjR*`i*jMD?b-vl0p@RVyM!ov{M=wqi1jX9k2a*qi=la zc{}sF#1ngissJkC@u1Y5=}Ew~!Lk7}din<2{js?>r89@o;O!{c865tjQre z<#ObSqUGfN8U>eV>AcIS5$Zi5|7QVoLeuMFGZ?E(11)9hCk6I2UJElP{9v0Li1}PL zG$Eclug+V2KrkrV5hMf-?+CI)kyQnG(KAGo=7f7kgZ1TA;Nz}kSNYV-gZIRb^#CP= z)|;BSRuZy!eU`@auie>F<|wUo|nQTv)CnzCo`B2DtkE zB22*WbCaNpO!xb?kfnn$%n&x1CLOOHAxx{P$0*tMds>J*Igk7a89RH6Cz%x~Il#uj zd+}dJ4f(6u&L43Gy_WryM^g$ zgf?;7F~j}pAv2$_nE4ZtGLYEpcwaJ*b4{=_in*=aC&K%eb&Wf_eSqis#ZmG3IcaK= z_w~7=;CR~70RxXOnqu^P>~)b_KE2lOeQ$Lr z$4j^`@pO5s7i|q}N(Qy7zmIKl(DRim2?6WG1~uAj=ZunX&5AN=p7j{u&|7VCh{?}hK;Tp57w!#7Oi-A0qJL3KDX=e5QaHMU zZG^2`ce@FN5Z^;!Zq(mkH!F1`TUK5c&4Rwku50Wgh5@%UsOYmP(vv*w+A_j=`8b$s zI18AFt86Vm@9!$Y_TX1?&*t+xyHxyh39hv8pCjj#QmUr(8$s{MzWcNzlWzBRC?XSc`{tfG`#IqaZf*P>s)f8AaU+{kGY~X0~>``{rh?YH9%E_JBTv^Uzn_i#74e&dy#Y*xbb?$kOjh7xY& zS+7y15?-mDwPC$C3;sD3v~{II2~Z%Kwu!TW8u}Zn`iv<@C)@UF5R5xg8l6g@ed-=9 zN6CuRCMnwK-i$<5D~P+Ou?H1}RT`Z-B2?KWYk-Y?b}|zCVpLj_(B>8NjyK&QpMpw@ zO5pYPuhaOW+hr7#WN1RS{4U;`^h6*7;4KSG2a=!MHO0p%`R#$=%LpMMk+;(Xr+&4m zy304xp3@eR9Ww0La`6Wrk8OiT_eQzmDUu_zWZ(@Oow2* zHPgMF3Mhgzs9PV3tKVCGMvw<*@&e#{c-DJJBd2)kf=I{t9{@MJ;tIFvw#^@g%i{>N-)Sl>~Kfk!BST0Sl z?#_$C%a6n&AM zE?FFMgFnY@cNT-J`B~yNLDUFeI@#kE$0az@mIo9u_mHz=Q?84utWgK6nc5?g;cC|9<1YcW(tYl~{#8wgvSY@g(%N8##( zUl|?q!{{jWzq-%b6aJ}sv=6;2xa|ay2$>gL&#|^g8qQ!Uc1_2$*R-l%*2bf0?q~^n zUy5=%KGDuVlsia{LiXSBwgA$+@`S>#531e`s*{(LiK~?NTZBHHHeem+CDz0sbDBnd zvY+*VaIUh`-Aok3e26!(jgrTR$Zgb+Umld!+=mF#ftC_V5wn)baZL!k&B(z{la2nuCaq8GgkqTFw}SUf8B=RR)2g7z6lI9d0n)3a7W18ICevWT8}ly+Oafs^n8069-gfw02F z(uKo*CmqY)VYu}42AwQ3gkHk3R%L(_boEVJ`N^OnBV_9j37w)x)6 zK{wIpDZS6LlBZL{eu8?O0S9{OH%qshQc?&R{l?Mv5I)wc-gm4MiV>erb;L?8F8x4I z7u~*BKC?LN&GJe3!Cv7M*2pNbPorA6V-Gt=I1OQgd3wq4-L^FU%%$ei3@>wTnFO&- z2)1cDn;~jZ%ynKvKiZ$-bGmDC=bK^NWdC~q_U^TbA%pg2c-{0!bFrVORB^<>lE>WE zR@;*&IHfmO)V3j6L>798up%CUPI^No^0ze?3?`>;7NG-Gu6QV)|-qhu3>xH9&G(iagEQk=6Q0OkhHPX?jv0Dl1 zB6+Ip)8Uuhsl~{k%Rq#B_Q!6fUGYuAqUKTxwj$2`M0TRF88(GP@8&+4mF?}$YKt4| z*7m>Z2n0XSd&IY(+Eh8!grZcP{&&+kcw?2XCpe2re z!IRXSlzV85*SZwxS0@+0MIPzwJ-!9|`u7P}GY3~3{5Sz4uFA}DxkII+^>%W$Yse1A zTjJ!>TQhdK{RNzNOuE0R~^I8X#)>3EK{Jn|>{U5-$AKk-!wl6!iY)UpbUzk$@*2 z=&*B0iUf7~@^Y`Yb!XZ}eG0TSuV zLo$5oFPT|vN!azF`%zBeVv2sX-?|?<8LOh??r>(Wu#MCs$!lV7P!1B)U8UA{(g}Avht_v-njowXu%I`BSr=E0a5~iZ6ec&_0#Y)*e<`FmDFVb+W z`7dF%Yav@M$TTZ9Pc9wO?U=Zes!`0z9*T=&gZKJ08U|C6CNq~p5+ikU9~DD8AUS~= zHZm&sU}4G0{Ka}I|A?D*zU7F*GzSG1%7jWNE@31mN#Nh7hqjTHnqiR3EeMaF`Pyei zoK7eQq4P#4VfPNw_VBA>eN%V_B%W5}edz6tU)%HC5vVA7bc`RC2go3HT!|}z6X6Pq ztWBQofHvzMC$`~@FZUZ}Ph zCVf0B(@;wu-DzR(~o$~fq7!H<4T?r9(XH@%kvSE;?SR#L&Sr=wCITZqy^js0|l z?%UJ*Xic*~=;TR-*BJ2k2_;h@8(w?0D6$yc&5&yB@^CO@Nz9y2zVd_MHOy&^ZCjA* z)GO6OBnPGGZXCFECLdKKLBn=mqN$6HN_G&B22ow|QTooF=yv9jFzeO$trzZHFAsJo z+C!cF#QnlgW{wj!uI+RYf+z{m?DvA#du00WO+Ze_Z6tb}mVf&3NQh!qX+Ss6sISy* zf7xj#6LlyvW>+lAI+*ztvZNOITJ$RN>g4GRmUqd^%QCoqTG=)I=nX;GMLVP=q0ao=)fVTFSsshLQzx1%A|y#q%mZjK{qDuZ4Q=)|0kka z1~GfOw?}M~u}A1qI#0~pP-MrvpO$18bFtS zd5#cvh7SVOS^YUsi9+_Jw2S``+M>dOnBTsMJ=`l&lYia(tV_?OV#N zJ!_9|eW)J;x8)7>pn%}cBth{d&_ZE7lsngMn3X$x(-xwEHAn6*?u*l$ze`&qeT8~x zA1)Fd#3`kG3r{xxg-&vy;;cJflIIvD_@fFn9S-BB=pXb8o0G@r)|8JUO<%YJzB1R$ zc3Z**2C8UE&yosu;C zQ7`ipy`T1hwoHx&j>v#S$~6YD_MkO8Ge_^{c8K8QdiVy&&kUhdiBKB!TRUw z(CqcqO^F}8E>ZF}^ghzT6Z9SG6~V`^GQwA(weAqfF>5<}{o2LRkbm@ePi#0f_vn_z z=VkRZ!SX&==gz3W!PbNW=;A18>D2T%NZfhQNoVR%^$KS@2dchy>a)GEKC=Fiz-0n= zDoPplIM9`~1_GPW{JJf!>SA$9Y~&#RfMV+6?fbGfJLR5by8=vd`DSL=+$g_!r`qvv5W4;4B?rr5TCc&13dgY)ZjC0cL6X-Zr zIuG1TaQ3siu-Z@g8-2 zOwJs>Eh}XV>onD0&Vs#6#>=yJ=mkpmjnJy73!Q<)ycaC6J~{YZh`#laox+?Py!r{+ zd)^a|zKI%Jmhe#S2w)=7snA;4A~`AE3Z0B5w_OD(05fd( z$Cb^o>#P?@WveVPIT251=ZNe*39&82-K@3K!zcU4(}M5Le5sN5|#zjREHo9_N5r<`(>Ig*ycrtF`fpyf zTw2_A&Y?J>EKJ$*^yIKRuPj93khbgr9BDO!u4(X~aT$rSuE+3%Ty7Z}5hxah95t?f z7o30T1BXXYJ~uP@-h~8j2YB?gTT|%M`@&IF7U62Hp>1tQ6M<=m?(ivzLuCBaLoZ+c zeQe0+M#w8{69!LDZ>DT1I7H@F8>GZ0AHvY(!p)jM{Haso8dU4gwr&tFH-tW8)$O4N z7U$KjZxZks9+`F93&V?=o7v8XAV``k4`1bHK8pXT~2f%%F!&P zc9ed@S&6?-kyH^G<&(aP@ob}_{Cv^vW2CtLlIwxVuP?aQ2CTMgSsy6-(o3>U)kOd4 zNS7RJe*}1FWgX2qCwI_0)QwR#qzAO?;qxB!;O8!w$4E1fLw7%|Bc} z9wZ(D;2r*lAqp3=Is;4Uo0@!?!-^3C3>uE`6vIP$xzqHxwfQh=aHgF7^0tO@PUr2= ztnL?%v&q8eO{mXCtVJrBT!KWTPyzr`4H896Rr0)+dqN}{&CUKJig@t3$#anN^q6}$ zQ{y6e2;nXItUAwlv-_=4|R#57u|83zSoXQgPDr5HW$FD78gbmx*Ha=T`SgStN z%RZheVtUqs4n5_>C2bdlT?{IH{Euh~4B;h0cPIJxL|xXFSTu(S#^2YaQX~GS%SXHK z?^YSX=X9Ae-PG}t$j`NZkwK1^amX3DEz=#=i(=v!Vb=xgn5tlBb*qB!?NqW>osb@J)QN0J3RAMc9d{-qC5g^NBrRYGj0|;wqo9SY`Z|^{t|>)oSl7k zdIGXh=pVcK@uxz^he4xN>e<1YwryFDfCiGQ^AV{o3x2?E1!Yt8Y?^p8rGEG+@NU;7 z(7*1~^x{oI`a<+AK~_S{7#Z0)Z6~kP*ozOY-00{)w_Ub15gN@`XQ?GK=ylGe%LxGV zKtAYM##tN^i|G1`J!KL}O?zzgzpN8Lt|)4p>eH_~u>1|2r}0~}?okRn1>N@vYz>^- zMqwqd%Jh|(x*3f=_m@V=2igXKcYW;mI1-fDO#r}(oRKwBgy12XWEX+uV}tXG)LZyl z8Ih%wrCZ(9sHM-g3@udqWpRYl_j1QC56QR8JX3hcPa^CS?23VN_R0&cJN(40$$B7G z?RC!~l3}ogWfC;Yj>JYRI#wwlN2DFQ_7E&O4EZ_A@g^5 zktXIqDxp=F4OlQ48FYoK_FeE$x zDeqCC)RC-PCuREr(FpHmXeI4AoaA?ijk2WA*i`Bf?_3XEH)SQq>^eVoc4Y6FC~+J5 zZmq!?o&e7j$O;C3+sryL*L)E5SvU*S{0SrUB6*XZ-GRzK$N{C=XR01S17GIhuD)oO zi}au3|8rSiggI9Q41>&Y4g-G`0*(zP2g)JE-O%RC-X=6&dtN1_@-Zlc*)P0HJv38% zoikW5ogNOp!gK8T(s@%x`F(2>`ax0jkEFqb=~iXhlliIqH)bmVeoFq)4g8{eb#={@ z_Mh~lXk3kxv8msLg$)Q~c7sIwnFyB9>bLQq+BpvC=bd~NOvN^BdFQIv`R)3^^W58v zZ1PWBBI8V7?^-41O@-X_vj8PmQv*Q6^HF2L-b>Hh4C^f18#&C4FZOH5kjk^{%%tqb zvS%*qSl*gBIU$9Czbwk$vGFVjGfr8rfY7tE4}vU_UPrU~SQh~P%ni$o%#9cXcl9Qp z1qRF!f9Jx%VwMA?obmXOh=)2`LIQvxVA8Z#OWy??>+%WP7lnC+k|K~T!jzY$<3mY)!j{1<Bsj8^UwKFL7&_YhO7A87?|k$Xt}XKAC|S|JcnkX5aQzkbR2++R z=*;k9hhYf0OPxf?K z+;a@-NxMHk%MNLKthNs;9kdPov#x#U?zz~9s%<-e_|SLD@10cW8%>{Pefd(?^oFdN zwIlkf$WD`Rf3J9B#rdCSrdVI)Sw9J82+>{^3ygvp8G?$cenff-Z@Q{SRbX! zsJbM3T)0d}CJ6$CTUPwH6NTtmoyWfoSaES~I9|IBN4#`;=f*c8>eY5=-5j8BY@T(_^ZxmZ?9Iyi`K?t1#H(PqkXpWbNk8RH^*U zBZysDys|sxQ&Vso5!_LIoOi{zrxm^k;VYLyuPAUuOVFtQ8k!0#{{D@%o%e20mjVb{ zrw=2X(MInpW8iZdQLEJTO3R9JGobli?=_L+n1N6qyZE&>>35%jd8y44rF&hvNG%PA2wn5Qi$IgIaN9Wq#O&}mBXh5{6v;~WndfPjJV#m&x zx$`^o_xbNU9_M^MpV#~Ke10z6Yy^mz<;&Yyj1TaC&(W7g1rhRZ)Fd43j)WGOssKoY zn=DnHMVsS^nl`RHPuM0}9?lCItW2jwlvzfPv^D1;v(05CM!hszV~+h|PCuG?^v%j7 zW`=*jT>4@6`?Juw#@j|R7CY2{uKJg>%MTckNCY%ml?h90o=1f2osbx@>E7dwqy(<8 z0iUQc0yVWO-)Q~1zSCSJfBBPlE}(U>2iTv~SJb@y_KW$U>#rskM`ITjE5!IKjHOtA9a@B2I(>IrcAu9yTYdJ?YJ$eh<`(- zh5Ff7>Y-Cb&)8!U#hC>Z_3H_fi<`3e*1~esR&?U3?}{Ijl#B%{Ji^b;V`7&&jPeV- zUw1`?Yo4k%ysZA+iFnc}f0%xSQBjYZniEUlR&I_g-UlnOuuiZLMh-#!x=vij^lHwH`dwS?q*yj&{^a8$?W$B$a`^Td@#oV$o z?->rH5ue#|xtFykzqKG-9{TqWRiCP2t!=72vWA7l?_J%Y06B`ZR;u{2ZYYoASjM7r z2A>x)2)wW0d>G_*CxmG|0c^i$hpZsWC~)8tCKiUJktM4p9IqUDNF=C%OJuU}wx){8 z>30hJZf2tn?Ns#}F-iBu5`0de#HHGmIv|b$4)KS>BS7H-H4?F?iHuzR5KJKm|2aW3 zgv-oDs@Z;*?_L$Ei5~QIyR>NAHLsC%wT#k7tbDO70}~AOYEa#rKCHglc{~kao#-*g zX7PK_l=wgP;PQhC;TV(EK=RGHeu)IcRw535vK5k`(TdMoAgqoLp#6KP1nOSvF=JBAs^N|AvnkVV|Nk!M zxT(@Y>2vndT&q#pbYF6o9GDts3l!`)K18kJt!&FQrF=M(kNd*)7?BOO6ojH1)BS3(J$Mp96rV*T?cxwnlNf!JU? z80?Ewo{|<`--^!{=_U9m)lyn{DvO`9&P;Acq+UW8zTF2-aLthNrV~Noa=-YLMfp+z z;`wYZy792>KpHYjG=She5kfdXIR{A)Ei8JYA2?@MfFCF1`_df=uOHC;V#p&&NuJ8I z;0-rDC>E%(%wRs|zQ;gSvGdYTnnLU&VVmckr%Cc{f+pg*E60>dTG@0#l)4vsVJK#K z%=H2z^#$-mEHvU?d-Kqm$9kOU%Dc(!JN@2?>slpQ@nqy@jCa77ABA@C~J2Gj!5#BfI*i>R}|4P&c0?4JrL%u6RdI=0JfiuXx?YMx`@n@ zv(DrGR_>Jh)kRh*#lWP*?$>>dj}A^3;Nx5z$kcMqV0*K=sLz$6%C&qmn zDPO^d!q=6~^HAY{SS?Psi*uD7h6kvqlmTamZIQp7n`KZM@k$ZbEb~hx4^v;T5kA0A zKgZ`IFXtURPu?Ca&)~+EQ=q<$AO0$)3|L3cQX8XteAiNG|o~N?m{iTpq!!TEps4IIb3T@x+o;T(aH6h~!x1b*{jB%cgqaC}XNHuye5z zfoY8xm<|b;T~>rDxnj*9B0uztbHhTW+wl8W1HgjpjlsG45x7E3Re6t17q;7U|$}_^+>GbHvAFyUI0TuJe?FGzoC; zy$j@zcgwm7>AiL&etXLNKK!}sD@_q#^V5d#)N&(mY5mVvA0z6o3EPQVc)wb=@kE2h zh)8Glj|Ts(Tn6Blipw%a*cHpj9N1oY^HeHpfRS zN^NwdZP6Xkn%a-%3)MhJC}D2ONbS>qkmK}oj!AXSTH?CRH$J?$j@sR`AinW1wJL{k zN%Qoj`uoCcP4ewNr*=~<);ttKGrYk7k$ko^9@q`%=NEo;cE>|G{v{G3t~oOF4-R8ZnVoKCs;o~ zn7>`4sL?L1OwO=V{TS=YbhvrF#sqd=g2P2G02sM zcAhvP696al(`oi|hH>Hmzd~8rStiKd7&&*PIpWEG;2`DPb{XhXcup!xD(DBm$a1?h zmfQm8cFMbw415mH(Zeb#w-3?rexA{g)E8bt!d$_WCg(|~=Q~{R(gn9lFZw})M_;!$ zgGsBc#w*l<1&I2w!nhTT?UNWc~NFf$mzZ-~fA(OT$d9SoTBMq_g7VJk2ZkvZdnl%$@``$ zuqmaoA))(s&s%k1oYqBEmn39I^u^fSA!pKcVkE6oc#eGMPymsAFYxIQVQH0*E)L3Q z1!3M>q6h!B=2-_N6+TsgH&WwJ5V`M_j=Ui7f==v3*iZ4qe*!k521_`*wK^C-oZZjpC3V8Jwp-HIVV zdLX{MfaMPOFVOH)z~jbhn~pf?D)=Id#crXN+wRCRj+Ny;3~{VIx5Ng6c^%Nkv2RrB z&13D4Sk|fjS4cUZ>|!0AFr7)snOL;M=D{nON+EGZAUIyk3aA&+kuu21pXU!#RAHg zRtgepqs1;o5sfS9%u~Q<0b%f#i<%|==PdJ=9dR}$Hsv>IVh;8~1LsRe&v+7LCBi!ti+bXD|LW4W5hvo%q$lk#A^eA7|2x20YH}w9qx!2QJi?!`obbz z=#`6855u!U*3U@6$14gD>=aKTSJYRLob<)~5Nh4IY^Y1%@c1CU=2i?NGw+=nMs1pd zBfu(0SRim~Hg0H>DJo`~7K-Nxy0I$c>h-=KheMJ_Dxay>egY7|*jdjx z0x=|E20Y<=u8u9lXn3nYI@@y8S387rMYQh_ly^lgEsP>-k2 z)YIWbcFINy`nn=bny!28_T7{UpGz~bdH}sg?wJU=e|#U6nc#Kie0sR~`Xgj#`MwaM zGP)n(vta+cPI01~mDXd?fTq|^XrZ3EmCMJ5!Ce9|GWKiAMP+H3-NLON9C{QOGjhcy z-Z3_R`EW4gX^0;fYzqR&T(Jw4(665h_?kS0{WL*ksK?c;%BhDTI{d$F>yY?lVGB9+ zwP&#WjA-h2a6RrqXFn3u%Htc7DN|R?IOiMo+$UkXDZpV`r}N2Y2W!bYBsB9a7e;^( zV=*tvr6vP5#L;5^3GIHo)12WYWqw(vRUo&`wLYQ$e_Y9!bdho0kFu76>Qb3lhHmnA zgBVjHmUlZ_fG+@cZf(4%pCqcCZ!P6DgUgp9tYz>yO|O6gfjY1qC9|^nf#kpUI-2s6 z+>eW6*AnmVIL4wV99+`49I_ z;?b#3aakn^X{7ZIXg*ie`cG>_(Gsm?L)8T2p;i5Cbq5mG(oy?2yN}p#K@}LJA(+vM zM(t!ko)#K^qp8UBsWFJJi|VP)PL@xGfJ)ETQErz zCU^8bO(Wuv4+4&?2qQn{T>%4^gIj{Tr>NV{tA0@92~M&k-nVDijl0%2;M9Q8-butK zOZ_JLIId?TsTKlwh`-yo7b9}MmT^M&%DtfS8-rYudZIU{ui(@tkl{fNt?WNDctrdV z63m$LJvgaq^VEUD+o!Rc*hY0C#!)3L9Pr*6$ILS660oW!8z9q0xS#DvHTKxx#?d}s z1YyynOd`F|Tv(rVw_-Gue7|}`;3r&_B9CI*q#FIt*^lx#p0WlXvGbxuLMWWZ5pQ{x zh{#qnSSJitw>oioa~+oj8y5I7#NPgu$OJ7*r-)DLwYzC$padX(p}LAohJBbo;Zbf| zqu;0ZTj7xbpKPR$-_>7q3~J zCYvnb%5R)}^O`Bzd`LRH?#pf?a41vb!0t<|mpoFNf4ZWI&0M0?M+{n@jwW{$y{;$k z+FjKyOyc*DrFhm;b!ZZ-OhCV?Wt*U!zK%<(@!XyOx&_la#Ph?wmc?H*)ou>PQ1V94@dkKfA0OMrt_c%qWh=3m{+3wLU;%W+4x7!KqLC!-tXT z2P|OuSJGzTPevq6JnCMO$mP7i$kwkbM%Uk}fdJ_W)K(v|CrFt=RNEwz`E5F*1o`9O zxs1>pk!DCe)@zWeNY=_m(y4t4&G!iDV828_=6@izMh61Lv`lWl2g#nAAI%5;4Eaem z&fXQT6RF;#!Gdq&=q2H~Ful`%;Ef$h#YtTsVIJ!c_^(tRTftyIO~hj2Pe7!};ur^X@ILdRGzbcQ&@0*fL{X zq6Nd$M{Mr@9caJ{0vj_m19IgsUCLnjpm80=79yw2|AoKsgAN&!mGU{7_>w+?oJZg}wi`4_Z27#K%HYR!a_;I8`+{Q+w58SL-=rIZu0=aq61resbtA(LJXl|- zz!+JZv1}JoUS@X^_cJ6mV&u%e2Z85q{y41hS+;LiRZXyvim2#*ClqwxcDi;S4Rtlb zJaL4Oy7}jQZsdiDe5^c&zc5Mtckg@-!bB52FO0>uHb;@&)w0-}QozN;?nr8+{(Er1 zjmdGv*Cjca3zMi!HUii}&~dDojRhL^7p-J&b`*ubtVUKvdu?%ySqQs-fQP{VVV!)r z1enr~!a5pssG=TSGRh?Rn$-A65t2&0r^~MMGocmRN~)@n-Q=?vz5K}k;mAwGNGnJ! z8)+HfvMKJQrzk%;YbbcZcltG0L;q- zr=$~--CRya-?lEFvSJBhGr!Zi@6CM_I?2I(Jx+L>V6-zxh0+-e@47)uJ4$CjBF2BJ zzS`e)ZtDJ<7lzVbOlmJ}S)x(E)){yPgp9oT%m;nlqq9LRc8cVXVD*@kax3us5cG8h zxr3w>wyQx%NClufCxEV|O>y!CqEE9GUoFze{D?~a&KZfkv-L41S#L*DKM@)wjPi}1 zQ9?xXy#;3dzXC8oZDHlJjm0M~fH8?4uN9ZH=5=P!X!FG+=~~eUl80SrjbJKtFx#a4 zCNt&>@aaQj&0M&3gX!X(A)0D&pKje(pyN}SIoCwO0AKuQWtNIr`KsXiw?@@bH7JDI zKbqpkgUm_}z!9^MA_|5q@n1jqnoJxUl$^m`mbXch*WqrNQTJk4uT#dY4=u9TK8PSA zF(i{n!q2Pc^w^sa+MIY?=%RHkv8~9%OLlctK(dA|Ofc?~22$X%T1m3k1tPI`_1_3N z(PHA2$U$AY-@UR@A4&ck^k{RTV#SK5myFOwInncCngN|FqpZ=4%~881*mG2&#%_Z~ zs!z0`4uyExipXBXmFvE%%O#OlWBN^Bh;;Ta<)MFt@5U?pJmUfTSrz(qwJ^6W{si9X zBl0vl5Y$#L4uum$Z8Fe*z6|a6KZAgBTWN}hU_5qQA@l2UzlF*6`>jn%rlM##=>HfJ ztDjnAHp!JF@o>1qi2c1?EP{gaBm)0rMekTNe)^XnolHq zf0!X~d|!jIYc=dq=4GZthTO;2V*bUfL@(I zW}b_XtPfT>ne1cL=T8TQt4hN;c2Xu0yVaILzO zf*xHimiqaZUum>t9Y5c&J5i?`+Dt}luY-QFX*C+cJg5-+M2i(FW!dPy(5wmBHoWF; z8#gszj(nO$!wy`h6UxIRIFWE)HXxrLB`M-k^;vaKyryiTmGO_$WO0b|ir?~`sWkjW zW1I@}Z=N%)W+YN(O!|Pk@W?k#_}fe0m{kQCm6BVL6MB>@aD|8xJX=2sb>DKf2zkZC zcwj=9o!M1aHN5lZXaf!(8{IQG;70VGRxo`Hzdsb-ALOxfbO9R1J!EJp^*%*Z(YjL{!#D+!)bhsoTG5f}$avnAQ6v zr!$mKL$=0xGd68VfdI+NIadQ~#ahvzG;@Jn?PgnbWAIUDqtTx{_g*bX@bYq_31wH( z=~9g@P2F-bS3)Eg3^eXP)!H9~+lo#$Shf=Ge7AtF*{Wb5oF+KplQ zf>JB-`f-H`v&PTG1Sy+m)e?uTb%>DT(O}WjHwUcY>u2Af_)_{Yb20AtRVQ*Q%K>ES z;9|9QK&_{gJn58VBuDlEBR*1F$wT8{fkRczVd$&@(cdCQcb@p*1<_9y?XQrf6V(Xn zSA)+_M}drm0yMD1$*Iku$&JTT-0_{L{IO$=N~^}u0Y#-$n4w+Crn!K=dv!D6NsVKg zt%QPqIFT*1TTWWjq$%g{fB()GBcT*_7IFRDR5LlE%#aO(et-24cSbuSSX_#evAyku zw+a7KSpwue@fCV*zRCb78;IpDfI&M+hdgkFXc8CW*L!6Aog|OqjcKo`^2^eRN}t zeHBaSc7o9MML;2SkdK_E6XwMB9M0ghh1XIy03cUI&Thk{YE34ub@#lJ`0=WJjJDpZ zvr>;nS0H$M!XJPbX04@JoOp%&i1>G5Wlur+V9C|Eth=1quxrV;B8^Na$RfZipN)v21kWv;Sx%OH+C#ll_bm8!LL_a>>4a1naEOhwQ+Cw_)lISsGXUEZ_!%-4@}e zbD+>>{=f+HJ5Z<=kPbaCrYluPhLO|Qq6e}Ybk81W+v5DWu*qnR^ne_z2bqxlsrndi zFk8a>S}e8|ZSIiTh(O+eI5ZI+w((l-7D*J&;Sc#0npaRWBX)K~HmA-4VTi;}?iP+Wvfzzbapi zPe07V2gPVSIXps#r3b8u=C^M+TGk=;qvReO+D+;9WU}8y!n`l2_KpP5%U-|0P2azT zOv~+>x5QuKX6xu`Z`fGWp_KE|iem`=XMUZ1KavoU9rYKPsdNE$i?W7l#jn>?HWv8z z>VO1E!VgYN#h3az2KwSW21)L73@4mZnDO>!64)N@OIE^Rc%7cu0CKWT-G)C;Czhz;VpYuV5#1pUw?f zZXQ81qqu3Ndb7w73fxMmV9Jjb6n7A!j4n%6PC2#y&AkZIaa_7#TtFE&KK-W0tnFZs9@R+Fnx4~U)E zfpo#qIirMk2`xYaUZ*u2QPW>3#8S3^1k37x;+#t%S&> z!6BonWWb3W(P-*$uiPA(>TKY$8+j=fa)sOQ>jC0J+lrkm5$WZivTu4Q`%-^vNKWT{ zY#M%-j$-GZiT0SuJ3&1PxnV5Z5{%P)^IYJwxUA4x(3)K6;Z1L*z(+LyB=7>``MNrl zOM|h>mw1K>3@Sg|-XRx(_w8W6q zaR|a&2w!qvfQh5TP7NW0@&E-I%568(t86+qG7sTsFH+|kv=ol~t}n&U#q>6!tq+Lh zZ+|2HD_)V&PjF`#B#+BIzMx92?eNLF%YI4{ir|i0@Lh5XDHn&C#U*{vfgAJScNg;K zeum#BY^ipH?t2Gu0e1A?vdYNZ1^1@+aFrB!bN3ZQSyq9>Rdr#GMFF4l_3psL!KX@~ zA4gd{o0(z<3#=hRf;;xTUV z%Qo-~N$Y+uCyd$bGImfIGzv7qJQiyopZ^qB7})f=$tt2F zz${+2D?~Vv9e^LHep-M1-_?NtKB3V3qfuY@xfXrB;=11!x}!)31{ZSbz#iyFC(@@` zE6e!8(Co+b2Evp8{afRTW9-60%807B(Zd5Si>P$tpNHbz687O=uF$I3#SJD z6*i>$vY^Kj+ivB;EMV`gx=+4eGz}s5FOpBEWI+ZcZ8BxRq@uWv>ii=+Y$_4T;LK_t zrxnXN9}^K1_zmhM^T`|bD;ecW$Ez;~kMJt%IYh+tB=W!MRTg4CKGBJmFByEAzdND_ zkLbNuQ8d9%x}n@ZgVpQgNxiIG;uCoEjLw|@(W#7t#7xS^RmnbD1%K6V* z$yjjSEjU&)U?*ht>W1Y>*!D;Gy?+!*DLl5(bk^LV1A#8qvt}HJ`xWz#jzyq{=zn}N#(W1oEZG;V@zg7oQ0XqG*|-9|3}pjG`Kp192he>Gj>)h)yT1w zA`UV~NFSxr2I~>o*H#Bc3bBmOF?ilw+;9-DKc^1E57)PmG_$4tr*-FcG=YDgF(YaS zbL`1H)#Q{Js~n4e7xEK>4Q|-0GHxjT;wBjalQalCb+X(&?Avv%QcC}yLES9+EThL& zq@692)~p9-CRCp&q)uNe_t$hm35_Bbjdnuz60i`K2@BHqrH5u|dPjWoW+krmun$NY z$nZ|>6Y39`U6^2a$VK@q#?yd;x>=*D(jL7@Shd*qDDYJ0jMt3a4{KQb4juBM(YqKJ zfS70#Z`F_rBCtSiwC7!vB%%E>rnuKI&W%u0T=@2%?Pg}qhj}???~K(*Hj7}TWlq^j z9lGj`#Cme)ZlKQhFE%ZT`L0fOhI408Fnj=5I)OwhqiLhjE}FkM>Pd*dn*Bg+Pd(-gcjYv6 zt($9ZmabuqTWB*mDh|0OEN?rmQwoTqbHg4ZUmRdr{c7h28?iIIAszHfVHfOj@tXD= z*l-sGAi)~P!&vj0ZA{5A^jgTYTow%s3ek9W>(|siLm`9|q~h?CzIPxoZ!>?ZgkyMXj}tXk)QPUI@zaUeS#&^lL-q87cJ$@>7KRV< zee=bW?*}AA1-A-7LpAX}#4mYoUr4b_91ne(Dsq(>WcMsMe2A)zOA&Dev6;aPjzuDofVic#e#`6yt4Y?uky; zL*p+o&_&WGKg%WHVfzXfG3u*x1{_i6z4jfSXr0&w-Mp9?z5dJzAK8jTwvhr344lT{CMsr#s^zC!Googf36;n0U(W?Io9#_0Q;vhIgRzHt zGKifpCmX^=Yan){>gfX@$88oX$7Wb#;bvP!TI~~KTiT(IXzS0WEH@%D=*N6nw!|>p zvQ@UfX5k!(^M3xjol-8;gFpLArG;_1(_=>7?i=z!`}a9*GINNRNLfD^*os}K`E`Hf z6KD~ie7N)4B=8sP^tkYF3TzHu5h1uTlJ3s7!WKpa3JuMle_To^q>lwUh!4)A616aV zERFU!1*2LIH-Ze4c&9$;Hb~+E9M=_i%=vu}fD6AP8gkF}vt7i+pY07$af)4v48LimyF0SGU_UaeWi2s)0kSdX{UJMG8?4q(vDu_j5TveTk2}4 z-&348iiwre{M86@>fBu?(sEyannzdyOley$Yayz+(6SXD=`QW${OyCsO?=Wlg$VjG zS`6vmB_!7Q9-GumkiPI5P^9y5=it5_je9OOmOGu!h!jcAkbUl!xwM zXdtz~r$F(3`5it%WJiV!GN8H*{@sY(wC{c{z%oob=BMiRl}KETQP}rQY3WcUiYlEo zrHfAO_`lIdNWC%47VNW@KgNW=BxCHf!=V6G-^QgK@Du3}Xss{0o|CCDOog*+Sqlt7 zv~!gDy!W8|LXdd|LCkUJ)QqfV#3WKZ2mDU}kG9e2Gg%<9iN~?4a*DsZ$cIj2L7D4C z-->NWqWozKPc;EyjyOUzEQ!4C;S?*ugL)PDeGcvD69y-k=Uo}i@Xr+Afl60Um2;9y+*M>AlG2>H-kdl zM#?kq2!b#nG4DhE z$Z-!X5WjIEt{{BOW4sh;7Ea?f20Wi~G4qc_kwr7N>$x+>`%slLSJl>yV{zMH2c_XBU%_e>b3*#5blp4?hrnFQR+V9TkT((3UU0D7iN$R` z%?aO4z~DRb&TlPbf3?IE5ssby7HLTNC!*=e%?dje_}%LYSr=k3=XO}9FvOh@0?fOX z+5=loCTL?wUw$iPd#s^gB5==cPjiWnwZx1hbQr|M7JhFF-C;Z8lT0f-5PG#asLNnCc;DJOG5uGW~=axOWqOE$j@zJct6_jiXwFDtUIiH`M3DYvOH;0 zq{6$4Xzf^U?HFL`%)Fz*b3>%gGO3Ake||l-#Z=%K3kkoePfNzHX;2?!G}|A9RqI3u zoX=W#2J*FtV!}_-gi~#ubK34i!knjx2~5%?ES$Z{!qnZPSRv7eNNJHIMl+U_)8SAQ zKLmRjGR!(Y5J`B`iqzSAOh#zWzx<&zbu|A0Og9Djp*1SS450L^Y&FqT#Z4ggn#Vt) zo_W_v6UXvL5}`553=Z;*bNSGsSL;Hm$zv&nRdJaF-pR(|HxyKSgArV0HL|Ed-r;~OZiRV6W~Z(* z#z$>>9Nhi9!3~T5ewEXvS1Zi>kCjzE5K!*6iFX#|jy}N!Dzst1_nn9lAcSRTiSN*a z6`nYdov-0tdHnXAZfiOJfj$YG!hCLWza9#I)zLYY_oZjSLrd(N@>FNO&^3^cPHGnr zvec`JQ<9K;NyV2fTq|m>h0Az7mhSI$IE;^#Bx0is9sN@a#!aC<7{6041%A5@>A+E?-O-+G}{*9Gt(g5PzJI`0ya($D0rK>Rbv_n zJsV==!$*$*P2u*Q(k3fgzDxQtT#7$I)6x4(ki&vIEVmnVv8&5O}j*3?sONw)r*FC0PIdssuCnL*G5?NS}bFh{^&&+6B!DX;c&YFdM{sfYT% zA>P^TA`v6Si(KllIo%l{mLCUzUA_GA^OKH|vzCrt&-Gvgho2`2=rnNhfgRA9lLIkR zU8LAWg_D-K)`vrW!M&1BJP`}+r;sLD1gN}r;i2qac$m!hmS`X>fhPBEmFLPmZBV=1 zDCISzyC3_uYP?O6HO-!e6WiC)B~;P5=V=t>Y`>AD6@?Q)TFGlG381eI(VMX6w}Gc< z2)lFFs;KQ1Qp$LXXJ^@^XNaTKUpoE8J ztWNL&D{fqn;>QcV*;*vLSvpJ#tDS;hF9Tao#YO9UZmRok*RvuGqX@L>EqOHT#eHKt zD`G97EhgA;QI5w(zp-VWFoa9;B6h^(A|o8Fo)B%XVVFt=kA6p$@^tP!%pG1Qo~&L` zBep=-(P@IsP&(!Aog#PO?sxVk`5B1vh`1)|mtTLs@%Q4WcX_Q-%t@W*r;G~+hG$t{ zT`nounYE_eT9q|N-X=L~J2D;+HQt%FE|_IM6Gc%UZI_4&mf8^#ZUj5~r#pie&24rtXmOtvPN0{IgoB2G%JwUM z2O8GJbd0;@<~fH9nXECSA>MaLfdpl*y}|$4V)oVc!vFi#jXY9TsU&fPmb05`OQ@3g z^^E7u>3i>z_xV*WvWbWPRvYG)){2g7&kz=P9y0w?jy*!%SVQau37pog$d>oZl2s+QIiJ$w%p#3dGAiH3uF6WNH2L(7;le=gx?x2YYsMUM1rkDn-XvKAjvPvrhUA zw#YICK1V(E&@2hT5ok5I=>|3D%N20bFO0D?t4d|@;M;9+4dlqhO1&`ABSlw!O!unS z9c3)Y*8kTB7CYpnT<2yA`#$fWMur%jDz3uxO*TNBj1(HD5#*OLzZ=i>lM&e>79eX4;KAF00yx zcEX?ScfNb{Vllp~$Vn9jvotrNP5~e|78dD+9Xg)}PHl6<^bPn^@#E zQHJ`Y{O+duOhY!m8+Wd^`5om_U$m29TQ>eFd>2iO^yh3@GMqFNp7cAwBFSL8%IU7@ zLd|EsKb2+o!K~)g!?+k)-DBB$FX~Zl{%nzjymP_@W7jtzKzuBgMgzm-8|3&dk;yIo*OD0C%gXsa@!L3>=93``lM8s6 zV#*=@CR$W%&+4n7qp7Jzs?Qdz=@Kb8#A}Vm&vlqnr@ds?C!Gv{jG^7Xg+c}oMm*XA z7dOnPq?Bd1N`lQFiTZxFhj2dU=3qV1uf{M!**hmjsJAWZUDBS5^h#y+&Co<9As_MM1j1!&MM1;aAyGl_(oFRGXt+KPfgn}cney&VVj zfv1f6Z?5@s)aWjuqKP3R66%-<6U%ODzF4CrbHri`srG}oom9|L@(@87 zTO2v#kL^)}z6fdTBBxek&#LReLZt=IGESb z&S3Kv{Y})I_gYJEYhZ;FgU+lr;(*`y4fOBsaoR{P8L)iyV^wm$f2Qay1*j zE8+?f@F|N|ka!|v4twuXzmZ4|F$^8a`0BcidG>`qodS%Nju8*-ZbKZ3ZjDHfUi7Z~ zSkBzY3EN9c`8+@(?S&+v(%DvIX9_sjDY!=zTlyB~;%xhUdM~12|8PF)prChtKj~e| zKp)q? z|7AoLPVy2SzAU?=cf{Cf^(k}STz-zV6^;l7J!!PEF&qYeDkuH(J^uAeSEPrr1SLj_v4HVmen@>-EXSggJ3VJI2j_ZB^e;QP=SzZCtJ4ok)dMYl359oZ!I>Kg z5awRA=M4Gej{Z^KZ|(c=2?E^}9Of?*EEnAz!op^~Nv}>1j$}+ZGcK2Z(|a}|UEDyI zs%bVVqnfVpd(<-MLJsyA%pb!)&L$jTW**h_&UR;V3ozk5s7)GNkx=q(vL|VTUc4Ot z*fBloLj(iGp=r;gZpqMYND|z{?Ws#_nyXM!TtDNk%5XxBlJNxOqr}(Y?FNPNo8>8S zEBADJykQL?o=PL_2AtCfyjziUGlj#%!!{*;m06!tVKF zOAF%I7txM%5bCZTMb_S59xL4?F9x%B6JZNYWr78vdosVMM&G*()g}H-F;K#7bT-@tl~{SEcSpT!9eIzv_09;W*xR1RSxv6`;Lb7d=8k>r|r z5S=h)!TAKoztjm^2`n)cG*NJ{q zTI~D2$}~jOoYxn;Ufni?>ybk{!7=s%0-Y(hk8BxxGxXU1--TisVnYb8<>b*BIHS&= ztH|Je)x4@aoV0oe@yF{p-gK}k#PC2+)={=i29__l*+jd}O-i-)sER&W#Jtp8kw2LC z4GfoFmJ?>*TXUqo2-}7P5qC;9-aqAYlz!PUg3F)N5TIk$r@Qd)0xKS*lW*QeDQMjc zyC%~8c1pD|qRs8Q#;VierMgmy;#6Jo#cN4f@ZEWA_=14%qKA1pd2~gjQ(79a3KQax z`wD{BPtW`Dqt@=@vM9CC^5Vehwx@*Bt%qHBd>~~k4BTrNltX{Y{-ItNlsr&(jHh3lF80&0uL`GqsClvo+g( z78KJ;6V1wIY(^Vr#`-j$#j+iDf6eUg{{GKH9`d*k*ZaC&*YgSACq8=K_7p4N!0wTm z&qZ~Fz2e1G;#nEnK~pfNKQ&O=Z*R!|j0QXH`A8Ag1$FOyI>TbBPJj5n{Fy-b(@jQ*|<5;W9`9 zXYq?@B5C}s16<%EW2f9kt&Mj9RWGS=?KGf>e-sp*4OCz29Y&M0294ess{TAET;roI z9^6+%A_t~r$t~wiM_&eI<6ruqlgBAA*`cwbQ23=b8|@YYHj_)&AFjog;frc>=ib8K07y4%Vq z^W+xcB?U9L<_0!yhpsKD?axCcG9u?hZz>c@!*Gxo_2Pc4Ass==EI|+=~P5Ga6p5) z#EALQB0Gsk7Wjyy8l+-^5;PMu)!bUC_$_R$r+DDOdNfi~%OHRwFl-gBi47i0cV3EH zdGvPNkth}nfJgvY66Rc~7?$k}7b}U|=BMk41e8{W*^h*8VL+BGoPy#km++45G!!uixCQ`9@&%u1P)d8Z`iX|8-mxB0E3oUl&AQDxG4T~?9 zpb0&l6NL4Ab*-O&jbIFPKLclr#}bNB%!_kKChK*J#86n8mp7yIZ0b$ifLq8y7Y+94 zo5<-(M7Jeb(l~=k`&`nPe<#S^9kKKlrpKF>*Ibg}TLFX(Sx zIuH-(C0ktevd!V8(B(W*Psy}(+F9=kT43~y%~5&s5W8I}cKcJ?r?N%9tkKWSSP7y| z%-T#CExt`p@!|F_l zw#ei0djUVX4qO&A*Jb-++%m=TpMvVzO7bqVoe>KLJ?ltPt1p?a^CrQO(PV7lBRB2 zZ+%SkiRQb;=|vH}oh4yHd@WkqBS~VQYZBpF|ACii`CH8`@rJ)RIVFeA&;D$3Pe(17 zSQ5g|iZJWH@w4;^5#q`vDi%y&Oz+orLUOp-UGn`4OHqI5UFWbjw?1)%B$@r9ne265 z2gXt2S5VCUD=2xS*MyD}iPi0?#iFVT(dBM);(JI3%BkWc&s{N)qN^xQ8cFl64g-d} zTH_XW7S#y>iWUUc56Npcpx(bv6Z?!Gk;WDK*w)97Y|@-iyK@NiAm--y9`>S|(g~**la}^8_3J8=>f_xlC?Gs-u2(5e zqa&HxI%aTAv8PgyOr@(3P(&?LayIjIFYX-pb3u_J;n3{f-klLYiEUhpZVAMM+X{bT z9NqQqA!nbm%@%Z`;H{EOA1%V2#C4jbOd(rrApLUPiaoSPCP2z|wt0+f{Gc8EQ=3Y! zCuI9sh#|Wn?Gj7zW?vnAQpqk1iC__ac5IJ|e~cgx7ohz0hN?g#!I!Dw;^J>1owV~> zlz|b(U8(gc{ar!Ar14z}XbTd|$NXhtdIHQ0MFj(`{kF$$8wRwB8lMxi-6dNsmPY0I z)V3BzWxv&+de*PjfcOjnBle&`^8?_aai)^OCVI!Pd*>)B{w+~qJIx2#0}K6@%b|JD zEPE#LUBcYU=|@FR({h$3w0eYKe^U>-F(Mmav&Mq?R5j~lZaWzABq{T6tTCs%f{mU` z3o28q*O59S+qaHk7L9%<|P6^*4sVcTo~` z#&e*4*HoiTad7jAsF}?s{7ey%N)OlxWfKBU#XKFjWAl^BUOK>Fl;9^m;UE>*reFSg zimm8W{{Rl1C>}=HdbDX^*qo)=le$Wigi})n?z?wJ2xk1cY^xKS`%g?`d6O1vXNlz? z;}LzDTeX9ffvHG>{vv9t!x!EKQ2J%QS;mwfbe!Q~?fTw2_)Ey^ zwp*S+Z;z|TUGYYM@p*JRV#)2kHR?UHKnD)R;`@gc z7A*pCD($m81S2C9_VbtbXSiWV^ao-_EuJvtYt#>*DQl_~2g_LgWZ3VYTwadn?MX3Z zJ!Mvx6jd8yZV3vg(VP)pFBEK3{JorQ`1DswraS-qmO=@9>Y{cu3OFaeq%R1Hc&a+- zCoX;vl_uzJkE5&GWd|zN3yDf!sSDyz1e(i~OqSL}nMlY~P>=RGUtseQ!vKb{WKSM`d;i z@DG+QaYL)9S6Z+IAE1I*p@C9yx`2Z(Oc2rTL0zlnG4JDixINO7k3QFem>2=Y=g=*J zLSc%kV=*!BAWoHt4bq9Xw= z&?nXw(3MdHgDog>%QT)$pO>S7T_F);=*oDS{wg!CP!{+9#1@)y^GJTwaT$ACaR`F( z_ruJH_3+=Ir4Y2+q9ZOkEQBe*9l;bZPuF(d;{qj4dlKYKa*G@5RZUmDf!7S#eA4W4OPR>yyi& zgh7eNC&Rq6ly4gcN2BobW-ud|c?O7eN2)=xvE0(dJ~{_Clj-rrqvV0oBzy>yrB6HY zV#2BrkrwtOl725ZOs(lE@9UDEj_X9N|A!TAf>H0)_loFhnjxpFPiLZRYlJb|6wttE zRPdTPjVdWJr-1h-7n^Bo`DUZ=Kz5?IF!{E1Iq#%`&M?s4s%-Ksc~s8I6kFR0JV|tKx!`^KXh}kmSwT2Z(B%mLfWC z1bzXN%giA3XA3GXT-oRRlsPjQLEDjrxDii^>@kxe8|vCnJ_nfy#XZ)kreW{Tf(ve6 zaV>6J1u<5Je2Z)W<*Pvm`Lw5@oQy1Z5^0~NG-|TRB(fRr8}&-B&y0E_7yA=WM2*nC zMD%=!31v-HTr}ZnI8FbfpOpJ%_H7+DJzmqWVYh|Ogf0{uMw22HXEB&ii))d**8BD} z!85NzjI-efCD6s}ihD=p^}~4;_=2CLB96vd@9`f~g(CSkLd89X?|8sedDsi2q|vJo zpL|y69*;iLE3-aeC*g)o6`lvvbF1?>hS;<-QKPufa|;yTsGYrupi!eM6atSp^Qw)n zr3O*aqRjABG}PDV7gV-?5&=628sI&Duau*@Wbl4mB4*H5OR*#(=1n{uOC0#Uc+9hB zzYfDZ{6I=C+lIKm=*^w_d%G3aC`-OMj<@5Y?aRYX+JP9;1Fx9@+dLNf=}M|7SG1fT zwO*ucAcqR+zyM{O0)Y5}wU+!GS)cL6DkfKr*xbN3g2yH-{gp;$Oqk z>nt%oO~_i65wqt9rIr0}*acyi|1)EZInENR0xy4S9>%`31P%j)c%9Ve0_)D>!03<} zRW4LUHHYt(T^jag2=}Sh^X4KR$B-<#q^oHc3mYJhdCfMk?z5(&u5GbH*}yE@L}AEU%-zR{H7PV}m&K;yz@d+l4V zbAr;jGo6N1=&r5gQ(i!seTMaCX3n(V7B?zTk<3IIgdgVF#qNKw+x5t?Mj!{1x70R~ zQLc_*{OEg$*t6zbHkxv)@x6?=GnD$he8ZYWN>1p50l(HK5i>D4@on;qoZII?-=ovH zjm`Y}wudN+Vde}$dB(Mlu*EG>K{aKGo)M(@;jyjQp&$2&8deAO!Z+ayI}T;z``2-T zO}#r(sdH_qWxiZmK9QizEKbo5ASUnEx9v$p5@F_yI`dq>2s*c{cr^3)JSCs_(N339 z=q~RD6EjR4JowBzqLo2%veTGX|E(P+jLG#9-$+x3m?$(Aj}l08ZJJN$)Way?tHKXwfRZ_9_C5;K zXM{g+5fTvrLGf)=4lI~xr)uC-Na42hfN;H=J-iip?t>mAIa{`8?`&=k$DBVs#t>h# zDioOK`dQY^6WmAJ2caWl*dtEF--RvU3FrvUoU2OAhE-`rsM!31oQip)irMqcY+ zJGQHU0b?s`><{R%gqP@j)$uDP76pOS(&Lp*B4$CO-o~u7=ux`eJ#0IuZ4m=t<^;tN zm5!I#@ZlD#XLK-MZ)61b*%Cu>d~-Xj>VsurQx=_(arxJxI?S@tA|cb2E<2FCrZ>ux z@fBmz%7N+;!Y6lZQ$Vw_a5M-tSC~=JFJ8I8@Hl2coeT_BxTU{aSwM$nvcJHw;5n;= zLu!;rJ~uAYAAA(h%1woix(eliq)uaNp5(=fDd)pNEj1+Y`fLmS`$oYi>gKUUXyfd; z(K>c6OIXLI*Hn(sw%mOxJ)HgFaRVt-SZvHac{;+Cdw_?3D<=I#?P~4c9^o1XFAQie z<)sJpN~1`k>3bD>`+oHN^jCb_L*J-761(pixxP7{GC(CoP@n40(34B1&$9nNmu3d& zx>oAAB!h&tG|IU2v^4tn_jUO630p_vM+3jeg1gI}Owf3@IXNEweL6HIhi{Q~MO{;G z56D%!THtSc5O0rq&Ol)pl}nR>Mr(93RLHW4y(5OGoVagW99M7pvrqk)f|UDpIy|%=x3-Ue@|1@X${3dO=*4u-c&$uRk%6Il^0`$7Y0B z#&7CBo?_^5#1D5ip|%!V3gv#3k`(-9e0*EO;mnp>_rXf`8nLSW>S2GsDR{_19V~BG zf}!}qrVYOrNF}ORWa%*Qb^k)0$97jVni}hpNGiUKM)uU>RF@}D0EB5VqZfH*5@3q+ z7h=CqT8sbjxkeF*=5{u1fnDk;R{V!)RHhfRtpHW9@15$+u9EfQ3tRohSgWWJE+vuZHz ziN4E$Fr|40F|*n&GM9XEL(ufStxRhm2n`*1+D_TPUyymfFVyCgWFx8{Mwg)Xe#o^n zj^qLyT`WcaRS$ZcebA>hRpaM%btT*9BeHbZMk^^goz-siO>;A(MRu!^ya6C{r9%IA z`>4&kLd%sA2|X+Fbw#2K|41C-1^{81a~DK`vL0`Qm#sFkLisfc+mwpDr3I5tNhAfY z8S&XswUiup1=fMY-F-Gl@~4SjLi>5xg)E^tr-{(>aNQ$c$p`69(w%F_)`yRvZJKI$ zo+t6-$(-Zl=kY5bP;iSgGA6f2cze-7e5@p*tH2BD z_d`G)(I_fu@JM4)Ot~MyS;W&nCRp;75q)~S`^ECE)XY7bf;s}8uSTG@Nv_SbplSPh z$2mzgg|9&O)kI@pI$Ve%&_vT_+I4kpx+hNN8jbodojzB(Ws)Oh^6@upK_07i*@ZnC z>#|SywDpwV@w;pY45U!&Nq9&@+ICZ3JO>1I*4k)1)QH3HHw;&*9PH-IdEt}0ZCC#! z{_z)(QKPF3nRV03$@o>yK4?dOZHLDX%0L;TOSWhNZwa;X!%mdZS(% zcfiDLR+3QDs>49P)I@>%uHFYBRj|VV#ZQ|VwOKdZ{exeYAk57K%Gv7Vkax-J3Ma(& zEe9qHk8r_iCvwVKl1c&_m2E~!y6f@xSbRL(1moO79JB+;=JpMtL3>R{LTg@%=}(oJ z=EPWjNy@<=G+T`t6*P!~ve`IFz(yq@%?X zITyfI^?ynkDwPWgl?oMT`RSG`@+|?;m(RhkB$f`p-E(b?0k#(rm~EsrR6RLI1Sk3Y zSFlpDFK!@KyiPS2@FTfEqs@r(94h+Ky*$hM<+?0bD{=SwlCmqLwe0X^uw~x%B&R9P zP_`bp1UmztjmExqL(?-2(P85&kF$;AZ8-lA&;URT;|r8pzZZ3vJuaYpyi{EW1&K_9 z4`XqZ+tm#kc< zVCLO(V0n#ZsT!xRn=ldJ~vf=$c_G4)skUx-ltKS}85)gO1q!%%>VyO})NJ zDGQU-)J@ma<6e{xGF0rArHEug zx)K@cUXO0yMyJb|cDL!es(XG#GTSs-dW~(ur{1bPk9S|1#>b|QQ;}TSMFe{k({~8~ zljGl+D7ZfKgN-vX^LW@V305lT$v)Ce?w;p7iGEaVgwFQ@}f> z^&*@lw*r_G__pa2Cy`*j(sW}@Ps7IQBSyTMm#q4oq>K7T!OD;YSsc0Ki*fLie&E`3}& zCs#P{eX)4&h$Q0Vklq;m=J{Tze)D1!hKz2EP!X>ye=}_$TrD9cj8y{nbm?e=SS6LY`Zk(KbAx$kj#DRR^4*gnB;zN!E7pX77llL z=UD74{vlT)RK?@upchx^O`gH>k)6=_)n2>X2l-f{=tkMX6R$QgP3>d4fMVN$Zrt^k3l2$kEjG|^OJZDE~V7u4(o&_-km5G~4GC2Z3 z@aI~ScgmmW?qk1s!`?>B%KOm>llt@kI@2Z?i>ebwW$->ySZ%Vu*+*r)o78$UPQFir z+V#&hs{}Pu#g&vQrw4=Ed7wEZ%Lk_0(L`ZvE=nPqsm&zGU>OMz*v;tLy2lgBoO11_ zkgwLMkI511RsOyr3I;Pi4}9m9x%3`|IbpNY%iJ_W-ML%20rGW!?xyAw|D4x`BL7+V zWGuJmr$wddPMPcsb`$-rgn^s5rH>_5wLcQxxLY*r?T6<;luY=g-gjxzjtwDYNy(e( zWmg`OjU{Ur{Qi;6*-dmm7IVgwd~C%nje>tBM*Wx1KJ5X81UOhYr9}#PAYcgWN$l}2 zr<+6!<*Bq&vrChQf$3FA*hi=*BLFac!?k2^jY*X zccK+#$Q-l}MjY;>?j$pIqz_g{M$h&uB#PBSlLw^Ho0~#KmPP%xmd@eXEMkChtE!)6gsLF@DVf=Dk)jDtO=;nZV0S?HU z8*jL>r4LdvC%bKZk)l#(W9=1o<>>Ygh{+rM<@lhUnSiItWPtWmJBrbgdvL<7SDTtkD^!Y%oU1$L_>;I z*g;NLi|tS)ohpIbQ^2~U=`*^-Zj$gPG=N__oP6Lf7pK64X86Fo9d| zcKjThazAUiNV?SaohnsimU5NIH6KY@jYN4!TINug_~)fE;+uiOhWKSS3y4yfiqA9f zs$Mq+$-7O@i0wY&BwejKGvSvBxpu50gnCBTzPS?v`= zG4uCifqzlT$e8e}&NJHb#-(qg-L=C{e1d1cwvp2jpa54^}OWv|wcf zI%aoI)d*_Y{BAovApC}cOJYA(5Xy4S>|}n+H-AqgL9YCrTa;!a!NPWSXIbLgq&{1M zaR1ndYm4zt%(1Dk>076Fw%! zF+TqKtn#IyxluWF%rq>Gmm!dL9{z^2fuxVfG+~cT+i0Qc$4T6a-9y8^3jE>ZdY<6s zsjL$$OJm%BYX(uppM<z>|a8qEi9VO)C)Tn$ITm0gCdsP|Ro9(`R8KjoO6x1_3Xsy=^7+oK&y1q%gEWrw=(m&K3h`C42-x4cikF788_cC83H`! z7ttwAp^*bJuO6pa*LY=}H)G55N=BVvLBlU1+GA5$j)Bl`naCxJx?6giIn zsKN-X>&DDub6oEr=*jAmIGuBM*n~C{+I=xM26RTsCz3sStZz69i-G!d+m9Ock4G5) zcz-rT1+!k-fbAk@GP!WTrBVY4=oIb^e1?NcDl^MczwQ&%q1T(gAP{3rg>~+ntJLTC zD&`7`te7u+h;Vp%&j2L6gozpw8SeR^V8%;tkN*iYmgJNf>Cnx2%P@VWPLdbZ-NQcJ z@iqKZ#Y3+N##lZHD-&+cJ8h_dqVdu??Diq8cvEn}QqT_i-IdnvFWN5OKQegR@|g)U zuM>x6HI3kC-P~D}aXl2k^w~3{Mf}~`OQWpgP?i(mDXsTO0SIB{!YA0yro`DfEl3

*1 z-hFH~>VKyz55OlwxRGIYs?QSTN{|%_Yd-w&#>Qc z^?+aIJ~mM16P;OW@EIyjq3)+=pQr6{MG(M$&Wk^%DaPAmzSPnm8}VV*(uT6X49!e= zoSf3#G|)tpEVTCxr8+Z!(021FIbqe?4s}I%>LZgIr$;jG+*%oNi(VGsJZRuqH}EtR zhu+qOD5}tZKS$}p2R6|FBBiVySMPBf31ul{a4O2S;GC%NDc2lcl-ZhGwoG2Oz^1$zri9hPo%ETzvOAlIAp2bd1E4ZYm?0S&d zk~21~ons-*ZWCcJ&+fi?vE23aqs=BRZL5g&X<7+y7v?G;t;K887{`$N;|3+n0V6(Gjr;m=(oOr zB?Nv+{!q4LisGrYiNvU714?>SXq1WrR%L*?q)&%*0^&JSi=Pm zBn5vKbhw@5^=7fl&-ltb%4*T*vJNHi1qe#X6-bs#%Svl~Y0>a(GD#(cTCnpgj=Y2A z$R7C7JBks&oml~Z0etuz;cd9y1}uBu6q81Oe>x3b>kGuwPXgZ)>!lst#%lqIT1x^g4xBuNt1-%UrB*&>k;>Y;fpww73b7*IAm*NcO2PD_|Fbv}@||te4Mh=Z zYuQdmyA8ZK_i7le8h9op;!2iMm=V42S|Q6syD~(gu1@rLK2WU<`^&|+r%{-Hzjw!M z`G90^vD{`eBT6%p1X7k45;1$q-a5exF4`!O(p~fnjJr1b8Rdf`rkV4i=0Be{czqMd zBNG~fK!y93n`g8+WtFwGLYecvXJ{W=$$f?(J$+;5C`e(C@G90{8(JK98=lS)ON&^8 zPYLVq0@Ub%JAf=FnC_+j(_T#5*jFcw%!;IS>%?cJQAhg@7pQqM!2&slFaC~Y%$t73 zIC)jvK-n!uid2<#Xo?ZPREVa2Ye`A;zk**uVYvy`6pUNB(1HB}){8~y5FioZS9#~u zPZ@PBZp>RnlEz|H|FD{K#F0Vt``y}s^u=@p%I>)paZ0wRm&DjT@K_P1#cT7zc=&9% z8K=~A5`HHP+kEcNwTah}rVkSk)P4RdLb6(|RJxxF0Aff3ddugFa2*-5v;SC97+!4| z2D=5*jXu?SW9NZGP^Nw3^$GeTRHay-#o;2LZ0NMw+#a4j9*|U2&=iWrrm3rTGWPeV zo0Hxz;7uuVsyBco)n0cX2Qb?hQ3g>ISSc@a2pFS`&We8PP`##+y3kTB~gY{mY~2>O40 zhw#SZ9k{Zkd!Te&V)uLA)H{JoK2{;EheaHz2VKd&NA*=wv0!MpbDjGXSG_)R3W^zn zNzkxk^FO)Pdm#7PraF)0YmwWC!g6ihcK47=s!vFwEO;yRT2fv)q9~V$kN5PAsXT=@ zZCJ^QWLlLKPS1=ysQb9xBTBY_j@!R(D4VeT!+Vs+Oie@N?JIHJ?z+D|9*K=d3wU; zpffFZenK5MsLxvWji?AB9pthbDaP5KcV^R(3IzMl`RTcW$Mfwt$$=LuGbDrKH)8sI zu5*@lz7Y|Eo+R~!%C_xVnp3fjuZ+rM-{nD@Umdr1C$Y_33Sd-h00@3N?s_U?&fC7F zb-n4UBft7!>EI!Q?rp-0gGC@}+e@rhT(EiIA|)Nx+dYuBOd5(NsQAw3Z+8W9>Uu<9 z4@XQ!&?%xa0X_tpa5>0*Sh_brf*T)Gt3RxQob+4fIB7Fy%w18y)PABLkjEzg| zbz745_x+4b?QW<8=viBm%yh$w?2-3+Z8O8bEj^O#_PDH%eelmK_To4wf zssj8(&&_TesoJXcKw^8Xxy>@fz7~Eh2uqpb->^lKWX_~Gbor7(b1Exlt{WnsIdv$3 zj;`C|-m{`A-X(v++@RnutS5(Qwa_-uIx^Lj&KH!llcU>O)g=6|^oug^ zstCgb*VhP+$9 ztWi&Z#*5aS>x$SExy*g&736-Dh1jz_k#Z9Li|YLZ(ELG1-dJSIz|&eq>{M||c<-<@ zJOp~~4QwiuZwl?!+m6&J7e?Hs=oxjiI|yeEAjL}(Kk2geDdSp?84ba_YpZkgv}8*? zcCWY|OmthbH1fXW>S@%fA;Bc;J^|Z-4;ikY*QC!$l4?Rr1Q?Gr(N891ztNVou7Q{F zPyiHY(BwOarVna*Vbk-r#z&8VIgOS?F-~+wnsik8vyJ?NdguwO2Ckb5+ESneh!+cx zT;6W;<3^Zeifkt@DeL z3}3Y)OnwB0PcExCoS%^0qsdmKFFa0eP5nRUAvrZdVn>f5kND8Q1j9om-ZG8e7OlT! z2V)i@#sKU8yqpedx?&|@@B@OzOYW9Iu*A1d>{}hcc?^etr;DCr^oLt8MC^iD`VsK& zL%N7%efJ$W`M z+99-Uel551=zh~C6*2HhH5&Se9q8~08m|~3y0^crRVJ%`8-;*yl!ni`69Az#;p$qC zKP>QQ{_o>sx{y|H@rghkDv@z*o>_3*4XluMkK$gh~ksV{#lG=liVEjxH1oppG)z z`Y4iqDkd<^3m0ZG2cxeq*JKBeJDj3x>;o(0^lqXWzeD3jE$| zl=#m@OxF?Cy-<&^P?vh^$z7Z5I7M0rS4RC+W-!KQx+MD0F-@@oY%#-AK1V}F>e)Ld z1!HI<>$N$tnXl7-fxg)Pn*7&!y52L0Ew3x=f;#8Db|WXguRO zC8^#Ge|{^gki9pVDY4PwOyWk~^6g79ZvQ+7+ir&lZQ;h}6mJN+ccYfl4BY@Y(*|gF zRuPgnr}(nQDCpxt^l1_a<7^P-A2~umX+D&g&{ncpP9m|yhSRe6wh=nl2RVIK+kV8- zGBD~h%wI#{UcZwEn)=O=CiYKK691TRJXTTg+aoaC?Xep#RAf&FEhbqOnZ7!NiY5K( z632yWbb4Ohz7;80B+)+MpL}Bd-fa}wI^(vYRwty$h#%{N;~s6`x&1w-P82fqSX0A# zT{X7Zc>FPLGjk%N7UB+NShBk22cH6zrNK|}8yeq38>yr1@=ruCvhhq$=)D-(1+6%| z&-Pdh&_}|=M1R?bk<*!*pX=ud!0$Y=Y9qHIfa=p3*o!xoA+)|p#2|L2X7#NNJTEfK z$?Nw~k@|=JMJbh1kW0s0X97JQtxpUlKg!&Vdw?6JBz2>y5Pt5)@*dP@_?Y!56`jvc zLbgDukD`7hTpD(x%Wkz1HrPaM*zWm75c+OY1to`xsGMo;eOd?a`<8pBNI3lAQNjyK zq*uUUQPF;gQ1EsKvyvIyF2;Y1E@mSs*WoBzc!}p`%{&lzpkt|5=|+~ltOc?QD#qwx z?Dq~9mcC??b`5}aSby&HNbTiUR#a)UU+T=pDmP5$Ou4VJOol^ka zg@(?knCyDbR|Svjq4aP;19I0n8KGBu1S9MkR!}>Tz+ zKymoJDWQ*l78Y;xB;#%iShA6&!EI3bn?@BOzaUaYRpFy0M6#Q7yyoPY{4+QPCBByw zPU_ZyCOuGQ3nME%owv;ky2`Fy6aslDQ3&cIO7{;#b*kV z?5ymw^rg3Hxwt-aTRd-#AVEhX8+v(*!od4o*OC1k0W~GNk@62dGJKh=G2=HUm@~vt zc}tp8p%g?>3133Aalj@^v1I*TQK>0jDr_nX6CvF?_Edo+33KL7;n)DP+dM7oSQBf= z-?U;*RT%Y!Ao~8!=#JR^i1?Txlv$b-cep6K-__$;)425b-lRb2UxRWnrEj$sD$fge zn)z=vxAAnkDRVmLXiLB_;5kb-z}3rkI<~0w`Ir~Fw9-b8f-1%c*}pm(*#G5Q%b@Bx z0V(KvzS&3_Fy8nyCbw)5^;O0ui}X-&CmMy>Gq`6sLmfg*hT_&Rk3WxsmI_bN-Pq~O z!8gY*1+2YaD+$XkGs`tQ&%h6{)vqGQ>ik|vG7Kq5S0_753)k}qOwB0nhnm-D?;vuV#xyZ?Z-@ty? z2T`=nCOnO~G8yHO)i+7r-G@we_pDr=^SM_lkp_h2NfN*GEq6Uax-F;{HuF%&cV7Q0 z=SWcB_AIp$qaf2|Q)Wx4cwcZiTO!8iYg5C{(&e_maz7_)t2?sa%^Levrl(vNJL~;x z_zU3X!p5N4D=DdzA3P~bW--dRRNQO6urz4<SV>jKZQJx?3gP-%s~QZD(UX1J~$A;>BW8WIsC z6o_oTNyLNvcE8Xe&w7u{yROlU2Xe6l3Z}l)gRy-JXjIL3 zM`PmFaY+NI7%Cz8pFd`jZg{j>IJ`C7bVIzqy|vR6AHrTR1>C{hLb)rE9C=u!tQkzN zL{c~&yYUrgO5-~(jptP7v6sw@w`BId8CqMGwR_c)#BH_7S5pz608qJf65CED}Gn zo#MMtnu^}7T)RL!X_+WA#dt2+FZtEX_hN$#XPn7gV4pS#BQ<9rUY3GE6zkR8k81ZK zo&?G}?p(lb-71P`hd&b)K0qJuUN{qv(8fkQLe+3$6se%M8GW3!W~xl)_O1)P^mE3j zH%ip~NGZ)~8s`?cAI(g^e;p?6&E)sDx^GTZoAJ*~LPTyq8bAdps-q3)fEt4&FkRH2 za`?j1%6M)4Ey_cNJ7K1o<}aGY(PEbe>%1#qIsjyU0BPXHHRl}epD1KO%;%3Uc1e~e zwGYW~7G?J#xMaO-6ET4UUC4Zm4aLN;*!GWGQ!0Lr&c4@~ME%;OnWZVX<_T2w($kcp z17(pXh(vW@&AIPi?`t>T2NlwBi8nXCGnRB=rWsEF&mW#ij;EahXX^s}lX2RTd z8NS`Hrw<;daf_m!dC{2>$>1*bP|8tJ#7ZkV-jRp`uJ~!C2thjn1=0cHBX>#2k79ST zH>Hk8V%g5o9UR1&MC!LP{3*g4K=iC^#2fp!J}t9{H{Vv5_x+Hu7|srAv$4`X{7c3@ zcyLi&lV4`MK&BvQVA3PLE;~$WDmL34CA8z%{bIE%qU0#wCxL&h)UAQ0S0ezTpRUWeUl#KLg)I>nKs2i5zc_*cv z(SCE9WZi6ONlY$G|c2jnintjpKUO z|C!nvFrG3a$fWVw92j1FZ9T7~6-p5ufQn4+3Yp&CcF>u8CK1v*sx&vw)P1+IlO19wV z*_0dLv2I?M2sNM6jcrO3$B@dZm9*V~4XhOi7oxmjC;TCzzji#8iGBaxP2d&MV2FNYQ?a0^nD`NYk2G3QbgBML)lB5+ApAQZbst^e7 zIj`GC#NzxR_@3Kw<)D;Hx>jrlTN}OIAj~*&x+U2HS}8F8hWy}pTiYuoV`pTd*Yy^O z1bS6o3`4d^G|)i8YS5E~ z5caTxq9%ZXitMg~H3?xAG%PAAwgJJSqf?;R>b&-y1j1%P1EN)nEm&-6-}ZZfV#oG# zX1??N!xoa9K+bcX`@XO1`l;9fT}x5qi(1*AGr@%ixsjHFAUb5Wqkj=rI#XfqwH@= zhGF)snq>p})yoQM=E~3`R!UA=EtaHD{|aPiG?v)&NVX)_1tDVvrk}Vz<;hyxpDiRT z8c^sEK39jZ0)Z^}|OZ%?%kg&t**)YSK)lXy*zV9 zz-KRv1|8|TYw$~$>uL6G{iFOGm0q)4OSoyPz=wnr=ASb4_P0vy^=@-J-6zJ_#0$&$ z9+WIsUuvOeNstJzpCT*|v*Q_s^8QzA*R#Yy*w-)i z?Os_o_*~)#!)Tt#haB|zqTMyIneenxffts`<51wB6?s{M$fzO5aDjZ_msJGD(WRk) zpR2#npTe`bQ+^()Yh$<+unHA$wyw>Ud0sC!v*_hG#3CbUznZ=^Beg}#Vpk}#7UE9D zg`VtgGeJq?S)=B^sk2(PlpHbOvajvamj-mu7r$GGjy$LI~oE z6dB`mE&)R>&d>XxIYM*|KwxW3KlHEsJoxY{vpnHD;gV6@bHo;lp4XyMWp-(gSHV6) zS^$$Dq8p>V(-e+UwmMatFc|QVf_y@nip7Pc)uIG`F^)z^`=0%kJR>m}jcFbUd5Wej zXeqeYi*TLfKZ`vC;Q9l=hS?C#p@ZKZ*(_Z@J}AMl;zVqk1t8^ zOjAi!EeB6HVE0r6%jIBP=WnJ{nSGL`xUDm0hWA8Hg@S;MUVn_149q>v3fDp!_EbhS z3h;Ni4tTe6QN0k*#%`i#=B++!{S%3{#a1t%y-ASucPbb_?4c(_=Fx4`!|ytbRO~V* zsfYkV$uuIr0+W=bb=8r={#=^yoU|OvWl;LeHm0#}FkVo)%Cl`DzI%!<4tQRU!Y2If z4W-_TYi%>FME0deUP2X3;-C^=8;YRTT=1;!N|j0K)LbRYQ^z5Y()-q$O9I%o$Gh@+ z>JJ6B66oJhqH9i$1=UI3UNLn+1}#IXteEZxWK3J1J9}YT3zgUqK+2I zT5Q|UVh z8+LE+{{rx4cpxqeyoap;`#;iKNdoeWqv2Fp5*Y8U&bSMZ{@LH0mN5Gc0-~6FtG@6^ z-@?l@@d9Ud&wZ}m20(UmjLN|7xfS+zL;@as_+M-OxQ{Gad-{&9g;Vn2_TG z<>~6C?&Y!V5lKj9(ii>CHs_P}c9`y*5v*dLcQqU10{C)S3Lq0Hz$WTFn0HKIw4~Ea z8RJgTi^1un-H8TS6f3D0_7+E&T0D`BnosIU{Sb{PZsoaRX4&5M>XhSB+i(fu#J9h1((oV8}o zI<|V69WV$?Z8;lh)(byI!?V0_R$YEgz53U+*@zSb#XBH$&%bRBtc3)B=tf2$B5ySX-w z+Z%VZfdfjz@EU?pRI7rda?Ez$dxm%upnLidJR?7WT+;*lJ_FooYfGFU}yoh zqkroB9`bV$LPX})U*0117i$Tu+&0LSJC*ivW?hd5dzeSjzyA`EL}fb$ak_A{ zw#PYHYEuV|^rODof{2LPnI^T?cx<|pgDIUz>X9=l*L>_)0fimUm`yCQQv93l>PruR zeZ-x~nvMiw%nL3)n7hv924FoB+yir~^REKkYxQxd7w&DbN3DbUjVbnS4%n!>`k! z5;_rrHO_b0ea2Q4cfy9xh$bz$^Uh2uk^atex}Ph zmDU-*cFyTjw3{R8%mE?f$N=7}4bUvKVs=&OE5}?>?k58)A;% zNds6sX$ZD-CW4zCvYfMSL#r~}7?}rNA?9cxDg6a;CH4BOAH~aIB^)!nMt$tw@m#F8m{N8GK(eF#oZUerlY z#fP*Z*#ftcqIGp(pzom7j~nL$7VFru4!YAH1t>ZYxttpH4I)Dog-UO}aF0mTHISz% z>IC$#2j@F6QwRYG3(7%M#$$qPlpD8(cvwyRbyB>{1JzuCk8_wUjDd_2QMQvQVogTN^jPks6Exww zu;9~;gdpR{e0^cIP&AJ!4iYsu9naBUO(k6x#S&X&+6L@3lU#PDCL(ptH-A%xa{CSm zf@iWiV`PE3Phr^+wR03X8PH!)A`pu~g(5szQ_>;NpR8965T1FlA=Rm!Ku&4O5r7Ph!XPYna#CW>Ia95s; zzjmaej!1*e@_x59$E2UDb6*_%4;vE9!@J|i{9xtY(gd^d^R+BS8NVE04_C_#>fMG) zY)GN<{Z zwoP8%25H7oSN`Q^#2Q=J;{Z^KvIhMh(>evc&;$ah_k|pFK6ynn;kl0mX>LuM#NT*i zNZZstdvfP$6Ik7A|DT6yR(SctmgHkK1iH=lMh<7x^X2pMX{VIp>t|aJ-?TVdU22D-nIR8t|h|Yt0w=&qwcvZ zHnY!*t<)a!j-zZ)f%O*}pE%8fzdJ=11i*$ImZ*H3xufKJqi2+`d_pae>XfOH?gQ7s zkgmJ+N(MX;v&~SxAH6T-B8+J{Uw(8P^AyyLtC~JMl_^$rB8BXo)w?iwCQ5Ji+4|qQ zdY{vrKhH}5-pf4D7l~ioPE_{p4N~1OuV#mfxF)*)2+t2qhGY}b=|lu7k14Jux@8rn zUZAJn(~e%IM+_NHtYc-baM-Q&Ol&LbXH_YCX*0zuWgrqy#Uo}+FP3tsm33!e?G{?L ze-qeQ(2QX1|9+%=(udRIVvrnH$2MoDJm>-Zu&wh+lJwh1{@xkx-4&QeZG_0|sv&{t zf=&cuO8-))K)D|>H^l9v&GeX=dk07K39YH97}QbHoU_l*d){1*y9HKbfJ?0T7J({< z*GPPr=Yos!Y=xislAJ=^w;+kGgvf5U5(ohc)DcZl3LB0DB6ZqsK!n?X>{DV42=K#N;uGs-!aPd zeJ}_xdD7*^Slcg`<${+B7f1o0sNk5cPz}bDiDadX(HbmI9=cx$(qOLy_PKoP8o7!9 z0!E4p8;JZPNcovk!{8^ZQp8`O{3X1H%*j-1H>NtzV5YP3oNeJ*jkfa~a>iTm9HFtC z05T(FC>&9M3@g8tM!XWQEQC<7iAuC(E3!^`kq*CbHfRG94WmB)l7`Hiu=g}Sd!`)> znfKfM%App71nANjxCj}&_yodb!h}T7Nm_*r5w-i;F)y9@47K7?1l{teBfU)&3%Vf; zLFT1KZ8t-`*Di%swnOCK&dgx%kcKpzxZ`UZ1ZWYI-Z%B3eQP%r6vOTh))4@dCF(ChIUlyW> zVv;4$x48Rnha5QHnxWY(l*DC(zoabJdu2u=Xsv33jR9n-EAi#D9Ic%U-X5O>f9!I! zq1k&vK8AGH>MsOD>kWKGkgOw_l%-j@Os6lI_7%7{iaNEnl8X%i;F^-^=X<0Wj7ceO zZUu{^RVd8s)#?e4+AahNlAGC<7&tHdvk^mHxHL-L6{*dLCk<}Ei1`ThBK*$Hq6Vm0 z179d8S?{pp$VbdAe&nhAEV%WE9*x89*&r~tecLn5G}fT?*m=UUi`S+j=qe9hpGHAW zkwV|0or3wMQs9-wK{?I3sBHD`GvjFjW%(pd%Gr#+5?C^Cq%1uMZm68?QtF+HSiZ4A z*JjL*+?yk8lb$Tj&#-qX<&SUtZw7Mg^jVK`1l|j=$W*G};bVMJpojqGmkerpPb=F- zcS^c9E{2?LWEyTEYsNTBm6-P8Y~VenVQ9ZwQk!J zz~K;fwuk1Y8A|sUf#d*b1(&-$np?oa@Y}jLgoug>64}Bz%cCugC}OYr9zP4YlvgUe zQ_bbmq?J_%8i(&hr8i_Lamph6&5du8=+U}8Q7KY`5lUH|%hLG{ssQw_g$}361=jpi zcYGg|Y`WW3j@s*AVzLoaG)txfTL`H}oY3&2Qi$(f|42?26^x>5ibuPl!P^V!-H_<5 z)sxKnu@hnrm{E*ZU6pW1GFX@hURk_Q>vu-$o65ewpf_hxk8sk+rsqRMH0uE(_>j+3 zU^3H;^AFK8iw=2eLXQ?(5Qh>?eqxM}Ir+VF{XEG#`4Yhwrb#>Qv3Ht}qn#-+c~^ z{P9e={$Pp;Y2nr+KD~gkV{p zZE6e1PSJ8FY;DH<(ez>SJH;-r2D1f=#Hqo)WJm#!K5`yya9-!??OA8ij`~^A1{a1c z*_1eXKJmYvr@HY3FNa|g*^ameFL<=eh3?fRy_%cb))JI8Wnc8&arEsyO{XO*O{I_N3A7O}+ULc9*;Xg%*{o8ULUwzw*-oF7E&B15?R)Gaq zb&eKf0a;5zx&2`|oA0e1F@&+wv5KKp;b~M`v}}o&C3Y7nYk+` z9phG=xS{p(Q)-@|HwL(}DEh)66SuFLbr$Vba#vY`5?C~+#g2=1;y>b+>cgC-($0DQ z95_?LqIu2#XT~pLuOMbF$tAq3<+dzq_ zN3K<3Y%cUeZsy9e$;%1k+j(*;_rM1%P-`!nLz;5@`RI0=`&=7+J7 z0I}>v?gZLrPftU`VWCoy$`d+Tsi8w1Gn!*}jm*7& z=qG&1_nyXjGddlKK7wLx9Ttc-*xfZ_5ns*(bspT*xJ_ zC(zM!bbT&s)v9!QhT8A|ws!mFf(v=&aa7!cf>F+aLGrBmzHjXt_=N22hmK!ncP+6{k%NtX`6sFQFZh)#CUW!rqdtU&qU z8<|+?*+?zkGp#UHPGxM0z9Q8}TGR=~9P}0UJtVo!!(D@d!7Raidezs$6;xChxbTqZ zy2sq;3^c#RGGu+6dcu(hZKm*G8Qhqh_Nph&n;-?S9p|5S%vWJUe^WB%-iR{2ou8-H zy4Her5>*g^_;^_ZXjDo*BvO#Czc)b%MD1$@R3)m$<5H;RM?rut(&~F*)$Y$N`u0Xs zZ@*zfBR`XMn;1*GCfTK^1gJhrAM#*BvvcJ8-(jIY_!b0D4u1cmeBqZ9%N~CnG3rPx z&GERy0mVPlh`r!3xMlZ}WB1`J6`wrVq$_#ES7&1J0qi9!DcBfq@vuM zo@UqZRjnQOLdrBrylyr=tlm&oOriIcqfCT9O&fKdgR(vc>%MLMD_429%9P`_|HU~t zjk|yGgqY8i@sk$^5iUh&?abD%hVG)#q5J7yqpOfjMdlQtM=jpbr-cO{@EvgO$r8k) z6=vd@pogFZK27hOZ#Y$zHy@+RbEVsoPW;>1B=oPjtEZn{`^@C3Fh6;MX58+OV+4aI zraiY3USCg3=dXYP?unn&f|{X~uc$+u5rtUF{{l`Z@?f9CbgI)8Uae0ZwUA2-PgWRS zvUcv&O2iQCB$gUy)cfXuWG+u=iP;9s{azPKE)ipEQ8{g~Lazn%A!2a<0wHcrs8^8= zAQLCvk8vUv`ZU)(3fzkM(ZnQ+f@8dmdfu~S`~!L&GZm}`NQabqg#TEq)suehUb1HI z7~v7HL}|4GCNGrNs;3s|^*)$xU7f4D1nFSCLy2s=W~hz^c0RANAuQ$IR{0U>-8yvX zyOD}<+~z$lcspgP;T!xRkuLjCHgRb5aMX~JTH?e((!6Ql($oj68f&ch4C^lN*Q{R~ zje5_0c}=6t!}iw?N!fjab*HAyM*&~JC z-gPJxBb>Ya?#AbXaY9?Q)_{p<^?>Z__)TRKXq)$@8MCv_xf4I&vz`u@*}LmIug^e zhVf>8S!HzQ1uo6-Q@hp-3JM~F=j>k6W_U9MIyvbUVZHFQ^FR@1E8SzzK)U1s=?;c! zppVh!T{>e#D@QW!Zd6yrMjn`*lSl43ICN31>$2J?9}tSp`u?ok|D1WUF9eS&X$-LY z0Z6Yg>dv2Q6X;nHl-y{9hYTUqKhy*_zIL4g+B-$*VyUzTeJF-B*Wf3*uS17WpAKrM z-hFEZ;M$$L)&!mMpGl^b=0<0Y=ThL8yCBQb9q{lJ^7FBpJN^pi`4&6YJ37@#~b ziVlL7P2}`M2_eh3Z%GY4_E=Mf#=FWi+mv~|7tmsC=Q8EqAC^YJKQ>Xe$^fCOj1k0g zQ>hXI&0hu4XFfY1_M7R1o#=gfD|KFD@gn&Q*-&;cq3^@>#f z5cJSHM}TW}Wwp$Z6|=jEQsg+XQ`@S=_dWamv{p(MF-eAg>P78Ucou?ID{Ya>3h|@91&@Z~*N0$Mj9Ln^JII}V{ zRM1BWIY#8|G6^D-bg1Sj6yriD{2pg?|M&H?;NmZzwFRDJ`{1$S^*Ob>5aG#cb|C!8 z(^bxICo((ZwXyEbQwjsHRXSHo%ST2he@PKqYW)~}9Tax@^YzD6Fn7s@_qaP8XGx8) z+Nu97J=v5HBm(1|+arR(^TafD%1+Fs3CJdgh09eTQxw!GGVp15IRCLn3d%>UAG8VPqho2_BQ zS98b&0dlB?PkK6xPT?Nq7__Qp_)_Sd&U!17xTdL9tMZ;Y+zUK7O84T0wJehSfn(Zv z+o92OEb+|!*B;*QIM-?9w>=v0S=aoOM?j~K3H+G5mU$?pWyM%}oUMxm`BLt;>9_Pe zId~kY0!hF%zR_z|DhCe4+*!jiy_T!Tag8h7Jg)#kTWrOu@;4CuB@(Jm7h_}k{z~tf z$7AUzP7G8MgFpj_AZYbOA+gH>;r_XKAc$*jA@0giyk=b)&?>~cYq(&@j)lRdfXiKw ztpbu3n0PN)_T5g~%THniO@rb#Ux9x3$xa`TzDEMYpk8WZt`%b*N4s{#1tMij9nLAb zVik8wxTFBjG;8$5KEG%yV=|1-QC8tdu;Clv7y&SVBMVSIzvy<`*p>o;K}kHeRE@z( zlxC$YTp2OWIya-@GP9+pWbEY2pbhs}Z_EP8t5;+G|A1>BMJVN^` z9R^8O^kBla<_tO@Ncc(r70p{MiT%sh8cVCfpw=2r4SNiZekKd65;RLXd^mlpHFH6+3RCRkMR93(SN-giao{{{(d)6 zJa{1??ons{ir5|T5EbwQmCF2=EeD#N{gv}-lD7rdi|se`%$5zcu;OFSOwDir(4S)< zQa|O!#QGl_$QFtT=SCPl3U4g-ZQJSj7exAK2wXhyrR-$ENvsQGi^x8G%M_5Z2QU^g1 zh9=BMFE&i+IyLM92tyg;E=Ao)4K>%rO) zgOy1=(mx|=3Os5}+Ow_$*=__GkR4(8p_YbFVC~dU`t>O{*4p=@j!7PXuZ(jIu6 z)u2zB;&-)bzgwJ}M31rC9ufLO=U1o!lnZ!eXe8#L@5R>_?Q!_7b(1NK;5Q}Cm-e)z zF^WHdgEmG&n2Jl~VX=&l5c75|O}Zqfx`z(i#5qlC*4-yhut26&K_CYweEf{T!x5K~d{6ShEhzaB1N=g?uHTZWzmFuGpP z;0mDmhaPP9@>me7*r2DSBSTL7PlPLWT)3tD1&;pS(Mv1ZTqC-R`IeuEt#p))ln%qT z%(wvvLU?b6MoaG03ACt0YCC__Nz=Z+fE(wa7Sja7GBAs8PGqGZo3xt4BLEKl3AL!V zeqkmABa8SYB-ZEl?TT3UaL2HWzLAqUhB?tHPhxy0X~)aSoB~7h4uYR4<+ETQKeV;Ri&OPGt5bf4qX%y{sug8PnM7kwb_&9g=bIluzOB}qQT$>Z)) z8!)S>eMvoX(;ouDb97xacPO|cf8pIy%Aq$7$ei2ZVAe~vHHOB6+e)QGbz$T`5 z`nUk!C*t;@RS_{@2;dvu2qK$OFiM;;^bM^p?~AzW`!cMqsBJ4QW+>&|+L-(Diq=k= zZtRQ&>FgmJB`!zf#Jj@fm}Ccf2H~jtCz*)Xo^91~O2700fs_Sb1AWJxrH4ceA0Xuz zxm9(TqsWU9Bzrg>0^zVFhDKPxHS_u&)DkrTzjPLRXAr_&-FaQQ6e1|>c!XnR7-ZYF zAGGDNc0|bMveL19LfQ_(TuVTmZjk7-5y=h*?}@<+?cDl zQGB=Gu-nyutbzv}*%7k@!yaPUQ$0ZX?BHu<_b0P$vAJb-b6ruqJn)*MOe9i1{m4pub8?c)prtPct$ zY49yzNfFY(IuQpCwFJ+*GorfN1?L;W?pNA*x5Vv_ zElYiqwiW#Jj?U`4ZzQC{X}S0}iu?4)XD$iSJY)pgQU2%n@#Bkl6Xf3wO`+Rf!CwKN z2w~)yS5g&W_czxG-fh>WCbw+a=mY2pcC?Mz9vzlZGkXMoF3gtI_E81%RtS#>bh>KV2%rRzLO{4DKHhp zh?~1%Y7kzi)zg!g#xQ3I5VS`faq!1mtUycEnEJH&f_2N8*Zz%6vc|rGt}2ESY7Pzf zBH63#Aye{=mu+7Q6WPHxYUHb;4gw8veKN5&t=ud0MKIP+r50y!XE_$kj!;5c9qI4g zh3z=T%phEewB=`jEVkiMm{B^Yhf@0>5PMomH%mUmX)L_M`HZtpe^Uz7yPJr^@%(_s z@v{L`w#?t-@F7JGdEawx9_DnX#z-SrFUCgQFC2Y_*xL&6-e=aa%O<})ua;*$!=?y& zX!!Qs1Q(Y>I%UD_MU6S<2XdS&%kNeOva1&jnB5=bLgXG&6J+1crAZhi5(F9HRxQ9Ui5Re35~$O48mM zM^MgPM$Hl0Ye|<-P!sme)TO!YrTh+g>}iSyBj2HZGU@7LN!MBq?&=v${P|x)a`iPJ z3=3RgbHCbhQk#Z3F{B=f5gt=l*R!wleC)?s-yn%0vd)dOMR#UkS>q|HZk$8CXNnUb z*^pDz*}jRdP2nxm6MS{e+OwuOt?zdpO{5Z!l&7^D7J`VhjkhV*h>nJc$3k$g^ z{&INhEK#m>WYNft5`{N;DNDh=LXjLobx7tPy11yBP=b=H9p|>X4$*c-qXn%v27Bl_ z_WWx3*y(+y5fs?@3-c?Kk=;JOMFKeOcm6UVLu5E9$unwkgwvF^?6;0_+s zf{esM=jksIt`)zPxoyCy zFkR$t>vauW$AHPNSU0opzta|hVI_Jw|Ab1^5Y38iGji^~uuO%%9=H@I=%Dbf(VT9s zsS37jwlR_)^R+VKfzy$3n1_<8Y0|%Qe5ZH155h_DXO6Vtcl`j;VgIHswan*!7h?k7 zwN@2dnXb(5Zl<6n>_N^eOQv+Dp>3A=GuF2?^pY7f?)iP3Wd=#J*fgCOjqGU-+WiXY zMOg$(82ks_sF%z??yDzS zr72v2$%EPzMZvGmo0#ZRaJkD#z;e(A&=EGdvmA-{fu^JdIqbCgEJV+BTW3F)c97S& zGib4VkM^*yvX;=XG9=(FY<4JNg2oOI$JHHH9L24|Gq@s1OCY%m&3&Znn;lNrecixr z0aD`HaA#QTWmM6g%Q3QoOW+U@q6MLHh0%|e+H$3Ds>gb(^ysu*x(9Rhm`Kwq$*9uQ zrIQ}>%O`>&MQMzjYeUbNTo0XA$+!d7O?za%)1%Sh_Ykaj>EQJy_Cw(@Jw2lCfGNvg zNpH#^vwObdtxgS_BFK)Vq>>#XH@m!w-}dlO*HXin)3?OmFGAlChHX@*RqbS4+XHzr z?|<0^c7NBj>r+9^QbUqJwi?R7Mn4xd_>HbJ>#|bzDr{=RqwN5E0!4jL7&k{w9aeLG z*xLp_drD0vuD592rBCX%eP8_0wo_bf;)I?=RWwmGr_m})0Ay5cBS&!CPMKY0ubJp| z%_n~J8f}9x-?C;WPZzSV#Fr6>S!Qy)Hy7Lomrt=0j_8?!1=vNoZBk}-l!$V*b7+)i z*O>2o3u5M-^%y-@=)+;fHFP~hN5uI86=C=)^VqB4ElRR-v;j+P>Po9RCw-00CMYW= z#JNhw!^F=DL+Y5#Gn@i1vS#d&he-t5e8LGEG`67?l><(^Wop>7kqhzc21{XT`hwQ_ zaU^@O?;I9C)0W7R)`w#RJDzJ}gA89rI>my0oJKU*bN#93aJMMt$bF)&4uc@#ji-QM zR66(gjQMHOA%uMArCPAWd1D!Fm)-v@ehKo=%%)xkpE2e3gEZ|AR@LD0O_~ zXS8oTqV5~~r+rr_VeUmB&|kaD9rEB!$u~^D6^m1Wx+{q~3IWx>&5`N|-x0xyArUTf z5{|l$(38Dn4@7BOEG2B2I8)T`pj!@Yjug%_IFV_^EGjtZWx2 zyVi?dJk0?2HD+}B&989@z|WDx-7XJ=4kfMitwQD!aQn4Yc`*tsK&3Ch&a#yM=E+Jg zN3W~5mfnVwBCeDyrIE1gxkgSEfp=aYK9Odiscxq#yijaQBk^;-PDPSQFjn6Ko|aE| zdm{@?61rxPND`)s#!<)f+F*!2FgdyRA(Hp`m)0&#_NjM*v}3DmKcESzoTT7%UDDjR#A_qp1!UQmqg65lW5U*9BANw!Vng^;KB`nXGcI(PqV#nMSiS3a2}%}K?RILi^=Mb=Qx+$dT0l1l^qgbqA{ ztTA5*tzqZjy^$BQm-N3J+4oe+0Q6`lW06ip!a+92!$Z4}4NeNB_X3Hf2fm}AIGI=~ z)2=X_f27egJKsD#pH7Wjd2QU5LO>M%XX>>1Hu5#ypc`e|bODI^T$$hGmI7X1PR!3J zALB5Ro+^%}H;E?MC)q>oTJ^g+`4N9feuJ~>fffLp-sexb9)A}j!=UtO`Mg7taXo}6 zX3+9yYSHAD|Dn89QjuQV^sc7GuRDk#U3wtq1S>QKLnuzgc$eL+j7tV?J0__r&6>TL z5?xy>#PE-yiwVDoN>C-K{6WH#myR`fIOQWsL=Ep=JbON_Ha@b=Rg z4*l-`o*q?sBaNEQivKb!|Hyc$raN6dM|gdaj*fApxxbWF=_67s4mwGbQ;y%4IMasu zkR4YkOR5Gca>1t3@+MmH8*V}*L6C3t>k|@XLKUZ~fqA9SDD(Pe`E$)BTA$ezw9~`v zj7xC1lr?2*VE;F`iJ#$ zg6;{hTf&?2I9F4_>?I>}Kg=Cq&Ln>uC$zo zenMj1i2~suv+7J3)SJo}Gpms^b9H^3mB`sJ(W7p@_9lW_-vz&218rb0cKVGxB$RD~ zy+H_)3C;_3qs(jUA=D0^)InftgWwZvxJo|Q;v$!(8g#YA8!^uE@})74f2Vmv(;7Zr)1xXBwb%M(O9a?@Ov-c16zygb->z0A+7^82gsB&vH4957&vZ+^Qdzpv zc8k0bWFppCHa{WnvuI z(~={g*i=P7U*=GYQwB#e;9Zybqjk?`((36!qD?F5@a71aLgiYTZ<-KH&UrEC?g3)> zOlRyi+PgJp98rky7ta-cZ<1cf3m#Sa3oVK0eGkURw$M#t7|eEErWWHq(&B04L~c$E z@x%xbyYyxbU(RqdpoJ=y{3CHIh%84qLLV5~TpfcdBCiessL(Z^hg+8XE?*i9J; zu)mq0{ok&oOj4CzG>^V;?@JCJz`=f|;W8&M$g;ACa3Hq(}X9IxWoso;RQ0x8q7WDLe z?7rRrWvU#UpIN8wLKL@$&YjH5en<`K3z>Gk_||cOeX0kZ@{tP$Ol;;O?v$^N)qhZ? z12baqz$4C56L?;p3RcpbhQja_lMPkpSHSoe6dJQIqZw z8Q)Nex0h?3SuFl2<_E*jgmdcM{k2M;Hychke42JP>{5xQ5i%lOLrl1ZiNp2?G_%Y$ z-2(o@cMkmwHhD>fWq%p{9G{T5*VxPycL|artAtA~XH){Tjgs}g5`|?fXs$Zl&*Ye} zeBa)$cB@CUb@I$UXoXwUy{tr*a&1FU)fyMj0-1HaVKPGz2Uy~(iEx0^nMevDff)WdrG38b;eXSbyt^?=L+ZjTK z5zA0cDayjsOw^Uy6`2jdch)j!|A>@#YgaBe$Y|lxWgO_s3yuMIA|y1V+d14zmavfq z7m;3mjabzJ=*41|smWcO%bm)2xky*XR9Qx^5%M<;N{-V~C?jzb)oa5wt%OYTosK*u zu>wYVmYIiNO*C*`pv-2eF>7?{SpkORTo!UfxQYIVb2} z6DG2%4->!hxOvD1h@X#s&Ad|n!x$LQZG9?B4n#D%#;&buTG^Zh7FJ5>48``-NC5UN zY48}}@oMPt)sKJfY;lr!z>T4&IgSZLT{GlhhPNz?o!ZgE1yTQA@SCU33Njre)hZrK zy&a^-9Vxh7f#J#{L`dF%eq#(c5$_CbABP6rRqq!+2Y6d&R_dwzy*;uk|GxRftm}N@ z0Ne#My)}`3oW0N_X3sE5b@rz9MhER5F&ps_Rj|b!8#U4c0{b*;3Wg6+x8P>p5qu#^fP@v2{3UWcsh;$< zm3}ENRH65X8FprWwzTC!8TsC_-6ui=r;;S>A%k?F?X^i2Ai3UQ^yAMpcs2`8jpE6J ztgEr~2rQVKIwR|&hc?pBh7qMhDPCbxrd_FW+#Pj4X`ur#28r1oc5@K?hOu=m+n_B2 zHz`D|l#oOFqNIEh@0_num&FO;DV$#&f7Ihv-*M~?GNnHtlq_5`vE-a`E0uM^1SrC_ z8r7AN;xw7ljvXO&ieDSo6~fuI{PaC{yfcL?Ys9;tUN2V=}Y4&HXSgin!qE zhMX3fl$-)P;2AA9xNld%p3FxO>J^Y7Z;L5Oj*x-Qg>83L@op~PH}Cy+(Q#E=^-3Eb zl?4vkKX&@v{5+Fh0-XgY$oPU8HM4@b8F6G#k3AHV-+<8&hGoaMy)VF%uJ&!tpc75) zv02?GC6hRsTbt5rcVfOg`xZCd7`OF#XlqDqpLyQ%mwWpsaVh#y`>9KsbPZK5)x%+j zIkl!xzr4#zG@1XIwZe5`UL8`k2;*br`uvMbmqoKYs^OhksWlpcW2tx!f|Y{nvb=mA zvoDg;B%3E;Uie6e9=fxnGkG@GEP*6Q13?k_KWBWMmUahL(9Ey2u@G_zZi9GlgFC_a z{-t<+Ai-8kbVtRQIKUs4kGPDxh~<67VzqBO)ZdrGk9sb8>9K7OuZ>QYI1+DqU zf|54`w9FfdHZ1wxoTWb!-j!Mq8eW$potKJ6>78abmz3TG@Q9oh-e*ly_~pEIcqU^D z?fmS#71&+c^>ritr!Gfx8>VgawVV2tBi8$WuGznK@w?Sc@NlNMhN5tgE1PuSU}aS` zr&M=jzcbp!(u#BG{*vI*+fDHMTbv)kBMc?L>?+&O#>#Y+Wu+)rp)=@=az>+&4-oPR zg+jicx&}CJlqWhnXwE1G3bm5V$09exmzD0X!UIH>uHbN0St&qfr|FIytlWPJ@GefT zE<0LHKz>P~J5rv%zoZ0*e67({9^PLGkXZ3$m4&)efF#L2Sc2RWL~cvif215BP;zyp zML?8?>R{2){gq`pz>lInT2xFdW2cqn@2{+`0=R6ouCnrART(jsrK?8nOvF~OE6ewn zRwIwx&Q|TOtVEs-$fjiNFVZ0os396=6X49ebL zQcw(pP_oO)3IPvxx~@cbN>>Rmk^7Y$2D+g1{U@r+ut88#Sy|6$_9(s;mG1pDFt&AVZ#}stO=L>Z1n^kbuuQM@vf&mKGK3jsQw1LwB;I40{li z?2~Tw>733vo%8*D{QgSp@_F8u z*X8{@&*uRv2M3s}0Kc5aS&1vB!*mgq<%92FbArHY2^E|uoQ0^;vt3-tVQPU0%n-u} z!A5ui8cO6TgtsigyOeke0dtuCc&;J~rZPhb92^^!xl*pkCg6OIOs&P01jV1TQmaUp zXT#>`N<4cVV>=q4$dbDhSq5r}bhS1g{*D=yp~#lQE}Sq$id?Ove22;uN+tCIGZ5Eq zz+op;qQtc>s}WUxDwU7=Y4dSuGDn)4hig-*-YBlkRtUdw#94ABkq-}Zg4D`XLM7(& zqi|KW0#8@N%b9`djOjArNYvWw|NlXnrBSHVR1NU%>){z_46aOdiM0#@*#rTE!NU+z zh+HC74Gz~&tIiWN!xl_Gts*tu1?GT7r6_P%fr9E&Q|g$}a&-nST#WkRs&y19D$U9! zz(0}d6ghPcnaB^mW9dhzR77?*T#SG$Gj5D5JWha6O|3VO(p$TZ~#fo zQd2%yf;v?y@C?ct!E#Co_~iB3F5fXD2_lPtJ2`=B)dpNieZh%Jk!L7Ug%_DIcvfbP zi#=Gs3REabypyb658pwe^OY`r)8C#w9hpF;Opy)Vafd04OipfC4s#J;( zjJkZL9!EoPMOH;9BEhwcx2PYe4XHO0rq*VZAQ1!(ZxAFvK@@ZN9uh++sZw%~S{x3u zk!5&>%S~oO+C6SyAH*oxMm4Z;h>%_j0X4~WGZ&e8X zgZ%LQkd|^96)Qn~-edZM{3uGT5D;~I6EzC)&rwm{A#v(dm#YYPNyBV75CP>b*a2&q zez-zO2!BS!sdxsJ&CCIz)r!Fb^N7-6u>w+-_9DyFa;oE3rBxGf4JcJSH6LyT*~`#m zyYLYiv4L`Hy7*Joxxy!qteMqf+w4l&jz(qUs*}HGk zf>3S9A(X^}Qb)J|jhIF{caB63QlJbz$%)YBC_qimK*4XaC`};ATE%XFp2^7!U}ABt zs$wqUhp%(#f+Bi({X^4;kWFcX^INO{0;*JQ_aTL_ zbGn5===vWZazZ=wnHfR=co2Le`{TlsWd9sIL9hx5BkR>JG;vU;zXf@~LH`TQrU4;6o$3uKc)Jl) zokwV!7eV2ZbdYCtzR(IleR5WI1qv!Bwb_puB_}i*w8}i8uiX3BhxA&Y5ls0DH~#o!faq2(pY& z;p6ZxN_YqH%gJK> zije*CE22TeW>uh|64a{kS4a%LVZ;iF&Pm~|LC^@TB86hhT&Bn%R25%QKZQ0`K(&;} z0Uc&5xR0qI5~-;KK?AAF!%MB9W%^T)%B2X@yHY7$2g!ohF~gI!3W6dWIwGt=SLWl{ zLh>|f&hpKtA5Dvh8P>8HoG&D-3&Vxh9 zQJOKBatkFuA9t>10NFRon#e&j2??|45eMtXdkpY`ULvKX^eo z_*s~PhG_Aeie52Noo zE+wWzfT#iplC6)Dg4R2Oiv+Pm$82GqbZsVvpIa#fE4F&a+ z;m?pve+9lMUInSSGw9QT2Z#l@3kniyc%KbnRqt-yndzaqbfoip2NyuR?>=>fHYCKm=8k2tak0a?KjY7Q=tCp3{)z9IX%sADf{NKclQ569kJyMWWTIy&XtY9*&)$ z!q!NjTHSmA3dEMu$m#O8v4~i!aQP2liI#hZCeDBesS64Ofp%m9&BoKxY!j)78i=U~ z>>ES4mz<@+&N759F007uJgOL#5vo+)Y6h`R;7Sfr*;*oFBA7_wx;T(>iS_S%ub%Y621)dAKU8dmKnj(QY8{8g(per-D6AUJ3TTRAW zGa%_zfMs!(tr_A6*vlXCW0f%k6HE!A&Bd1&BH=26Itvh&mo#*mPC8J3Al_L@8ng^+ zqG!;^;Y!S%4tGM4*lW6=44|Ka_XcG~C{#o$cOmMKmU-Yvcpg?x#Z8z%rYf?zopgYQ$qg}r?+}0=rGH_c!EMk;fL6i%83C=UmHv(d zCrC-{G9YCs?1TWL(d-UFn{kyR zO97OWTJaLiCI}^k26~Xldplnr0ue2$X~w~>nZfl?j2N6p20KsPquBwzQW1(&z#4Mf zZ&=0eK-J`cz@ULZS65uc5T?+S9FT);WHSz8a+bo8Gu;D39>oHYrKGanVge@w6cB3| zL;bb*y5{3dKcMi`+~pYSFAUxdnlU4}c?4+JJVrRsCtBp;S1>xa_{mzix@4N=(a6M6uSepW56TFOjn+6eB zFdHIcO&mm}2AjG7Ak9fvy3_)9l1GR&EMpAH<|WXPXgs^r4;!VzMPd``zhpw$d70Hh zZyuHDm{A&FqFH^W03*}D;@lBfgvxSMYSttb$Y)|bmd=y{#*yP%$|DdiBO)R5QH-YOUL^E0|GePwwVfY3$GM7@PYj#R2sFGwFTw) z0e)6%FcJy{a!DvNQm#%VxRIvf?cfni1m0Z8i~?Mc1v^8L+^Ycn)j8Z@777fU7CUT; zH%${r9(4vJ!OkLrC7pD_j~s(nbWLY>p($()#d$uZlk>GwKVcU+MbY~VWI9`}RyEHD zq7-|L3FcuDrto)|8CUuNNHT1QHJ*<|s5KO+B>*QpMVljB1^HpWF}SG=GB90Jz_)`L z11pQk<*>P*Q1JB{Lhwlt0gPD+7H$EnJIyF+1c)h$niHOttk$Ze%cq@{f@R0HlmX=| zC-Q{nf#%Lvv5uf9_!6Qe5=ch{kvdcaNEKU;3WAYPMUG(pw8$JP2a-SwyfIEm5h1Jdn{CK}ik6$8*SV+J^-fcZA6dB!Kx9 zLgWZF?nBiIu=#Ns|crfnzYz!dkxT_^#(J;e=*;PxP}+q4x)Zb=6a0(!x+ zVF?CEGo8@#o*)3<*V8D3_=O7%IKFkwN2kT579Z~iLUNq~IQHhP*asTsNEKECpdM?Y z0ES!sFEbRl9;LJp66?=2GzR;6pXdo7`5%~S#00HdH`vsrUdP#wQ>1ip~i_Ipbd=-%Am zMc92RHb{jhvE^)@KNo9e!WOU>8GL0uIYNN}?e%0@y07DA15JmqEN~3lX9-(_1!E27 zpj9K)z=wZ?0x4`*NDh&6yTNW^xdgX>jWx5dP8wV|?UAySE~aFOvf@{Av=%cpV{JE^ z4ky=@4a6Gu8+KdwgtRt=>gkX?_>Dh22&f-P0fzj0hqM_`E0`;fholrOrr(-)f zyclkSjzQ=8aw!K9X)ZB2j{RlE5CE$II*t3BL5|Aik<3^stKjQoMKHj0O9#uXBzNFDu5iQA27*^=hNg#7-0&O zTg@b|B$jirLum66M5fNC=A%)01i?eh^$XwngWst8ZGhj&l6F9$wtuoQ76T&sKZB~* zawq8=Kz(b$9OUD)hR`a54}daoWeGg6OaujWua(My)dV97-Vg)AF%nD41H0(ui!T6D z22kM7!!EEGU{_DpB_8Yy`Yt1j82=ty!vslt#)SN_aT=}?%!7h>JNV29tvV%JaSrP? zZGH&htWku#Qh`A>LVM8J*+lv{HqAiXEj&;+Yw_lvrbi7_D&!}aCAbMvor8JNr4q{s zFyY&HaR1_APZ=&RfYQ>Y*SSk45hc+LBU`|S^7PYOiV7^4f(TH(O?>LnL3Nml&%C> z{*z7A0>r}NP>Kr_FlIs-1Y;a+wvzV^1tZ3cWF-**v>L7_;ANP%6+{phr^{SQ0jxDW zTPy%IkRMzQrYR~FE|x$tmK9^Dl`K&=xCjcMD3+ZmW{bd40JDyA1k>hKiwhzlX|I(F zjEZv=%JESsq+);>At#g?iW&_Ba}`!JI50*Yi2k_>jLzadWMJEDd9Dce5CUg`jZKMPJmIYKP3eaU8%a{&DaS!ud7XH5)+D(T~PZubqv*8+dhB+u` zB9(xD5*z^j6I+dxIstK|1d94Q&^(yV5-uhCsj=gTzzIx3;3(c{J}btY)r$edPfg|4 zn4#k735p8*7s!+mHsBYfQXm67l8enm-3W4ocFc%UIIg4=PMc569)Z`P%LpK17hv#W z0J~SXJb#crInG*RjxGZxm+QvEYG^z+9*{IztrR8);uy+-4NMu43dnymwuZ}_NoNKp zPfxj{$WolK8w#?&is@}oLQdeU)ifY<6>Fy3#q9!Su)+iw{4*11HBhT*rU2UWsUujO zCD#E1Rx4+%f~Q~ucdT!P=L0I2T4iA3t;FZ)1;L*L??E$WRlGp`)p;su4*)!FD~BV& zm2#j3PMemxaZ9Ns|QC+!DX!;&7 zBcjYhmIKu3<|E-LaxU2(jR%uqm$_moC7cP=qgD+b5OD`p;`!Orf0U(YT`{w9}JVoWMoPHO6{5PIVwfE@WB`Y;k+_F8BF-7fcw8>AmH4B)C7&f^YZ(<8mO2zn;e&e_i`kP zEES>U_L+i8#Iu_1rs)n$`V`IUz=6N85F~=-=i=N978up3w0N(E9H7J;*c2QPJCsUz zlB03#leGXsqQJ6?%m4_=h37c|c#d+NYWeTYv*CD9Iayi#`IrnN6M5jG-ithBT7JRP zNIk86bJB(W@H5y8)T}atErtaWU@ddFJ!~`p*xHJhP#9)yCuoFXI{V%p#6lTRI_wAf z%)tt7K_qfDFVh^WG=o{7NwbBCWIrW#lv+9fl$MHHh60tCf!#u2$u!#Y_+aBcJQZ7v z!R1)vOct9C3gRmVk_Z0;-n<^Gpu!6puC^?kZuj8QiZXZ!I(;&Or3bMAk*BOj#Q8XP z)COIq&QGDlA|RRAVQbb6IxyVf)CS=Fl9O4dFerd~frCWkr1#Dxho^FFOa;jF$U>c4 z@sbm&-hivrdNt;XpnwBYDL*5@+PwZZz;Z3)fCBZy?0MW^8xZ<(E-Tdr0TWqnIE%Z3 z24z#$P9KbbBGiJvX7IUvRMsZ}H`xlf^z1ajUqVE>1pWfth$6U)9HQoaoC#*M+1aK2 ztk{_ZF|Bmf843agT5=shzyTcKyOg*FrcJXJuVg0p>~*X!<}ep?v23I~2{5<9Pzx zX*N?44}o1(<_7m5(mX|Q4>>?FcAA>POaW+}ukR;^YnzvW;_x@6$T&DSm6ZkVUc@Xc zgb~v!59jTm&IcGx8m ztu@1Xl5GLxhuBy876E;AMn`o@)%nq8oiO;(=X5~b6pucz8{LA38WLvmvc4PhnHue; zyEjb~oA{F9VkQUOX=Em~guBfa*zGJqm$`QOv%`E~DF+ME8#vL?c&~Zv%L#vOyb>l^ ztX@A^ih)Z=11#e!#Q|1nb@~9_+7q1tyo`&J#{A4a4-=a;cfrO(d7}bf+YRqK17~a+ zoiMQzlI{EKN;t{oW}EGbr1p?=TRBz<+53VgRow2KS>w=K7T>b{>&;oyrM06v+0y^MvB3j!-z1Ue!qn;;zTm=xVKGr>e`7HM zL%x2nOFL8IE*IQzVvIZ=mU#NUT;@-|XjkUXZh`$toE6zaeD3O!7 z!*!g%Ap7NAh9HNU)Z4+%Zf%`*F3nB5yk~DTLPDhN4n-uNi&MLNpusFtV^0W*3N`7Q z&&K9cBX(n66`Aa?!c2|PAUxE*E6UXSLk}BT@>fZBxGzO`H^NywkF#8Ab#BPWr#-Wp z{%QWCY2>IwvxKzFjWSFRRZI=o%N9`vqavij8V60^JiX%$ch4G&_=qoIE6iha%wvSQ zsv3IyTWByQp|f7P(lM4*x^j))j2(+tEa0NnRV%pA%+M4rlrAW`FZ6n&>vr*;zZ`WR zZdy6Jis)v3JR5@Xhq+jH>6|0pCV(oMnJ!AdO`S?GCOKm0sBg<^cEIlgV!+R|=U5|7z zD;K=7oV&WNnuGfK7(WumUM!WQ?ve$>5$3~5j|dchrJJp}>tN=_N!?SHriR?R(xhXz zjnWTn8;vtPq`zhmHgpOe%bRM_9xMKXd~Y}u`{*Cf5QXZ3)*k=M?D|^t%&QPn+xAx! zJ{KN+Octe1_6g%HtFNzv4BY+7@(QyWS|T*_mh$m%jP?9i4|A>M1lYnzcrhbs*#6QO zWI?<9FSBCtm2nbDy=f|UC2?wa?1}AF*yI{Lp6v8g6VF`vbhfAE_4}E9!lNXtHTHg_ z@_#;NG%3{47{wmwPZ~f(O3yCsU`AC~R=mUO(yZ{8Musfs6UJSnQYCZ%^-lN{nO(PJ zEW+n}@IssPyOT}vIhSG=ZnY>!N-UrUaj&5CEQiw7xoaW&Tj=|qU4d2w{tNWjV~S3~ zhK5?7#m0AZuFlh4e0~7?<5+_FYEgP6DxWADD=x>)&etID1GA#sn(}rVWRwN zrLV2+_$3V#{&0EfL9fEC1!tr9T?ejjSKH=!BbGkngT=-d_Q zJ#qI?;s17iG1Z9-?!h|I#rjg+X~tKQp2AyiS42R&?b2y;_Rc6~_tkInx^U6Y24Y+u z9W9;|2N}{5dsc>&_(nIAAWONR!CV&_8<5cED;;CbmNz_DX#QU-lOObN%2=uBm>w@9 zMssy#r$)IoQNwi;w}iUEGlH4znuj>w*0EVKYerE}$ZUbpE`7Nu;c)Mg8LJOhOFPLm z*UCwq?Qqh!orr%${9aLaMXepnWJ`Pd?!!K<4tJq6k=fy7cI}BidrQjs@?V4IblHXT z4sj&Ul~+2xKA2PcroKAtowCZB9S!efdKMTg8@ZnG9*xMBk#Y&Uy`X!!&|@;9*yka& zf$c7HI4avFGvF_R!rhGx!b?s1Uz9p1bLMvUI?BrJ-oN)XdM%lH^I`}4L_536eWbYC zuI7Y5A0+nVU}(F{pDmo)XrF2>v6pww^8Z^X*%`1syJ>AunYdpV@8GhM)Lw12V^?xS zmz8NqVkiB3WMkJ$waK_n66rYKC9Xj8aMZ!{)Do%?ZMP&#sb2mrB&_yZUIiDb7K~RP z{L8O7*2F!AY7_Rf>mhr7G&O`>8>Ym}AJ>#l)RX-uh86l+K_4lr% zVyhi>ckO*PzG!CK)rfaE?buk+y)aHdU|Gg$*r#QC;jaYs-gXD+P2z>^5V4i zLA+iyH)wOWUfosE4QK9Q-by3--#%FgIV^m=MOJxF5-9^F!)&%K*f-d_Mz@C?rP5gO zv$+AQ4m^BxEs7gF@;pws|Djc&|3`k_(VX^CFOHN0cNOy%jONE4^k!-*(DuI?o!mU$ zU%F$TscbNHUbB9zBy1NI5a0^#XEG!Ty=Y>yn{0S36h+)VN;Ne~i6FT+C`@xYwD*J15qBOMBCBRS=(gm?$9mez(O|6I{Fk z@+kAa*brEOy1Q;MYV@7|x3S&!$cE*MXWqRS#UJQs7-|%qXr(uC6fM0b@qKoy`P(O$ zEpu)Jy`$~0RvjsY!k@Mjdx@Z#BTY5RV3%~xKEqQ+o$M%_a@GA+Tyy(GY75nqSru)2 zu~*N7WCismMUm<6@99b`O_Y)3`R4J@F?j3%6#K_CzrLcH; zJ1jq9*IV4!{=QRjwD4JP$bp`3zH8=e{?Z;UeA*PjfDXPpV?ahiyLO`+j=4x8vqK|% z$wP*5Hv02MZ!+ec6QkHt{|YWI7nhAuu!e)r-+Fo{3Eq)RejA;y$$+V8Rbov1*Vf<{OK5U>~DX5;qx=I4m^Gsn9l#7?>_pE9sV!(qYFYG z@zR489^xP04QnfGmm0F_euOo*_f_S0Z0Y92bMB$J9idYt=~qv}>*%cm$GcC=9KH;> zpL=JqeOLTL-gU}_=d??T_ zuF5O#Xe@41xrUe|cyFuXoSn^WAR#}N$ceMubMRve`qQLCXK$@7o!LGz$GDIE32Y!x z-?)0OgL3WEf*l82AJuGa7afaySl}M&7t-QgUBqCmmOb(J?wkKIF2v3CT2$QKqNAbb zoa>X^&dSO&`_DNg-5|>>jC4w2e8#-s*iVCohsloy&X6##vHA}@Y*aY9x z-amQaO~LHq7?V9`Y%V6+7D=qy+jmy0O_HMQG3vL*-4(x^Q-Ur&e>dg~z4#$dQBdxl zOGTf{oUSdhF8a;!)|6>K-P)>*$CWkLQP%G{0KmeXP+O zvwOaGCicS7SIz(OXRFhsrt&F@lq-q+EjGu;}Q445ywEcyNk!c{6d#(wxpx`w9mxwU(b+vhV0<0 zS;3o0*qr$dch@!iZG9r;Y@PXa^!wY+y;mg}na&279$LCXUm)lzEM%kTkbM%%E}gY$ zL~_RQN7X0=+w@IWPK0qPNu?3MZUdZ_IF6ce$!IsQ^vM-d`-vv)B*qMGRcdEzO$EV~t z`s+7?`}R~{sjYqW!{IaCrjmqFrRc7cUk~daV+iZF-^@Y72lwcH4(sWg5y0Nu{(dRv zs`YLk^Hp{AK9%5XwtLMKGu=LAV}Mz=#s2YSM*|@b8|c{=`ew>Y}HoTf@J;<8VBr*|Stu=lOdYIMULg(8U;0Z+duW*{;w@j`@P$bj?FGL9eXZ4bg2T4Wi+^vkt1BF&ZOgRF`$LcDN531Rvg}qiJmTcxv5ad_b~~4Xxnt% zm~2-fF%k{81yC-WtSkyK6v*us)(7zW`r@l%uiZSD5fH-O3MEvyiL(7Ex0e014BI&V zoG_Bsl=Ml$b;HV_TP9x<`~o9A&hLt1eXD!AVtx>`S@tWZN|G_t(rNavc=P8H&rF07 z3wP9;6qO&uAM^;9-q;0WlHcTk5rZ!YJu~TWGNw%u9y&HU zU~jqcIpS(s@Y^h}JY!w=`hW~aUmwS+`ddzB9$&YdL#r(h4oG3o@^o*9C%-un(DD^D zBDY>LvdPEnR|ijr@@-+Kt=x8Uy=H5sJo*{lXG%TIM*KS+51X6JfaN%)u&qn$al1BQ z7kn^CC?kjejMrD+4!NFXmJu2{o3t?AIbmyrdvmOxm0;X6J~DXTGrv2+hY)1*iD}mr zC-WcG-+{hqoo&M{O1rD=Za&?&r}~ElMt4P{-a&UG3mc!^6;6#lJzhL4`NG^{XV7uu zW8CQcO+Og>XT;p_wZT^X6#b%a;8a;LdVIw9IzOm9e?>%AW`a?=v7+$|S{BsiXhX{w zeunfGwO>*}C9C>Zi}e5OHy*74!Vd+h63OZ&fD5 zDQA*I^KNfk*@J_f=4ZqkmMjj8)UPhmyY4oeIU?5DXIb0X3oiUX4;nIrYN1;V9JFxl zo9bJOpTD)#*w@LqrqinBr5naALI?v_mry)!sB{g3^8-r-N-nca5J2f`M_Ew^;P zbYfu5v7?nxx6AQ~D??t49tF79W6)RHW?DB?Z0TRWM2xoI?v4vRp0PN(G|}Ljc$4<7 zm>yl;G0DiaGc-uFQNu~4-N(o3ccPn}dpDQGXSko}I9R&Ri==B|^gZ@Tc-~9aZnUpy zb+Di5`XptFU|@5erd2Pnd{IvcrMVW7P|x;le9PPfBgB38H^?ANR|rNIlHd)mXUyu` zMJI)ACA(I}+_}A{z&HEYLGTcXrqcE$j|OHr_mF)yKe=VwVt>AhYcujh(^%h|DLJHw z=YvN*gk0$pc&@ODm+Pfn;*WV1PR#GKQce;GkDl>iRNg)Q)mC-N5;P&>FSDs z9L9^TraT=tV)%$!kK359zo_5z$&mU?cyo!@{j_*zlOk?b-{EM6f&8GvT-=yt53O%0 zqFB1e28`IT#+WyX79R_m60(!(Xk|MOk9=02JkG5ytvqr@d~ojm7tkDC{|5}3KWw!9 zP$M+lQ|Q)MFI|weH~;q2*faZ#p^pXyX^CscN;l(h1(j@rt}#jO`i6Z47R1r(8$Nw?55?pxXu) zUD#?NeX$x!D1@Fqctmr2EZx7{P~zn4DpNL72JOB@U2`ihnHOWaZSSjD30CsGCac6F z?tMQSzdA{=uYQxU0R8EC{I>47b>o{+?joNp^a7K}OhsCiLyc=!f?9!NGT*?KXp2+fN2K zv{$#mOHaMDC&7Xr<~{4L{W{e4_;q=v0l@0yLet{g1HPv39B2=P76#?7#x;G+)o#I^ zl6{@;g980Mcin#I@0vQR)|BMlwDV7PEo3VyV{9z$nvH65D(FLfx5-uK|Gqn8etSV< z+3}*IBzkRwXYy>YB>$tuC=BttAK1 zPmLN3GdBH1FB@yV=1JPWax5atxYb~C&ujbL`k7FfqrWfzRKo4|LXXZ&(epzEF5L&_ zJ)9`sWh|P^wVC8lKATpooAg(CC;egTZuRYq{<(2|y#EHDf|YSrzj@)eh0Q3!_OgR5|b;kd;f&}(fO0amfBx0>(`_f`3d1~^-k4t6XJ+Tljl=@3xv-PJE{^yZ5vs- z`^f0<3T^h+p31PkH*dbK5XN5{icaEAT-^Rm|95@OSOUg2LB9xSs45l>W+cQ*4lBXY{VdR6m|sKsCPq=iq- zp6y=pKbNWBH&XIy*0uJf&B&Wy zeQ22iU&PGy$y{W0@0ch2ir?Ew3b!DBHgRL$em+p>J(2AHid;Xm12cU4P*iL!x3%$oON_P5VLi#+ufvmV#1ACu<&`gBmzEUNEV?Yf`zx!N^lfrffb zkk}RX^Df7&#oQx3M%S(7Z@~?d`Q^Jp*3Jam4g1q}RrSuzDa?X$Cm)e~Q%)J|w#NL{ zIVu&tdLPf{r!1QyAv`#0y;r{Gm%moTx5qVhM&%k0at>PG>v`=c9(h#n^wkI4c<)+9 zN~ie0;u`zk2Ai6Ecf6z)`PB$O^6V?O-MsAg9+SEjY6{*T_fGI^*Lb}+Lkj==rd$+V z>#k!Kr}oeMXk}friOSqJU%!njF&o>j`Yb30XOK!0)50^>WhNNcLZdYI%JLSc1?d|b z>--JX8Ua`H~MWw=C1$4 zS1Zhmse1!`6@))N_x=yP_tu}sZshG;#wq7o8$R7!T3KOW~6RtzViO@@gdcam( z1MsNs{(%nxBZvHk#dOl+iKphdkKe=KNjZaiuk%G;mtEM_omeR}OVeL?1iRIr;e*E` z`$CXsePVssN>b*%;P0D$x0c8zUi#h0G)$Io^Ui;2T)QgtZS}iz?-xWBrv96`-!a~I zfdh8hQ|A{cT7PZ#(%SNv(8vs1VR1lHZEK6n+s)_(nPcy_bptu*rb=G2%z-3r(APX& zd}L~hN^>8ju898U?~$^|x^0l**pHuCX1^(rh^=1nWSVdJU%ei+@xIZQ{!}Ml(N?Gn z)*UV7&A2kZ&VH$SRGME43C!lgjqf17cg@Zyqw)=KpKu_NQ8{@2ZjOO}G|I1ZMf2pB zo>b?jKY$pg?_KH)4#UJtlS)5R|k8?>gtN1pQ26#2zhTSxG~=X?z}J>F)}Qy zd2`7#S%liWX0^BfCBIR7@$iaQ_l=f^pUe7b;q`+jmVOO5sB2FgFiwFDkBmOfX*+ID z3^yD~gnZs?s7-j|UWe^zhe((iw0lE^H?JCVhaOIh$6fe)e**vSFaMH}vQUT`I&c z@H@fO&7jV_*&TNVJ_AD^r<_uu#e&Ov4t2wS6jA;+G0CpA+dy(U1Ts!{U^BqIAn==0wsvZy))CH?2ue zjUx7CEIN5qFvmFPd0z78{v+v~hMkJB@!OEC#e~LSEvq_FW9ofOZt3=8RoG^MNOWXS z11A&}bH5Lo^JyWL=I-paG1oCtwEvF?BRAK$?<#_7WAlQKIC58tL(jY!zWF|Jw4NOz zed$>YpNcI_`Dq}=NG6|i>vmj6@q@@Zh# zsNN}2(#S_cZbG$(EPMr9yuL+FhByrfp~N|wq>ewc{~_D;UzwNo->cerNKmlVcbVMa zf{z80*36rDDE-htr}W0Y@`N_z`u7QKf&ARWUB9e#de)ZnVaerDYb-p0*zNFF+2XF7 z^3I%9w?Dl<9Qf;1SI>Kg*PX6*?2GnkzyvEwMdpbh)RLz8)_Bp z^LUPdcMyLDB`=i=V-xD8{=~3^9~(H}`iF!3ooe4Ae^>dlzVpW;#;%hexY7;8{@aqnZ2RpD>0Xl&;7LPY!yUV`(G+!f zVQuB%-NbIq)choKO8yF)U(g3VTkD7cDs)|NKrdELAPn#QMqRHW+^^67TRpRKEV1ya z#~X^d(*=DZKXnvb!Fbp%@?NNQov!aKJbFeK_D@IDu<^emSw-2Rg&uR4*nj_F(H2=9 z7`o72hlTJnuI}Hmw=h9Rl6&%{B(o(NFEL~u;jm*Rvjb&TDAlJu{3nujWxZ{ED7|f#Ez?A@qoeB2GhjRI z#2x91hM8|ZZ~x;7`9?*t(c1>6uI?~d(n2AcbUzk3+& zeBZAVn;DM$H=Vh&R{aq)L(?-d0+|`iO(3XloVFf*K6>=y#km7(*6beEKV7U}m$^2{ z=-|Df~TxlscPjN(^&b$&mq-PO&CX>v=Xy1edAktDJ`p_J{OhnbR0 zL!XuJv<$KwUG>bPt#iebsMKNk2!EdM~aNI$i| zD)^bKN1W?e_$pKzT&~;}_HEgWmPK~ZqmFrR;PU7%YYt{sJ%H@On}TXv>jYV6eQJk` zW-`JRUr~PTUJ=Ln>G-qqr=CX^Q%m9<;{+Pw?RSlCr&rjO zt(}Ezs_;3!Q5#GlzezRN8x2*SK%3p2%bc!K9XiP|``f!B+DmpLy6122#@;ybU6ALE zD^P@7y7Ua&a7k_xVbCnRwn`_ddAX5qIssimgv`Vd&;a(p4mjbcfpyd#4*_G~=i<>qrH9dwl_)YIfFIzN+7lXefY;qw!pskxcF`Ihrt+pU9dilc*a{}Kyf^4mxEd^z(s zgb0@zldyFaef6ZUkv1=uQM6yNPdn6J!Mc6&$FkYG-kH5nCr#9Un(TP9_j2jYJ+r!R zCp!P>4!waV;PF5EOYPkXXU-peg3e4LLL+~+F!ue?bf5}}zxGf4wuSu}nhN>BpvQT0 zw&u_IoximRT#gGnYoJ$9fsw&J-l09f#gju+s0y>YX1T3%7Y#eLC$!yjT6k6x5_|hE zl5hQEh(pfd5ddXdU)DR`sxuq}(P8udAZ*7O8)FLkdfDfQ&+i?@~LXFn+I zIplM5tMBf6pSL@SKC@;qRsS0yGw-~+OL_-3a}2#yz9cG`)2bXWGX1H?Y)#~sB>qPO zLz(qFFXQ8lZl}MF;QR7OYaWv^~huOdU z?NI%~;r8P_0c+T!2C6Hq@q)zOq~oFG&V!4#b}wbO9}Lm5nl!-?lVeq`m<(d+*!>4a zW8G$~eoz|woV#3R1}*BeS`QwQMbLC~vCTVI?WP^`eVU(Ty+xm&M9@5A%BLgXZM#^WZTdUwB&GXB_ z;s7)2*ga*l|8SdIw;9$CADocUa!H;-gcIz3G#SI!q%g^YvzSGY60T zkuKY15>GEo*tHd6#CH19+SYEDLb_qUt>Tmw_xyfbatAU{57sY;j(kShRYs!M?ALEN zB>MH$57vWcw(Y94`x83Sj=VO9*2%Ms(}%yhygoitV|3luXLZVq=!>0VeXjPwwNoThtjDA3rO^(<)FJgmLh@Z$fBA z3P~KYdEo+JR>CK&4I`#Mi|H#h6)(16U_N7sU3tE+TI((M{~Pfs`16U;5A$IXRhanH z(+z5GwTkwvvh)?pw|OKS;q&dbYIon;MR6PSB1cAwlu^_dk`)qTU!s0(xyhczer8n_;ZHP<}s1e+FW(r=U|*@|2{+2tNRNb3V>hr5}Qz=+(C23o_RnJ3k%_ZaBj6&9dyB+C?Od5K%f5l7IO__XQ zMbBiuS+y;-)A!fK3KKUQQ|r1+s$ZBgqCajhsgtC|9ar_6in3D$DFa~SlA!d`caM29 zeRXIfTy=Nvwy;hMn_X`kdzZ~qWL8XBt$z}^eZz2E7o2rxneJ!!pGHZFMeLXB@b?{W zxUW6upZAA8$vvF=NlV>J(eyP%XSvRQ-zmb{*EjUmnDk}AZq`t{-@T(EzXOj0C}&z$ ziXiTd43h+w@BQoNlX^I7X4YIL-S&FMMPigs`DfSsb&Hte27Zt~z9eD!eh+E&9w70qo`!RGE5=zXIxUo)B7 zwSw-@US=aca99>2_++uQ*8%NgI?u($fK|CvBbBjRoN|0V8HE=B)7du(t$I<&(A z?QaXJ(CxdM$!_b@ZMn{PYZtx%EBizDYs8{V>ZWapS`X}0#y}~y+cLL;n0$JqloZzI z-tMJyDDl{ZzNuHc(aIeR!b2xiC0>b8b$8)U|2wQ$Yd-wR4cj*(@@ZVFw5lvjwElcu zz#${gjCSV}i2uqLIBLS_N^G0WoW&ij?^2P5{U5GuojPvO-m(1e%#LHyC3~ySpe~j^ z@AZ;HHBee#IZfkO7WCZM(8g_lIc5a;PWb*UOltWcn>em!DtE z)3UCGQuF#|t@p)GF450hV??Ija26HL)O@``{(a6cnXjoFo%*hDU>JH?oxUKmPt~u` zOZ-&87V!Gw<{7;gF_r(v(3$u%{l{_qyNk^Z$CzuFV<@J%7gKYSYnUS>i4bLuN=3$; zIdW5!5v5CL6{VVc3pHmdY9y&tBb9z#KmWq_@%VnfpZDked_7;nglHPtDMJfC8{p%M z@~BcvPreiXJgO*FnrsxEfGAt+V#?k>`er>>DQL0x2`tn>Y3imn5WX8r(TOz2@&-ftO2TedF8y)ctKL)|-wtyO%O zx9XbMn%eg~{HCR`^P+D{()*%pq1GPg#Y?ALsX;6tmmQd1{vKTnRyy;|J-;GydnM4$ zmnV$(5TAr-W5l_j>uh!i7mCDa{lvAnxtZQ#d+d%cm>j&q@NRiqqz{Pm zskhG}`jA{1Sp>>xJSj6pmCzResqQo(YAxmQS#`(1iQi$7EL@0Tb>M0gwz9BoC=UOx zg1*%#Osg!?<9cy#(UVc7y6CXD(;G)Gdw+Oo8NExWIN+6~Z*NykQA9Y+$Rnn#jaFu> z48S6ApS9NcdDsY96b-+Xx7dCS;}^H<3>Ri1A%T*K&Zu(eS|tXz$3l(Y&e}Om=RUZ0E>SOp8c$=@@lv4_0uor~CiV}6|4miK-DdTn77f7SomFGD|U&L7yrq%?)*gc|iHUpp-Np{ohHhSh7wc`LUkfqKahcQLS;Lo#(JO^J&68TV7p{ z(0k3Yk<~Ys9wg$Ow^#*uZa(9JMh|RTn}42HNCrTQwF>litsU9X6hHRc(?ox^Is{D< z$>%@TLK*YmBhs}$HVR^uBlN?&Y{FjCJdm>I@}-Oxf)|YJ@9TKq>&l}6p86HCv8&<% zx${n`4^5N_q)sh?D^odpG{I^ms}H2Oae3fKId2(;Hjp1I$A!HPbHv+!I);;?MbEcl zxOX!K_PfhwcqSf-uISr(RR+?6%G6$VP-_wi1yQu zx+xiKT#9AeL8ek|TZRjTu+5g5Lru7^qnc{c_eclez=pC2SV_{Ym*H=TFBmMy`Kq(Y zBhFlR7JK?2iOk7}I)HNk;jiI;K=D{HJTBLDp@ zF#+X`oJKSD#;7WoB}tjvn!e5_lu}1eW>(!iJCUHfK*NhZY(}l(BP$!W{vL!;WzUzwFf4}B#Pn}` z>0ZJJayN4;*pZ@r%(XiJOunhB*He-iQB9r6!V%26Y>`&HK_pUnS@;s`VvuRqVWZak zg`Ti-f<6?)ZJ7W$6pD2mJDb}+obFi`Zv8-6ku1#j1o=!&ca8|BSCm_u#@vcS%02M3 zxc|b!!CARnC*l(^010BywMgJNmL$f89Q1LAs8~q{{c@R1#1w3rKLwSt``Qu>g6;uI z^c_~2x|F~@3(#Wsa3JcbIU8p{y?Q~4^b=E_!=mT}za6m=tPQJntd(U3s@V>!TvC+> z`t{C;ct`Idt9B&1lQN7IF}C*_kXZ$+q%hNdqTCG5`W{2exz5dS zhNq;wh`v0OE#`r0F#itOBP&mujb)}Y?q)-zy+-B}u|IlPGAMvdXT-nilyjCk@6@cv zSnBi)KSY$s?7K%1iL#|a`>Tz;3X6`g&TQYjZ}xns-EF>U8)nH!Qy(`j-48&J#}b$! zCb>9V99Bh1-Y+#uBFRj*%&E*RuLh3YK#TF|t91q@vXArDn=I~ePUJW9{hY2v z1c^P7kaT(T>Y*&b_P+CQU3a14cd8Hx#64RkZm+W9C@kDyxpVAw3c^s&ikxJLEpU1tgj%5zO_b7vD zjg@-qZ-|5i!c$Ai%to*dKog8i3=ZxCNQJi_@Z+md(a-f@JmDOatN+%I=uCVE9$p5C z=F;z!U;%rVH&u!)$N5WiBqpn&Pp@)Wd>uC(tV;#t$s(pE%M2%v@?9HHzLFis?^EPX z>a1P5JC*uXU3(*so$c234Qd)$$oi^!g*@4S)Bi2D7yK}6oK2<09mOkQMVvm%UEYiD zl#>icy@Ba)NlA<(uS795wvB_{`KDL=WqDuBEsYS^`t!#tt+U_zuvW{RwigSGO-8cO zuf5w`w-lBXsAEpj6y8#3uye0@_lX;w^pGThj!^-kQA5y0D68`b#L!*=_p1}*LK!8{ zC+6!f;$`qR&kc9*n716}X3ym-Db9vhB?Z1&j&6D6U(aMY z56bhFQurq5JiPT6+OOH{U&*%B2F$r6|CCF57^fnU9gh}AdJJEMYI))PZshZg+M5?V zC;S)a6ymRB+ z$+F?o++u~y{0}IROyL*X-4XzKXLpIV)K2NL5u^0QU(mw}47zKn3W4Mb42D#^JYd#; z7-yC~fcbqt#@3bTibZSugGv(fY(W_AWO*(o;re-eDVM|S-7Ptw`QP!%z~!Ntd?BFV z$KC9t*#N;E<<`EB%jM#1LMjCjBVNZYxLkQZ@3^IUIacyVH(x9GDmS|QCT@#nXHOvG zHKKeUt2+(5AJ+0oVHb zEzx6*OqFWB(4os3(h+MZsXPV6+|>vGE}gX~(Z{DY&D=0LPW&0&eunk=jydi#}P zu9Q-A>Vkx`RQJV&17*;@}MRlo)3h=W?3RWP1fHRKNgXzB6!29(jVkXec-A zZ?+g7Vo$%#vhO<0v_RMZbngya$IT$*an?b^NZ;l0g(icz*V=lK-kNJL*C)|Q2n>Y* zVjv<}_kEsso8mE&>?BG)u+iI#Pt6c4M(EEXen1 z;^kfa0&itNDm;W&n7InE9h?1cw=78<}liM7iG>e?ZbKgMILkks+4G?%}BpMB}hDyR5P8eQt*QQ{L(0V|CJ0Y$irN2=N6wy z-GbKjI&T4#k$SUFcB~CLLX>;2{hvloG0P5#;wHSq+kDpC@?LJy#z9FPt_}Di=C{-z(H{)`Jy;Qh{lgkVg=Q(qWAeCk+q3%DF#ykNu_4>#Y(E4} z*fCElD3(aE_#t6CD$9_bj1*>lmD3zolS$D3c|e`0>+0?B6qq0GNG3G%K+xi6OYfrX z97?+$f!eCHA4o9Y9a8j?Oy^H9AXx~Gdw2YZD)295v^tYWtvW$~hT z(Bt?pDCSML>Apst<#v@VY{JiQ)wD}V5H#94u*)`KR{QC$RnN>0Dkuz=Wgz;V?3y~W zC-aa|cA9O$h>k?SMMd;+q`lExK976vP;bcSm4|1Q^uPb6Ua%^FLQLPltL9OIyWOJO zn@H&b+fTRBx%@Y#BaH3;-Mo+H(2&ySX~sfM9`EP`#mL_mmY%2A_+1ZrHm!QFU<8DP{Gx zEsdDqPVOOs09LU&h|BE6J^vVRLvhM)e2ZFoyW}iV3o<>p7GMyLPmUlX22R13fPHlDHBoH%1hmXgGp6A>MFsTv9p$0M4Cv z`Pk}}b~zH{exVCF7Y1`)xC8hL5{sLravD-)w-cv%Ra2igWnfSdjjGMnk!4?p!up%LsjKe$ zQjf3u#@$$IKueh#hTqAr&pKx*uPqL@?F|C!Iz6*JaUC1G<=dg;ooTM=2(YJ1t~6#` z)8{sF3U!vD39MYp38)bEJ|%{S`dIk2IS2y0WeWfDz;axxTJN$rIi=jcF1ng0ypKjUDb%+nyw=cnnecA~eaho{=zPxnd4J*NvyeLV>!poAyQ16N|XfKYz1wTSIV zyRymU%dYA0;4*tmU@nd^{R94kmz#UGxABl5JN~j*@zRGTpN?K;h|hoUZoTS=2YGol z0}^D;aU`ZyC&jxl-6u#^+N+b4v0d_fLae*1mZXCQ3NnqLV(_iq2KFK+mn=2#x~hIU z-n?}Cev08?=AHfqE!6Uw$r<%sSn)3Guh2%eK_C$Jp`2FdhKNn}&m&^gM4UQW z!{{yWV#4ohMf47_$vH*YW)oM$iG@o9G#O!e@H4kYX6U)O80{MXoEE#_ADUhOl|^|& z%{;7?gIpYxS)phQFSV1M(+rq|Yw~KYkTKX9jtwx?Sw2!=oVY+8_D0Q5`+hBfvDER}|iaQMfUSv(fNrfxVDhkp-pu7#rupQEC zhZ=bwb_ciwJX9%^8y`6y(KuHSbz`k||N5<`T^DO;e5 zbAE|5A09xLlV-`b#hT}JZhq~bpO_!s0k8@(6YoBca|cko+Afxrjkm{LuFYDe`Drz> z<7e;c1`}<24H4}1zm`;l^J-5*zAKv2Q+D0-6NtsuHE#}ZPLIKqqHbG;5&%6Es;^JIniPK zTxLHu+3g+df~qdCXYUQm)sj36Wwh=A^z?Z&BZx!IP;}1rDY24V%z(t7EfimNdR)td zVSey{+4KTIh=x;3;^O3lxkXVQCsUr)(?;~;FXKFrd7Z- zMmL&yZEN3iXhN-ACGFHT1ZJ98He7R2A?&V+wJbnOUI1$m_@R>~No4Zky$qFHRxA5+ z!=dUN&mcMzK#M*sW>i>F!c3%XULH8_X9gSD^6VB7;DZ2we(e z5|UL;HmlAwr$i^a2K+iLNy8+ieCK#NMe8W*zojby-a#qNmvu^uFARgyO~^|CjO6mm zvoa?7r+}#J-jTcX=iDf8jQ`OLgz?+i_Is!2-d2x9fry@eFE2&tfVD_ke`$&3w-q>D zFwf8e`cE6&0|AE7A!!H&2josr3U_yk3{MuoO!%$k0_Ey5{Mjq2KWa~K_Iw=}W&$!l zrGs>S1|w*cpB~q&lzfVdgBLjX(A66$90kxjh`aiSHWj60-RlWcia-j>?&AGN-cy_( zDU3{st(}Ag>+5tWRGNV%<81a=;~!36kRy9S)fgIQIMGo5!ESvt?aq!DeakYZB*0fXuTr+EnP5D5cqIX zkEEbB1U0R&vsSutihcE<)q9Rj3hKxR%S9VMWPm{}?iZ>|zBdMevs+iARD4f;eSk3= z;k=OVz~Kli;K^ty>&Fgyny=cg7D(buSZKDM1Bt%+oSDB0NkaeGBp zO#e6-sRJ~qkE%771}2vt4|s4&_l+ISKMPBt=tyS4wK;`}5T1^8kEI;gcA_X*#qtaY z8?*1GqMrQ5(2XtoduozjzIh8{INtAYOX*m*0Tzu!oJ|eOkPd;oyFd3`x@5nl_FK-^ zOvu*iP4|)GFnujHA~fuWxdAKd?4$&BtPw`4)HyPJ$XnqKfL@&~Vwa(HyUAtaG#h+<#Q;1SXi?Kh^m%cejhlFno7(MiI~S6Nxh{aV zDKmVKhr60)Bi0Y-4oH@-1Il8JD4aM)aXu4y3$WUE?6(o*Jppo34qaZ$@C4eetD186 zdyfILN2l~s1#cX0KbjXxY-4$R<%fs1c(@`tVh@vTt{`4So}_yl2XZD>+diglOVP_h z#*U}kaJ4tuVHmR4e+CMJ&!Ot2_a)n$vB@taS+&^tD+G+bL$saG(RX0l9y4g!MNHB< z^!EP97N_A0_4+rwJFMmc@+A+Phw-2QK?Crjgfu04mAO1vCC-^k7u)q|Y~qH{l`KT4 zWYGazWw1rjXgn=W)J&X>!Ri6HMRx&o+XuQy1OC?hPQ^~km zbU10lz6Zy6TSkqQAW-FxW8`z#Ee^BCmR=xz2w#*v_`ag!m7=y9`!0rjq|Z~H9s30Z z4+83=_RKXXM!2T>UrT_0-EPr+Jqz+kpQ_E8>>_L1yH&RMM}9_*AAOgV9I@u)lobkQ zTU@!--JE8Ac`J`2A2Y_Yi&|j8i%#-gPVqc86BB#?csIZ+C2gmku+#hFS+&7=cx5Iv zV0XVPAb=V2jTV*d>QO1Rqp+XA_n!^$2$Sn#sCuW@Wo3w zD_F@rWq+MxG{o%0KzqhEgSDLZtY3W8X@4gCmCAX!HUy9qK+}pT0`yB}m;iYh1Q; ze5*99X4wnkf6|Au>}T`VxiSn$d)hcB8R~KO6J_?STU)hA=q*Ta?0 zPfP_h2=qP!!VS3pL)NU#!GR86uevmQ*1@&;5|H~6pUpr0^A#!wx3hEMx%^a(nPh;| z$hiW|DBH>kMUVA6ebD;Fwi7<)k0-J&Mj2*SV1}SFl7vW}91Hcw*8dygq`bix(aQl; z=m1t>K}eej_2e{#lSm8KS}BH9?i%oyKh4~DE<)dq2Dy`14iQ<2X~!ETHnx!Tdb)B& zydM%T(g&Yc+%Az6nvZRrWp>R;{{2Plw}L383+|IcwuHQfJLIl%BKcP3gSp-IwMLN} zJkRLKj^-sdrkq!F4{!lfU6J6)L#5uym;+y5ci7=5AUsT{)^S!Ng`JY;?X^;9mSCv) zOaa0u z%0@*@p6bfc{EUn?MIHEIR+8Ic%Tp%Dv|S$_b7gj2f0;RVZFK{Kv;sIE6HSC#AWU^2 z?u>!*0`sXvE19)^;K_LJ?(b%!f7q+1s=`0+e>zYKlA^dO|Rzi-jWa zp3t}t7F*)FKH^W;&n(tqP@*=-@NN0ZOsdjH)}|IMRog9f3m%EOsaE=A>R2@ta5fBS zTJTckiZ5uv>h+t=T37pXA&)SzwgtsZQP@}CHFZ<+!wW@A{pB}P!V*vZJOmuTzW(!F z?V=Op4Hja>XiU820^Zq@)76p{fP7a(CQHSg>YbV27Gn-3zEE?3NmU0l`o228DEh52 zl5*8qg#vI0jw}8qm;9@t*O|)>XGFW|xql|#TAxX=fZ?p<2AX3G#EJ%_?^kX8D9Wb+ z)V2fG3Kq!I)~{_|wlL?!rSeDBCYb(lq=B2tw$40YKtCh;hp!p()KT-AkF)q^&? z^A)#%Wy(>`8HHfnDqoyc;%GRYr9cW<=nZ{1*0>+Waov2+Kwr@i!Z*6ZD!7$C=CCCJ zvKjU;Vv+l63eprYH}U9&T~s^~O2f;Um)aX8`ifuf7~Rv$!n`i5c2l*c~Xnr6YB=k53f^DBqLM%rj^={EGu9bS-%2_NhoT8eS+h;L>wF4f~XtJ<4Bn z7Jc#xs0j!Keg8yOtx3yR7N$c8Zi_LYJ#DD%-dLg#qLzie+w)Si+&)%z(zB`C78h=E z4{xSJ(5GDA+vA_D8Weac{fIhH?U+x;9|5;+?cr#Et65XFH{%h%* zyX+W-S8mCM{Jd8pNlL!Bh$BT(l{P&qTb55NUOSn=FSIHYe&qv9xz~CWIIl8c0R%pb zZ~=u)U>*0pL2dik+yxJ%{D$yu(OObz_UQw}rw~@0`WsJGq3^oCZmOLlp4`MjML9*j z$ikMz7udJ`k~$@@OP_hZb{1QwJx!PJIM$guY$BsWj!;b&z|)6l#50&@I=P(~8sH!c zWc0O7sUsek<=O^X7elmQZ+OoapV3cBXnoRcuAHzeyyCG}rS{%{I&YouFk)29>xXsu z-OBDoC7=MUR7poKNu3?qUI1wMCzMb*vs>EBF0C<0q7^P)w&$9Omn2-3%?Y;a{BbGi z);iP=L4a1-wfC;`B?*sDoJg1oe;sybx1mF;)GLW{!-Za*Uw@aTgJ{RunFm{WB@DM} ze046yJw56Dh|7F+{cm~|%bUi}t4ro> z+c;)p=82Fgy;7Q}e9rZq-GP}gAB*#k#Q<8X-8{+xYc~y3**RLo)+NCenzDPv-`D$z zAL9jkOSddJnO{#Ou02W%De*g1-awM2b-K3>gO@2Q% z^TW^Ldb7z$W#DXnPBw02-^W=>%;MM}Iy7Ybgv$i1fm#zcGAlJy1; z)FO;rhzfbs@?~Cx>7-e=h|q2cxNA6<86#kQARf7Fa2wUk*zTh~l=hGvD*v+maj4aH zSEAJeh*{Uk|ZFC>l-#LL*6>a2$x_)e1a~*duSv~cp#7UHAit3l-fr1STyM} zc=zclmXsc0I0>y>UgWey`S}Qlz~br;obKVRv+7Do1s(b{Az*L(z+^e7UeEy&b(5oHIDnSK{&$Fn&GDGlUacBSRHC;nAt+<$F$i~?fF$ahr z$buf-O{1aSIXnqU)K589wH+quwEM2s8(s~H-1X+9 z;YFYM5Au`>nOkVH6mlfi{HUaojy51mdAtoU+l#cJat)FSgy5moyl#^(py-C%J!B_$ z1+TpT)UAcYwj{|_SZs?!dd$}3{<&LUh$fKE*~RgKQO`6dTQB& zIBy#)6fl{!VGfkk8oyIPlePs~Fpz|E?{A>31vm_%z1-eA-l!7fH`+Y#n8IJsn|Wk@ z?49s{Yyu+ejO!ooj)RsUMUICOAgNUt-&TZ6|2^2E?uD1Ry0tp4Wp;vJmj3%2%s`%V7I24>d(cWGePi?~%)8YoDk51a0O7dE(! z>;m9>2b6N9fx5`OJ~F0RI+)kTjX%Yo_{~ESZjUygF?Z8Eo`7=WWQ5H>Vr3&l$ao6t38wL0uU;S3nbq z0K3SPF|W?9+0~~+nn+p0Rj@HK16=}Xg}mnqJMgGqX5NtZw?oWrHElg?Z**UrVA|mQ z585L^6VeG7J*)R@RrlIm^eH>t(B1;y0-$%}&RU@Cz#a!iiLFmTkq94^F7YVDC{dk^ z!)z7JSV(bO4Bn}8v#z=kt->saa!DeZTby>5DQMB=w3wA*7Pva3!lmY^ICjSzsj9j4 zp!GcwrWu~;Gu^XKAHxzOvnpXNG^CNMavf4{5IY7ZMTMN6u5i6klcjDJ?O%$_>Lp8Z z%np+?&Qen5f$>6Uj4TO2sXQ<=5rqj;@`&5jfSwz7dR)2J84*aW+d<$T+}{A?7)V5b z00)D6PGbsW>TYPFOP;uODxI`NzT!G$*}12fJY#E|;x*d4m-B_Xda5a!7US&}8fW;{ ze}=bG+T)10qbE7FiWsQ7B|zdeJp>^bMdVraycRdrd<#GefPH%#hLj+j#t}O8CdApY zDsf>iEL&-4Upp(X70CL8E(ajF{&AdC{LEqj7*@*Fytr!*jbRBz4>J@2OD>&K6;I7s zC$ehfXvFKDy`#JDx90dj#JGX9XkJP}DmJk=*8a_OiE^VFz@?e*&R^rP>#U%bMEp~I zv{$VcBk^6CS-1U}dOkwN&Dl#hu2!%UJL^z$u!!y|p&xnO9tK2Q?1TS8sUHfU%16VRp+XEWVw=@l>i*6tbxz(Qq%WBnT0 zL=R#Afur7BJ6p11PM6BeW+mng=*kHkCgwp7-X7W5^ilgSP$CI(UPd~ML%o=Yar#HI zXi{#dUGAlkg)r%DNj2eQ)~d{VYt)UHRkfx|Pkj-jJ9EcrhA5Bzf{1GV*U`ueJ5K7=`s$ghZ3^R%wjOC4cL3<_(~QW>_3CCp@Z^(n!6N7zyDv+KG`adT)i?BBP(cgX zk3%FaqCq{c|xa_B<2OrrI|X_%LFhNyvL#?5IVo9LP-8-nL3$wu-qOsZ9 zrtx1*UsR>tf}7!b6IXQB!X%!N`aQs>@)A$U1yROMyUc~3xITUsH@+U3eej_Cc1zOb z9|<97N}D_@wV=`mF%6c3t%p+A#=&?B-mWS@ZgPt^^lID{RZLP_j{Q&1_k_7Le?0lj zf&0{>0C$%N1nA1;1w!5%bUd z5R~{&FK0ZW#~k$4K5*e(R?)w^JRkkrJp{s(OijIZNWIh7*-jO(4>2EOwUV6n^lp0! z&#{Oi2A8uS`;W_HvK6oD{2Yb*B?QtE$?;-Oz3|PN9kgwRi?7uAxrCp};K1-_y@BVL zc)SMj6XHlQj1aA?Sd`$p_nSkPf6h@mW6T{PL-bcic#=_7G6FxsOD~kfecGR zWlr7;!K1!#)QeTXP;o)8#wehmWW`0Yb{_zX9s2$;~Aa8`Zq9*RHnv-!ot^J_#P^_mm*s?M>& zBT&BHf36A3Vi84atnmdA%2}piTe**a!q$Gv@GQFxJ5QL}9jQVTT-i(@CG+2qODlu= z#5u;f9{bJ;I;~~6h~5j9pSi*3EprtwmT9o7j5a_+dVL`p3VTgL!oBI+e?Qx*ruCti zKR(fT`>sY1eF@mgg^p)^+3qs=;H7^^qvKL*GCcVnqVS3Yh8wxwuTrN*21~nilYFj; zT{7Xv=sj(tJu2XJHgunUtO*{o<|e5>M=I;H#F}YBEHT4EUWHdv7YHx)aySGZuA9P& zkbK!-hu6QJ6#yyajmg}B)mfE(7$oNQEhny71ZZnVv-w|ti%ff=VyT-72ei-+;M;{CJS4a^vaO&EU$`on~ZLXnf3w75@KUi%N1=^ zarJljr?!UzkGr<;t=d23>PDhBAM?ftukJ=Rx?pKdbC>BcW<|Al;}!h%d5@ zEUo7n9?O?%C@Z*w`5YnmM>k{&+)C{)w?PVS5 z-k^<>tX6Fo12;s}@fH6^JG_p&#s&R0)4*5sjFVFHKv_@dR>C^_ii$vLexG7fbTO_p zw&!wNhzYyMpe%`O8j9Z28v{IZxqmfs6kZKmxw7_Aiv*Jeu&05VLu;X7TIH2ZJChpk5|7VrI|GwYNnJiObn2xYM9ya3m74+b;qu?>yFD#-&R3|`z%)J$%=EaxEn6}b26 z4lF?Mb#KUt??}ZB!)u$=cv9N)!t)lqU6+_YNAK{|!@q;=?n1U1(&T~r6ZXFk<^8iQ zJjIO;XUAVSuUs3|xWEjlrm8i^c!k9nXC^4G-#3K@QW|gcE$W!}uGoUxO@pHIn}dDE z-)Gb)o_?|+;jZuQoU2$6Eoll;X%3QrOr$MI`jg#tbUG!97CG-|LN^x9qv$ro18JAf z2PO2EzTfngBz^6FR|U8vT5YWKq#YF}<94RU-N3Ri6oQ=<@v6=V*O^sOi5W&(_nwQF z>FllNW|*(C-p&K$W2AuFVv$gSYxZ|l^??`$dF-`e`&rKwq&cJdVkQT|{b&mUWGu)) zW;}9a{;`B!+&e>$IWtJ>Co2%o{fQgQQT=4D`>!VuQUJi(f)bX!uuZ$v8V9Vv2(kc$ zOU~?l=lT=k?JL?+pe;>#Hnbd&3wOuUL|g14Bca1$Pei7aNI~r1vsfWYLLW`E2|QE3`he|}Zq1`l z|Md?u!)~zQtjPswc!CR#Vbt)3rfwgtL4Uk6HE1kLPwLB852ACE*amZ+(U?7^HE?6` zVt8j;JCb6!#zJ*I=>BlcNOcD|Wt@#A_33^p0LTpbFt(b5FeeOU|?|{ z%API$?&-g(ooZJQ!|b2kMGOT9yY`}1WaSO3wdIIimpZTKP@DAv(7?%HX&Ud}W14*( zm)AE1yT00##ZS%z2Oh`KOa<14MhB*cp9?b&oTn_a*h|j7M0?_%_ay<3V1(y&r#!9~ zEfAT2)J(hzvkE&N8nW>CVydtaf?55;foM?zcO>;+=vftT_6KNf7>wtSZ?GrtkS#G? zL*5b2jF7c;|NY_us7yha<-FG6X2lkLASAiIUU0*~07X7Db^Kb8yAS{FK)PZtmG>N=x z-pG^sybtio1W_B5v3u%AzQv7}X*=(nl~ZE-3Tg&?^O*~u!C;f$F_^nWoB?(IGBEQl zKskt}c8rdGq483UHlX#=A&-r;iW8mDIr_%I4S#<$g=KI(&vNKeYzpZ7iq*)a?F56f zZUsm)ooA)6mxVhZWWk}U+313a%)ZOj@(LGl#uaqZYh|wHxJESvA3{`zefuPR>k2M+ zHko~ay zt7~sUoNLCyo;@uCNn~B50swe@PZkj*9m!ZLYJHR-ou}@+A-^A6Pq0VIswhLf2-h@Z zG03YuJO!yN7;L!1zsdNkhAHUclKCdscjs7Zo^#Lvav+p-@cDQTaOQN0OP@{Yh6@|> zMP^;{%od%YTM@j9t=^(2gyuc_>YB+?44|5M=n>&JgiVV| zSTs%|U6g!ETtyujEVcUGR(r{62KCb!Xer9Hwtc#%(U}@p3X$1#3b@(k>A-l>zM?P; zDaFPZBn6{24ss|JohCmie2f7=%LcR1+AL>#yw`}c+u?fh9sZ6+iMFpq6i{yXn)HT= z01e4_%abFf_2b@b9#eWX03KKyg-rp`@CTVkGcZOdErkGj{oA)1+j;whq_R0>TA~Hv zg|2|rzY49PtTS8@9(?hb&QBw|(oVIx)7)Sva zS)a4Vo=Zz4W9kF<{<(C^00Y*%^7ZM$9g`olkY*Y1ZmPAdDP`rwAk4rr!|aGD(;}9t z+9C^t;3R$o19|JHlB3b{q7kT-jgB#T$)?@()7r}X+1*+Z(boW@h>rdX z(?-c!ECZK*WUoBa!SBx-Ux!;Yn|J`LZ#QHp6P^&n(A!a+c8`y(0g|W9lqCLaj9@_% zjoIfsRMO`QnRK6gXOQ(SWOdyd1Jn^Q{_9|VM!dvKg-TW)+eOX^+6`N0ghk*4dweE| zR-pY-q*kHa>#M+`oga)PEO9du6XcE?1<_Us+B3K>r<(&A!o$q3A@CtrU#x5{pyn}8 z@cHuFuQlS8^)I$3z+8Q?Kxz5TcHqt(an&p3$@iqgfP>3HM+*Q{ed}G9`2_X;^&Eqa zLYg)I>9y-GgbAIQFDY!mBbv=653WSpICqRh@gwKCpn12(}Al5Ve00btgdo z$&kIfobw^TtnzD9s~Bz9qhZ8|kXQShS}FnUxHJ#alMLhXz17!YkBPBB&vOJ#*g-nc z*~PP=ZAmqOvG}e2W=g*AS!hOj!xsAlmw8@)OH*1>Gp5T+cyu)e-9Nu;DHC2s`taketzD;Q(*6DB2y&RK8K!ibo=>^Clgv4*i$}qh+?+nb<4W4 z26Vby&n;V#zE<)=TllUm!vrn;!ydb}<;~%rg>xZ%;{F?hU$E1MSoRK4UkRK0zLp%Z zmMSw*@Zrtc{|ue^Bh!Bz$3HvFW;0`MqZtdi=3c1A9CP3INFoxMqoT-|V?yquNJEE8 zzKEi8ZgbRJB~hbGr5fp0efJN1etLgi@AvEZdcGb{KpcfWFwH%~(Qj&hEnS9&;LnNW z(&*&Yv#}XuPV{;CN_X`Vn}*yUK+*Bu(ov{6SiOHt(i;Q;x&#c!{_s&!S(dM6Y4*>9 z0A&c*)~+Yzdv~-`By!^4N<4VRl=DjrwpM?X=cy!cKC`*?G9m+SeB!UBe_lDWa(=J?zNQBDM?lUj^nbe8Z6q|LuM*-zmW}=>LSan zfF4sWz64puwH$C+*t-1z>rp$CVN@d!TtHHOG*e|r7w&Y+*T;Y3xxTetPsDyL0Kug) zZhz=1B@47%3XLlP?t~@dv4nohdeUR_>5^{U6QRW55$D#BJz)~4!N#HZcpgV1+@yyx z#rZ2>JcYYyL#-&$MJ^t$HS^D*7|MlZnuvWZ!$gp4`B&QcuHp9D`=^alZM0CzkK+&a z4>Th~ho(#D`7KXu5rB$asP*9`zifr*R4%<8Y81C zsi3B)hl#Q(G5lSF7q4F*W6T^#X}&9t`P+4>mnQhYb&;nG+U!to)4n_j<0V(wBUTM$ z+BG@x*iV)WP~YhaC%^Ejdp@CU6X$y`T$V(Cc&L&uAMQ#ihnUr?bBCUkr$$GMaVQ@@ znYjI&oOlvJkJDFXJk3!r8fvO7dqU7JLH`AlG+n7?!wJQGt30JsEG7UFvG>?XHyNxqWA@{;h5E14{eIfl z9C9VtVw|T{-Nvv-ptYn+pGGy03|1TNn(FNEK5mEk2rJ@|oYat^FE7fsX&!SDGDzm% ziTBkj$BA_ZC@#dkv0=L`W|`N9W;c8Ib^x|J#ObmZcO9)drY~O6$bK#NqC*8g(<*<^ zH1X1xx_C&wci;7nTR6#);n$$^F*kC|t$~-yhK2<#Sh;TtUAp=&G>eZ`(%XHGA*k_u)2GPCCn2nzcc{R4F;ezP{5$5A28@ z6Agzrxp9!}&+A;RCkISbS=zwGkR_)W$xSMhT6a<+fuz0qhzCG`IWv%_ z|H=pVw#9w$ME5#oDdmCid&9*#JeqfplPj*(_S4oXPZZmDd>?nQi!Rf78df>;f*3!Ww3qfQuKUWK)twiqF!5DwpyXe6 z7MSGOaRXGO;!lt^7i93bP5m?b-Z43@FMio1LS@*jLUGCfacB6HjMe1c>0#+(cepHf zN&d`|m6O=7>;>^0%bSn2i`Blz9)oM0Lk=XS7bxmZ?%OA7d>_E~*MKQ5wr^Dv5u_@O zf(=+z{qZGY8%J>a2${}Y(ay?ZA z;2n})+k&;p+#5D(#No&}Yy}tNe|J)3VZ|1~3aGU)*Vy#)CtQtEq#ff?QTpW=9}yR_ z2s3iMTzKS1SVhX4jMv7v?Z-;P()V9chu~%YaK@_05$*pnGyR8bO8tI`r zbJ8fFqg%ybp^|%ngNJW$AE%rD+ z;-QH2bp+QRN94(romVaAO&^;_ikykIyA)WQgC}8QekphQFwnzY7W6BCw{A5cFC<+C5|Nf= z{mb`C+Csl{Z6)TPQy57`r!Dv6ca3Y0U3)q)wI-3H9!x}qYZAM+{EGOUwKr3Q za>;YJICAQ1Nh@o|4VdrWex_52V#?Pj>`AMv{zppQiJ(go^ z@kmh11c(dmI=*sQo*2_HAZ^uoWYWThFsF)d^G#|%0{mYeEwA>6YMJs;{ z(2!t8QULL}w4*+;HEZ-)yteSwT;;1M(P#!edS!?xTctc|ks!H+4$JJh+kwU5!P4SV zXVmhkdM{`)r>h?IL06jyQQ;-YL6j9ZxVpOX&Xbc$U zQ(RnaIwQE*{n&D2uH%MEk(Cpu>;{KrK1rD)eXsH*s2T>!7Wv{_;)xq74c@SqE(4l;SZA2Z|#Dh!Ln>H`5DQZ{QPUgHNE=cDz7rw;?hOqYu~Mb+KN z|7fVT0peBl&T6~7$g7{>;Pi@cWyU{hAwnF-?8;F+y+b=Ap!b@M3Ll#3k4SZkfNSvU zze-Bkc}ipTDt)3*NrG%VkkmUC zkG;Hu^&Yp`95f)Lf{n;S0k=hSd_0cZ3d_+H56tRMD2!RIl?gc0$sBuLX+zPrOvwub z&587I6~+SB$$qU5;#K5~TZ-@#Wc}tWUSI=$riyKY3 z$`HpdH8vhB{ByHAVf|9&OJ2W*yHcyUI{~HkZ_Ob1(vf*6&29SlJZcKOux6hF7IH16 zf&PO&gWe~;2TV5?9e|%p3p8b{bgGjN)3`ZJ>vb?ZhX9|V(`lkp$ z*|RPN-%T^ChG3#r!zt2~9AUD9JRzHH8wG3cxhJz^LRHh9+JBU&?UKXN&&zNtM379^ zzvxKG6ldFzt=sSujG)ay@Oeginzem$m@TOI>as12_UT?YQ0Y#gdR!{F*KO^6n8Qj1 z$&kd+CfxAZesR5WQC4#G>KCFa%ci8~K#WKVhk8Xzztt;yC1nY&=ToV>L&0yYxU->P z)-^}ll}kSb;daL|EWv0rIYwOxpKkUL2J=75fCD7!XA4#^q22Sj;JDJE62uoP)49(~ zj6`XDl=hh?A!jQ3TqSuJ3ZV4^lpQ>Mugc~{e~a{49sB+=bijjNVqy7%4cX{J_YGIP z^xmd|2JaZY*?2^#N!7cFoHqGfkR(|QlsJrf71yG=Ic8aasJOj=k^(J2QpexdK7)a{ zrMPHC&geee;!er4A*c@c0#T+)pJg51$7RKLR7&NWXPtosdR1yXWSyrcIy`8fZ58`7P%80Otte2zSQh@UrhdzHD{-?O!rJ?8j?&>la zg5N?*CkU6w-vFuj1Zvzwj#b|2n++6@^z}V6AOTE+I|PaN%;G~niz_=wnmpnO3Y;5s z5NOw4?Xq4Pgoz$O{(8ZDlC&jpdL1h1Q7iaDh;FfYL3BujS(0MczGs*ypIWtzP+9dP zvnQR~0kqHc{GvHRulM#^kSp+_G31X)k+~W!fR9Hlq}$7yqiY834x&!`YAMO10u zCifOlPHKTQ$<%BBbj7e();{{(EBm9jmYUze3hs7bhQ}&y#KyhR3HWFU{D|D$WCNl$ z0|^((ecKC{`7^M_T}JsxcEh1h%1i-{*VM3Y@nXaInT=NTo9%9`QyVz0kq5w}3>jSU zYouaNF6@7ZJ-xF!0lPS3Bzd-ct*qLSrVoW;x)Mq#X_10aM6oa?$TT5Uo!Jh^QH~=ob_1 z_OcxoHx1yo0en9NsS$bHNymZ}IGT|VlGWeicHgHU@<`!ZvCkpY`2j+542ZhzRNig% zx7O0mEaXyEN7n9JX<4zj)1VEf%JZTm-Uiuak^$P4)%eYJ;nU09M5in^B$XtB=xKz# z_DM@;GjGM#>1zjZ1M!EiQlpP@0dcf;Ia&d@A3vSVHG~$nTpZj)(+mMxER^} zyrnAm3>b%h1e$W%u!U7bcI-PUTzr{)_(iI^fM{*@YWU{ug{u`vW6ic)`4mqo1j9u8 z6sB4SHSS0)`e}ni$<61F5{q`0nMdPpzmpi(X}cPH&Fm};?1~;4x7|H^9mY-y%N>3U zzvaeXdZH9^|JuUSdYi(*;5hE#7?EJrEFWC-60Cpfvjp@;=6SGf4%$HT>)NN0&l!i1 zfHN4nzlslIKRPsd$f~nQvcm1#+~!xNEyBmk&TO^{@%0hN05X9f@S z?4ZUEm8*`?N?EJjUfJ>tZr9Ye;Fyj>N`ie?-m37eX-Iu7xJ{wF^9kHc*ZrNkg_lPj zs}b~j4;l`{7Rm~gt~vP&3OX}Fn{pjIr8CmDscy&ZzGgd9V(kHKq>d`IfK62{5vooT?0l>pv!8;;^Y zBkK*@TSx2bH7Xg;*DPjlP6IijPKiZTe=HKdtvyi0!uq=Dl|d(Rj1R9e_>Ija<*uAe z1U*eCN6{DmVdUyAtcopXxJACQjSkL1yBfEp=1T%(foDWB;j3QLNvYjf!d(w?yEe(J zL>CMk`eCPnGUyV&?OyY0{ha%|K^yL{8DZ#h*ug>2(-OC-j=N~%EeF=o&wkk~?!dqD^mpI!l3ND(3#_?no8|o0#0sjgk{_)p-n{T zYHhy(BmC7=se zI(za?-OejJ+;Bw$N@EeO+C7EZwK_F|68Oh9*MNgn46oe-4(9BG>-Q*%nKt zMxB(s&W5X=ctUDPw7ihPlC-^JA zHxK=`lT9vd-hJ5JQeFRf;?=%&&9s>@yF$xshe18-K7laZXWsrd{0<}jy0CP{p-DR_zE?>GFv7RL&=-K9a zueU|xsY2FXOFeTiT0_>m+OFo+pY_QHJKrBx;zhk5J8(MX)`dd}rmdH6n0#A&ZAi9T zIZg~ou(GvK<4=znB!Ooh5f9qhw;D?s{~oepPTI6;o@|g{Zf1tv=1_rOe}i=(GJi#` zQYWQ8$DX5~jn0I>976$#AdLE-iBhkdnNY{pqQ>msmE8 z!^#+dd{^LuG=x&7qzodPW>l~oqY#Pj+YfF-J62v&B1l1xII1y+yi=2cg`Iv=!2&Rg zYLHG)>#KO$!UFBOk<8r|tuDc>X54NPGlqU_V#y`Pbinbt1Tew-6_*>YLxQ&Df(|SL zIU&Tn?eDOiE5aFK`_(CBjCI_CW^{pelRb=Hxdm0P@y&irBEQC6kxU-T5t#?F zBmTS)b2B~k8oNrmC!Rrs7x)(Ot1CFErqAMa5;5Jzwi0TqIh3z3ZhA*q&CH1 znrU*i$0o>8J^Z&iM6pYc!Sk8goH<4WZDwdC`J{f6_3lhJ&Q|aTZrCXGp$ebCd$x&c zt&JR9ZX!A(3ywMU>Aj&o2?+n82}+(`q8-!iJ|xwaHI*F%`c|3+EP%K{<(5BmMR~2W z^5Q_3X?y7&cwXQqsmc{kPpCeH1(tm0yAy<&S0!-H7bU)Oe(=Cezr>os_jd9q`uS!Q z>=9@#5{7b*Apo@Zlf)!d=a2QMP-P}-TX~0WG%nE>Hj6l+{U~=v_Y3W{v(RlF=4AN_ zd?|W^-cL>P=l1F9*Q@*GyON9!ra8`-`8gy1;m^P@?OJoeT5rfl5z)qX9sWjyXnUZ_iU%^F~NjJ8YY8<+C8qjz6_?oDJSpD$X=v{<N^5$OKP|+~DfffyebHry!#69)whUpvq#4l5!4)mwJc!6`z~~EVOR<+W%r%f2 zA=<6gr$8c$q0F$hTVO|u8)b-^6ubrFs*vKQps>?F&BgJSZaV5X>xN3?gzaBE7j!Y! z-`}7uu#A&(Ts`x0sRP9PW!pm!MHKV6^(Hs0w#nlk<|-_U ztf)do>tnd8!N*X@xqU?g#PBt*f`-w=M}NdR3dDaBdLDj_|2q72N4b3HGSG67jJvfo zbUrg5qF?8<`PR8%nZis4dZ(F3)j>^@Y0>MQ75A@)*$xuJqJ%YOD4-+IR?FX8Wuj&_ zJ5tmJQh?!Gjp6R(^Xopob&12t&&_Bb8D19jZqcH>Ybpt?msMG3B}%N zxY;F?;|hdO^t>D$9IwCSn5!WBnSt2pGSXy-Q~&9Y6)U`ZU=?;~d&`gmIS0*g?-Ecb zPHkB_h2oFI2+9+OnwkAXKwI%5nyfejC5gNK)&$sq?AKCc5rpyApZ+$sv71g@uZdI#f4)geCAGJo`HOnyn-N`D3^I3!gd!mIGGr zu^?!&Yu`Nlp7L(rHoZTPQ8hOn#~!;2DhoGfcQXnQK0^M1(!5X7z6OUsG%g~AdP&K; zTMVbEO7mjox+Z3-DT0_R#CN=jm@)xLtot1SMzL`%zPImP-GB_ss@}aK4~yPBG0xsR zf+&l(rnuBvqD)4bRZKm{h?}k;rk8}YrwjR7gB&fgi=O<9*=4%ikr1bh(={5((S@52=YDLHKnwOzmY_MzvM%G=k>KWu|#0|d}zl(t$#6VqZ{t;<&F3Y;! z&+mZLBlZ&!G&yE{kbOM6jc+ltxMt?hoU3bLsm`;iQaNgwvbsNiKlX7H$dGLixac`A~<^t^e1+ca#| zMd~sM%iKc91v7=Er8JnV5aF8y?tmv<3GJjMp-3qOnt>}!DlQFn^J zudU}mJ}?8}Gzr1pY}kuc@6*ch6{9PPD4!ojENft%J&;fX_zze-bOO(R@Yx)qNZni( zw4%qn@W$v>MNenX_^XWs92AqqKJU|{qC$W8kYj({>D-}BxG`WvGR|Z(R_uJL?(`7_ z=4IJY=Qk`*l0y!4Tsf%1^+llWNbvb^GVblz3$Q5q;#KCJ}) zJi%1ETxP#_?ETooHbbPUmkl#M!!xC=u#AbmJ%tUy%vV|n{utkcFa3)6{4h(VblLM zLi;RG*janB-QX)i7dISlND4F(ApALO$n*2Q+mYs$MHqSex*bI|Hqxy^CHt`{WBU}6 zHB~IU3&Y8?M(q5aKPoqoHPM?mKjJT8_-t9iEg$8!1wJ^~I1D2~b6Gn~b}1G*P1W}5 z?gzS}_th;_8i{N3q*ZqY=?9#2xT97bKXtWxmltJ?-lbl;^ic<^2#>mY;NcxlBQRy8 z(6dX}a&R^LYihV`w`H`rzOFu`rs!eoz_}Df$DjQ$F=jMLxJkuH-0;fEzz=?3C4Q?G zhJqCTGXbH9MUp^1g0ND?C|B{CC05dtJgXB&%@jOB!}n7q18AmL{jn(~WF`Acd6j1U zS24CcD2BxXQOjC&(fXF9w!HO=!RP%;vIPy}nS(LQgRfrnV*Q8V(;%0I5grB#=wqOdbgz-*vaalL% zRkzRQWmp>gr=0O+Sjr%?SrTm+=x&#EZ90iE)P>{u9op5}BZ(`2#~M;F7gENp*;`&C zYO{YZg}YDhZOT1E*|LXRc`V(!ISl^A=Ky5vYf|)KJ$l9GwBD|t%11nV``atn z7*AL-G?L+dDDjfkIJ`RsYk+%Jlf-i+xAf+x)8C!e%p!#se$0=EJA&iwGp?2`)@k7a zpeo!68-y*WKFf$VX$dZ!Qixi4+x++Ctdc-_$2-Ga>)8V+7!Wp^L|zO+Mur9Ev(8JR z9tU{^h)|25HMnOQM7grug-+apLMXf5aTQGSd{@NdZpmQ??mEIw^x+#T7pw2(#^IRq zkjxH8gX1*Y{Qq)d`+}`&Qsh=VtBvwZ4c~1wiA94xv~NGvFlo2@U4oF`e|WjJnh#1; zK7v?6(_|d6^FRmH@iQ=>3WI`^rY|VzpEbJ=3 z0Mbqq{m7-~TOke~fFvhuy`*!H8o3fIzCW`~R!$nl27{n$WWAt=iPOZ{>0Q{OC=9AI zup=)ixQ8=y?&7zT?}@=v#j0@8 zJC>n%bubO5_`%^k>P{1|Eo1G9#lzLO}fiRF@UM)5tV^$5`MAvvW zZA!EWNE$%wNe~FFL`(WHj9OjKky?*eC~~L38C2vseXuz3Jy*<*dLkxy^KHllWWOUS zG-3roAa7DZsG~d z`2y`w@pGaqBrSBT<`WRQ=-d(cJ(t)gp16?C9j-&C{x!-kmDD@R%XUkzh&DK#b}9EB zM<_Rb*+6?Z`9aK|3%OCER}^J9MF@sUPLBgeVVAr@hi~l!t&mXG1zbpL&FB`!)N{xy z>TZa-e;?7ZtYT$9w{<pWZYcwzV5Q&RunrBUgl_QXX z*WdR=4Qe(5A4p6oMbGb%7LPf%Q91c|RXZ*>dMUuX55I?20|My&<-O=j!p zpqk2cmbHFh=REO_AbbGi)5WIk&D`!L93-fq&e@(dzAJV>=|k+)nU1by=76HTJ?d~L zOj99m_n88Xz+N#JmoQu-LU6Tt@V~j^v`q!~QWJ3PO zr2pKznW@ue{d%mu?ic8e{W0{}UBO{zykDxQ(st%3FjmLMzY#I~8N;Y@aX`OtP-j}p zfC1}?=;fBtYvTtz|0<{16ENuW0_KG4qwx)C;KEA^uq#cEFlR<~Ep8Hm=R*~9YgVB_ z<{O#<|c(->+@r3EmgvCbV&{LhJw#lbXbKJEpD?Ee&H{IG6sx`MAU(8+%>< z!K@Tzve5G<*O=dFaW|QpRfU#vJFDs#e)C}KgQ23u{lp#G_|xdA9$Ty&8k_DK5k?*> z(H*Ny3^r&Q{0lVySM%fHTN0{iNX3o6PaEJC;m>*06baqXTOXSTDoL$q(!3Y4D@00W zCJ8~81E}*$d)+B^wpdD|7!Y$Pk%BjN6^94krRTu%v1Bj=qo56e*O24B)XSf^SZ5+B zy$^+}r(Ccg>3dE3Hlo5YM-bDSb}8w{C|maz4t3>Ri5;uWk?h>)7mz+K1dF|Xs>KTp zzqJP?a8mHyBlr4%kRLrEuGoWycV_e5mjMSc86YGORSbS%` z;ht&EJ^9MPldsaCrbsvd)Iv!r2wg5JeX*SyAH#J9Vk+u(0;`k~Bt7W>ALQW2 zIyI*2gM#dz`2;*?kwRq(QRgemk#eZ0S}llkZ|R9pYrX!;H2oMen}WdU3mSEA`MZ<#9cC_s&FmwgaSN#+`?n9(!2=+PKtO@; zUteFm@48XPJl5~{dieQ;9y1#x6SqdmHi`LCX1P!c<6{^#ODMuN!I}akauZyFPyhS`ET2YD)ThsKpZAkJ~sV`R2gW>zN zZ7}((JqiP2T4{Yg$s-LP`EQtbeOASfa$Ea?81A6b6N99x@ckOe&1Od0xb?4ozBl=+ z{2Uvur){u=HJ4gxuBpb9@yHwOJ1u|xH;gtq4>h}S1hB5~1i_+=-x7j$*ryef1t%7b z>~~GQAg%{3*?E5D<8_n%Y5%w23=?p~Qs?6?G0mkXSozf+UsHL+F2EazFws;9PExs> zK1V#Q_zfa?KP}EY?b!Ru`b1ru?kwE|1Yb!p0aX0Tm8qzyMzylo=L#sr*aM}E2~g3t z3ddX)(xTePlxi7Asz8~1U|jIX1V7}f>id1YE-gk*PvWt*8(C$Q%AUs~%p49#!TaYf zJbR+nRhcHQ62|N`RO!;}$+u=HHiAjXR+h{~AE0r8=J0v_+IjTQoeosTy>Pb2_l9sx zP)ad|q;E213-R{E;1z-n+U^~O!SH#HyER_nTmT02A_?TMKA*42O@@wchPi^c-0P#B z&Z5kykwO}@NSwR#spYa2AUf#)(A={E)*iEFvY`KF+gPYhP-$Vlhhg8`ipJoS9J=`9 z`nZZQF7dU+yk&uFOx`laaq;S4;&5v~l!HiZ;bsS#T5ow3PU^Wbc%BjuuHf~Nr6W&g z^+Jgy)*ULB>1Q2|sh3#?i8!W6pY7k@e7X_hSD>H{N}I#_q`u*NsXGXrlP6cI;H!Yq zwGRsmo=s&a(|>?tLcnQsfCp(bZwyqSn^>P_X*A&%AUm(xI*SIR@q*1mMr{gI+Pj=iLB{b?CW)6zXiq2gU ztF;1=OlZ@?r!Atjcf>P(cGcnmerz^#espE^X92zajvfKi!S0D^>|mn_&6xQ1_X1rW ziFn=Rn3I^wZXy!}^$jUnrI_ZhB|tsG$sCPAqc_Atz?*ZBg!cxbb!uy8t>3Ta zJUNLeIq?ZoV;od#S-{UFC)r}9{a$qN~?QZ3byX^OS;VE%b1ArB}3T% zr7XS=IT3 ztNhM0HVD<8;1~nS!;%-R4@&eC)AyvLPIc_1+XlpCK3k5PZUUnb%XzeM4?7L(Qmh7cvsFAXM}s}D4&1RUs*y$CfBO=Rk~Fkz1rn_rC9dvGO$1#I zQ58CSUn%pDBFQEQ`>{{(-Y|UZ&larL_WVMKR7^)ZH)R`r?y?Jg`?WJB#-`F}+~=6a zV9)lV>S!6yy-l;Zp&kAqEN<;_F4SXnD#sn5ADTsMe5}WLw`rEmqB~cOY_Lk?iIS#A4W8lKFl|5F$GPacIgs2DHxdL{uqB`?ok;DV z=mQi6w?JnLX zIj35xDgo#lfUhIQYvwY)3fOLsx^eBdz1GjYzO2W71s*!`;!oMo4CR%nd z-W9s)uP6AeqIxEwSl(JMqKKOev{K`N4F&@EGA)7-1g|C0SzvxId{KvCHG+Mz)i`h$ zKAjDfK0!WGAa<6zy?moEypYPirba;uv79JprTuho2ek*FTUqG+J4lNLKJ%B8^o%@K zl+d|FinCa@0nZY1ao~vy?$|23xcD#n_(QD(VHDNNVX7Z!et*0C?)Zq~+3Bm7#s3sy zQ*f(wu{&2OAi!c`S=vJXo8-Vaxt83IB$fWOe!!0V#=yFs(RrkqJX#R~xP@zloaowa z2XQpUdq?2>a@F7I^aRL^K;O&*D>zSGY9*oLb6Anf+cSjpyg1(g1(^V z3VN504U#_YXwE0MYzJc~o*Usz(v@Tc{%)%7FYfXye$IX5eLQQM22BVvzCS_v30L?I z15m7gUtGE5y@UrXNbvO za85{Uu<8Al(uVW--Yoki%p`+?TDle zqhU|8obabNbFJ-<=nP<&%um~IC#OmaefbbB3ux8008QPjnCrzZ37xoJHCDEC{`j$@|g|;~w6nw5O#6_=xibNrG@+DEx>Vz?N2%Tye=^ikJR4oY9Z;Rf)p@2_k;gfNeuxd(gT(23x6} zNhdtAdQtk-10D!6GD98T@Oh-UE75elfj24)1u!6NrRJ^s?KU8Ypk49sXcM)z?q$F9 zJK4f)k*WUrH-=wIav_K-Ca&Ervq#2s`LpK!ET!#e2PL*z|F<$V!1n~owhyrBZ~!*;EvU+!y zhOFt&C(U24ijxTRi}rh&6>CS6byP@kcNlzdx>TaykFm^mprXLa<GOGhE6Ux^1Z5s0}R zBW&oePCQ;OE5ydnPc5psUWUuti_^KV^*?HlRi1h%M7V=$=?=I)M@+jy*xZ>yg)reQ za}*gk{8V0$WLr2D5c^W|y5nIjhE*u1i@@(ZtyA_#;MN+9f&%|)?$ zmpTTmCAQ0ZYJaW?jaWMHioBA@1T7F`{F5yENQQR--xV+@X{Nj|7z+rV!nFke zQz6%X0HY5Yz07s5U=~75$bpNJS^(;|4O_y!?BisR{|0O~U8af!Qt9{cpF0*u>N9u0 z5LToNz=?$@w^s2vt@8TBSmYay#DJNIT7^Vu7HAb1m2TF;b!GITkd(UiD&Oj zZv?my&WX6t>`XakUqAV>(%xZ{PTSc?f7=*xV7h-B**D1$a=o8Gox?B-SW8d18{vsk z<4MfiKj7`{kp?ACpfRb-Oj>7~fLRY`FAk!aLpy#G_Ls-g3p@t2uR)xRak-<|)H$8w z={vy95PU|9Cs+{h4@NbQkK7O=nNplk<`?#jDaq}PD{h<-M2wBUUW@TMFN;kdnd(*e zEI$F?MNyu1LYS+A&^#lSDH`g_aIz+AZ~sr36#kNF?Qc2;k=#fHLF3pin@Ce~IE&Sg zc2G-(^&1|-``G>d%9I~TV>Xf7<^&poR?wTuG~qqNm-r+>yE1{;T1ds}>vjFal~>?z zxB~)C&B*Ta(fFN~;mOrqb%8ntQh~1h_S^;4I;l;dl)Rz6>#ZX1&QA*rJF{9)4_p`=lmjz(M#K4=b zpFgiM9QW!?s+aU-2JEuV)pYAx-%E76yNUNd5I>%%pb7%l$OBFrrLQ}3kOcM9QItWy z@>SA}1~q@d7D!@Co~tFXw@D;pp8Z;EcCE8nOrO}L_ZWKf-mDcs3!^q={?~fv_5eLy zZT!<)Ujh@s^ZNS=H~KEf*|K){jsyR|`oc@1YM+gZ7p$?J>5~wff;eh3l*dL#hkDC; z1Pjc>r`5CEXT>(U@OkL(PH6}hq~^-P0iDQ4in`^2XBBao@9o5ZspW6tK)hJKru^`& z+o00$PlN*Zn!$oaS>t-M{FgF%00oyn;~)dbMr!yx1IOC1;>oQ>j#w-}p_Vg5oX$Nt zSRqK{v7uMeYJgL2$DpK33+`NxGorjLSHb))_Y7<$nk4?yj?SGu_KJv^ZYrrv?Z-=@ zKa3hkwJ-d5%biP$n|ITdLTeAR!iW}qR=ljpYB@3hOqi82H6!m4M#=j_VpkdFQkikW z{x6hCC3|Nx2d#P!;yEAu6wYb=<99svRpT7)w5XFrk@c*N~~lI!%gGDA8Ucu+s(&T zXJ6X)-*?NR?xK(-M|KT^K#pvdFBpQ38jg78ilsM+NgsFp>4X)L6RDzdoU35IcPyMR zjZVs?dhPLpKB|NEFifq^1KhUAKw%dSwI0v8{9yK;DFSuOzkJ$R$xM*j1QN}H&Dm*W z@pt4nXAqEC6)v9ft!Q>=n3%vga3PTd%3&a2mnT`T7Y0x6AeCCW=0w-Df&fC_B>OpZ z`#Q(|PGXqG&E|U1X|N~p%j=PeBw(w_Z5hM%dDol7gUWJi>-}G;*C1;E-;MnMd#y27 zQBaAdUbEiHV=+vb#b7)f{}<6?*%%Dg-#VErt=%=zxQG~Z{fv;zm5A~+aU;cynfHWE zm!oAjF_Tb7f1vnkn?NYYCIb<_)UGBS;EI$g%y|ZLULO)yIp5AaUCwW^PkJfQ@~Hd0 zX)9XBoj#q727VyGGPcXrGU#XV11vx1cP$RrzNv$O4>oQ9L`d<)?sz-%RI=#2{Q9!H z{y*uqhu5nem^5X3-F2vMOfEv!luNzHy70ZP?+>dob$w6`--KsQli-4nVqr1(C%Es> zf_{Uy+Zc#%w67`N1$9#!s0m~EzEek6babU4+)rRW)MNBzl)0S`C|4P8e4-zQy z1vTI)(^W77^AF60kX{v&96=TI{td2kFBJ0{bC~0hp14*81K2Qydd0U~%AeRSJ>$oq zI?GfZ;m)>FW5b$NP86aC>F?>E`^8TV9{}{-$*WFI`H&eO*RZwlc_Jenaz3(t z2s$%)h$cFtjF_xwWOpTK#MAOeb^vi@ZA~K3{_r!s(T`lbgF#G?JcAg_Fz!3^VNCcA zm+1xap2giAg%m5wl>Jpf2ixU5e-X`Ve;w#5`czH%o-%?)UBapbS342iW$qH0X0@yaM1_nRv;^0BM@Y zJFwAP9t4ovVJsa{wjW0xPy`^+SCsDc-f;}u;8q4-$WxK|PQOScN|eY|2B!ucxh8G# zuVPWt+2Pj&Hkn&yb+7L3-_!CntHS)^j9n7J#X?&c-zp?6x3chYpJ~?h8q=w`6DT7C zK5Rlj8B}o7G;4W`3L=GXQ;GXEs6~7T_KDtT-Tj}e29(&Kb7022&&&WypwHEAR+R)YYGI7dXFYXRRz2~MDW2xFZB-s219 zWQqGefi7}^D_#Sp9Yci^VcD!BPP=9Xi7)0Q6W-W2&gnd0MS&uu0Dx0rx(I|1!M#`0 z5UM+R?)Ay1KeB{bDwVRnLJS*8BH_fZ5t(Lf-N9Du9x7WD3r_8rvAf~<-{OZWu{TBu%&tZ=FYtGG+zpQ#SMD8^iPqb-il=&t$ECx-sDb2yNc`Ga z20-VOPRi+NjaJr?;Ee8;wV*RpY*qXbn-~*Up~Gq?SNIIXuB`2Q*w+djabHN+5G-bP z_Zh*YnmUg-NqGcPNN6A?wIFTc{^Wd=JX)2m(srTON+i2_nMa3m^dD!Ue*P3+q!xfm9>rAIK;w%B4XeW*1Ypl?=r!wQM*syR0q+gz02F9g zf}KWd)q_yp_ZQbrS$1gKyJ53)bn3tPmmJA`nsB_MO|P%2%*Nev3pqe)ciKdtu1;A* zJ-BMrUb{QI)KyCIL7q;inz&H5>xd6-YH(3>upoxFM&E@c{?Iq>8~BSR7ic6-@L*9) zQ{(;}@Gu$2AGv2(X7o1M20zaXZ?U8!s5 zdo4@KdhLSSJqJw7k0hV)m;XVIS>~$%jM>`#$6o%uAs4|VwdLw*@YhX1!4IXIE@sjp zc_k5%&8#)Snf+i7QqV0pkaeH2*QoMv7K`qD{iKu0BhO79t=Co1iPlGe4CL+m_%-s| zlMpvXXJQ+(erT+&QgqNcSj zTGp9|gT&dLH@gE%mL4CEKWd5J6g+#2C4@nrVaqeJH;8xWYeMMvoG>+#% zq{8^Zm!K;PIQar`yi9nBdXSfI@kb~DSt;U19Kv_@e4R8=;J{J(POg~zW^zyXZt7Er z@DZ^WfS^eJaA9lhU{$M|=f#$O{?qu@xg1&z=2(_;a7zHQ^cGe4I$`5h`~5VvgBJ&F#< zzN5Go4g{X8E>yV;zXGD^lUHB;7fY?0$`PMG1s#woG?N*@oko+z(21qu6nUs8;k@|< zxDO0DHiz97y#=NM7|Yrn-a5(6n2}j?5MkhD%?|%ZIO2R%u4V#_1(J1V+&okdAll)H z*Z!yIJp7V+|2BRYA_^jkTQnS`<_KqmRv@?+?ld)Xq?s0uOv@5C=ExaVR%n@KW@=bj zRw(XKI5S7#Dl03r{mc43JpTaa_2S&;KKHpk*Y&>QNH$oRgO9%8|Ld8UKBIcrp2UCAtiOAD}bAI)XM>sLsH>=;)zEGS(wEI){%x$spP$ zds{%){T@L}3JjtTNt9cpZ_)GN+|Gj96iiyo|B{#-Bm_;TW{Kkw>Gf)1QUCcefrYT4 zZ%WNUOXmiBC)%;ZkP!kdoK*dCUEv}KJlYK6@ptZyfxgGcF;>kW@_}fk7Qp^(V2xT7 zBuGJH&bP$2>>+WzBk3Cv^R=T9;^O)OLB84VzDvJ)aE(VPo-eNlYq#|geZ0CJ!s;{f zsHfLjuIN7#$hDZ(E%7ScZ8Eg@*G7y(Sux&pCk-*`UL4W;7EP&IQdFrvl%W%CCx^QC zi#Ck^VpL`8MMMTsY$!Q}S?G7{LMqRvVm1JiUJx6~iN_n|^6{*PUM40S28=a3u|!@r z{dZ2TLTNU7qSswXX6inGV8MOL+<%6H!~bRcq1`66@;vf{T^ZBN<|dy6&m(-HgeH@l z19=92cMJkx2q=hgb%22?sr9wFX%SfcaagIqmJVF^J`(GON>rdmm~MXhGe_4;b#tRY5*}uAr5CZyNs!_Mtjd zEegeALPq?oHb&oTimVf1(}j_9+q|t5B~hCC*_Yf;gz4pY$=6)ZIbhCg;^PBD9j+># zm?Y<@$tY)T$?D9j%A#LBI+tY#J~SI+d)h})6wjRjG4uJ`0PlnX&~*&*am@8BrPMRu zp5xhs8<3*8>xHBEeuGTYP{BA^7D-K2j*~I+SUi9A+FHC>81*==)RFa)d3Eo`#i1F_ z9WR2Zyf1poZLgf~`Y zuUfAE1D}5SWgB1I_0{47Xq`X~*4>P_j(<)u+m?>qiR#3{LFWu2!3C4jbpe$5I zCuMyb(km?ZASu?*!=Redn@MLd%g3TxabL63h|o~G5I5Yn!EI7L8^sZS1&6wmVmbmY z3e8cgHf2PB(9C(mV^ZJkAf5H_(zh|i%7H~Uymy-#UZ2=_0+qnf4rNGI7MRD@$h?GA zqIrxxx+i8hM2l7&wiU$a0yPG}ScYGc>vAB~S9+)L)uzm21eVbfHppPhVK)gZ+T>L3 zizN@&anV>8PO6ROc4obiAT(0Vsg*3CT>Wl-VB6nhDWEc&Dk{_(ug#fw!?v0Bz0=>S z|5-HUpP|{0KFMPbp3ns)&-=V6J#$JR6z$#}l)#Ji7K*iRi|+~u0)6f_lY77+;oH=jHVKYAsSBSpO>GdFo9C`?A40 z4V2Rc?G=6|PUph2h&!+QRu#lmq;eNNhFZv&tc(r%Tjz9`5FQ}qYm=@o()$|8zhc6k zq-lFO!67I}iPAM@vgI-xz^Qq^{=+?E&=Hyp9_#J#wzeKIr8c)ICwUf= z)lIa3@<2OhG)(erE()0ox5fVR2|`jjKsD|MWho~b!6Z@7g2^N*OXP?nTu86IxckB( z?=)G4#nnIB{u7XF!w#h5Ff3-!*SX+=?k88(kdol$7O~L zlHCEkJx#PIJuIQz>E=Qt`TjRbl!z5h+vzwaX{~Xqg-xFDk7%HXZfR90R!Qg0FV`2g z6uaXKODfXft_Jy6eMi&q0K+P0(q{Vw-b?)1cCNNe@t#sNz5woGjMF4`s-f5lN@HfZ zpJ)@7?4)#Vwe-<0G798Y!{2vN_s{1MGlxK)hW*5!8CWETr&p#*pYOxc){U6Vp5a|LNaH9zgKxU9U>wbk6&YWC-cGR*G8- zK)6g+?X_3VQmFFcblRkCoVsh>{zKP{azyn~eT$i>2mMX`KN>eO&1B3nSE&waZn(je z|9pxwlec0aSpcys8bWe^V08Z3?m`(4ESDCIHDko`GH5wpOAIidM(9%Gefj>iSI<4& z15Wee-=O~dy5nEgEv?4&+AAJgzbf;LEl@VckIN|lsMpH^6)fHdECHAw3)M$?tYqp_ z>26NI5OJk^5uO-o-rM~~w@z|oIi%7zIyBb%I4EPxF-3z-m{5-JX`9WWj1Wi~wj-G3Y`hGnlTFw-T&a7-gQJdg1_V^El#cd(ViOT}Sx-WL z2(92YIsqQU1nwU=)Tx8Jxy>#xsS5M_K!urHLTepiLKmpWaO~FaqBF)0aZiQcd;f2j zIY4=%mjxOQ!QX;yX?H(?(iT#{`WNG_9LaPW_1o_!eP373C<6XFuakw>loZ$Iak^?6Gg2&Za{45C+{a>M+A zp#LJwD2qc0^_bRcvZBk#faxFI=Z_s{_$7+(JS9c$B#cgL&aN88zfm1T?^F+>Ti!uu z{fg|dl)fg5WhY$%p%fIKjn)~PqyN|3?hP$;GI<8-FJs&o`(!+uty9)lKmqxB#QpO= z5Skt>Cm(%$U)Au9U7#>K6?+R$i*E8Ei`YZe2D^1~22Xsaos90-yasKUSwCPby9Q9< zpIkN1u=sxP4u*a9{V#{4FG&?BB)QHKh~+}cvOIcU;n~o8<1uzBUiOxiMim({B)Q9f`TzmlzMW&9)B5402%_y81~kc_ zh!y%}`8)Y~+xk*tISoj&l)9W}iBN#*bNho}OH;o0=Yi;t?>TJ01C4!X?ccb;fHO9p zcIxP1AwW%FRK*M}m35+b;c+40CKQsGlO^{vfGj&a9iOjz=KID1=3#vt;OH&Mb4KhQ z=@RzBmt(@|rrT9&gZyWO97uwVP0bWtD(PYY-#C`%i_NYM$yLq~Bn-OQY`rr|4b+vq z6RdR53nH#nn)tOXP+)0uQ{7A`2TO{9Frk=`%quzbhZ$I!w{Z~!-C7}6L+3TMf=&Kg z_fo|M&oZfAHOi7xvVp>vI!F`Rce)p2W)gI66KHXpH*g2b0B$TsxXqo-tc{kjeP85I zn=Sssu;%rTRmCLZrQdMzTmrghGJ0doRq}-?2z9au=nm0*${RE4M9+`d#zWVorNUyb z*TR8^C6-*aO3?qhMkE^iGaS-^Od2nhNWkKnK#rEn zjYAqM={*^m(7B&|nsp}CS>5>Ht;5o{)}%X}FWu*~`IQNA%ThkbDu#tk+e3hCcukNn zt$T+NR=GYBJaQe`s((0$$4+a@b$|Z%7y-SanKpfF$WB3NhK|WaIKBD^k_N&iwLk*pGPpFr+Ec93tw&CoD0gc9^Zu zDSKl&kt1|VAtN{4P=krmW}{C@HL*B^%=?TXakRUIzaa@o@_2gMVCVWA1NI2tL$yt; zi51UOgP2X){a&!8W?WLxy8F790`?s-yE|7Mn8+Hp*y>ldSuREl;1`T)i;8;~-vEZDIx)3?A|Cw%G{;RdFP-~@w8b6rDR=9SjEUxdD zBL}|h9|K32SY#Bnof385+a^^hBFKDol~a&Cy|Q>9^I7m(rt*=U zaC4SJaHY?{$pWuShF_4Sfk>>!yu0dZx2{$eYoOZY+_4|Vq_3I3L)NSO>qmKT%*;_5! z@q9{FpMV6*wieu{u&VQyMn9wz6;gQn@IA%7-P!*#6i|}fab+NDvbM!*yc;i-m&unR zL$BZzk294iS&w~npf8;;O{-_#_>wsn=3f>1E{PSH3wbm1U*)i8{dMTv`&KXzBVTF8 zq~JlvVFd2fWax}5L>Ag){@@;&X>AaAB;gbg_tqyt%1T_)<^*$$g;^Raxo65KUKKH0 zP_3$gy?5QVNKj+s)H^34;zVDiQU^t!qQ!FMIe+}7Vv;(tX(hu@hwfUcJ6QKgJb zzeADJtEK&lubCNH2i@kK0MtEcpm0An1K@#7Ocb^elr}A_kj}5jR2jzphj9$eU8Siyvd-nh&afyFCx$|AEzVkzpezp+?s@~K*3J-C-0g)z_`1Vz);a4xy!a8uJMmwH>sA11@?QD4@EYp@3 z*!e9hNjrO)d+d4^0>HMf3G>nWsuxo*KL-~S!)4mj{IK--66-;BMAm@lIgbwe62PLvO4*ALn^Ey{A3S4WO?o}{!7hXj2d>5g9 zO85cHv8=F93j*d=KUPwiU}dZn_fWT*;shLj;ACm-P{^peLjregnSt zu%q|fRJ*;IQe$Hf_ATtDhkZ{b1m)b#D|suEIxv1jfAXF5J{AE?cGo9=r?M0UW-I3l zZ*(Dy$w6QToRJ%c6|O@#CZMHLFWQ72b+@3r!aNQ#?XY8f6uDoS;EO)tXMM`-jkeOx ziD#L>kDg{sz5~>_QXupf?YlDe^C&m=I*gC%=0oZ|wKA_;?E^NxLkFfTPd(mF&>Va! z*)z@qVc@di;i^g)*Zs4+qXU>};(%J77Ex5%t=h{;T{4JtE7a z=VPB!W=flm8xC;B+-4J^r1d&G%G!8ntdDlvH%qj< zPeGP@ZG=D$RdR$fT%@BFla7e*3Pxpmi1`(mY( zm_tEwXH`weKWaj}KAM11OpVw63$l^M9u?uNq)9>Do_ov)J1#Q4{}oVe3$9cQ!(3Aq zotiw7t_Hom7X5w7H=gB81-7ukFCuL|P8k&=8~g_8Mlq#6Qq)peB-)PG(ii_K*me-A z6;tXE=G|0CLcP}cL$kE{`B}p6K+u7~UC4_i3e@t4yfL-P{Q|}LFFdrvnmSN1ugB#H zqk+75x)BPLc6b%Dt5^U=3O^O_y^Ae+Tnve!Oo*IE5PERf_|p%l2kOxh6!fzX)m_rC zwgpZY8F%^S$6duCl1CNgQTU6ks4VFV?QoV#q27b@$bY{I_|~8v3b_ppM^zL;Zw>a_| zZA#Nw@yDpltt@D-_OAU=_YBh|k~mpI*KLt)cN`{IoFUBSxIIP+sDw)A>BND|)pUg*Jp^zVceF&84@;76{kF@-@fgl^eR!FYLh zbMQC!?2xN<+ZnYMtzKB-%7)V&c1!G>34wE4T2f#5dH!+1n73AX^L}O0IFZ}pba@Q0OGTJ0|T6hDIjPL56NXV_GDW;Ey%c2qJcuW~$9Z_pXqe3l)ycprv1wUdKMFs*wV0NVl-@ z?CgkoY!=_-MUyD^NVSj=!eD_ebZeG>wIJ<#_9B^#iCG1O_`kg2=6mo@t?TeM(uTz! zh=-mpNYhNe@CJ&c+u&isG%rE8gsuT^HB+iq=j+QEKVHaPz4G#fE;J4b_h11!W4mDO zEm|$WiM*nY!toYI*&(oEqcYbuYa0JD{(PoPSoB(N;|~enaKrF+ZNB~71&$vKoSpYx)sZaO8 zxcZ~PL>-jE>jKGr*1`Yc+R~*4tdqRlvQ-`^%?{4lpdwWsgkIA28~zFR@Zt8muNHqB zu-4ZvIltVi>n}#fUK=yeI~M}h#J+KDgH^>H8{*h6Af%b`0w?oJfgqLfGet2S3`qM8 zgn>1#+nHPeGYpn+%*%SREWO7~OS~QLQ(#n+I?39lN))_gGB!Nnnv=!`MHd#A!E<_}0YWMv>*2^zCm-`tQ>5tf5rbR zdHmjU1`nDL`ythO6Ac@O+|)ksLcUZjt--zs^Nt@OXhz@m-kdvDLmZu4+L_<$d7BCTwycl9eK1j!qp z9^!jqjDcu5DF86KIcpZz6W@*B-6AQe&2}x$#5jEdvm~3*V=VxAq^kyrSE~(tddx)% z0QaTbPzxcYvWL7_U#?&Y4@||pn8EhuA-uaXeJ&;H^KDOH{)Z^V?J95tdm>XiiiBI1>>1v3}grh94mLAT$An|m1=44V{ ze)ZMV?8T=zPVZd@<3c7poT&d^?;%Y%3x9;+oz?*!1j!F$j)wO0@VOCYdu%}2;Y+eu z&ax^>r={=icFQoE9kOAec}c_X{)|IH_c1o|L5Iz4aef0`7&WxaijpRZ{p0&6+;o|P zm2uKYI!9AGR%QYtRh@rcug-5k6Qm+qQA>?lt?Wsn-$Wf62Q6~4A7A&X=L?f`}eH8O(hTEXe zmo*RSQ9*H|CBOgz`wfblmeBj5Eh#jSi$-k=!L+?J5`u9dI9PT(dp#p_!m(}(r=$5Y zXBpw0LP6j$K3=;(TjNWHea3KUR*B!<@cUA%ZJ7fIXRR~eK)ZGfNEJ6@q14M=)oQ+N z#p+A6s-bl-Z=1Cz@kTtW+gMC`^3B1#7M`te2Gl>_nd?^~EF{)ok=l@}+$6ri@Jd zdGz*B$1zcM1cvi(pxKOhPi|>Qjrxba_AnD7^Q^af4Q_CpV;AbL(PDj`Xt8tbrsU+M zAVw)S%_(27lh)&F!?DCe9%lcLLY7T|$7b;(LLUF~B5H*F#%)|PnC;r@ju&@@I0GIQ z)=rMZiX7jWa3P^PbZhToui3q;9@(*6GG61+kV6+AElGzS>$LpUtf2rl`r(kD#ji{3 zciMAE?*Us#*mj8jVkxI9{qgC+R~Xn{&b2e+PnNHyK5laU(mbT&S|D@Egpgr&p@vwe z4B|*oF`RbJHu3uL*M3Hj`P(P?Cc5il1W`ccJHP44U{SQjm)<5xUI~WXuQ)I4l>>N8 z+skZV{EAZL_bNN7)wxNwmBeVU)S7aZFH8S@w%25BjGQOVvn;6%gc|)~%R_pQo9q&u z-iyeYTWhp$ttY!Tz{%Nfzx7r=tB4;YD+t@AAjgBgjbssfK+Lzc&&A}xG2E7nK3t)C zP(v~)S_vmH0K7)$mdunYmDLqMq^(i(y9XQJivcX^=#nCAn3&1TzYZ!LhFMjtyBt6y zC@raHxPwzVBh z_<}!hzJ69aTSY;?yL#of z>;8b{sN0m`EWiK7=<&5dLUQj-kmTT#t0p_`dE)*=IjQk?pq7Cm0>E(bWg6h;@jxN^ zupRf8_t4>LQ0P2NzcUoErzQ%>LV$e2{A`b%Z3EdFO8=x`2|g2*d;j`q-(!$OO9NND z1>l4`C(f$!vYuox$B+M~Wrjaj7L<7d<;`J!*bEc~aSwki*)`Se9kE~_d&^ZT7sd2B z(U=*yuKh@9?$gk~C^fZ4M>YLl2)G}l-ZQ3cJV(I+Q24QblGKOzX@? zJ~l$Fwo^u%aMYWDM9G+-9NoAykR-3lpvM?CTNl5KW|A}{;aPsd-XNztW(tHnl$5O< zIO?1HYldRmZ|^Og$0SltR2}wMNA~PekV)p@FRRtoBg6A+Bt0b2M?GzH z=umA+KKo%w6QX}E0)n2;^E|Cy`ckkWc4M%!{%1#}|H?*?C96ioRL2P))Ea=r6YsncF)5t0PD@5L|hx zN+frt0M9AnC>HUZ%X;7YOY$|6+1yXP5{R1*gZMu1O-RoNTgC{Mo@ROA~ss7%o65>nQtLw=jg?EWs_ zbH-!kq9ZApv$Y>gd@7gq(#t_SaMW8O;TVr8XhTG`t(eRn9uwklN?gZAUq_pW7DQmT z)~g?=0xY)+ubFo9TxmQH7gE1nD0r!R&Ld$R7BvbT-L-u-br57x<>F-GAB&ikDW`E* zQiEd*O@}mJLS!DXnlt($|5tT#;-|Q|hW|NL{}~saQ=fds$r0W9MZNWO z^dCLf{Rp(K+j)vs9_w@e{mNJbJ4Gbr=c!Jkrn4n$J+V9Kx`P7y-QyFIR+f=hBQc$M z0IK!e1z5BAIG^fw*%JyWV+U21ZM$K09P*UCCK%C(dIaa|jc;FQ#hg5&k#_KlxE0Ij z>@h+wh|9hD0UML-bI@a$k$XRCpDU3z9{(`G36I%--ZV7Ee*^1(7_o`6eM;?N1;b zjuAx$*3p7W;8soN*%-9~#i&jNOIsgzE-nR;?kL57Fmp_>QSFuS%}ob z{dlwqO0B?kUsU?jg#9ZrqFpN!G z$>+lZ_-HLzE*f+TRrWUQTFi&{65SY?yb)p8hJ>xA>rD-&&#OkQT)+K;N4~nhy7ujH zm$~-CZfe?P?Hq`)04um)1S)E-6eHx3d;y$X^Zt{l;ftuWz@-N&lgbB z!7Qp7FCUzoaN}7gNKSbUd#z^)Psj`?$8E+Tf@M&~sKH>5d@#33d*ls~RnsDAE#$pY zU^&KZLlM_R5Vm+(K(y0Ar{B6(SBfI!WOO7x4UbWGlfv!Y2yirV@Jzy}LZnvgg%icg zxezt~iR}mwT$$dd8Fo*>hUY(!eQq_gi~0+2cs`tzGZKZGx%$(9u`35+&^wA`Mtg+JMq1eHuG<%T zp+SM$_0Cr`#mn^?MXjU_!bY)le`$i=crXtr9GJj}Sh^utot{Le)A|XpY?RuQIT}_B zRXh@I_?7@am2=`1ll{M?xQQ7!-}>{!NqeZ}FbG-5RO7S0)n z2NW#;fVzQnFv;#eDuHW_BkVfm(=^bcmERsRz3*6{x}u-LlSlSR!y~xq%e2I z(&~w6W}MDQkivMINE#rZ6mC&_Qnx5B_X7(<`=@N&In+cIw>=tw^=8X1Gqvv>2*twD zRy|{NLLp1utY?S7UR~vq4?|K{qK`wuHihBcELznGkL`I(gp)yvA#y^sw!6HNX>6N7 zs1%b_kEZEwC-9er;JCmBX8ubu+)p?4ctd<-6oLHmA?OXZ#8ffo%wK#;_v`9O{59ch zX&l07j%E_=U18XqCeVK|u@Dd|p`>$_^waiY-=X{RT%RyruY}xhPsF}oT~Y%eNl~k= z`N4LQtci%Q-Pr9HO#+gB{vCPvBI$pR3LM><>?vNO?bb5V6HRA-gP@ycU%}9$i9+7a6V&Zeb!T1A#6Xzh*#Xhv@2djUo)-M_f8 zv1_5)nwst>^Jn7svsMNJwpJj3xlODvmdon~5~BB*+NnQ&V+KK2Hdzd6-EEKU#uL9K z$mW!-pnpOTtPOW82PBfw{&ydk2s@M9{|jHTav6@|1&he4zoB%g8NxEt6eVjU@{F6| z35+2M;)@Tm(Tky2VnenJiuQ2)1$~cex1%c3@u?o_+Pcc2Zp%e&v(yZxEh52v1?;W5 zTbJ3c^g<*kj2R3fR7ff7CW11J^(@YUuMORRjR#$6*Hx?Q<- z>-IjPPbT>D?*KRt>RcoYjy3XTMr?v6D-kcRA2kK_P&>5k9V~ikBrF0O_E$CXTyZ=- zjO)2fxW411P5kqA6X5LGW(lz2?ImWG4ITPG-7X1!9>Sw?pqeXqDC0du zEr94mnMaF^_sB2fKpGb7nf|6~8@bev7dx%)s+>G_iTLoMt<5J&=*o`+bHoVCpxPBw zMaD~sh><#(g^R*+&2+~wgrxS>r_~C9+pq4SSdL{$es!fh=^@8)zxi)r;0hhoJcsCe zRw$3MdWmiVS!`=RAxqJIM#2st{K}#sSAp$8)|U(#jw|J6W4l1mHvBj^SBz#gwYBWJ z($3GKE{9e>R(4czP%|?Mn-aUG?^&Wi6J|fxqmGR2S0K;m?eVDlSJRF1P`PfFIdLWu zh;nw58;0t=Cl7) zTXbod*dXofwVMOvI}<9i~~mKu;ITOq%H8u{RIHU=JC4V9DwWi zn30y*kJ&YK^nX^#L98Hb4}?CV&BkfV8n1ALZ9q*SZ zwlUpBsXLoqc{cGju3|8LOV|!^`p~1@;>JnAmgb&VS_5T$MgzEihktj$4^&rGLiw{7GZ5p6sd->@VrmDyvuC5%@`RQtI7?Qxfnh?hT&xoHn&BX|E?d)IwCY zeAtBCdzT_i%$<%XA2p{Olyu%c%rE@gb#F}Pf~i2)QqwI}KS=Z2(khJmfoQ6_{e~i_ zEHI{*emv=W&orq^)FWHHevTkruxIVdejRe12ua-#Aw6(Mq4vJ0Ifn@pmqV>bLxA{Z;dR@}Sg&rAWL% z>@RYUCjJ_R`pgEQ23|@OiTYvc&;;CWG<0goerr9`JT^%F%>noNDtI3`AF=UAmTd9g$LQZ-)}~b8j*@%`%4*0ZrfRdXg1&SfKy9=nl#<_Lp%rj5Z-J!UxKS zf00wfLILg46Np)*uCtxGlF?;#mjf!-RghOc&?mX4JGFuYg&McG0;@t_+)?YF_&(^p zCZDAdSud}!ckgm41o90@Yu`+^O-Jvg-g^@~AE>TBBNKIXLl|PTdMat*6kBC-BpqN9 z%A025`tb4U*Z{ZbFTi!BXO9dK*~+*7fF- zCW-NHdhD&`YCDrnFmpH11A4!$PDp>L8UVG-N4Nt_uyG#DEMFN0w9#duE4GNZJ^RM= z^dwnDR7T*phmP+<#Xanmh1q2+Upz9GL;%6#y3fzpyuPMuVpI-1N)F$+qx2=MdIALB zIeD(`j?ZQ((g3)@=XTholUZ30%jW5*sRfp9Qnb$CU4@!++t4z=W)5P;Q@$S`8SUwe zz1ziv%PtQv*R`(zkQeF_%=nk!!uIQZuC2yt#tJ8xT^J-Q;Co@b+QoarMQq+G&`1yq zgvJ&7Gr-lgF*T0mkDZ*ao9XRofuL;Ae<4wpmt?N<6CvBP49lFssuet>bfnxX=~LRBB$S%E<3OVp1ne)-=o5KU<=qAxqeQPlPGxmWwj?NhvklX;TES zm6!vVU$BQn;bo9;LC|E#=0rjUs#Ae%wRC2bIY8tvKev=`my_L*UN^6UDucL((k6#-i}~-FQTHSV>4!w+rBeZ|u8xH~0hz71>aeb*dUG%tJjT z)Jx2He}XY>=zi|xjvMZt%t1KxLhf6`H8~Rez^yET+uE`3HD23+76Ta*T!G`JMy=g2 zzB9V@d+(Puj+`Emc%iyvM#g?j{b_1b^;`Lz-9MAuZd3Ql$zp*v6$Q)ICl5dO`B$7O z76Lrtw*P(X{Cw*E>MZZ3%~pbKBjU>ANt zk*dh>i@R7&k>E`I&erI(v?GawsZY^|v8cNmyC?zXDLPG0si4E2Z!6E5%|8XVX;ER} zh12}dbM$p*lbot5^K+P<_(4Ld5I~`Rqg&fsTHy`7*_#p0{p(M^C*^;#GnolW*R7}TUsf(BKx~Xh!tt-M2CSD_H3YIHVJzSq@9V;ZXWBwvp`=_sFX_!&izt%bj9HW0vmpO zsMV5R$SV{XDF(W}q;&^2$iN9_c{S~Zaj+9}coHDo+IBTOco$A3>iqNL&O=sGP$K$6 zv6lKQ=gVyR$VJvV9*M0?_|8aM?A~VO9|aG8VM!^xlv{W@LCL(om=9wYW@=z};UJP9 zw3f(-ZQ{8b7*!*ZI^4nzb+{-C!s4n)01tQT^B?&Oh~8G{UV`+UNXnRmFRHOTAWoHj zZ3Hovpl3lrJ-Um!rr=H88HiSPTK#HKKWYs$R-L8Q zX-Rv5sfR5T;f>!kn#=9a+1K9~m$+!%xZi#*NT@TgEMwk$!FE_7K4rSxT|z-00;zja zFF=ag8|^S9S}cfMfsI^B)SI3f2axxv`25qyv{kNNeNaDZ6-1yXJE?&>eR8{5L1ni0 ztb5~~mi}YN-#;#u7&3$L#${GVYRewy9aqTWNHkoIOdlI)Nq$=8Kb^YIB$w8r4FJ2p zDGs>?2i4|`_ODvl`g-)lGfgYDrr*MuvEfFjomIV8Q0f9(-_cv$molF}*T^1*b5u+t z4S?w4u3i1c2LKrp@_k|0eF@$$iuCIJ5NOAXkwN?r@d2go2n~mA9;{XF->))Gu)Sl_ zK#k5H2r3f7Cs#NiQ*csa-BPPo;BfIs=MiVPz(~GEDk;gvBEUfo>3uE( z;wzI;l<6%E3Dt^@sv}S7nnZi{0fg-61vq>E_XC=s=iX+_a(Qx4 zNjwu%Dy|a77noM}gPfj6gd=P;93G9fljPbo$5jWIhR6nrOOELt^O|KwMBSb&<%Z}Q8Q=}zeY%da zYCmBoG4%RUigTlOrSMO37or|Fdt%V4C8A;ee2h%w@Odqiuk{}F=#O^0Z~(>S-e4tL z7iWL#N7(H?3kwM0*uh#pNENDFsO$3?Hv3t_uD@W{))EKy%csM|rPIPUn!Z!QU2rUV z9|JIE9dE1}t=l^kGpQffy~dD2Lgt6Q;ME^;juHqd@qTQp{VxM!S(Ceo?vod>eAZ9s$Vryc;qxgR*LeWD?`` znAfNNxccAOT+|Aj1wf9e&Bw&5>%Zs$GZwKbs{wfp9(t$9X*7fNdbtYrx}xNSg(Fru zR#E*5g4)!3Y^h^_kzX*9C{=Ixz>c&^s@@sQID4wKE{^ufp#VwhxkXJVK>B>vuC{9* z=L}w^cd54an)ZYEMS^bGP%$`pdS8;X|D}!8jI02~@PS$UvSyh}MI{b;332|}v^xGR zwB23izV67+G&CgE6q}UfT)VJT(F3qR<^Xk7 zz^&fBQ=cw<@7hb+C>Hxj^)^WP>(kKI;FPE2~-)q zjNI$eB1$y_6Se*Qvl#@8a`aWU_y1>N zLJTglfgLfexW4^u_hB2Gv2#1%HOsPwnPmdKVRVTTOBy1}aW9dwu#hNxr>ANk+%}hl8mGVk{4hIbE<|iF}~&#N;rS zbgykm=c;M!&Cg;6#-1V}{dF5*5Y@E%t(S*RxeK{(j(Oe=(CPUq8~OvhDS7+B_6Dk$ zE|joArAQ3Dc&K;Ai>QP_Qz)wONvvA6N)-hJ^0ov{C?2FX6xj?_wmR0haV)h7%j+97 z$ikzoh#gjP!>=mHYo{}0D#A}VnfO`6JK)8=9Nx5pMhP<7DbI`fIeR5?j9lGVQEpO; z=yS=%j1>JYT{y8*egie&ND|uaPInj=!nS0LOa49g@m^4#0 zZdC_HwE~g7bBEZmQ>l;}NV#Hkt%k(|EKhv#xq!+`Dmc(_3Y9I03DN(VsD0q9z_LLes zwPDzH?6axuVKbFpc|nrES$Yyc!@Q;J-*jSRBDX+}2@EKG4>EGUfyAeYzS129of?{Ct=*iG9r zl{Q%|=Uz6-n0nhP1Dqv2t^h2nOKeDn%dVU}mJ>7BQ7e%sam&hwVNvq+L6vrIU!cG^ z{IMhAe>VWC;h+Bj0oSi#8HX_CG$RevkI&~W$9{j3u>pc?(}wi-NMCg*1)@s}lysUn&~o#X8cIF{%N+rMhy_?_3>p~1q)IDP_GzY*kb~H++-zk!5-&dGa$_A*-N*=w~ zXuixtLJ~Ae9B&`|k&%UX+8g$K?-rZj4nkjSl3~g&=k(YGkUl zi$L(b@#NkPT*bQcn1C@NFLoM*jMI#Q<6dW(McWmbK@E*y&R_F)z7tHFOXoRi_8^z; zSRQ!0xln)M$SH4m*3+-~PhVOH`NQ#0a_L*)5{_EsAO6z8$m8{;Q-kKKV3v}+cbnq< zkTLDcSEFVL7Tu5uI!I-92nq4~N!i%EWXTjQDz;kjt1ZSr-!j7xxxt-V zMfn+CP>Y+c?bil<me09AW)5OX;+ArLJCC&Zd|N18IGdv#ifU=+*Y#eOF z9MBE~nCNQ-E8?LXbuj9n)1Fq_d|Kdx)0Jb~&c41Y2+NF;`lbajwTzMe>E z1-?g00y2!hp_gNFEVX9z_9j~$Hzo4qdk`B2PvCd1(>KKhWv4S|be;B`kGyB)_#D8^@p|E4BFVJ+f~yK<6G>gDp-)bW{1pg*HiF&4+v{C>ejb`*R3|=bX=6 z{G`Lks;!!XIv4H~5dxkAM5P+tH?==yv%&Kq#2X=Wa11qFPml zV%ML(PYl7_2?ZPk@eLY)gly7&-`XfgrT)X)ab=X9zZrdKe>dz&tEH_bQfXb|XlSqU zn>!!M0wm|z*2pqPef}|ac0MHPG?3|7#Da|X{2xVk6{Ur(MqxPk;O-XO-95OwdvJI6 z;O_3h-Q5F&ySoI3wx@-<_q@4z&gwq@?6nuU;DeT-Q~jeSQw-A0`!~y${mZ6=a=^hf zncF10F)?y<_T(!nHELC7TQps|=nprtef?WU6mRmbVN9JPdwSzc zQFcStL}|~~S>LYyJbs_@!o{!E$Brv^GcfxwtEJ=g4bsN2(_-+BJrhWPL(xA|B_|uovWO*>VRr%2Y$W2?{l`bi|)*5a&vX5 zv)SVOuzq3x!kNBL9nW-6AqrLer^Cb=W%E`$@<+$GF)pOZaJ=Ba<2yerSXH6)q-V|I z{P11jsaZd!+M8CU&Hg?|g_3#ur$`VdUa=IBOQ&64o@Y+s7FjA6|Mqr!VN#XM*+w;~ zc&Tri(k5~a;1MCJ?Y>5TU_|L$=w18eo|+zl*oVX?;9&?{%HD-sbMlCE#5v$ zx7UN4O&l;iWz#yDN}r1PqI$Pai+`_lKJ)u;?6tK?&c2xz_w>3}M?2Im)iiU>XS=(9jcgY$dAZ76f4_aZ#Fv|+9-qtF#qa$qW|Q(;S7$iX zhTmAbyK>>}q)Dg!QM+ZU=!q6(ZGP8Hj5n^x*5W-*4VV&HeQEcFneLX%+O~N59f$XC zo3pgVk!qV8)ExLAh8z7jdZMKTS5}XbbXtn)FIzn>H?sTDt}CnUTGX}LHPQM)(zVTd z*y(krO`ble{WpBi6>}&5v{ZGrCM~(HYBo_a)}_oR#k&jNk6S9{Eh=lSi^WYQM@f{n zkP>4mCM~oqYh^h($FJvK=Rcn^*|LNGjE|b{_LuwJBKBP_@-*htJRj56NgjP?WbtaR zi~p~R`jku*{OSfYS$W*=sz#T%z`l|J!qA)*oA?;yqgrStezX0Y~AF7v_$XFk)l?$zRa*UxBY`W;_ea^G)5QwY5@_se?X_TD}> z9#(pLA%Bs7?o7&2_>0Ie{m1TSGbAW-;KH)Fb-$EJCJz>koSEZv>WJ%8atxSWx%3ZZ z=f@V6Mtr?;|LOHBGuA9_&}Uf6#czC$p$~cvu6S+Xr^mlnYxY=tmm|){uSt_sI&`(+ z|CkK$0KHc4)mJbC_Z){&2wjhQj=@tx=oZogV}XKlLrb*4>gwlrPA zPE&TwPE&Gnk+$pWR$2P@pr3b4j{oE2)1!_yxE%AVIvh3g+b+{4q)6SORovn2`rmJU zd)X)7f7-DgkM8y_-M!Pk;pNlY$Q1o%07XGGp@j=ejmZiO#bi5u@=jl16zkMVf)mEYQBZF(wx$(^}Zr@D}L;rpoKyM^wt9wbewMuZILX|5EfB&@V zbCuf1SD#%Tw?t&6#I+Yh7US>uOrbR@0C<5qL*D4Enk`_c5=J!Ew)ciuxd!$OWBf+ z$-cgB$M$V2^{xMNid*YnZ>k-+z27B#%;p$Hie7H8?@N&@yQgFr*?rxG zGiSQ5D^)!Am!GQTDIc+^!2X0cS3PX;e95NSb<*GN(70&7=<%Xe@AvNA?EIy_%l^`~ zs*&Pe*64YL-L9Ll_1@Pp-=r!uEqRpZ-Ipg=b#rg;IBJ$?F|6R|$nK&|OczbAOSK}_ z=jD<7%jC5y4oqomt~biQugvNT-z8qPZc3sAt2&&x9B--c6T0=ju;AF@R{f8RuaRw3 zCKu6a?Ym?N#`Zq&W6Wu{`@C#WZSC6O(`)7(c;&*hVsBq$-F?4sr4h?=Nw<=DwW?6rWo~Ut~2?^rWoi%0Y+ZZ=QpM_$XVe9?_yEXnb`>x@R+^ zOjwqxv;3A!x_H@pBfGp&i=&kN`f0@6 zn*9pDTik!{@%53r)6R?g^R+4^r?$M^>)*UKW|Ps0cc&fz28|p#<{()+=#7GojuB;{Y2P~>J z>(EYHk7CdU)NMDStG)?k;mv5$Z9yM%D>dpXHTKyyYQ)9u#Cq=zbSrkEoV*K7zuo9t z?m<;$FWUV3P^8|EI@*D-=6_+$oNp`fqC7hbz>` zJ6DO#iEHRLT}L(V2HKG~QFOV5y8dl+rS722dKXQCdtuG*Va?n7Va<&P#Ny~f)axIi zoAnswuqS9bJVjso8LHyX(aIMn;=T-PehF({y$WltdSY|%HTpGgP)&b}cF;Q%?cSrV z@c~`ok0{fBLKEwASQFkzJ^v=Gx%e%y*!LakmH$IG`FoW8e?Zgf3;HTwQ5EWY7(%kvM)6#t?T|AjT*#Eyuk z9*Cbi%!{yRqL7R2QBg07hHh+hls#ggX&e)Mxmc)jMxad+n+EX@L-fZu3~_hj63dhE z&}@#6{FzKtqY|R+k_h>Bo~TPFL6ND4SGAQ@#fJoHbD;twn(ik0d_Tra<4VL+no0MY*LOn)&t7 zk7|IbYeTdR8lfoD7H!1LwH}DF+8{Iq2cu6j1XYZoVNG}=@pM>NbAC9n z+dTs1@{wpJjzZsOG^!S3&{i6YBHuXFDGjx&GK)}T zUyM4@5^7v{Bl&(QHRjebVt;Hos*Nkq&RL0K#46OCR->!C24%^$Xfj*$@z#YkzlAlg z*M~LNHxTQ?8`0TKC}(a)Gh_?;_FGZa+J?61b`%+RppLyWtO;)EVA5$Z5 zKOr{9pQ7LR4AtD{Xh*z2(fK9ndauxx^eD5uMic){SQFkzy?GnfTz^L_j=V>`?gP4+ zA5jkdghqcxzI`sLV&9_8_#KKk{|jrv@!?*6AJ$y?f!G}Qf`0W^RMUP$JMbqIZGT2x z{TFnFenpw?H#9MS4{O33sb_zLH5dLQ7JL6fz2a|llm0>3?_V@6|BDe3&sP?|yjRil zM(dCYZGI@LoOgKLL z=L8g}`w5Bd=|m{DB}Tn43A!;!QFc#;rcrYAWmBNakrHj;EH5gV4{ecS<^ zr7MXtR;jQi93TE!>9FQv8DhJ)EQ*!oQ1h?2_;|kxC|fBsRVt#-UkO#}%4nli32Vaf zp&wTbYtB_8mbSy}`A_?oDC*FxK@Hj0XMQ0J+OE_pqaqJCJz@e$$gHwbIaG$gj$ z8lhO!81>jD=(;yW*|-^+a?R1_Xn`t8OA2gwBk{2n1^P~FVt1kq%FS)j%xi~!WP4Oy zG}`(dP?YY7I%_9%2|80F!|h?-cA>`K=t``Pc0;?qJBryoP!H>gu0t=Bb$X*I-Uq$x ziz;rvuqM2b@cqM@s{@GL!GS2(4ni}1F#175P_-M1w#G0NMTVnJKLTB>kzozDM}&Dk zDy+FUnpo`{gLdUu6jR2b?r+ew8jrH-1T+OEqE9^uRrJYWO?V^mWJ*{QIhEM$nuc=O zbTkuYpzl2sRr6VBE6zrdcMj?lbJ2-;VNEza%!B!1&6x$nYWqU8ixwf@BOi5-CFmM2 zMOl6snw-ngCs{#(3a5wtxRL^KcNMWdxf)lt-ExU?x;x#mVuA^^$168G) zX!G7ek@7a`D0jk|@J90C-LS^pBlg?xqgwm`?YM_1dOkwk>EYgeq(PIUk!??1_VVd0gc8iBa~AkETTe^pz5#%9jXj z%ETz5CJA%G?cp9K4Rg*WBQ`sdqhFi?l}U-VXDSp;Q=_hs23@YSD3hh5z=Yewe@ah* zx|f02p2~<~t3=Je8RFxkGo$R51x>@O=*wh7l|4J!L^-Gt;r7t)b5i4O0YKgXKD-^0V>fCM6C2LE847Z2*+>Qc&uRXCkt(&`%qb_L5c154P8>+{zz1T@noq8~U3RolsEt4~2ucq;02)6m779@d20!#tZ2)?AoL ztoF`AyJ9wq$#YQmn~SdHJd{=DqshMjeX511qAdz*!W)Uli^H08ONiahr6`v!LoxHIa{q^{&V0mOeo_;VGKl&(Jq}j;i7dw0T~l zNbw4_@L^5359Nc`Va=I0#D3dbREyrB9s3?dj}NFDe?(XA6Uv;Q(IokX0w3N;ef*XJ zd*?f1aq@qtH-C?A-VZ28eL>UZEBg9BqAL9p+N?jLNcamiI@})a?XT3xo4*m8qranH z{|Bnsf1(}s7mAL5qptH0y5j$$%=8~W=a{(Sw**Jo6JAOAC}GdlsNo9^MGIfBHhTDi z=`q3=433GWT`cr9B2X2HjW&H86cKU5o^XA*=kdaxOYw=#z69u3CPXzQ5!(KVQM68i zx@uB%1(KmmlN?R-6yZlFypnp7GOUSAMJ#rwM!hTzx(R7f_DP4Pd3y8}GoZ?w5iQ?b zgU?6F6xM|6!#&6x*0?OhW_woji?X2_mmO`794MOPL|r}?x}3RDCe1^E3D<}Jn3n=| zHy^P*nIFZL0;uN~L^rAs%C3dcG$?|;bWv2Q2Xj?Z%QLPE;f=$t-X@)XJ z^ROlyAO1;;u;zSAV!OK)ish|QPi%v(Pg|5N+M%h`9(`VoDrE<>Q96b-;rP%GJB2l_ zGqK#!1`v%{M3 zM&jk1u;%hyVz+-D%GLAHOkIF}z(Q1Q7NM=S7)7BasM9V*7h_pi6OIq_ba_~Feg(1G zvl8v{RVXH{M%{M}x)y6uR<>yJtwW!3J*uc1!kX|#;?c&i=Ikb7w_`KPC0o##t>}Ah zL)COU+6p^R5-43E_cnD?L!)US} zL7(U-H7Xn*_Wdzx#O>q6`uGWS8&9H~dkW2n)95>$K~>M8EqNA2mUF1%M}{@wjpUp2 zVa@dm#QyL_RO>FGop~9>&?~6*Rdltkp)7VCO~xDOW8VyG!tr5W-U@53+$Pos?x0(J z7v;2jXa?R#-{t|T>JQNtdW0g~W7IL9gf-!fM>Ws<-H)z6)!@@nIjm4{OeSAl5rSqFeF_<@nENdVPbw>9?rVcW86}4@L6t zDbV5gaG!smK;HjCY)*egzx78{3x7g8=4TY$e?i^oS9E26Lz&}uG>QM9#)mgjAO57q z-u{bNocJ5{rhm}Q{TJoP|H98aoyG6rS3C8h5UEm8(PoK;B0=;pCtM%yO^h(-MoeOJ zBo_L05vXRxMmsbPiVks6*N%s-SbUT+0h%}o!<_I)>Q$mJ=SpH?aWDz$HA&G;ONMe# zax`sIps$`1RpC@<)1^icD@|Avt`GMtZCGKarUt0Tn=KfD<|ruxzLTzjk0$hG|lp&SNTxo$&WU90TiNOSQD-f zcfU|rbEYt{*;WMo!lJ0g7DL;;IEuz4P?sx-E=MVpNlH^-!u8=ll%YV~DNAfmltZz( zJnDHB(2Z0myHrF|zY_XVl~HA_f;K@_YDBm`^xJCGxEs}p<MzOXD>gi3<4Qht6U2`-wTA(l75>@(E zXk)bwYr^fJpSKBXF196>``V#d*&h95jjDeKw5>X#sM-m2fzIescR?AwYgiL*5C6DZ zSQFWu*zW3qVp&hr6MCWR-5X`|K4>cTMW3f1sucaviUDCwxIOfPfnm*=LBw+VU^I({ zpdULFRgYn48xKcOegx{ABhe)pMS%>rhxs^~0)J-=u{t>x?dEYP<{Q+b#-r;p0cC@U zXi85)pLH^-gj1-o;f=(*snqD3(}>;C=_ohMKr?$L`eCzBb)1d1&Kwja=AzCt4_(~( zVNJL_j9(DeTwO@44lP2vb}@<>OHdD9imu%kVkDZbVUV6Y4aZ(M8`9)`Z)`JlPu7L~bKiySJlVwgbh) zov8clLf3pZ%1V3C)|~%_*zf)p)$;GqPW&H=zTcy6@dLU_Ur^@z ziYDcc=%f4;)`Z)`KKwbXIr|H--tjBC#lN96zoY5-2l^&|qN?y0+FXC5NcIl}I@})a z)4vqRd;f)>cut8w!cRO~qJ+;ch#EdWI$HRAx9F%F#z0plCd%xw&?JhW$A?E!?_<+r zZ^a=N$K#^j7!Tc?_$WstK+`E9`nriwl}wB_a}pHslZH9r_;9b2g*n%g6Pv>+(A$)# zW~M?rBsB`126e5p=!&L8nK3<@*crl_@JQ-q#<1qHBo+rUp%|n@)bszstB5> zMZ=o#Na|6su;y%WVzILX>Ln%7nNlcwl}6LF4EhRXQROa&Hd%QJL^wY5=L!_Kdx}_| zu83x9CG-m_qZ(5MZMUi@8dXDGwmQ1(HBct5NsS4|hksv-8g;uiu{~Y~#iqKb=hj0v zqCU#b4baqUh`wYaR9PCMjo&1!3AcxS(=@EP-i%ltX^v)H3-mKvq8i!?t!|B?b{o{i z+M>(Y4rQG7VNJL_{7W6yTr9jT(bK_gGZP$5CLz8;NfW1^WJYVt0B1%54+TES!XX%w$yE zr=V>#6-BvesB=t5mv{y>GTa{K!%S-Y?ODX?#B8*i=Af827xl<_=sM3wS$_eVQVY>% zS%fOV;;<&Xk$Af#thuq2*d1Aha{Y2NvsR!Vx)N1~RcLFkMp1kXYPl9&92?ez+rzwC z7uH-^Ppl4ZK)Yrmis_qB588~b?G}_ZwxTJ#4Sl-psABC1Yr-3e=R3oii@S*3-rXoy z?m;toFZzD_P_^2Rw#oq%1rDN4eF$B&!(mOhJUt#A??uv`depm~aAh?~~}7 zokCghG@3kT&?k4O#M!VW+=t@9xv=I;B(d9e9_6A7XvSVd-~AG*#+T8SyMiL;Rn$qY zQQ*QG$&c45Fn4Yc`x7@&ZN7ze-fa}4?x5~+7hU~(C`;c*ll1}m1P`fE;r6g^A5kN2 zJ|@;jpP*a+6y@w^XofvU-{A$SIxo={e}y8GM;-TdSQFkz`Zr=eu0o}?kD5rcy)Bi{Gt$spP z^=GsNenFAuSJcse3v0p~$tS;uHIaW1`(1ycTJ{&(34f#L^AGCg|Dvn-A3x{Fyy8!O zn=Qhmh(hK?)UYR9ANE1Cu*XFYU$8w!_<}_-QI3m+rbh(&#<7uabbvNzTog&;QJ};1 z;XcNvK;BJ2Y)&Rbzc~@A`H9hvN`j(mQq&ESp(~vnWwsP(5~ifahgVYXQc+`XrY080 z(xBdu7TxT0D2Jy<(=h}3IvG)wkZ3byLJ==>SQD-f=d*-0*Rm3uL)p-;&5mkD4zz=F zqG+EBblD_6<3qph9M)XxLM#t=MPs|6pV1xFkRE8;_e4>v7wV$D(Pij^ zGIrmvCLACBMZd7-a(`mGe*lWr15r;Mgl@oKlx>EfsWudS!C|P<4o4efL|7A!5B+px zSaW_9vD`Bn&GIqmCyqtccO2Rl21VuZsPj!gmvSQVXAi@gaD4cOlf#;`Q;6-3sVJ6A zLv5y`>p25u)0t>0%tD`QHmYQED6rv;#OJvb=zH^s-KqH~w=O`lU?KX^i%@l2jJDwt z6lIs9&b|y?qUF@caD15eE2#0eRuZe@tI%#-jbiQ^)FalS>ts>ZTZg9Pdi0q$po+gS ztO;)<-fRkMu5TuGhqs_yw-wFIZRm$=N2Pb5t+f+Hv0bP$?nW1TPgoO<5A$+wSaW$F zu{y9H?dk(4rX55*@DRE-hf!8Pf~L?>^l6Wwig`S&32!8xod|0#oFsO8PN7_J8qK6L z==(ZUEzhE@d=5qaNYts$ql`5d2}(XIeUp%?YxY3$rTjiucGdC4PDdgDAf%# zxo@IRc8dZPjt~3!HU;AT9b$d@F1oGvP%gZWX3PWh-5#Q9^ayR)$0%|l7 z{P2t#bNe~5KmG#Mrk7~vzCtn5qwf3~UA;FbOT9&tKLe_#SC-8Bgx0H!klvv#C~UNR7>Na9Um7(?|7)2#Yd+Spv;pHP4Yw( z`0z;To5U2@`$>q!nWU(_J8{Mot zD2L@m(;*-F+WAoxFMw7SL=m@8SQD-f_o{GMbF~PuIan0^nqsJ?7e_m&1d4VgQP(Ji zu5f9T>C2#rRW_^%kEEWL3u`WxCl>oEpkAraO|FQteUxJFnLS(Di8s)c@OZB!HLpzU23Me};7E7nJsrvb_o4bg~3VNJLY)q}=i&6y^| zVtZ56i<+Su+Z<(&7HAr`L|?8Is+_ISCTT-~2)Bp+*p>o!rya38*&fYijecGSRHHhg z?a~QFgU+Z+cR`o6E6Rl3s4?O8@Nc_Qqi*&fwnuxS*w730?B3{x^+DOOFPb|2&=>EI zD$@Y8aR-Jq;r7seP*`(yFtI!|1kKu^=%)`uHF!AMb|X;K9ErNfD0JyZql_36)`Z)` zKOY;`TpCAg_Zbwc#-pAx0bT!zC|gfLQ*|=>0#i_>nTj_0w6G@J9{S1juqJW_vD`fq z&9YhOC(K6GXAautb5T^9hdS?kbSV~~jIuDS3AcxTuqdo?i;3;_B`6jzMLljAx*p3> zHd%qD{7UpWSD{L}ngSc%NPJpDfxf$zm_MD3a?3h2^Vg#vwETFxk zCEQAl47Z1Qw~ZQqb33s*wgc^kohat)LOpypx{iBL*4>My#6ILZ)uW1cAgl>*Bwim3 zYpxw4c83n5v`5g)IEsGoF;wl3qpf)YMbVR}Gn_&faXPFCw}*LgCak&Sh}HhHXjh#> zF*OqPfb-~DUqD&yBASAi(5JbKD#n$tCcKe&dNr&$e~sAfzK(MF4Kx#PqVIDHRg2qb zE8Rhn?=I?;_s~VTAJ&B1!#sQt*0_hnYR4nAiyxyfPf+)Kimu5slog(%$@Kz#(w7vd zaC_KKuP6}rJh49Y8r_yRC>Oj%Gx{C+uJ2Jb{D8L1M-#o>6-|Bi^QlZCqbJpDT-9d zP)AK3=7dL*k5YsmoU`5ds#88xl)PP9H@+bbrn?8s-hiO4Mp4PsH@jNSEwe+bhXgL ztR2>bH&V~)gf$oH5{td{P_L+uZc+o3{TiZa*$92*#;Ec)L7S>6ifGNkns9r#N6o{U zb1jI?&X(wxv_dt$HQHWnP&8|cTD3!$yFJR}ngSDU5C6FX1?qlBVtcw1ifx@yFYJPD zOjnfMyP;{+9evpzsB-i~o46M>BHSMOLvL!_?LNfvL|-(U`k|lOAJxbKXgd!?QGXEX zQiIWD8GV=e>OX;xj2W| z?wyNb7c+e1HI7S^0wPAqq=K(llu`thq! z^%FTPx%-e^4=y9~`PoS8667{fC=sKK6S?3Iz;tst$iz@E9uqM2b@R4E7)$_#e;02Ux zFQS=#3H_kUsM=jYTjMHZ?^}qv|2=fA z?xU>w08N31=u$s8h1_><=7<3 zvS3OyX;Pt&o;s`v*N1(QCagK1mRRpjhi+MVloK5~zCbBU@_CbW4oqez(rb(E}O zO?V~wFk4vTvJ?C5IZ!RmiFRBr6g_jJZjuLG`MfA|j;c%vwAo6cNK}d%9j*`et~53BRvBV*tStHs|rWxvL&CwNXfii7NG%;F*HQ|la z)7D|l`8LF2Pg~T>+o7A-9%WyRrbP$zl{%uz*9mRP&M2aG32Vaf;U0DkYtD8fHaohb zU)%$g>4~;yFBDCCqpr{gU9P?;ll7y(gyX}1>Q8~XH-Oll8i-=+Ak+&6qZ>U0Ww)Vd z8V*BWW;m+sBhV%qNsS1{hkieb8h2|nu{=Ho&Bn3l=Zr%&!l3Om9!0$gs7p>nmw6J( z_>;q$aD4dJQ^K0-Q;F^2X(-lBM?G@}x*;=B>RD)N%|>5z4yufE(Z-$^)`a6jznmY| zTwXvd4=hBpdJ+1mi%|_+g0{_46xElZF0>q7+7&2ct_*9!@!_AY3TrN`CboOlpjfdM z^(2d~?>dw%*Q2St0e!xWs8Vf08+CJ76OIr4XiHdgb}O;mxed*d?dZ)8RK0ehZMqAE z+KoE*9(2j}QXs?eVLtDpz~9?XtWF<5yY(Q7g@;g&IgGB`5tNONqA7a}efHz15}%;P zhBp!)PEw<9pCWe0Povy)2F+ZDe#BW+ozJ1I7m1?OdDK}hpo@PotO>`5d2=bOxqg{g z9l3&b-BlE`uAv@!9i6^`vi40h#crX`cpFumJ7GV9v~wS0@R$~!dq z-=k0U0adh*VNG}=@%U3%bM7;-+xZR3rQf0%{~h{X|3lU6d$j5Y6nVa&PX3hw7v4yI z^CJc3{!hgI^v|fa{epJkuPDa;hPwOj=oC47El)bRPv(NNWoj^yyH?N*~sQN0QGogf$m468pUp)yhn0Cuc^{KMU$sSDb$TiqbpYiWzMo_l9Z#sheuK$%Tr+QR3H{773$3u(ao!b za#UqBU8d=4fs7>+7JJT^H@JdMG;9M_s1@ zy5bE{W@>~cZsV{fypi%v!kVj1iN&F2sMj_}H@yYQ!7b6WYlXf>Yg9$rpiSQvMMS%> zCfpwGdHb;Dk|s9$I-p#XKQztzqpvssRo;PUQw%~8WpG#%ZV&fhNLb^B5}WPA&@UQ}YTO94 zJw~EvG75G1(dcrHL78+c1t#1c{^K|b)LlbtPmV{iWdiE?6VZ*DgtF^oG!3SpFFh4i zwrOY+PNznM+e5#bL5;gPlUN>`g=WKS^t0!n8a@|o$9X8~&PQEh0lG{JQN~*o)`Z)` z`^90+wI#&%&{7n38R{9!(G6aKvi(XlHCLf8vKm!}HE1K&hBe{#&@XIQb7>v1+`k^p zstxF;Y(zC+6WZ3BQB>Q4y5LrHX||z^u|2E_w}*eSBdj^Uli2Rwg<|<`)D!oh>$4YS zi+yM+?MI*Y0IHM+(MCBG)`Z(bKRg`PxFf`J$5AwkkD(uT997Q~Xq%iwQQ;KoT&K|` zJwt&Ew}<)UDDZdB601|^&~AxDvEV%F(HGEly@;~mB{XF&qtA8)RidlZ*ziW;{WWUz zt?R_@*bS5$Z=#uV3;pohs5;$2TlX%ClJ`(&zK<^6gRmyt9_IDKu;$t$Vs-d2TKfdW z%%`Y_JVV$1Im%iu&=h@%KEo?iv3*z*-blQB9oAfaL+tjyMY;MNnyK&65BPwp%}2D= zKA|Y|8Fkuk(8c&RtO>V=dHP*gbN+wCYR~s*m;Zob(ihZyzoKjLBg)D@p~?3%`jo$* ziu!9<6W&NX`Yo(E`#Z7Q@dwH!f1)veq3`)Os;2*-t?)03-2cUh$f}cx|H3zLa^aQa z=O|Q|dr`v|oQf8{U~Ba71q)(?FBlUOb+=gP8b+Wj8yijbIOr3_Wr+&chkYN98gV;5 zu|A#v-Nu9{=O#ikA~E_-Nl?{Gine4j6j_p^j-P@W7hXxeNg38$Petqxr$)6d4ceJ$ zQ4CFoTBk=>D+9`68PQ~v=woLJYr^$mUuF(#u4EzB2eP7Doekx*>}Uq&K;I@Os_MDW z7RrqxT^`gi^M*CymE^O0Va6mO^i9j7QWenVRw$BJq(F!3!+owqfxKUt*qp9{ zerr`!3#*|WQyoS38mJr9L|3*J${e-PB(6h^4{xMC)TPGWu173R)JMIk0lK*jQI2ec zrgLNT^_rk6)f8=(W+)Og4{O5l;oh_eYi_h8Hb+{aU)LJdtTt$ewnfpQ9qQWc(G}Au zWd}5II)*jjjnu17Va=7!#NuEV)N8t;o7N5Gpzdhe_CQ~~C#u4|(5CB+B37TUCLAB` zS>LecVn1TDw?Fz715iyKh_>G#6s-oMt}+B&{-G#S4?`1ecvutONIf1A)|?wjEOw1T zy>v9X@ncZ-9*d^gIP}V($}=8q@(C!!#IPnDAMXC7u;$EUVzX@u`h`lz=Y$&f0#pox-*y9o|uPX^L*6v7N8rs5M`G|XzDLUUup@etV_`* zSVoNq$A^BqoEmpy1+hH363zNm=x41)HEa#q4r@`=v8ao$LnqgxjJqMM3CD+jwK1%@ zx{25x+>B!F7Sz+Xq8qdgWxMTYYV1H?cqgj#yU@nk9oB^7LqFdW)?D06Ecfk0vvNQB z$p=vNKZv%~Arw^)qb_g+UFxGKqaO=v!tvoB9}jCHPY~N(Cs8arg?hqibiL1@Z0^uh zJc~ZhIaDbk(Tek7O*lUEg9~BJnTy17`z17sE~6iN1yzr$Xd7QcQT{sWoHx)Vxk-Tx z$A|fNivoY=HnBQ+2kqv&DCXZoJ?cKXE)P&Pc!;L-BlKAxqe}RM8XMk7yn9NGzWI#U z9es{+!wWRCU!ouO3ROpsw$5u5CElRU^cG#*cVSI9K8$}K)?EEStPXudyY>@`8J|%P z{svvUZ&B9#4o#8&p-=xks)!%Ln(#*A#h0+=(pO@)??;rYenKRBsl>l|pgcP{&O7c@8 z3e4Ta#QtOwR9ljwou3TF=;Wxora;#qCCW0X&}2)EK4BVaR5(BEyR_7ZTj_}PvGnLR zWI#D5Bbwn7eaB3w>Sjh;A`6PlSy9K!7S@DUlCQIeHP>Yj+^TWO<6xLiWOsw}8LARqDpf|AuL_!!RnbSO7S@FG!#=DY z)|{DSR8MMdSiQZb2Q2k9nf^@h`w$o zR3$s3&D;e={H|e5I6mC#Zeh*!?!@MB5A?Pts+qme4(S~uqQkKcNBM8-`|{7OXg`!0 z`=g0HAZ!UQq+SjTTP_bG76%5SUOfce)S)N`4nxysIQnWMP!$@9Hti@BF-M0j;q-7% z$Am2x#uA%7q}qwKr^O}&liOKw7yWi#6N zTf&-fdgwP>!va=lvs-A? zZS=YCph|w10vp~)d~=Tieg8hOJN*FVwufjIK0-g{F{Op^?Yx^h48h@cF{5Se^|DcNXZ&(xFNId^9MxuE7qKLAN&+m;AKEE<*`26H(;q(2X zhtIc)fwoFa6a`|TP91?RTI?_<+#cp}oG|BHTw=8=9@?exQA|jHx_3f!%@U!km>5l- zBWqOsF!W$m zt(|4foyFb?qJ8+vDL9{ zcWm1o+jhsc-LY+VY}*}UY|nhZeouY>T<6qXy;t2et4^LZGf#4^79;itilbUn0`0Vt zCSa(CE{i5zIrK5hhc@B%u+LR!bEyKc-dhpfib^OaS4PvX3i_5+QB|pi zHh*;#scWE)UNf`_JCaXog*NAE6Z@TYP%W*Cc6>b)z3ZcH)&O0FhA8tiLX*5PdeJ1b z373a`&@{Ah&4~53=I9o-KsmN0n(nR8H)@TlTpP4G+M-C(jsZGc9`2)NfV|tD*qrEq zep5$O^E#m&*%?KbE~x8wMOUgD%B=q67>*?%&cR#VVdnTo2wG_)D;(?%d-$=ibZImEDmkLj?}{?p^aNgEVeI0 zy=XbQu`5vaSc#_bD)i-6qsqAkZ4%1>5iSq?X)OcX-F3wBY5a+IA;V)I5c{$Z2%x9m?2eLYr`V_!nnG zo6F~j?Y{FUR$V|nN^=cu#P!f7+#dSrjnL-&O=7wG7Mf+Z z(NDO8s?S}t&F`V8bRTuz2k25fL>cu_XcKM^|L}2WbLI)L-ToBC;%BJMb96mkpltFI zP5D>ob9z)sUo*gl9f{9x7@+UHC3Yv@q1^Hw&HNAOM}0)q^%L3#pHYr+t7G4x-S9n%IX|Et{v*1MKcTGqGnx{=pwILxs`$T!HepBN&F`Vj z^*@N+p+8Zs{R_>EztIo=2bKO8ZO#AqH#?%Jh#J0Apff}vbFrg_KH>T>FQbJ%m!pRt z*dGyoU{wqhQ)8kY5DQ)F*eI*TK~pd;`ZV!S#fTsJge{3@k)h9p1jKH4LX^uBp_!N% zeV-(#S|mkVDH)1<$x)|Ffi7yw&?a0T=25E9=1gj0wIdDM#c5HDONY8=dUQ=Ppi~k~ zu8ioDW@3N}*N6R_nE~Q{7GixWE4nS&P%g-hW^@kpU2~#pm~}g|=vgh**(ZlT=aB<5UV=!iMCl%Aw1ZD&g_|s^RhKYT@zJ z>L>=*K;5P$x@xsh7OIUVZ5{M6>V_`i_OQ?Dg)SHC6YD(<&@FF>a#ABSeH){1(F9fH zrfBmuLy@XE>S!%Oo3J7IxMgT_wiU78(Hhl~HfYDSMbWDr>ZTfNrskb9Sn;RpE#gS2{*NsLua}3I% zW6`uX=xdEbRct(3IRQo7iJ?t6J>08Fq0QCF#OA;h^sA?$nl=sX!09O3&Olv#Cb~ki zP^O!WCgz;bChSN(pBvg-oJTD7&PTms0lG;GQTAJersZPvm6xE(zZ7k%WhkOA4{gHf z;U2FDZO*MEHal0LU$Pq2_%&#ISrpCIqOPzGUGDWLlW#yHHikCgJX8-hg*K-*6N_zI zP%qqyZp=26-M6D@v;%$Fov3o`LYsIu14KAI^v696aCi0+%M<(1Y}${0?g3OI52EdS z2u1zFs7oC|m*ptR1jiU-!s+4P9%qcYd4kv;If-KZDb%x0qZ{f_b~uBk_F44B&Y{Y9 z9&OwUp-nhFw7(eIT)jjr4_-#I<_h|0S5XbRhPLf>6g6(3E_@SRx?3n?-41QS>EWN> z32iRjCANF-p;&nz_2dWW`aMM1>JgeMkJ0CUf-3b>w9%i1HsSQpPo9T1=Ux!YT`$or zeT9C!N7ef^+GcN1RD6p%&pUL<-=mE3A+!mnhkx)fv~iz^?Y7S-7JWfI_A7MVzed^k z8#Lva2gF zOYj$CWH>#{yT2LZZ~jB9j{b{w{eR(eXSRrT!9*VxB|PsC6=j`hXo^QipD_YeycnTP z*pYZ0GqkxDi`X5Ejnc+JGd(W)LGe(vi;uQOB#I&lP^V9bE>@z@CY&DTMdHxrQW9dd zFDcrU$xuv5j=FyebgfdNteOfL~GHs{k5yImPjE|X{`WJKRP z6RPH!(N@fYB5zjIDYBu9l0CEur-yl%BeZcjiPiR8Xcy&1Ve+8vkr!R#d??H3N0YMv z`XmJzpu*{4KNVttxL251pDcoIb5WG@i=i1+9DSD(s2Y?+Te=jAY^6~rEW;QVb|l}I zWsJF1j@Tb9k7|QLJG%mk;T2JLtc0#kWt1hVpvhDfeY|R+O*lR5>*}G+^%}(bP)&5U z7Rnj5(G0GGzFl2ZHS3`*QXfTz2B>2<3~j=W|#eqR$*tD2&n(hS9b=BQh@ zKv%UT%7U%Xq-l*lqD^QM4iEdZZD@0$9kJf6(JgC_a$*NGeLA9V-U(Hu&S>*?L6Nd6 z>Zsj9o3JDKsC#I0rU$X#-V@d0UT95k6g~T(ZqgTB`F<#K^+%I*00Vs3k@`H40ruV? zVsUCP>McXi%^!+#^e{AChof&W0#%ujXtRw%k!UnybT~ZR`!S4>x5pBjV+Q?(aj52u zM>~81icS+z*PVo}#AKA2r=W>HHM9vkQg5b(HrJ;Ui^DTeubqi*#w?UWW~0$_(AS)c zs^~nl8Rnyivmmqyr-ysFFtoX{h}i63jDFP;R8yCt9k2{Vo8_phtw2|BCCap`(8O3B z+JqgcXKO;63zk^yS&Mr4I&>4)qwKo@O^c1_D{VrRZ!_AITTn#X8rp=@!#&y-+ML}^ zYN5?9@J_tx?KBECfm;d6HX8R`2Yjd{e#5z)FBjG52Idi1l{PP zD7zg))9^U@GAB@FKZ!QcDaMF!dgu?Q8RKp{VtM=wnvG}C&pC%`#Cf!xE}*D)5p~H+ z=rUhM8F?kN38#mDb2YTNagEp>zK&wu4b(Glq8oAxW&7J`YTZF!^e!rS4{eOwEjrG1Gq=Bv;qoF4v}4{a{KCboOt zpjh!1^`v*``o2fm@&lU6AJOOggeui%w9&qVHsSQpkG~3S&VEfScYcFr$+zgoeTS;o z_h_5`fTF^WsB`~>F4@m0#V?^vI6eIRUqhSIzY*K5zoS_A2kJ3@qU-h-%0_>qDf