From d5d1a1320b25fa9f25d0217637726c7f36f7fc2e Mon Sep 17 00:00:00 2001 From: Ashvin Date: Fri, 23 Feb 2024 10:07:13 -0800 Subject: [PATCH 1/3] Avoid unnecessary Iceberg datafile to onedatafile conversions (#330) This is a performance optimization change and extends the improvements added to DeltaClient to IcebergClient. The current code in the Iceberg client generates unnecessary objects when computing the file diff to find new and removed files. The process first converts all table format data files of the current snapshot to OneDataFiles, uses OneDataFiles to compute the diff, and then converts the resulting OneDataFiles collection back to table format data file objects for writing. There is an unnecessary round trip here. For large tables with thousands of data files in a snapshot, this results in the creation of a large number of objects unnecessarily. This change optimizes this process by skipping the unnecessary conversions. This optimization does not change the behavior of the translation. This change does not break backward compatibility and is already covered by existing tests. --- .../onetable/model/storage/DataFilesDiff.java | 103 +++++++++++++++ .../model/storage/OneDataFilesDiff.java | 38 ++---- .../model/storage/TestDataFilesDiff.java | 119 ++++++++++++++++++ .../delta/DeltaDataFileUpdatesExtractor.java | 17 +-- .../iceberg/IcebergDataFileExtractor.java | 87 ------------- .../iceberg/IcebergDataFileUpdatesSync.java | 65 +++++----- .../io/onetable/iceberg/TestIcebergSync.java | 2 +- 7 files changed, 273 insertions(+), 158 deletions(-) create mode 100644 api/src/main/java/io/onetable/model/storage/DataFilesDiff.java create mode 100644 api/src/test/java/io/onetable/model/storage/TestDataFilesDiff.java diff --git a/api/src/main/java/io/onetable/model/storage/DataFilesDiff.java b/api/src/main/java/io/onetable/model/storage/DataFilesDiff.java new file mode 100644 index 00000000..efcf45b2 --- /dev/null +++ b/api/src/main/java/io/onetable/model/storage/DataFilesDiff.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.onetable.model.storage; + +import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; + +import lombok.Data; +import lombok.Singular; +import lombok.experimental.SuperBuilder; + +/** + * Holds the collection of files that represent the difference between two states/commits/snapshots + * of a table with respect to the data files. Between any two states of a table, the newer/latest + * state may contain new files not present in the older state and may have removed files that were + * present in the older state. In most cases the data files included in the newer state are derived + * from a new commit in a source table format that has not been applied to a target table format + * yet. Hence, the collection of data files in the newer state are typically {@link OneDataFile}s, + * whereas the files in the older state are represented using a generic type P which can be a data + * file type in specific to the target table format. + * + * @param the type of the files in the latest state + * @param

the type of the files in the target table format + */ +@Data +@SuperBuilder +public class DataFilesDiff { + @Singular("fileAdded") + private Set filesAdded; + + @Singular("fileRemoved") + private Set

filesRemoved; + + /** + * Compares the latest files with the previous files and identifies the files that are new, i.e. + * are present in latest files buy not present in the previously known files, and the files that + * are removed, i.e. present in the previously known files but not present in the latest files. + * + * @param latestFiles a map of file path and file object representing files in the latest snapshot + * of a table + * @param previousFiles a map of file path and file object representing files in a previously + * synced snapshot of a table. + * @param

the type of the previous files + * @return the diff of the files + */ + public static DataFilesDiff findNewAndRemovedFiles( + Map latestFiles, Map previousFiles) { + Set newFiles = new HashSet<>(); + Map removedFiles = new HashMap<>(previousFiles); + + // if a file in latest files is also present in previous files, then it is neither new nor + // removed. + latestFiles.forEach( + (key, value) -> { + boolean notAKnownFile = removedFiles.remove(key) == null; + if (notAKnownFile) { + newFiles.add(value); + } + }); + return DataFilesDiff.builder() + .filesAdded(newFiles) + .filesRemoved(removedFiles.values()) + .build(); + } + + /** + * This method wraps the {@link #findNewAndRemovedFiles(Map, Map)} method, to compare the latest + * file groups with the previous files and identifies the files that are new, i.e. are present in + * latest files buy not present in the previously known files, and the files that are removed, + * i.e. present in the previously known files but not present in the latest files. + * + * @param latestFileGroups a list of file groups representing the latest snapshot of a table + * @param previousFiles a map of file path and file object representing files in a previously + * synced snapshot of a table + * @param

the type of the previous files + * @return the set of files that are added + */ + public static

DataFilesDiff findNewAndRemovedFiles( + List latestFileGroups, Map previousFiles) { + Map latestFiles = + latestFileGroups.stream() + .flatMap(group -> group.getFiles().stream()) + .collect(Collectors.toMap(OneDataFile::getPhysicalPath, Function.identity())); + return findNewAndRemovedFiles(latestFiles, previousFiles); + } +} diff --git a/api/src/main/java/io/onetable/model/storage/OneDataFilesDiff.java b/api/src/main/java/io/onetable/model/storage/OneDataFilesDiff.java index dc516d93..394a6904 100644 --- a/api/src/main/java/io/onetable/model/storage/OneDataFilesDiff.java +++ b/api/src/main/java/io/onetable/model/storage/OneDataFilesDiff.java @@ -18,27 +18,20 @@ package io.onetable.model.storage; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import lombok.Builder; -import lombok.Singular; +import lombok.EqualsAndHashCode; import lombok.Value; +import lombok.experimental.SuperBuilder; /** Container for holding the list of files added and files removed between source and target. */ @Value -@Builder -public class OneDataFilesDiff { - @Singular("fileAdded") - Set filesAdded; - - @Singular("fileRemoved") - Set filesRemoved; +@EqualsAndHashCode(callSuper = true) +@SuperBuilder +public class OneDataFilesDiff extends DataFilesDiff { /** * Creates a OneDataFilesDiff from the list of files in the target table and the list of files in @@ -52,19 +45,14 @@ public static OneDataFilesDiff from(List source, List Map targetPaths = target.stream() .collect(Collectors.toMap(OneDataFile::getPhysicalPath, Function.identity())); - // Any files in the source that are not in the target are added - Set addedFiles = + Map sourcePaths = source.stream() - .map( - file -> { - OneDataFile targetFileIfPresent = targetPaths.remove(file.getPhysicalPath()); - return targetFileIfPresent == null ? file : null; - }) - .filter(Objects::nonNull) - .collect(Collectors.toSet()); - // Any files remaining in the targetPaths map are not present in the source and should be marked - // for removal - Set removedFiles = new HashSet<>(targetPaths.values()); - return OneDataFilesDiff.builder().filesAdded(addedFiles).filesRemoved(removedFiles).build(); + .collect(Collectors.toMap(OneDataFile::getPhysicalPath, Function.identity())); + + DataFilesDiff diff = findNewAndRemovedFiles(sourcePaths, targetPaths); + return OneDataFilesDiff.builder() + .filesAdded(diff.getFilesAdded()) + .filesRemoved(diff.getFilesRemoved()) + .build(); } } diff --git a/api/src/test/java/io/onetable/model/storage/TestDataFilesDiff.java b/api/src/test/java/io/onetable/model/storage/TestDataFilesDiff.java new file mode 100644 index 00000000..83f8ae79 --- /dev/null +++ b/api/src/test/java/io/onetable/model/storage/TestDataFilesDiff.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.onetable.model.storage; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +import java.io.File; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.junit.jupiter.api.Test; + +public class TestDataFilesDiff { + @Test + void findDiffFromFileGroups() { + OneDataFile file1Group1 = OneDataFile.builder().physicalPath("file1Group1").build(); + OneDataFile file2Group1 = OneDataFile.builder().physicalPath("file2Group1").build(); + OneDataFile file1Group2 = OneDataFile.builder().physicalPath("file1Group2").build(); + OneDataFile file2Group2 = OneDataFile.builder().physicalPath("file2Group2").build(); + + List latestFileGroups = + OneFileGroup.fromFiles(Arrays.asList(file1Group1, file2Group1, file1Group2, file2Group2)); + + Map previousFiles = new HashMap<>(); + File file1 = mock(File.class); + File file2 = mock(File.class); + File file3 = mock(File.class); + previousFiles.put("file1Group1", file1); + previousFiles.put("file2NoGroup", file2); + previousFiles.put("file2Group2", file3); + + DataFilesDiff diff = + DataFilesDiff.findNewAndRemovedFiles(latestFileGroups, previousFiles); + assertEquals(2, diff.getFilesAdded().size()); + assertTrue(diff.getFilesAdded().contains(file1Group2)); + assertTrue(diff.getFilesAdded().contains(file2Group1)); + assertEquals(1, diff.getFilesRemoved().size()); + assertTrue(diff.getFilesRemoved().contains(file2)); + } + + @Test + void findDiffFromFilesNoPrevious() { + File file1 = mock(File.class); + File file2 = mock(File.class); + + Map previousFiles = new HashMap<>(); + Map latestFiles = new HashMap<>(); + latestFiles.put("file1", file1); + latestFiles.put("file2", file2); + + DataFilesDiff diff = + DataFilesDiff.findNewAndRemovedFiles(latestFiles, previousFiles); + assertEquals(0, diff.getFilesRemoved().size()); + assertEquals(2, diff.getFilesAdded().size()); + assertTrue(diff.getFilesAdded().contains(file1)); + assertTrue(diff.getFilesAdded().contains(file2)); + } + + @Test + void findDiffFromFilesNoNew() { + File file1 = mock(File.class); + File file2 = mock(File.class); + + Map previousFiles = new HashMap<>(); + previousFiles.put("file1", file1); + previousFiles.put("file2", file2); + + Map latestFiles = new HashMap<>(); + latestFiles.put("file1", file1); + latestFiles.put("file2", file2); + + DataFilesDiff diff = + DataFilesDiff.findNewAndRemovedFiles(latestFiles, previousFiles); + assertEquals(0, diff.getFilesRemoved().size()); + assertEquals(0, diff.getFilesAdded().size()); + } + + @Test + void findDiffFromFiles() { + File file1 = mock(File.class); + File file2 = mock(File.class); + File file3 = mock(File.class); + + Map previousFiles = new HashMap<>(); + previousFiles.put("file1", file1); + previousFiles.put("file2", file2); + + Map latestFiles = new HashMap<>(); + latestFiles.put("file2", file2); + latestFiles.put("file3", file3); + + DataFilesDiff diff = + DataFilesDiff.findNewAndRemovedFiles(latestFiles, previousFiles); + assertEquals(1, diff.getFilesAdded().size()); + assertTrue(diff.getFilesAdded().contains(file3)); + assertEquals(1, diff.getFilesRemoved().size()); + assertTrue(diff.getFilesRemoved().contains(file1)); + } +} diff --git a/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java b/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java index 276f3204..e89e31b2 100644 --- a/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java +++ b/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java @@ -39,6 +39,7 @@ import io.onetable.model.schema.OneSchema; import io.onetable.model.stat.ColumnStat; +import io.onetable.model.storage.DataFilesDiff; import io.onetable.model.storage.OneDataFile; import io.onetable.model.storage.OneDataFilesDiff; import io.onetable.model.storage.OneFileGroup; @@ -63,7 +64,7 @@ public Seq applySnapshot( // all files in the current delta snapshot are potential candidates for remove actions, i.e. if // the file is not present in the new snapshot (addedFiles) then the file is considered removed Snapshot snapshot = deltaLog.snapshot(); - Map removedFiles = + Map previousFiles = snapshot.allFiles().collectAsList().stream() .map(AddFile::remove) .collect( @@ -71,19 +72,11 @@ public Seq applySnapshot( file -> DeltaActionsConverter.getFullPathToFile(snapshot, file.path()), file -> file)); - Set addedFiles = - partitionedDataFiles.stream() - .flatMap(group -> group.getFiles().stream()) - .map( - file -> { - Action targetFileIfPresent = removedFiles.remove(file.getPhysicalPath()); - return targetFileIfPresent == null ? file : null; - }) - .filter(Objects::nonNull) - .collect(Collectors.toSet()); + DataFilesDiff diff = + OneDataFilesDiff.findNewAndRemovedFiles(partitionedDataFiles, previousFiles); return applyDiff( - addedFiles, removedFiles.values(), tableSchema, deltaLog.dataPath().toString()); + diff.getFilesAdded(), diff.getFilesRemoved(), tableSchema, deltaLog.dataPath().toString()); } public Seq applyDiff( diff --git a/core/src/main/java/io/onetable/iceberg/IcebergDataFileExtractor.java b/core/src/main/java/io/onetable/iceberg/IcebergDataFileExtractor.java index 825424e9..21c28b95 100644 --- a/core/src/main/java/io/onetable/iceberg/IcebergDataFileExtractor.java +++ b/core/src/main/java/io/onetable/iceberg/IcebergDataFileExtractor.java @@ -19,24 +19,18 @@ package io.onetable.iceberg; import java.util.Collections; -import java.util.Iterator; import java.util.List; import lombok.Builder; -import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.CloseableIterator; import io.onetable.exception.NotSupportedException; -import io.onetable.model.OneTable; import io.onetable.model.schema.OneSchema; import io.onetable.model.stat.ColumnStat; import io.onetable.model.stat.PartitionValue; import io.onetable.model.storage.FileFormat; import io.onetable.model.storage.OneDataFile; -import io.onetable.spi.extractor.DataFileIterator; /** Extractor of data files for Iceberg */ @Builder @@ -46,87 +40,6 @@ public class IcebergDataFileExtractor { private IcebergPartitionValueConverter partitionValueConverter = IcebergPartitionValueConverter.getInstance(); - /** - * Initializes an iterator for Iceberg files. - * - * @return Iceberg table file iterator - */ - public DataFileIterator iterator(Table iceTable, OneTable oneTable) { - return new IcebergDataFileIterator(iceTable, oneTable); - } - - public class IcebergDataFileIterator implements DataFileIterator { - private final Table iceTable; - private final OneTable oneTable; - private final CloseableIterator iceScan; - private Iterator currentScanTaskIterator; - - private IcebergDataFileIterator(Table iceTable, OneTable oneTable) { - this.iceTable = iceTable; - this.oneTable = oneTable; - this.iceScan = iceTable.newScan().planTasks().iterator(); - this.currentScanTaskIterator = - iceScan.hasNext() ? getCurrentScanTaskIterator(iceScan.next()) : null; - } - - @Override - public void close() throws Exception { - iceScan.close(); - } - - @Override - public boolean hasNext() { - advanceScanTask(); - return currentScanTaskIterator != null && currentScanTaskIterator.hasNext(); - } - - @Override - public OneDataFile next() { - if (currentScanTaskIterator == null) { - throw new IllegalStateException("Iterator is not initialized"); - } - advanceScanTask(); - return currentScanTaskIterator.next(); - } - - private void advanceScanTask() { - if (currentScanTaskIterator != null && currentScanTaskIterator.hasNext()) { - return; - } - if (iceScan.hasNext()) { - currentScanTaskIterator = getCurrentScanTaskIterator(iceScan.next()); - } else { - currentScanTaskIterator = null; - } - } - - private Iterator getCurrentScanTaskIterator(CombinedScanTask scanTask) { - return scanTask.files().stream() - .map( - fileScanTask -> { - DataFile dataFile = fileScanTask.file(); - List partitionValues = - partitionValueConverter.toOneTable( - oneTable, dataFile.partition(), iceTable.spec()); - return fromIcebergWithoutColumnStats(dataFile, partitionValues); - }) - .iterator(); - } - } - - /** - * Builds {@link OneDataFile} representation from Iceberg {@link DataFile} without any column - * statistics set. This can be used to reduce memory overhead when statistics are not required. - * - * @param dataFile Iceberg data file - * @param partitionValues representation of partition fields and ranges - * @return corresponding OneTable data file - */ - OneDataFile fromIcebergWithoutColumnStats( - DataFile dataFile, List partitionValues) { - return fromIceberg(dataFile, partitionValues, null, false); - } - /** * Builds {@link OneDataFile} representation from Iceberg {@link DataFile}. * diff --git a/core/src/main/java/io/onetable/iceberg/IcebergDataFileUpdatesSync.java b/core/src/main/java/io/onetable/iceberg/IcebergDataFileUpdatesSync.java index 7dbfb16f..e7cfb408 100644 --- a/core/src/main/java/io/onetable/iceberg/IcebergDataFileUpdatesSync.java +++ b/core/src/main/java/io/onetable/iceberg/IcebergDataFileUpdatesSync.java @@ -18,28 +18,22 @@ package io.onetable.iceberg; -import java.util.ArrayList; -import java.util.List; +import java.util.*; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import lombok.AllArgsConstructor; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.OverwriteFiles; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; +import org.apache.iceberg.*; +import org.apache.iceberg.io.CloseableIterable; import io.onetable.exception.NotSupportedException; import io.onetable.exception.OneIOException; import io.onetable.model.OneTable; +import io.onetable.model.storage.DataFilesDiff; import io.onetable.model.storage.OneDataFile; import io.onetable.model.storage.OneDataFilesDiff; import io.onetable.model.storage.OneFileGroup; -import io.onetable.spi.extractor.DataFileIterator; @AllArgsConstructor(staticName = "of") public class IcebergDataFileUpdatesSync { @@ -53,23 +47,20 @@ public void applySnapshot( List partitionedDataFiles, Schema schema, PartitionSpec partitionSpec) { - List currentDataFiles = new ArrayList<>(); - IcebergDataFileExtractor dataFileExtractor = - IcebergDataFileExtractor.builder().partitionValueConverter(partitionValueConverter).build(); - try (DataFileIterator fileIterator = dataFileExtractor.iterator(table, oneTable)) { - fileIterator.forEachRemaining(currentDataFiles::add); + + Map previousFiles = new HashMap<>(); + try (CloseableIterable iterator = table.newScan().planFiles()) { + StreamSupport.stream(iterator.spliterator(), false) + .map(FileScanTask::file) + .forEach(file -> previousFiles.put(file.path().toString(), file)); } catch (Exception e) { throw new OneIOException("Failed to iterate through Iceberg data files", e); } - // Sync the files diff - OneDataFilesDiff filesDiff = - OneDataFilesDiff.from( - partitionedDataFiles.stream() - .flatMap(group -> group.getFiles().stream()) - .collect(Collectors.toList()), - currentDataFiles); - applyDiff(transaction, filesDiff, schema, partitionSpec); + DataFilesDiff diff = + OneDataFilesDiff.findNewAndRemovedFiles(partitionedDataFiles, previousFiles); + + applyDiff(transaction, diff.getFilesAdded(), diff.getFilesRemoved(), schema, partitionSpec); } public void applyDiff( @@ -77,16 +68,24 @@ public void applyDiff( OneDataFilesDiff oneDataFilesDiff, Schema schema, PartitionSpec partitionSpec) { + + Collection filesRemoved = + oneDataFilesDiff.getFilesRemoved().stream() + .map(file -> getDataFile(partitionSpec, schema, file)) + .collect(Collectors.toList()); + + applyDiff(transaction, oneDataFilesDiff.getFilesAdded(), filesRemoved, schema, partitionSpec); + } + + private void applyDiff( + Transaction transaction, + Collection filesAdded, + Collection filesRemoved, + Schema schema, + PartitionSpec partitionSpec) { OverwriteFiles overwriteFiles = transaction.newOverwrite(); - oneDataFilesDiff - .getFilesAdded() - .forEach(f -> overwriteFiles.addFile(getDataFile(partitionSpec, schema, f))); - oneDataFilesDiff - .getFilesRemoved() - .forEach( - f -> - overwriteFiles.deleteFile( - getDataFile(transaction.table().spec(), transaction.table().schema(), f))); + filesAdded.forEach(f -> overwriteFiles.addFile(getDataFile(partitionSpec, schema, f))); + filesRemoved.forEach(overwriteFiles::deleteFile); overwriteFiles.commit(); } diff --git a/core/src/test/java/io/onetable/iceberg/TestIcebergSync.java b/core/src/test/java/io/onetable/iceberg/TestIcebergSync.java index f1467e6d..e442e764 100644 --- a/core/src/test/java/io/onetable/iceberg/TestIcebergSync.java +++ b/core/src/test/java/io/onetable/iceberg/TestIcebergSync.java @@ -264,7 +264,7 @@ public void testCreateSnapshotControlFlow() throws Exception { partitionSpecArgumentCaptor.capture(), partitionSpecArgumentCaptor.capture(), transactionArgumentCaptor.capture()); - verify(mockColumnStatsConverter, times(4)).toIceberg(any(Schema.class), anyLong(), anyList()); + verify(mockColumnStatsConverter, times(3)).toIceberg(any(Schema.class), anyLong(), anyList()); // check that the correct schema is used in calls to the mocks // Since we're using a mockSchemaSync we don't expect the table schema used by the partition From b31088850f01826eb987ac68ab7de2163850ded5 Mon Sep 17 00:00:00 2001 From: Ashvin Date: Tue, 27 Feb 2024 15:47:51 -0800 Subject: [PATCH 2/3] Upgrade Delta Lake version:2.4 for deletion vector support (#349) Support for Deletion Vectors has been added in Delta Lake version 2.4. The upgrade also requires updating the spark runtime version to 3.4+ In addition to chaning the version of the dependencies, this change also incorporates all the backward incompatible changes in the Delta API. 1. getSnapshotAt change: it does not accept an optional timestamp (2nd method argument) anymore. This argument was not provided / used by XTable and can safely be ignored in all invocations. 2. addFile api change: It now requires information about DeleteVector as a parameter. As Deletion vectors writing is not supported in the current version of XTable, a null is provided to the addFile method call. 3. update transaction api change: It now requires an Catalyst Expression object, instead of a generic string object, to be linked to a update operation. This change replaces the string object used by XTable with a Literal-expression. 4. getSnapshot api change: It does not require a timestamp to initialize current snapshot anymore. This change removes the additional argument in the method invocation in XTable. 5. DeltaLog metadata change: The metadata is now available through the DeltaLog's snapshot instance, instead of being made available through the DeltaLog itself like in the older versions. 6. change in the update api: it now requires the user to choose if defaults need to be ignored. It seems that the defaults need to be ingored for operations like copy. By default, the value for ignore-defaults is false for most operations. Hence it is the choosen value in XTable also. 7. Update spark version requires catalog and sql extension configurations in the sessoin definition. This change adds these two configs wherever a spark instance is created for writing Delta Lake commits. 8. Remove deprecated spark config spark.sql.iceberg.handle-timestamp-without-timezone 9. swap hudi-utilities dependency for hudi-sync-common Co-authored-by: Timothy Brown --- .../src/main/java/io/onetable/delta/DeltaClient.java | 12 ++++++++---- .../java/io/onetable/delta/DeltaClientUtils.java | 4 ++++ .../delta/DeltaDataFileUpdatesExtractor.java | 1 + .../java/io/onetable/delta/DeltaSourceClient.java | 3 +-- .../java/io/onetable/delta/DeltaTableExtractor.java | 2 +- core/src/test/java/io/onetable/ITOneTableClient.java | 2 +- .../client/TestTableFormatClientFactory.java | 2 +- .../java/io/onetable/delta/ITDeltaSourceClient.java | 3 ++- .../io/onetable/delta/TestDeltaStatsExtractor.java | 4 ++-- .../test/java/io/onetable/delta/TestDeltaSync.java | 8 +++++--- .../src/test/java/io/onetable/hudi/HudiTestUtil.java | 1 - hudi-support/extensions/pom.xml | 2 +- pom.xml | 10 +++++----- 13 files changed, 32 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/io/onetable/delta/DeltaClient.java b/core/src/main/java/io/onetable/delta/DeltaClient.java index 8b2ffdbb..e40f4bdd 100644 --- a/core/src/main/java/io/onetable/delta/DeltaClient.java +++ b/core/src/main/java/io/onetable/delta/DeltaClient.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.expressions.Literal; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -202,7 +203,8 @@ public void completeSync() { @Override public Optional getTableMetadata() { return OneTableMetadata.fromMap( - JavaConverters.mapAsJavaMapConverter(deltaLog.metadata().configuration()).asJava()); + JavaConverters.mapAsJavaMapConverter(deltaLog.snapshot().metadata().configuration()) + .asJava()); } @Override @@ -256,8 +258,10 @@ private void commitTransaction() { JavaConverters.asScalaBuffer(partitionColumns).toList(), ScalaUtils.convertJavaMapToScala(getConfigurationsForDeltaSync()), new Some<>(commitTime.toEpochMilli())); - transaction.updateMetadata(metadata); - transaction.commit(actions, new DeltaOperations.Update(Option.apply("onetable-delta-sync"))); + transaction.updateMetadata(metadata, false); + transaction.commit( + actions, + new DeltaOperations.Update(Option.apply(Literal.fromObject("onetable-delta-sync")))); } private Map getConfigurationsForDeltaSync() { @@ -290,7 +294,7 @@ private Format getFileFormat() { } } // fallback to existing deltalog value - return deltaLog.metadata().format(); + return deltaLog.snapshot().metadata().format(); } } } diff --git a/core/src/main/java/io/onetable/delta/DeltaClientUtils.java b/core/src/main/java/io/onetable/delta/DeltaClientUtils.java index 7f261b40..6f1a997e 100644 --- a/core/src/main/java/io/onetable/delta/DeltaClientUtils.java +++ b/core/src/main/java/io/onetable/delta/DeltaClientUtils.java @@ -35,6 +35,10 @@ static SparkSession buildSparkSession(Configuration conf) { new SparkConf() .setAppName("onetableclient") .set("spark.serializer", KryoSerializer.class.getName()) + .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .set( + "spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.delta.catalog.DeltaCatalog") .set("spark.databricks.delta.constraints.allowUnenforcedNotNull.enabled", "true"); SparkSession.Builder builder = SparkSession.builder().config(sparkConf); conf.forEach( diff --git a/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java b/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java index e89e31b2..60834480 100644 --- a/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java +++ b/core/src/main/java/io/onetable/delta/DeltaDataFileUpdatesExtractor.java @@ -115,6 +115,7 @@ private Stream createAddFileAction( dataFile.getLastModified(), true, getColumnStats(schema, dataFile.getRecordCount(), dataFile.getColumnStats()), + null, null)); } diff --git a/core/src/main/java/io/onetable/delta/DeltaSourceClient.java b/core/src/main/java/io/onetable/delta/DeltaSourceClient.java index 795a6bab..d6d583f7 100644 --- a/core/src/main/java/io/onetable/delta/DeltaSourceClient.java +++ b/core/src/main/java/io/onetable/delta/DeltaSourceClient.java @@ -107,8 +107,7 @@ public TableChange getTableChangeForCommit(Long versionNumber) { OneTable tableAtVersion = tableExtractor.table(deltaLog, tableName, versionNumber); // Client to call getCommitsBacklog and call this method. List actionsForVersion = getChangesState().getActionsForVersion(versionNumber); - Snapshot snapshotAtVersion = - deltaLog.getSnapshotAt(versionNumber, Option.empty(), Option.empty()); + Snapshot snapshotAtVersion = deltaLog.getSnapshotAt(versionNumber, Option.empty()); FileFormat fileFormat = actionsConverter.convertToOneTableFileFormat( snapshotAtVersion.metadata().format().provider()); diff --git a/core/src/main/java/io/onetable/delta/DeltaTableExtractor.java b/core/src/main/java/io/onetable/delta/DeltaTableExtractor.java index f733f0e1..bb07fca7 100644 --- a/core/src/main/java/io/onetable/delta/DeltaTableExtractor.java +++ b/core/src/main/java/io/onetable/delta/DeltaTableExtractor.java @@ -41,7 +41,7 @@ public class DeltaTableExtractor { private static final DeltaSchemaExtractor schemaExtractor = DeltaSchemaExtractor.getInstance(); public OneTable table(DeltaLog deltaLog, String tableName, Long version) { - Snapshot snapshot = deltaLog.getSnapshotAt(version, Option.empty(), Option.empty()); + Snapshot snapshot = deltaLog.getSnapshotAt(version, Option.empty()); OneSchema schema = schemaExtractor.toOneSchema(snapshot.metadata().schema()); List partitionFields = DeltaPartitionExtractor.getInstance() diff --git a/core/src/test/java/io/onetable/ITOneTableClient.java b/core/src/test/java/io/onetable/ITOneTableClient.java index 9e4b9d0c..56e87c9c 100644 --- a/core/src/test/java/io/onetable/ITOneTableClient.java +++ b/core/src/test/java/io/onetable/ITOneTableClient.java @@ -676,7 +676,7 @@ public void testMetadataRetention() throws Exception { Assertions.assertEquals(1, snapshotCount); // assert that proper settings are enabled for delta log DeltaLog deltaLog = DeltaLog.forTable(sparkSession, table.getBasePath()); - Assertions.assertTrue(deltaLog.enableExpiredLogCleanup()); + Assertions.assertTrue(deltaLog.enableExpiredLogCleanup(deltaLog.snapshot().metadata())); } } diff --git a/core/src/test/java/io/onetable/client/TestTableFormatClientFactory.java b/core/src/test/java/io/onetable/client/TestTableFormatClientFactory.java index 6897f50b..5cffe450 100644 --- a/core/src/test/java/io/onetable/client/TestTableFormatClientFactory.java +++ b/core/src/test/java/io/onetable/client/TestTableFormatClientFactory.java @@ -45,7 +45,7 @@ public void testTableClientFromNameForDELTA() { PerTableConfig perTableConfig = getPerTableConfig(Arrays.asList(TableFormat.DELTA), SyncMode.INCREMENTAL); Configuration conf = new Configuration(); - conf.setStrings("spark.master", "local"); + conf.set("spark.master", "local"); tc.init(perTableConfig, conf); assertEquals(tc.getTableFormat(), TableFormat.DELTA); } diff --git a/core/src/test/java/io/onetable/delta/ITDeltaSourceClient.java b/core/src/test/java/io/onetable/delta/ITDeltaSourceClient.java index 5cd13a2b..4dca16f6 100644 --- a/core/src/test/java/io/onetable/delta/ITDeltaSourceClient.java +++ b/core/src/test/java/io/onetable/delta/ITDeltaSourceClient.java @@ -40,6 +40,7 @@ import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; +import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.AfterAll; @@ -127,7 +128,7 @@ public static void setupOnce() { .config("spark.databricks.delta.schema.autoMerge.enabled", "true") .config("spark.sql.shuffle.partitions", "1") .config("spark.default.parallelism", "1") - .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.serializer", KryoSerializer.class.getName()) .getOrCreate(); } diff --git a/core/src/test/java/io/onetable/delta/TestDeltaStatsExtractor.java b/core/src/test/java/io/onetable/delta/TestDeltaStatsExtractor.java index 9058c20e..d084a5b4 100644 --- a/core/src/test/java/io/onetable/delta/TestDeltaStatsExtractor.java +++ b/core/src/test/java/io/onetable/delta/TestDeltaStatsExtractor.java @@ -120,7 +120,7 @@ void roundTripStatsConversion() throws IOException { String stats = DeltaStatsExtractor.getInstance().convertStatsToDeltaFormat(schema, 50L, columnStats); - AddFile addFile = new AddFile("file://path/to/file", null, 0, 0, true, stats, null); + AddFile addFile = new AddFile("file://path/to/file", null, 0, 0, true, stats, null, null); DeltaStatsExtractor extractor = DeltaStatsExtractor.getInstance(); List actual = extractor.getColumnStatsForFile(addFile, fields); @@ -148,7 +148,7 @@ void convertStatsToInternalRepresentation() throws IOException { deltaStats.put("nullCount", nullValues); deltaStats.put("numRecords", 100); String stats = MAPPER.writeValueAsString(deltaStats); - AddFile addFile = new AddFile("file://path/to/file", null, 0, 0, true, stats, null); + AddFile addFile = new AddFile("file://path/to/file", null, 0, 0, true, stats, null, null); DeltaStatsExtractor extractor = DeltaStatsExtractor.getInstance(); List actual = extractor.getColumnStatsForFile(addFile, fields); diff --git a/core/src/test/java/io/onetable/delta/TestDeltaSync.java b/core/src/test/java/io/onetable/delta/TestDeltaSync.java index 5b275f1a..888a9b87 100644 --- a/core/src/test/java/io/onetable/delta/TestDeltaSync.java +++ b/core/src/test/java/io/onetable/delta/TestDeltaSync.java @@ -60,7 +60,6 @@ import org.apache.spark.sql.delta.GeneratedColumn; -import scala.Option; import scala.collection.JavaConverters; import scala.collection.Seq; import io.delta.standalone.DeltaLog; @@ -334,8 +333,7 @@ public void testTimestampPartitioning(PartitionTransformType transformType) thro .expr(); org.apache.spark.sql.delta.DeltaLog deltaLog = org.apache.spark.sql.delta.DeltaLog.forTable(sparkSession, basePath.toString()); - org.apache.spark.sql.delta.Snapshot snapshot = - deltaLog.getSnapshotAtInit(Option.empty()).snapshot(); + org.apache.spark.sql.delta.Snapshot snapshot = deltaLog.getSnapshotAtInit().snapshot(); Seq expressionSeq = scala.collection.JavaConversions.asScalaBuffer(Collections.singletonList(expression)); Seq translatedExpression = @@ -485,6 +483,10 @@ private static SparkSession buildSparkSession() { .setAppName("testdeltasync") .set("spark.serializer", KryoSerializer.class.getName()) .set("spark.databricks.delta.constraints.allowUnenforcedNotNull.enabled", "true") + .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .set( + "spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.delta.catalog.DeltaCatalog") .set("spark.master", "local[2]"); return SparkSession.builder().config(sparkConf).getOrCreate(); } diff --git a/core/src/test/java/io/onetable/hudi/HudiTestUtil.java b/core/src/test/java/io/onetable/hudi/HudiTestUtil.java index 0a74d877..8bfa8bf4 100644 --- a/core/src/test/java/io/onetable/hudi/HudiTestUtil.java +++ b/core/src/test/java/io/onetable/hudi/HudiTestUtil.java @@ -129,7 +129,6 @@ public static SparkConf getSparkConf(Path tempDir) { .set("spark.sql.shuffle.partitions", "1") .set("spark.default.parallelism", "1") .set("spark.sql.session.timeZone", "UTC") - .set("spark.sql.iceberg.handle-timestamp-without-timezone", "true") .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") .set("spark.databricks.delta.retentionDurationCheck.enabled", "false") diff --git a/hudi-support/extensions/pom.xml b/hudi-support/extensions/pom.xml index 368a32e5..1f3507cf 100644 --- a/hudi-support/extensions/pom.xml +++ b/hudi-support/extensions/pom.xml @@ -34,7 +34,7 @@ org.apache.hudi - hudi-utilities_2.12 + hudi-sync-common provided diff --git a/pom.xml b/pom.xml index 792d6216..b6093069 100644 --- a/pom.xml +++ b/pom.xml @@ -24,12 +24,12 @@ 1.18.30 0.14.0 1.12.2 - 2.12.10 + 2.12.12 2.12 - 3.2.1 - 3.2 + 3.4.2 + 3.4 1.4.2 - 2.0.2 + 2.4.0 2.14.2 2.22.2 2.27.2 @@ -165,7 +165,7 @@ org.apache.hudi - hudi-utilities_2.12 + hudi-sync-common ${hudi.version} provided From 5d54f5afe3d9adf39f84b9f8d5485d3152e0b780 Mon Sep 17 00:00:00 2001 From: alberttwong Date: Fri, 1 Mar 2024 10:47:38 -0800 Subject: [PATCH 3/3] adding blog entry on E-commerce Funnel Analysis with StarRocks: 87 Million Records, Apache Hudi, Apache Iceberg, Delta Lake (MinIO, Apache HMS, Apache xTable) Signed-off-by: alberttwong --- ...ommerce-funnel-analysis-with-starrocks.mdx | 19 ++++++++++++++++++ .../starrocks-xtable.png | Bin 0 -> 143101 bytes 2 files changed, 19 insertions(+) create mode 100644 website/blog/ecommerce-funnel-analysis-with-starrocks.mdx create mode 100644 website/static/images/blog/ecommerce-funnel-analysis-with-starrocks/starrocks-xtable.png diff --git a/website/blog/ecommerce-funnel-analysis-with-starrocks.mdx b/website/blog/ecommerce-funnel-analysis-with-starrocks.mdx new file mode 100644 index 00000000..91217a40 --- /dev/null +++ b/website/blog/ecommerce-funnel-analysis-with-starrocks.mdx @@ -0,0 +1,19 @@ +--- +title: "E-commerce Funnel Analysis with StarRocks: 87 Million Records, Apache Hudi, Apache Iceberg, Delta Lake (MinIO, Apache HMS, Apache xTable)" +author: Albert Wong +category: blog +image: images/blog/ecommerce-funnel-analysis-with-starrocks/starrocks-xtable.png +tags: +- onetable +- interoperability +- hudi +- delta +- iceberg +- starrocks +- minio +date: 2023-03-01 +--- + +import Redirect from '@site/src/components/Redirect'; + +Redirecting... please wait!! diff --git a/website/static/images/blog/ecommerce-funnel-analysis-with-starrocks/starrocks-xtable.png b/website/static/images/blog/ecommerce-funnel-analysis-with-starrocks/starrocks-xtable.png new file mode 100644 index 0000000000000000000000000000000000000000..d883e2373023e3c53862a123a52ab8983a6585c5 GIT binary patch literal 143101 zcmaI81y~%*wl)leBtQbegZtnX2<~pd8Qk3mADj?^gKI|=SiaQKJ4 z_qq2x|98(Bo_V^byQ-_JYSmh+-t{iRRg`4V(TLCx5D?JiWF^%Q5Rl6e5D-73yntV! zJ)N6?|Io6LkWi76kf2a;0a@DETOc6N$2!C)Dl|^u28EeEX~a5L^DXOE@9*Z`4TKRg zgRo{|(M;iW-;ykTJ@R9!45>jdY zXV!}hO;RqfjSt7HxfGHCDB*$w$rC1OpgVorU@#q+V=>!XH zIZH)F1V;Ed3c_fy*}u*aKf~`Kh^b4+$-$56W-b;Mj;_`q zw=kQWK==hzCs`d=1cX;OPhUiYuAO}_xbC9V8tEYq0({m66 zJ^A6M4i;`E6rK+Dj;{QkLR5d1;D?_-U1p=A_^XJUoe-6_q6&os$i;$!o0XlFok|#u zf`UTO#oUr#O;Y-w>hNDeRMu{8PW)_aU@({!%*hIJv0{72$H&LU&cVjP!2&PA;_Bt- zX5z`>=t}+fL;m#~Nefpq7aJ!x8;~Q#({oKsLGErsR8&t5{m|3kH>JAYUEt6zUNC;0R*eia)}3ws?&8wdDM!(UDKJv*1+ zU(NhKcmDT9|E>viwQ!LDIlwEr3IFe0{ZskhZ~p%){?(-R|2FxKgNx(e+x)j%|5o*> z5B$n5Ht^<5o(56)ogmx)bM2qT_lIhYS~zPFhQ$WENXejXY`(c@^>K^mgOBiA4ZfxOFiK@bZ&ej%cOXs$~;ty18FkfgTBMEo%BqhVx0Z zW3nT^M*P1I6k=Gi@?*X_wg0KW-<3l@Vrgq){kJmMM2OVo*f$s2oByrU*Y-+&V-y0U zkdKuA=OJbcg&+_*R@Td5^k5|-Yd(}QSha0l zx5E@;E#xj4`4{`7ztGp_Mj7xNsmWWllmXw7_+vo(3@I4dV3a)+G5`vI1$p zh%%AZYly@-W(8N-6C*`Nit6a-Jg@vR$^z*2TnylSLb!&crZQ(sN21|y#B;$vzKZ5o42+Br#Ah#LEiEmBudZ8}?4p@G?O(i2uSd$Z zc9_poOff&0uQ3rEA0IbgZuWv?WE`=7eTy_6gu77hoIO?tl!^HraCx|NLaWa7*!;L( z>yQgBGdv^?4~bCz=q>)x^OcagM6V7bK!#fS>#<>{%nC>&#C_K6Q|d>*99MAGV)Xgj z1Abil72fRYk|G~qOOwiV%%uHC!JW%obd^a^ap}PBS|My`SviX=#bp@>Pkk^EK9&Ww zf!tTgv9M7Zqi>_o%)eWNgJ%8k7H266Y44A3<(K0%80GqurUAOQ;9 zXb&nF7#xgK7P;5p;NT#+ex39@kR#y1cc%8w;gViYB=J^yTEN5gc&ne!Tl^1MH8mHP znnbLYkF9>!)-$cT)w1}k+7yrXM*+F6TO(1w&iqD9LPh{k;sj%uOgndvARI3xYK5{lYG?5p(k zVV>NNF?MFd{~o#9rz)FrEFKF*S^rv^YOlEsgN&j0F}?g5$v*AlI8izsF~&Tvj6_bCnhmi~hFP)7Bif}Gq^uj#Uz{J^#~;G_;eKENz@rjc7ccF z@vA-1M2_Dm>SAw3pxA_2PldI$3QL<&x@BNY}*QfzXdriJ4J)=GdK5T(< z%Qkv3=I5;1TAq%7>CC`nDr00##n>vR5K&|#P|HPv;Q5%u>3-Gd!1K5I6f;HvEw72W zhZN(#y?>vNsxJBSVEjqd27VU%7>@l4r2zZkEaNd=;IRHjP4`O@r;m$HKhPbspzKK{ z*=j{D-VC_EZXk>xZ)V4Fi&IkZTw_^7b=>oY*cHXbhp74 z5SuRKGtd=|Lu#{4w~HPrpaIE$ZaJL1k<-R^8q|D04D!brsk?l2g7i)U=OqUELHi`j z;qx%gL)EAoidYzvkwHw`R4$?yo=)Q3?U~6vmpj|`%*y2hugj>wu0jrZz`^}B|3;a? z?5SJ5R^A0d&Ba!_&q!n-WhH*$Iox^e@66<;8ef?3yR6H~6udw0#8Xqnv@H;QSf>+lb|+zwJVop`9>FYNL$adeirbCn$f@Pgd@ID8o>?gOL=W72vUkI`_qsc~ zgn_09<)DWRwlvD)jED1lCB@wIxaw{6zg+-RQ3{BY=iNzfT89h%+Zb$7`RStOE=+?? zFR>DIAGKE+X$O}9S3&yTJGrQdy$SJ@8a(8_$70vlqXCDshgybSxm9hOs9iT*NKk2- zcB`Rplp;4jqmhYICY@bFM9ru~m-0zjRZaI2`88W_51UAnxYM<`!({7>_TZ-y(tM$@Kq5VLv%qZfSZ=^jB#{O#qzl*6)h-WX!JimA3wAjW4yjrNl5?#e}xm=B4^GJEA?{iQy zUSTFv_cK9|N7@H}WxPd2`<}P9%0= z>iJ48dnI*^s5cL{M@Jerwr8c<^gpmf1Sqp$Ve{Lg{(m^FR$$!Dp{BKMA)@!xAJ-{} zi7Q;`4(8&Ng?D-;_P1r1YQ6OzkJ{DPk(2qK+>#3(>uV4@a&x_KlaSC!7mi~u9d`Z? zu78}@m@AYkRBQok|KpCAki^8T$?_K%B;Uqx=vE#@!xp`^(p(P{ibee&SD6!Df&_TB1Ixk(Ulkg-5tVY{tli7tZ|9_X|CdWLFD$AQ??zP}2c7 z-_@_6jp_-ds;LnKVGr9UeI=?c-md&T1f7^R!}m12JbPsH?WoREN zUFxqex}17vE<%y-{9%l%(E)d6JRQCMDiU#8>!h8#XXt>_o=Pg6Otn3 ziUYEDo4OcClSZv^#l_4G@OIY&9&a-~q;QvPAQ%I<;D(WiGDa<11-}o|S`39fGc(Q) zE{6t%gdg+r@MktM+vE)(bCgp=Gl((ba5Z0{WTHV?UR%5124ek~;DH0yUp z){C+tHg}1{fpNPJ$i6~{OHFlxcgH`TtlRX#>Rg#on{+I@{Rig1lyMk3VqE(dbh@*t zPguuHrv!K1eKe~h|Js!%3nMX0>t|!LR9THCbVs}mMO$~v#Ll+IB;bvLyXH z2@$)PqG(0+`e;ldB1W?<1i&dzco!NP8cw&fqX*y%cgx-g3uIQH<#lQ!UWVJR#?|>e zMSi2OyW3?dKiDPvfh?&>kCu%M61<>bvF8XxoamH>XvOy@GLbP4fzG55xNO3TQE_Pd zddUNr2NT)A!8P9FzAFvTAiQ0kGD=O4V&{~ zA)ArUZF8Q;S5B-N;UARCb4pWVUF!CdXyVw=s2Yc?k2x|xVAAFMJ7osWNz_#jf^D+? zrxvmLw>p`E>~?h@baZLXYUBHqfSjL)7=N~7g6^<41njKum|Og?j@N8+Ud6WSHo6s_ zO%gCx-foT@J}ucSO1R&a)q*vsEKd0=w=6y2!HZDD-`BH!ihwBt_4!wep1-^o%TEYJ zk22(b)eFC)sQ?3A^Oqv~#yr+z6i@E0{pw2BZYnv`>#1R+&Eb@-ePdPRfHv5w${R{Q zZii)NbvJNrQE8(*zuWdN`xo8Vt+0B#Q|1B;+13#P2%}WfB?@i1JaCB7fQ!~=yErR* zdA7V}NY0W}q(7$cFVd}kZ>5xvVj-SoP(|4Nm30;)Ye*XKjPAHx>MXi3fjgoM+wj1>z zU|pZ8Ef4okVz!@Pmu*nr=}bM!HX6NS_ZPhJY?(J;$F#zrj)%WMk}+mXkqusFbD{6%)@N({%2ze75mmXmBr;w}}J-B^HZWEKWOm z#0kfWn(nN;!tIwUCc5>6hg<|WHKj*RSmGHJG=Jw`-tl0^21z%WcM%YAaqHqI+jkZW zo(ikh-A`mZ&=~oi1T#fwvDsHXulh<17sfPdxG<(k*kWDMdJp!_U|NLg zi_6+Z9P(c+W>Ga>(2H*e~GtD)b(%(i6hrBFS>rqE~1p*{+Nq zQh1Lx`Qd-eJly%-()n&JYf6ES?DrUib_-YL;$p@)LtG5VcRrtePq$8RDFg%rNJ=Sl z%P5o!92;yWb{nL+UFr_m>)oxAUTYLG)hw_sL|pb+T4LRtNxsU?0eU|Ul`@>BVeLQ+ zgza@(vp?h&v&5$7`h4~yxUIHjDJz?0N%1(CW*WBvL?nP>UOrwVKr{|DuiiL=R&*#7 zwCv-h6sG&{$}|VFa`!AQ8XYP*%f@1L`-$jj(x$>B6D)C?2dafmJXKI+MK@*_M-BGK z1BNR;t+aD?sv5~(Y~8!EZFgP<@uVC#fPxsna9rbjNm?U>R-b#Y5pKRFNs%ZUmK!5R zO*xR0(UIJ~?~B&^-+RN69<+rFC+c`hb$O<+zswv9F(R#8D4BPGFWgZ>efIe%VA|$! zqRLA7-E9QwneFOGSVp|r;MZXGWe0Z*@=1l)+Gacv{er9syHWqVbwAI_w4>$}g!bCvNKO;%sm z2h^5o3(xB^E=|3WUg3;WGapEx;WINxce9N<^jPq{PTO(0&sH~IrTk&B*cbR48y636 zP$bWT{0I{_I0nU$^einW z*7EJ8sB^~gaEFDZWh45lySfRI0dtF|y>+96F(^VC?5z7CM6^ouV<1)9RsxHj`Hv8E zpTVvIO5U5<+pdva1VksT&p)2EdS(}wA^Q7u6PNg&yq+iGF#36_yWmq1U!yuuu;M8( z^xTvo07wvo?mWqz4jZ>d9i@2Nd?Qa4Khrj&NX)~(pidhfg96I6^g3SIoUJfh`b2P1 ztQ&rK>MZT=-@3opp#I_w**^BA^As~(MJOW!U74Yl!~xGJE5sg_sI@s?s#8f)bs3L2-oA4mEb`Etx7xwDuwZgznH(@G5vyJ&fk zSoOXtww1DK$eoo}_6b^^05OPom3nrvBH+j+&lNKRR`(wlG)Vt0tU-a2`JmkdpQ z35RJi1^ybe7&6TtX)_}oMZ65eAWf%!>$fIJ$ej}n5xCJe@LQ8&JjWUIfc6JyCD8wj zr-?3bC^z%YWqg;oo?bF-rFS}*V`f)Vlb>T)ik9ZUOdq#|<`QB$$HSy^{woA3P@``L z94JwVzFzS3cnfoL&#Bn+&oU&#W*vc&;b;$=agqXjL=rKXNP`JV^y{hsI_@gA>rxD zMbr9_g`UMtzN`6gkG{G)eIE?E5;(w@OEYq0J57%dBH(1tI zqk1(tJ);dsCQH_%2l)e;&!Daf>e5GvKh$#-W;FOLYy-b)Fppm>qc$q6MPoR!FoEL- zjzm~ubQ)mEg%$#}c9r|%Td%K67s2| zR)I)kXrEQZrErG>mk(=x4r9z}Ji7&rs9q0bOq~=}Z9f@0R4Pm(<$2^+);iqCwh;)= zWbIYn2je->R;Kg9SliIK-hF-h2rSa%E3DH7vV^T9rmz>!vglU#VMP@fO-?>XMU<`4 z=b~-2-H_*rEmmxXZiHTT&@}2@9(E?|<&X~w=r(&gU!`f`gQ=Fc(gOFsiv^`7e`PBQ zNE9CZaK*jHeY!VY5}|Y76I#+Ad-MWoqUo39B;9Ivs;{>-LBM4jzW^P||NSnE%YU`Q zW5UUx#Icmol8l&^Er#dB6qhOC0uNNYBXc7Glo!E-OL5e#(keihYOQLoN9JQAN$Kc2 z%TWLC;G<;@Cu7;%Qp#NheMIsFfkyRTZ7zf6%@Eq$0oy_-Z-cjFa;gYB*9Sw@uVnI$ z?y5G~>GHo)xp;w%32jpnEOJ`>c#VYc_{XM@yjw-!J>VKo>M~xt6ipQ}M3L+VDwQ2%m6QPTxY!oJO&B9I>;lQE9!X91zCiu`g}Sl@7>NraG{Rw_wgKw+h!M1-0UmKQCs4E;JpDCdp40t> zVqIEswS?7Rne@ntRGUdhg8!G6fo`#i=2m`VHSr#wg|rauw}64KU4d<(0Qlm~R2Y0vR6gNh7A>6$ z>n(gWBe%I*GA^vX*AT*_UYK2I`+vL4-kz(;<_W+WdP6dzxRC*|s z`hNGQSWc}Gp9kQ{RcsdZezez#O<`MUStI1SGo)W>S%-uBwUKeSW#9IklosXK?SR)s z1oCB*t`!L;P>Afc?sDN?M9r~}c^pK2e{ohUc;W^upD^@QlcU`A3ki(o?QV$d_m+yv zS-Hk=i0TMJ%QEp2rOwUXXw$TJaxSt78<4JRqgpuu&(_MWO1@hRmsKkUYy;VqT072! z32OqUVd+3pJ5r(C%ED~&uuHwdY!dDF@D0frG91t@LT#oQG+m==kK%aR>!y0HhDW{5 z3W~P-s2)9c>1DzZT5(;s;=e%XYXIMPrhGJOdW9P4yw=lzh#1!a-(A=DTRuMAn~gld zlZ)mYn+Ut_mZ6s9JT$l);qFw?MlS@G+uHt|Oq~KTE$=Xq)Xb3FnM z^DydJ*4C@lzG9v|yA?9-J@ML@&I*F_IjamlsLBWItA{4o>RF`mOZ3xL$|Y=o+$KZC zc*Ej%H8p8Ss(=b0Jw7blZMH)^cWgm%Ca zW{?!$!nR+(fjp8qGPERBJBmN^l;pn{p8Y z5aY4Er;}6L$_?{23rwg3?~NTsI{C?#{g!`!M0(z{Wm{*v{71+|^3r@~j8iX2hD7IlS~)gULo`J`atgq?M9@}Tb3f$XT=8km*pv&j?1 znP>(~S$z>WQs}Zsif`nJ;ARJc#;Nemvrap33#pwlA}C`EnrGUSqoIko@Hpv+FwDwc0}bm9$$^JM31$ zQ5h0v$$CHcob*r7oKfMsV+pUbVVF@hcTs! z#hi?Qghyj}ZF%(q=5iwllv@rd*z)Z?2Fl8ps6cC>wooCNGBA@3hptq3NL{qB#dqYL z4RMLC4zAu{`ANZsR*k4ht6=-8Ed#Fxg;yT^r5mqu>A^)XoIF`Oz$&L3Se)V~fZ^JB zUYyVW?!jeJjIy7TUG`)sss{?m@~GG7cKUQfC`2XG({|Omc=d~^(!r|KO{Ga^Y5piy z0pzRp;{1SChJ<@Ozjf5PgeWYd$tjnW;+d}4LjVwr)&m>BdJXul3G>l5SKpWRKNiBYM zK*e&`HPY@X1Y2&3QWOlG{zNiJmF6OYPntAk8+~p&AwL2P=N(s?;SEVtu#wVa5Yb$0 zbjWAEvW4z10=IY7iJ|)>TpTjIV+b4o}ovGd!F-f8&CXA<8l8x3{ju4{?Db}`eLpK2gmx4^pa_TK4) z$NB=^OBd?u=>ZI3?imzUe`wsjdy)SGQ0af@{}2ecAW8J{=3 zqM_z`&myNjZa$TFz!=P>{)c!{fxl)@EdcsPXTEL}qA(&NUCE(0$adr-mH=90UaHcw zOMkWgl1GH0Oynd023vgh?K#*m3!+P|oWds3aAvdgn@wsH#0W7m^7`a^sh3}PU!sv% zx7&jx#mH3?kdI54S6kLt$ZDhqNRuu?{6hsOLId*ennA_&DoZrDnQXY^96pm4zL+sG zP-Q+Uw%qCwa?v?5-$_lQc^1!Z*Vdw8sGWsnts_(5d}j~!G}{tiY4%WO8AOzaL>?qC zHtagU=36>Jm7UXuD?p!x5kKxaUROatquHx~(6)wo5^S1@|xcnU_7m zd3Ps26Wv2)mrzyfaEvq3NpILtat$erw*f0!Lc*MtpxB7%cRoRV;6k>!oP*Is&VA|#k8OGFnK?>M951)=PXyB;-xz3qv`wjdZOam!udi;JXTKN`W+&SsH6)>EcBS19<4GSzIZt#kH4>L zkONU)n3TrH2Bv9@e<@Fr7d7DFkU0IK!a_$Dsr*>?^VX1Hzr2)zLw8~?$D{X#sWxS5 z8b4|)O=B2m>uiJd{LRiP8gBFkaqR#Hly|RGkG52Ko~7wR$g_@tN7<5X>HDf1(cCK1 zoad|u^bhR#v{_34K6h}hq5uwaP8A&Mt%iu&qg>&=?_)wKI#{R^NMWB1H^B}K)RJr! z^1{c3m(B}Da4>V`Sl@<_qku>&}t-6(yEH13amI6Vfj7|}IG%J$KbQ?e{y zdw>jBu5BM}qrX_zsMU0g5y*gVfvd%K$;)<7nnK3h8V~hS-yGQIHR*FLcQK0~M&}#_ zk{ufcmqVL1i+%{+>8(gBb#9P)j$ZG3pwxp}X{&M8`OcC<1_8u4KsR6Z%b{gK(!@SM zPOU+#OuO%xIV|tv+80uks`!(=xx5cIwTh~Rg`y$J$w|aTZn{AHezIoN!bh?p?)3ur~H#X8)bxFdK)1q!r2xjDf;U;xo)6bsgKYjMuj*VGr2kZ8m@)2M*Ou5XW$y$}h0Rc}VO&qgX> zUjNP_Ja4HEJTo>Dv)fzOgm`q}-Dok0|5gN2_jkA}OtSq_Irj2k{z>~FBZ@d_=W&dxsv7;S7*7y5Yo`S2@>x70(;|+o zoWk?p4?kSZJ7+s8jUS}v;z|e^m}2>CpY5)%^)QCx2IH!-0b&GxOl({fR$0J-n+}Y` zLn&gh+2HY>cAtgfX$}|bsIPp+<*$jxB+x64pqleZiEN`S=b66z_e_9BUAb`Bb0z*; z6h7}xf+c7VQQpr@fC zcVg0!L;!eru$jbT)!XuM4XxK%{8N#RM6Uu|`GX`{UMx8ry0^LJV z#E*1hr0}l@$dVE>78;it*-GFSuN#ECD=Eu9+!T-&4-`^Ek=4vLL1heT#A-e?3PPH{ zWY4pp5PF~ZZ+&{lxkMl%7MlYBdav9-D*bRqCVa-gU|P}~Od@Yk>tk@L_jw>kb-)|I zc?@rKdK~8OIvH<|J&GSM&!+J8mp#y9Ds!FSd#MW{aG0BF zlL;Qe8_V}@>9B8N1G&&5TLZ;K^@A;jv=7 z2g#RL8|OQp-Y5Uf7pB6RZPLfQ?katXwW?wr2c>4gV?3M5b~zHE7*X|~npe}qHF)U+&E)SI+KC3=+;>z7;-}7#*D1B=2`Hc6_^WAu8sq{P)LCx&6z_iio z5VR4XMuABg$qyrClp(8`M!CFMfjifC8=$fQmdjSEX99JQblW=l^6paGFXFBJM#sG^ z`dkvKSxsWQg%8oYY8TygM|0BN?y3t~d^(Uq?p`80NZ%^*k+9K0YcgDb;RA;K6rh)c zk8U8G$b*Q9G1c+b&{OQN{!Gd0EEuoLZD0D^Uj@GhQ~wHwaXZYzxez3)W8b*!626k~ zjn92G;n-Uk$H#!;tzZ^l2cA-fRs(DEiw?im>CL^Or>&7-HL_=c;@ljf5@SYxEzrJi zuFBKaEJ6g=T$VE0A|^|=bC1?EeJSuD9u2i#OrBn_`4&)O~7$`qb36jM%7w_9|D6$&*jN%AAX>Q zy3VlE`JZG00pq6Vajz7*pNs%iJ_uEuvUslJheqwMxACzo=> z;UCsI0`lDpe7N{S;fyX(TE%1UhxS_9uxXyis^HEh`4?oI&#kUV__Lemc$pB2VH9gR>(ooSN1X|MJjIUlj=`KNfx6FmY2k+!sWc7mJCdppym9r5MoOcNyAr}2i# zAk07WB9HQ0pp%PoL9ug@%1xByK3t?eDrJd9JSdj~#!@h-^rt=Qez_0_3pPyybfltIj4*ok@TIu;{%cDH2xDwV|&3qvtP zMi%K84;|Sr2V=+#%>Sgit0m;VidX>mxd5tO6s48kxCl%--EMN}nQNVp$&ISm^Cb3Z z1J3`DPtRZh(}&*V9m(;^wwd>_EGu_w5u7jAZ*90288mq)O|&bRayjNPyTjc<+4smq z3N3s;oyzave2eh*YMY51IHt_ud2F26W*rpwd8poE;4d%zh2%$;gQNm6!!(xxhSQz= zjK;|Z@(2m}j#SFAajfIT$%3eU7pHu9&}Rh~=XuQ+8f6yN1m)>OUJm-OiBnRG(VWccZk1?=KoU!7f7a)A?zjT+=?OZ&jc1l$dk9OF=2RJn4QLCdf?)D1#BvC?~t zm}-`Xt%WZfP!AXTG`gTu1%Q@Z@r|1WOjldpTVB#6)sR0|ApzpStMTAV79`^#d{J#vEEnXUZItw9y&gG&g4siJypUGevo00Zm7; zB4{*)N_!cMIJ$KZMaF|{eaQYYkhU#Fi$)>7{8UMiy`PR)!Cb2JMpdmlK>4u2A6ybI z%F5K2cn6Khp$Avd0D7tX@V=xq;})_Mi{tIAX3$a?a9!ORF((khT_WXzv)}X8BQYfl zwF4U~&3Zinm;MbGhrJt1Hk^Q(Q2DPmOe)`X4i`#VAljJ*NYEYU-;8mRX#MUxMu3&9 zdzAfUx}BY91ljP{S`;<7Lf7~X~RdxFG%Oc<-c&3#Tg9Zcb0mV z0=A0=Ts>%*$0DDlxy9btXLHvtYxTq4h@{Xpz&O4kiRFE>+;a?&2t~&vk>z0Siw>fX)?!a#pKgj@bk_y?Rn#Vq zr9{W!9q$m+c|eBZ*xH4K@)X<1i~181AHAj8gv@$gTeVffX|>J^7bo}rs;a6%NWbV_ zTD?(7jChgTF9TQpl0Y;z;$+Gej(hbbERbUO&{FY!!t_rx{V&)O`xIKb-|}HWuX3$# zr|Wadd`-CUp`B`a+TFH4N%MEsb7~=V!bbENqdKdO*KSSjdo&WaV2A~yBbTu+-Oe=A zASyi`LsiJ|Dcnlaw*~wXIUU0(8)S+Tfm$>&kzYx#mFstLU90EEPSHG56#U?&mss3o z9&%XgAgPHHkdfdg@jV^0U@_T52w?s53EQzpp+NsAwi=E@*NDI!9*)j)X7RLuh!H0d z=Ma7r;cAAyaA11CajvX}(@1v4b0cE4r&_v@eWSdIHQxN0UH0B%*?}{tKV%3S@O$iU zyAa32EIY}DD{$UPYOl2VEvy|Me}BPonB7nq~UaGHS(o za=`RThKGi}8XbIZ<+M71v#Q|SF=G_=2CMO*p0{f~k#U|mniQzm{n!1%wlT76+_=uM zvIOrhnrVvM-WS1SjN>Sq&OB{N0ARlp2Yq#A&zhJbMAQwC%);f6N&}iVIUn z`Q5CYcvS$OmA5m}XjbOf40^!I9Uk@6Srp1-8C80ko0Jx{Y|u}QhI5WJel^}n`y)%5 z)H_8^fm45gne&~D^cvaAL~Lq020*ku>ZM4h!__GhP#V9vLUgG-Mk$+mO~>)UvD}-b z;4icpF26bU+I`7)NJd?G z3T*b12zjyc^H65Fh&_iTnp2m*fa2;l)_L+z^68b!4Ye~Qow!+3b!0xzApduf7v0bC ztgjs|H+$Cp2kDduNRwD$2w{QnNMtv2AT(_I`cG)(=e8ipp@NMe^P~7TIP!PE$>k409DG9HP#xpt(=XXFzfMlsMSrR}MhX-Ee9SsQP7}_pPZhBHMtU zOf<|0)Ym%hiCUZ{}s|N1;I!lq?m?E zw0w4{JqI6Qvh+?BEq;_!=*=zN)p3oaBikvG?D<~W!xXOHvzf{>&#;nUHu{zO{#VeO zgbK6Xfzyq_jX}1y_<;L8qtn#G){6<22MVzFS%xQgQ?122UTshe1 zrU<+LgSqiB5*Rv5lnOD4@n9F$-BUfUjALE{Px9;C29EXbjA();hHXI%cK{`&TcX zuDM#A_XYag?w+;+UVD#udH|DO4aXU8(j{RuMN}<@37wD~x9zbQ+sIAS{R;KOC{2U1x~LPe(_^$`VgKVb)f}Mv1@xxF04PIaUp(F3|{t< zdCFzu<%keIfD={02V4jA4k9CL%NjGd0;BF%p_VK84W4@8%@y_$_H#h`k_$vezX!s5 z2VGrQ&ZZ+oaUxINmUtLkxFB6V-9Q~Yd>hj{F8`5y3pvh!1#L-EDDxO=ccRPGh2C|1c&Ur&hC>q%8kHu4S4#pt|yEpwI27H zUKhj4QXZQMmRoLe)8~y~Yc`=t$8Ue5H_1du?()VzRT_KL=qUWvx@qE2Q7>D}K6TNc zOB8(=`xfRL6>+X0D9lQpo}ID&e9b~!9r}aqnJ#V62Q3xjLNicfZ3z!R@iNNULb3)*YXoS6ct1N)iBJ;z()vyIy$(51_*6ih$i3uLx?s8h<(a0m0ZjHz>Zx=0JMy9#Xv79CNUXd?dNPj@nI?Npd)$3-}irKY_3n9tj#ti+imt%8#ddv zv2nAp*|x3Co@%qL|MdI5{*ULuJeaw!`@GKccpqo8tRVF>w9S$|A(yp%_tVWGe_y0) zMpsDNSo@DxIF;!>DAT`LZktEhvllaPH7Mh$EoXm+82IM@xbZdq_a}xg?wFyK?;;tzNf( z0B`p3=2-aOY>fs??t05i`+5zprjTLjTb|C)Md{G7=ENqM67WvJQeQ|)R=sg}K8{~H z+smo#*V!u58Nb=cFW8@^dp3Say>D%8b~D3xk=SXvqsK0~`}1amMG7w*qZ;{kqYVO2 z8{Kiqeuy{1U)NQR6KoD~4!!H-M$Y<7j>j8nuOD0|zy^XM;=m z_J66Z!yFgH2|0oTA~u1e_(>v7gg*jdM+YP4!Po74C@!mouiQLfDyvcsJ_3q5K3;B? zzZji&$aF-BLxy4wBPC!DoH#kzjRQYM;Ap69vLBXw#Anp#F&Erx_7zv=AggA{cXjkPn>V} zCf2EMuIte?$PW96L1Tif#{PFQUI}9XN`(3K>E7W>D7b|^E2S?`lH=QUc`{od_2?C* zClxu7Om*GPI88Ghj#bc;VCY(-M=&ZhZD}r`_^{!Y6P7z9HMck1etjh=c@SQD-Lai2 z;7`(yZfVqdRQwy@aQA#t&A-OlU?=Um;W*A%rHEc!*FrI0chV-}U){gN_)%m})S)EX zd-CP*dFr@d1=oIo)^BPyN2>U!8~fY~sQwg(BXn(`ulLdvV;A7NZm9XUYC-+2T0tOK zzxKO?*7M??Ha>BJWh-~dMPwbh2vYC)+Po1!C=&>mlR*7onS+St`tH-_Dk&Ib+)%S# z5JMv1UL%*vn)1MHw<(}mGJ?BPK~Lzb<^SD!Y#06_;fm38VNuhNC^l8#scq~LnO3cj zyf)GhV6(B|wsm$b-u79?5Qg+Prq2S2>gPZc1w(D#PIh+uxpSJ&=Qzx|jLm{~5z!}G z>Nc83`9DLuut-{%Vwu>8D(GV z;_XK3T%$>-cb8R#cb7YR#QAR$A&9JGGxH)@W_1$1+9Ktyx3@je0v2lJ_S-hA7)T12 zgnnuBt>botDQV&MTY95s{lv4?-exLBbe21N;=0$*$~k*E4ear|=1e9UBERszjh#lxdSzl|M^Iwn!@=?RIpZe3k_m-FzG~obi==aC9MT~v zZKfH0ABV_~2>52^8)WGY2zkywBypi|3_&?rW)cUH)tuG688Wpl?)%WZG%r12+m+F__$Q`bpFZi61ZBA4w!{|Dzqj0xG7z(7l@SJI}!`v%+(M_cXK1A@k>+cAx!c_gxE<=yJ zf{ncx^{al@>NVL86v74q9E4*1Q)QJ@XuybCo9tWE;Ox?FD#=c~FlepB=Eh-$x@P;~ z=`g2#Q1IJ?0RG;551Pl1=w4CmfjG2_R|`H77jlwV3J+2Mops^KD8cSd2j7G@fMG$T zhBP78nI9io#b$1NNmRl0e7kO;1}?XRIi0JrPKxsHwtl)cT9ZsGm!54_T2h< zZTkrVJ`Drk$+liLAJ%zRm_y@jSs3Sb;L``ILQ?fz({-1rz_7;-n!fhs#mP^Xr?Q*F zjICb_@v#*%|pcPv1fbxJSaj~lrgd>3`gVr@M(!**1Um(2u)5x{_bqkAR6wIKE z$zY-W65-tdr2=0wWX(`gf~3$?7yD)aWIDIoX@**byCZ9+C!|EW)?2g^PkcYBzkUTU zmQu`oaV^uYP+v29%a%`wSu6ZDexh%sQ-Ye&?PU#;cdB<9EMh)H zq?%f|%O!o2)sI<5J7Ij#_sQ3D-V{&n`Z7@A8|jK8GrF#~G9&R;J(=2<6=D_MmFQiL z`#(bs2M$ytdClu~gnAY5<7wJv|wOlkQ5W6Ib;FPYsO8L)}T)zb3g292GIP#vS9ehz-S~g=abJ`#?WQ*{t%%L6%#9 zGo5aU`6Bsne%Tzx48mQcvC52FS@~G zZ>$7sg2uh$`MBb`1o$?(2*3G~AVqw%fJlKGL@(2(VrREj&V_wr_UMqS<3-D@>xm3>m5C7IX`f}&vU88bQ==dmjjvrJh9 z#!lDDiL7Svvu~_S_VdA=26wAJ|8qTARS89zOz`b&v+~2$cl=}W!q9pdO#%An&~G|= zeb>g*xvSki})q|S&VZaAH-Z?+uoJ0~?-i|oUla1O$fJaH?s8hh><84!)Zf>*!(+jZz$W76UM&Tw_-q{LKVh#v%TlQ9-9h_~ z&?DCuaE+Lv4!bj`WZF65-G47Q$d6)jOp>Q2#Vyq_Hk(5e`cUkXhu@MX?|#|1;9E2; zBxjYrV7>I!eV#QzK_&iO@51jXjUEns29NXjWN>R9lB@32E{!&J3@-UF4d*BnSLMP@ z7p6BTxwcu;N_uSE#@Ky9$!+7JXvgZASa#~hK@JH*Km-a(3+=}MjtD9N-!zbkE_`6X zC@ZF5!ahRr;q)jsa4u9h0Q_;T0(o}%*!Im3MqeH{f}!{fJBsgD4IUFR30Dc07|oVQ6SKsZW&|6U3`^#;mcui2p|Gpk^)TsOWO|PAT{NA z_I|qUjpl<82|8wR5SBJl?CDm!;C6=*8zDfT-wj!oWc&RBf|TO*c^Trgyy{Az4h$B} zciJ6@a6GLYZnGD%dzZ!oEP;?{^DrnR#TlhW%=i6#GT(i^ojYyO2%UB*Z& z-6F{Xez@)Gg6(eh-B+?H>WBG*7>x26iFR+#&Q9GCVR2tvECM>jL(Aw4BAt6iAlYQy zoNk%I#XJ(;3Os?Hua@i%(Pik-xeDT0DO?Mmqr?V17%Oq3rvMb9qYJ+euct%c#}S{t zH9ahRCL`|O0p`{VfYzv~IH4Am}RGWj`ey`h5VRJ-XN`1_&=leFXTgd ziwN#)VH;ElBcg&~LI{1Kc3~t~o~t~sq6hZV%{J&{21KJvyw9A{sjMbL!Bm=oTX`hP zds{j1PBQM{sHgvgUD1I{@wCfiw+^iMcb0JdFHXBZ;uQ(cC>1!iz*dMi#4-ZP+5Do5 z{ujq*yRe_HLGXC$DXuI4f;IUxR9=ouX3|?7xoHT z2@)s$;;z|2b83F@HNs&*MQTv6t9;UZVsHPd)?^anatm3?_q|iuycKL$YTj>JvrSs8 z&>Sx-$v(C=y<*(12??>HLhX(?FXeub7373@Q)%ZYCwD9F%e+4RNYhuCFJlU@?{EN7 z>16J>=L=f*crl1>duIX98s=e=ceY*+xC3WkqC&dg@2UwG3&o-PjAg+7Ik5PMle`9R zeHOBS%YDOTiFeiu?yxH_Lb{&`3l9?R68QihVB-GqDOYLFWHqung4P0l6SoRPbhNKa z>ZxuRHJ@x|mY++{Bwsv&e~r-NIfhSCq~uHU7M89>@l?MX@-vZk*d`UQUzD*0ziS`S z(o~f!N__7s40(SW>v>%ZCN7JphOwYi@I2Gmc_DCePgcE&V;3~5saB<0P{Tn_Sm?jnM3nL=u4FYwXrP2)`5H(6 zr4-S`94!onp9?Z#(V<<#Zl%+!3LCHNqSc{)-%;v7f}#cA|n84qJ^S(BO}c4 zHuXYiwQ!2UwRlfbhR1^Jd9Zg!`Xbo?3oar%W%so??ce@>`wdJSP0dI<4ORa`1F16N zBZ|tG2m`ipLnmSX=dcD+%;@-i={QnjsM;yon%16k^HQ(O5?A+SnL1WC?+Uu<`#(Pr zIPM{Mvbh2$;odL9fGvxhz|v~%GD4UOh}Hwmn>&Go;z*w<9r(7-S=#$pyQT)tGrS|p)(`=w}+j$cWsXDcWoD`IV)M0 zStaK{57cb6&6=F!FLu)?0er$eSS3ye6Apy?q?3~{fMgLIV{q^c1Om0>*Sin)VVls%uL%Ee5Rl}a6^S9jT_I`veLoE53o>A#yikM( zlmQyGNAfExVfxPKZa?_l__js@z9Y}3r^1N(Ka9Odm<0zUbLamS)^7L%9;;H+H>4lnE%VO?{2zBF%Y2WsCWO=fKdTg&<_nIndJS+^_Ir@AhHcrAO(~ zm*LP$rRkFh0J}r*%(_qO#xwa?toEe?cP(BlFP`fh?)@*G200ducX*u-|kC`AeIK%8F_T(oG zGHTTbeyg^oczS&Mw33eM>{lJGZ!c8=o=(srZ!T=1bQy=>r z5@?65*x;K(EN0-dXE>P!GvqP{IvRG53B+H##Cx;wuRk6!lTX0}>sl3`vG{lXt2@j_ zz^PGgB(KA+I4+*&!fE(mXu%U$?hx=S;@=hP`lSpVz@Na@l+QePwf=y6F8K?1izb~~ zhc60_h!N3Vyv%&vJWBx|1L#J7@F$IbCpb{HbD2yK)zh4kzkuCn3h(a^-Q&!DPjU|? zC4D3xkHg08smU?i1x3aWpG?~oMtFM9HOWUJmE5`&x92Mla7F$#Uk9~9%s=2iXnJI- z@0n>tomDNnk;j3YFNerD+8aBB2(7DM4;FxW(z>i3G+W3}V1FgSW@5H}6MFxU1rJ^R zC~^&_9CDqSOKyik%;LvG_lHZUDJu^OVn6qJ3}3x=&>Dt9|6@QSO&^P1tl{o1eBPk& z-lwfC2;mgV=j^9pl5u~STq?zufA<6A^RKzm{7xD#AAl_l0~0uxo0s71-I{NB>to_gMmY7b3EQQd z5e-Q`jD(U(C|6cSzI`GOHZ1OgTkFWUK`+s)DrBY%e3o;R9$+ieK0NR-KmaVavc+|= zHm-{w!%L09tm=OEd@w11p=^5PNoFniZ1-BMA`HA?*AtG(&}4zATd4LcWEBXZ#(mHs zw>*qWARY8h$L!B9;NO}WZp4TEr?O)4Eon6hxNO(|@!ja!b*eAdQvCzFwQxG3pVpJU z8TI-TMEogV2gkSj;u4Nu{x)~*FOV^*d6BX^zu3b8P6x#GM7+sXQ(>QD~iSFyDoZgaw!=m!N2ztCQAWhZkfQS%+;cp&qgHtU(gvW=8?+j4| zB8Mku2cc4(cz?K8NJxk+i^hsm zyP+V|GNBmMNw(Ly!*FhA@>`$HMJb}neAc@JirlHSbOqBLQ=SfxjdLWt0-=)dt z(ayn7|Jq%I@Ccn3C+$ovt3wg=Y896wDfa}YA&fW8Y?VI0ru1(Z%KP?%x0zWJKnjr- zUxg2?l0lxqKH`5}#=eh5dsW9_IU_mMxFQ@axJS(H3Iv4_8)(A&iXgT)f5jqg2?&SVv}3u4zMRr{pE`njDuBZ{M$ zzdtd0bf_+pLYqRnsqt|qp9LLGY?1DMPX4z1zf+vPJQ^kU()>{%`a&qxzpu7!N7PSX zLFhhZWo?o6+C5LUFaKkg&xb!R^mQ+%gZ37>I4$dR8T%~2gsSkP)+a*$U|x}5!M1fD zr}uV%$ZdV+v_!Oc_p^H|f)t=ornN50m{7~+vbGdsVYw{$i8@-t*uZkQsuN4gD;G85Le;zoU`o-RW>sRkEVlrT{HKFT&XyKAzm z<7TUeU0@tVvGa>%ir4!kytz`iE7n>XUe6CrZfifbmFhNlB%6PMo;Zn26BvsB=Lt;v zy!|+C2f*_={U<##M_j*+QEkHd@6%&fZzrwwT%rZ=!31&zKeZCPWP?2rCp10&9cglU zF#o3UY2Z$f9^7-{HGPRAwF%FCz|$gE)pu)Aaed=GXk3GnFMCkZOkD7Nlwa<^h$y(h zZv*pPC%bUAhxG}B`}1(NA+L7*5Cqbq$=P1?7ojyp!+HdN@7AaKE%Z#4s{9u63=vBX z4i}jX@9{fwwkxa*BOrfs`TE*M2`sr0K@KQ;*C@A&{WJ+>r2fCtRN zTA*zqI4z${^+8R|Y4>hW7UCB=SU)fYu#w=o0M3=SoT%vUkMCLWWA)mgU+1~(W2m2Df8q*Ud}P`X8=%IaIvY!WBdQ~trLZ+itd`Bkbq?{&;6-~ZH4 zITjAx9?kD7`8_A8CFc?aT|Peez3;h7c!U3r>#;Gu`TipI8igv*0~mfjJk(S1*66NX zF_m%7bW!trL!=)u+?)0l`$oer;m-0nu3r)Vt-*d%02h8=q)cneM}y74kO-5O&kpuP zk`gp0l!mqnVGv&DWgk)&-;SD+;QqChj|O_<4{w9_nCq_e8Pa|HsDe(@Bt#r&%*iLa z$)&rY5t2v$vJVG^tv$q`p%FDTg(SX%mo%-h9#N$G|16iR_Fk#!a1Fg{DSjz8x2Y5* zlD?`ONnJ;}=G-b%37og=0ebsm_qtFuGJK!hhM_Rpk>~yPmPbGxH&aPDuOOQ1L0rQY zSjy+ek;hUg--$j9;BCmOeWebS3|NWX5h2e#)Qg5>U?k%nE0pPXzvvqKRi3uH$Mp^SZJf(5r+~q9G^AiGdkLs@`ll7dY;mBW~j_jUm z`bg(xh2GhwZ9M-v)$t+72G&<{_}0bb;bt|k79L0A(0*CqQ)0Y6W}kr?ym}&g^Vz?K zJ4QBD=I&=*k%)ws5-25Q{pSmw?q>aK?YYUephK&dy82TPy#5kk8cb=l+XvbPJ561Z^q>~eHgKmDL$Ea7 z(=9}xRPFE{+59DvLq@+)(z-Bg@~M+yv|F$tC|4)Y1yIHfvT6sq4_-Rw^KAKojwz$< zs#%A0&j~fG%TJUZn?{(oG)+M4Dx2CaPY{GS^gZ+%!!X%Rl}#;RKYb{!iN!@ks)Q^6 z)1V=b0m4WMU5d~XY!>Cejy};^RWel4iWe>S3?eo^w(a**#+Go69M^bBy{%j@f7ToO zFFKV^93cHySazmaxR5MDO;6{~91oc&ffFKw(9U5dn9&q-W=|76ixPTzG)d=mqF!-9 zqW_p3q(wjhSk^8Fn&LlgA6~R$y(B3kkUGc;M%-7D4u***+il_Vd$_HbrSjUqOsL7H zv4~9leeUH6c|JcWdH5;y`MkCv41oKTyJRap5U}k`pCxkc6~<)}fco557~$%lCKDkr zO#QG2Z45t}rDmA;KP)APcTs4~3BH$dZ!G~X4vc5)^ z<=Si0B)aD7GF5GHA$$=17%oymKL~2BDvt<{z0`!0;QEUTrgi@7v1Yg9oCTib{>C;Q zjOaU~Zqr0rppiQBs93WEV0@-Zkh1Es8FaeftiT?e?<`l*Hx^zPS+URn8vv z$HoLfoEgpd%Br{KRJoY|~i zN-+$;9{Q>`$-XE7k^gGrufPH1(P3FQ!S}~K#mJCBKsOH#qpja`x6|qx*kC=#_!XT* z)lgbwuUI~vocTy}#OGn_BAoYm=`hD{^R7M~1<%)QID#~!grg?)J{|KK+iJ=_a zYV3BvRF~G+|4MzUFm1szf;1NR#?K-qdzdK4frjH@h3g{H5y*mKFgU>K_~|dEqUiZkfnnEaDvFtk)Re) z96eBZgIa*(FY?O{oK$+>N7eLSu0XLg8?Rqh{2`$Mw5mjKy|0fVC|D}i8do>%xL zge>nXol&M$U9yD}j7t{pW#zI@(EFH!}XZ=KU74R16K;{JVKLz`iFS z_A4BH-!1|B2NN{n{V+4aKnCL~{i^XX`&?mTSxFEa3MQ8aFSzU2c;;(Jt%u@R!eQJW*_7x2Yjyhx8hC^MkBTpWw-qN9TBkoC;W3e=45K)3qvQpq_qOt^DV8h%r3xQ z)A-TcCAk?=Oarz4Mr2j|%+cky1?Tgq)~l=zRSx~&ow75d|uPzepCVcoAI1=x*6)MrfmfeVUh z0?~M^eU)QD^jJB5r90n_vmbQzJt8Wqh%6JsMjFUi6UceM+J+677(`Nj{pU%jV?Gj` zoX<3Fu8)yP{BK{60zH>~3*L)A^hG9N4BY2n%WDbfd;&Mtu8UI+v8vmmw*BM)4{O2x z8lt4hX*<9KaJ0RaK4Mqjy8WLWuK;mvXn~+)Uk1h0LwarKFv%Mo%zsg+3^T$9dd{Nq+(x0($}}cf)#J05*JQ{8nz#>TlcdK!ge#UNj89SAMVzWPZQAqRP>S&xi1M zKVC6jwNgFE1T>Ohj4gD$Tt6*zQs`wqkx6!QFr8FJqToUoZ4D>7rtqHbQwBA^&LB3pDP5!w>+<>wND>cEx!I=dOuEtMsa zA6ttM`||Gdec&yY|2#hD%rh;K}cR%{gkg8&0p+e zJLiAH(KrpJWg7Sr)I64X5l%*u)QRODTj(!56;-p> zy#B@MbT+#k0*}Xw>d`DWA&R6ecTG2F!u{wFF@pKf=O#|#Xdk=}#+QPALz)UsGN+8O zc4(S0rFQRS<1k)z+Us-Hh!Q|NPdsNSB(dWNne88l^q@(4fH)cw(=_m*Bh$VQF>xLk zwl>9z8q6S?pH>U>aS3@)K97^&IF50j;PvtJ9mp65S0KkSs!hrW+<#-p5(Ug zuTP2@sY3X_wQ=>G6WWUvNx-Z zyeL$SH3a1TkAxo4`I3xj$cYg7iu{9At~OceoPJg=EO+f-QdxJHaV;v>RN5d$+po>~ zVY<|tg99i%ffT(M8ur^$N1W)Q?6I4?t*G_|h0Ze&L84{QxGdY_LzDh%Yj0G3TRsn2 z^X}jC(2u>=B)&}%B0Mu#Lp*B9!$9Y1wUFLAl~b(RFreGV^LszqH#=P@tj4z9Z$=Gp zLS&@6Ei^UK#ySjNAQk6<%`R2C3y(KilvY%(b4GbldzLu^>Aw2Tqw0>=i0$KxsU(cx ztkOxJ5)s`Yls?2QwmqUig>R?n9qX_BZ3mx~Y9%Rrhc~*XrzMZ;;}a6_mJ7&%1Rh^kSSmR zQxi@?>#ekd=C};o|G5dqkwB}sGVn=oRaQXC390V6#(gGjh%HEgW3zu6y9e_p97iFs zrk|!oKZK%#GNKM^T9(+Oq3J2LU7Ak}__)j>e*rg;`OxVY87C*_Cq(@1GFjS{xs_)Q zj=>8d#+gZRedAg1`;Xc&*#0v!MU}dVR-oVQ)qB5NGGs=^(IW6#`=~@kZQ<2}$LmoS zo=gPk5E9z^LGbKTc@0qmwyDoiW}w8O6Sz!tJT~24f9607!GDi}HYQVX_H4kkwhyn& ze{Mxdg|=8@WqrIOVmhQFCeCJ0aJuC*4YaOFg9JRbL;$t?@mkHzsM}BB_&-#dROzJf z@v71DQA&*tV}8o-XqDE{OJ(R${Hybt@N*5rsKO{;d1GSs$F@!BY+u^x(#^j7g|uDM z{frFRl0N~)6I@8qyur$ZW$4>HCYEV`bBo$qh=Apt27Om{2SjsY^=$9dTg4jBd;q%Q z=8AXkjsOi-eu#u`P=E{7;Kf$;3c%>qGU99@jD#=#;cSMG2vNh521=&$Mle(+bZ~zjHw1um5sdeqH~hKuk-_7r zZI!H&U|-Ah4eK2`8#~)7g+iTaO1AJ?NN!wARL!3R2-$DD#(Biu2Zr*j`aMeZbK}6{ zLLjW&oEhBOJpwLy(XjvsNIlfn{dz4TZ8I^;ZP% zA3zMZ7#=ttknf5?U+(-k^ZnWlL03F#>frA2ojTxiQnj`NTe#mK(tv`|)b4h1p4jUK zf0gD>BD;!>DCQT%v9gQSZJ1WJje81an;eR9`~@tuL{9T>at0CGGcu6}1HTB5_>=1D zX=f%d$;ljPZ`Akn{kpZei*aoIQu~)n)^obMd(R=4@7ajM0srziQLb9~aAgnFzx0r} z;i_(as*79NG%b2Jl+EzuuXUw>?XGOd_DtW0SOlR3OUMNGV^LN%UW1WfOaEsOrB&1$%)X)kPC>$l-?f1|McVFlQ^K}2DAe-!=n{s*Ah`i# zv_vPpHgiCvkFEZKh#y$W<-e=9CqDFf#>#<#t?H;JJPH5)6A`uuf}HdK?Ev5Z50)ZN z?lUjmITjof?73>w#46MhVBll}{=^g|r9lCtk2&&a*F&HP?M`S z1sIy~LCtT%&zDZBsqI}eidWCd;deRNQ#i(4mowF__>V%6dM9z@dQg$`_@;ToRaZtt ztjiGJC-7QZ0My;U=FWbAa?*JpN_&>!CXFjpCnhJvl2y}@Rud?u6T5UE#P8q=WH}eS zUlxOCMV=|AGot?X4s%dl_p$dIhSTfuyUoHi97`k#t?s%#+kd74hZ(T_j}Ci8Uq?R5 zcX1uyDH;Z5*bx3GjZ;M7Kyb^;xX#z1&S2sCs0A8j-d-%4qC%k8vL+@5Zgs+x9MzbO z{{U;G+Sbc6Si3*&D?8qHi5(yPsrhhbhOZ_|a6NRX6oKUQjZRCd=yII3fZRo$>%D~| zHvsi%_^;^x=Yor`IC%$Qji7FGr!#IY&EB)n2nf(4xbZ%aALwxkR{8oCw?P2HQAtzz{#qY~FTPZK1M>-0EvcIW(O zjka-1?L;dDazeiq4q-W*dAD%ZHn!0@GR3OS4>5%>hi2cyrIi8IIs=q9%|8dk;c~3_+xKmGR6!BH{2A zE|m3O_@Od5AK8PcC1BAg0MENe^je8-rw{rZGBp=P9a-~FeicTDQZ!78#0 zuquf#oQAS3mJc=?i+FrLTCuBfKhiAy2|Y&U4($m6<)Gk4Q60WD1RkLh8$o#lhA%yW zY=a-oUhC9;xlf*J|983kvVIQ#PA=f_W(gyNwhuI;i(oK3t=cPy2r+ZP^4P^f5X#GP z1@~auPqn)ktM79hPusV;T)p_aa0N>Pkyf$!o2n))-sQ*}Mj(zZL*~lVXqxBkZbW1* z9v^&A2l(KvGlyUY;J=_xW^fKewxHaZa(e?hZ>HKk$JW2DR91|5Ri88acrK-S6SXz% zP(u39DJePaB~2e0BKWgU^T6aqu-@kl?{tl~){qZSA!jV>UV$z_vzw^E4 zAIjNYKMILUuZr#my^7o}vTbc2{}rH_4Y<&Zj!y$9eeTQXdGSMNLJw>bI|half6eJD zz~ff<%#zGCI3Ts!3<$UGW^gaAyz-W7Kh@n#Z}9`v~i9?{Q6`+uI8) zAU9?vyUq3Nn4=pH%X;*5A7f{m`wV&hzX|yxli1sIB_Vh#I(UnR20v47fGgS}tM4ls z^Vk^$Y68;qmagQc^RJ0CKG|c=KaBAAHYLzb+0WUtt?KxFWkKiLy=Q4+IkRZ1+74~K z1s0mV#m_(2pR)$c-$)%+y8pq{DPI)S4gx~sB;`HDIjdAb0bYtJ%tBpKsDhwLR< zS#DE_dFJ}0A3nE~9*5Yl)z8JU_bGa*u3ZOGDC-=GuvK;D6@*o&Y(54~Pv(m}mU@06 z>#jH5MKT4Ae101sE+dXw0txDKZfx{+0cKBpDH!04Q3H% zu#Kbd$R_mSz_WnEpD3hhYK&L4W;#09GW~u_hyK$6BTl3YC=A}&+smntd2JDc+%(^e z4osq6>~97FXORNG{b=3BcKxtbDGy@(mVOnc`!_PQIf9CXm>z4ol}Xi_`Bllix=CU7 zrYHQ%N7!7Q^)QToFbAX*0|`#$+wc`l$y9BRgnx<9#1RM0Aw&(p_i-mNCFQ^$4`xs} zhJJ3GULt8|4*70rmA<6=itNKeA0KZ3k(XzFs3X2cz6l_qxCwnDnQ(BjN6n?-&lh_zCJ}T*-y)z;{8e_*B6kdF)N`d4kc%Ztgh|vdLz2oHR*~Slv zUa~z9LINFZ3jysKH;VQ!(%e`9UdGv`_NeDK8$1&)DmIy|yIOmTl*5BPdd#14O&sOn z(o-4>Pla)oE)>8^IotpZnE!x#EDolso#-}%<}up4HqY^8STgXc)N`_QYc;ZPvRo}p zwVqTNzSF%iT?D4Hd7Vq9^O#lYxt$2y!KX23>eWPnwJzi+g#T!(8vpa3!$OeD>0Uuo z$0v2G57c}fH7s4$U>sGls7?tXy2IH0kso_BqW?5kDM#OG0=lxoY>_nZ-k0)tJz_4> zeE3oK;|*u!R*A9fw`%UwAAOBOas_RL|8DA1WJKR2ug_^t%$gO`9VdE5=c>*i@O|$R z99L(g7GLd2@cEpjQA!(aBx*eHx|K6$hJV0oKu;u4BiSIG89o8R6;Me}(!HZzqqgV{ z%>P@#xj5D`7bK_Fzk3P6m&ZXHLuxIiPM)cCfjPGI z+IA!y2K#O3pd5wUwJ=oQumB7-M!4bY4iW|pLTgAe9vj#JLw1e-{tF254*k89k3jAn z0IgK|=XLcaKhbkc&pFOr&l>xMO&NOT0E}_C#!B&;MI+J~K>n`$qLsJU=l6&L5u6TF zGRI<^cNbiYRx-Qu3TXW9`u#F1IEO?&%6*zzfqGH+tSBu&TdOY{C1% zZ#(cLO~~QMahGsdsvdfr*(OC`#netGK|>R#eGzeA9B*5Ynbnd|t5H#efujhO7liJ1 z4S1rnRBe8_+-lGcU(#*<1wF%{-^+jy*6FRhG6c!vPeijmYe>2aGI9C&u+fkAo{#eU zA3iEX{+yDf*x!3-wWb$9Nld@z0mjxZmUy}MkBmM@XrA|XuQKoJqop|;%>*(U&eVI$ z(Ztj&=CV;ftCAs^8)}y#-C;hq$B~0NPIe_~r7coJ-*#Em6Pn|b#9u7{Q zbxqsOBT9UsU5zXgzIKng;)T5ApeN#)FA8UJOs$lZ?CxGmA9uXy^Z;RlVXMGN#SzeC zE@BZ!%ruS2}I ziq8yq+8Z!lILCskBv&jEqTnOSd;vW5B?5Zh9T#BWUYQHhRY`I`A0&~C?f_rtA&As~Qz?IAmBlq?e^me91VO#d{aSlMpq7j(kW=L)9!vepiZO*|?~l~j1JAAVqoNGF zgG0*+4|*!554)^Vza`zuZ}?G9iUAF;ZG2&rV+LN0NgHjAiSKWGhLzewUQQc>bPoI(mKDuQxoz;kPgdBx(ooU72P{^Q#e4Z#5c}^)7+q|VdA7= zXLjtE{xRFNCLxo0ybqmEV2EEZNkRQDEcMp8bjQQ^u@k8VFS;5;3S%)ZHEu95L=GDrdUoW?j35X4s?;vjr#lHif+IXc19uA3E5 zr=fvCJd?OKEfy9X$u$~q)8|27yTv|_eOSy3aT3ElgqDh?V&w$~z8gZz>#Q~Pgy!`D za|beCxr_@-Ox{suzM-vSr}cnn6*A)|5x+CTE|Lc-1NS!Da9#J^UtLtwfnjU{rv^WH zK9AwG;_$78k*_3gqif9W#B$*81g4sf6SKP)38wW zh~rlN0i2mflpk%YojT?IdB=UBV(5h7qOR6&m+X4A>dd`)%JC<(OS2gV~)=TfUh_W6XNr*jip^|CLcm6F3eEx~Ca}k-k$D+o~BF7vB zQR$AN^IJw7C`ZX;-W90*g^kL{U|G42U5!fR@U{T%{#TYdJy^6N1lELjbqKsjqwzka z;k4x?{^O{{Twq{z+uL$X;bf5e|1N+9V|}v&I~%h;kc*@H$KWw@MJqCIW%Vn^j#=}L z(_d|K>`6@!dCKU}A8^}d1k7=qf|x)7H`Bna08K9DVMqM8jth5yQS`1!4<=#ba0O7K zhu%~<>5M*#eMGO7?|oY+D9A}`Yu)EzV-D;qY3@}3);rYK&E4)@@oqciTJfOc_)82| zG1ZB{s%_72@FK3%f()K^bT#4c5;aO`>UM*<|j)>**9`Qy+%Es^>eN4|Em9o| zXh(0C{b-JNoSOV7l3qXdn>-;A_9)ZA0*_%rvss5joXwY+IcPiyezy4L&6-(Xh0Pe; zqW$DHs2cY81$X6D{`dMbUWjV^Buh85gh@gR1E|$l0?x7YCMbhhrZ>K?Q9-j$>&G!d zW=>|#2MxzgW@qlByMxy=AA&R_-78XN=`f33b7CA&!t{2w?mxT-oYBfPxkWthEJ#;$ zom|-IHTUVUi9CYC++d4ZqOrKJuJQdS`D&?(5h zl~Z#q@m3_adD&5{c%T%y<)B_aikV9%4Y}-hjCU`2&?_nmR<#!XRY9lMTqmGg=TFj5&K*#aQ+5raN836O?#L3`r*B|jSa$Rs9E_V zM)A9L8l=y2mNXq;arpmeI?KPP+Hecc&G~G-2&3x z(%s#8Hs}1#KQJF=WxU|aYqT6o&nczFO~9q z+c*WSP>Xl7ymyRsRV<75ySrn%oe`rxl^{m!hmue;G=AXwE|{V=h1My$blq#O65 z@;FdMx#^LI5A~~uyabQ0fh)U_dCj=gM8Qy$%>6g|52^Rlgjqx4l+A{CrQ{jN1!VKb z<1j1mgIzK`?`0qL;~V5Z6SBx=ieSJj)jol8Fp+~e@!22bp|PgpFd<64o}d6N_i4Mn9y(J4><)EwfBf4)Tb=AN&q358|-F-dN|~~j2y&ehMe#CIzBA78gpdD2^}ZX=mLe>BSt1F1 z6@>YB7y2EfQ7Z&u7$ybZ5F-^o!)|yk5*UL#a+$PN>sr6_zQEpH2n!dJ`5R5v*fpm- z7@xzSv;zU&Zsf@NPdniWXHN|eVt>LP$WKA$cW0oW%ZJWtP84Skln+zvJ+Z zpWW$0@!j|4T;WSvt9>8GfI;Rt%M^rA?4;;g!C-Si{_t!a)w+-8KG24F|pjzaqxHJJj(q3r!xKuTK$Qrk3~b9YG@DTM?2@xr5OF28@@^V z+u3gnR8o`+xoZm*?4sL^ks?Xi)$@k@-jRiDAP8=k$53Mc0*Bpk)!z??J9jw|{DXCl z2dN5c9>Xod_r!%(7~|Y#9Z}&Urk6E83@P`4*qGfWk3n%YdP|+p4tytyNbk?@hGssk zVUs?9QUL76L=d}SDia0wm$*R^#V(Ap_%^J&;zP(aEb(U0D6mT3HgY*JV&_ZkakQG2h!XQiY`tKUzomp zaLgrIn?ENM_GsjITqmGkF0{K3b&Zt9K=yvBd3Yd(Z#bpeu?fgyoC`>Q zgcd=@3p2?kMM~UjGhQnT2l_NQHy|;E7Zg0x0jToBGxJe9;ql{D?^C- zW<`NzOuftp*4tcbijVl~fng*@4=kYg?U{6o?vwY5h%ui!4LHqdf1fwe{*2YtNCH(( zz@hcYSkIcxC4Qb7y-~DjDGuPVUgTOQe3^LJ$6UV>=$<4swSX3bs-I_Kp*M#X96djV zD%<~RkNr~MJtpUt(RM1A)0L4|W35Q_qy1CuquGR_YZ)r}WvBaI5>R5SOPZrI=EXir zbuo&vnB^^g%QpOHe!p9BUv02#$FoE8@naDIMaK8&?>xFXjMMRV{NbOuD<+lSbtL%5 z!aC@hk(K@9iBXncRQTZbgs0gd<|(w33yCN|nWUwSZ*a{*6V8ttPOACZ?9{XV2q`Kf zo0sx??4RNBD(}3aqvt5{78^0t+_YIp4NJ!wZh=L_aIk`x9W^qn1-548srBcEWMWra zOJf5OJz%29`|VHewcG=AYyDq;5?6VNBjiO&{&zx^<`LJRNea@g(lxpyx`p(^3=?r# z4$uEyOw*es@q;A?Yj7h~75-4?i1vm0sEfQl)o>jCw~K5qDvHt>LX+V!X{Ms2!`ryy z;sBzT*D(}>je?FL1qR$4TImn_(CioqY}7k7gP% zUe5n<@ek%V0f+KThFZSSPfeCkyl19U&%2gX$Lc*~e<3O+Ce_7;GMey%ikQ~nfoR2& z$Sup@n45jM?G4n2L1&RCMxhq2$)+B8D!8vDOmc+fzn#7M)*bj^s0@-1jfvSV=s7QTi~{m`O_BG8w4&`A$_Qk_;Ob zEeFlOm6?+8wQ3BU0Xx&vHZ}_`MvQ=GE}EW zY-N10S^09NP|*hT(I60vPK|K<(p-ZXB;lo5I-1NU!=<0hYw*iwTDDZg?y!$fMS9)twhIAj>K)>uvzX~KA*8m2GzIE09%G_~ZN%23Zk znXI-pCLGHT*gBln`y6-vmvl~iAdOM&G3}NdW$~;9fX-(AU%FO{sa2^*JI){yM*k!q zyU3?Vv{dF`*Ms6LFAK9)b;E^*0)CTGYkGpDfod*F<&oE#t-vbpekRP}D88D%gG6ID z@iCFsOs-n5vtB}47aQ($_{s_xd;jFU2t^dQEk|RUug`1);d-3gr{zZD{$MI;(kqQX z;x;y}xAe=rf8&(B`i6_$lA*NY?GmW;Pg}3{d(wwJM zNl=%2-%;!GWFo+gSUNIaVGDbYMGz#Xu_eJB{X4SKziR{$1-Uc}6wTX7uHjzWSiF&d z&bY;bgP_4s`OC;2Z;si_%uJP6O-;h=iPsb3NP`#&x${58|0dJe41d`ZYW|jRFFdLX zKqkZSBPOIlV?1J9Z;X~e_)dyc#0Ye=P)7>-F)K2qwufZHGO{JvcatT5gG%3>g5l;{ zjcZz;@BZ1{ESR)(nLk|ZTY>$HC`7@@d`ISBakj)NEO^c?G?eaFT!V=aa&*Y?T{}!} zw;tq51eez9n<9!O>j|4nq*6EmMEb6df|=869L)`AhDn>ng(B;dqiPPKy`zCE1VbUg zKIv}UpsbIXUB>j?da-G(U5LV1uVw{j1Y)DLzsD831M$a-KBa5@6F;;}G}mHv?b6$1 z!zLu<3`^85LVfzmHsBU~1w-@m^m4L%0M^pS1FXR|VTdB-E`As~kp2|?GGl(z?@NgO zZV}6|OPr(&dZ>Rd-b%xmUl)aq#VIsYVTN*n_Cd$%ep`Mj>J{?Kg2guDX_6H5H&g$7 za&9kFkTMeZ@4v2LhuDd!nVcwe;;mzuf>vprrCd>5%D%sT`*46`$p?!lGt?ji>taxv zO_JCIptoD~W$W0O*H~ z>{;J;a!X5xCDb$3d{p2XPQ10VuW_KxkB@UhLy{+!xx z(of$X;t@r;P%4q$84Yd1zmKhYzmO&#(w%Vaxe)2vvoReEmRv4KlV=J^N zgI7A5XaI98kp|h~iH?D(`?tEbz5j!}rtPBrMQrAFMGiaOT$T8lj~m=@Y0e>`?B<|M z?Yk19#jjExWO4`LXBvfP84sV;euRss80NTcuypVyA?P)0B@WIwndAe3@N;G2`W1ob2Y1Ac==n zHj**{w>G@!pgt}?&Sc6r#)gbvnht{T29;1JOKtAYZb^9D@FHa#5alVc>(7=hkdBbs zjiN~*da44;v{^%O<;VuID1s#sg3FU=wjtKoRH?$NhEy&HX$Tm=tH zw#2%4(5J!nSr32nRuWqn{#K@w@Zxk{P`gR2gAdkT=Xuvj=o2rjmwp%W_vEz%1A{dl zz=nP!p+aSnBa-=7YsR3iV;}u=HAHez;q^uO&jSjK5r_3Jh{FA+g}s7HhjgoSQ@uf@ zV}^nKLSRX*_FN-%btJgoV(o6W*qUXuEgMzQk-@V1RM<}9Y@La%k>9gB61`M*ovR!e z&+a8|q1p1Iyq11`h#|gAdzmJS*Y;pZh{vQP?8T`3sOhgyaoY4W_`^z5&3x#)hfg|W zdSqH;9PB6R5GC$DS}4isQYb14XIYXA{$J2X>cZ6DxvvALR){WmYp|A9HRT|e|8f%8 zCUy>NE~GVsvB`!nAa}yIRbHKLKD7gwr<$Qv`TNh-o2;#{sr5G`_|C4c*dq77dhw8b zE#08G7k6(aA#2_ila3WTo3WLKU8-k<_FeSdbhGqzSVv6Eg~k98-4!^QA)pfZ!g8C7 zHRQoiNckHPnb(e9DSn{{_i-b2I#8O-e+wyf9v==oa~|BOdS`Fh+9xKMilUMBc@^pm zwH+(LERC6-^nrbkp(5j_d>z}Z|5P&i^kDYp$x-a|sRlEmf7$)P^9!S0n~R~<&rp?Z zvJ=KB3STA3gCLQB%=Oo7|3B-FV+6U{bP$uqS#lEI{rAE}B3Xx}PxgZ{J%8ju$y&1C zW0Aw@DIx5pUa9%}*5k`?2ut;ZnBN7UI+9z#PgwDoJwgZ;#BDj~J)NU}3%y@ooHT1c zX*hghaUt#WjZ8obsv78fETu1B9|B~T_1bO5}zU4(CLw@S9X{9oiP{-6)K1jVGDftD#=CXnsntBkYDur!%IO&*kHPwy5LD%?QphT4mF9cf zmob#X!oQQ}Gwl8`-s6n*`Gac$cmM))SH4CH8GhStiVL=hl6zqMZ>cW-uWyh-iFg@J zH$jjnKCKOuMz_U;+RT4bXsDw%D~a~o?Y2;h^(kWfLb}#Ua07ktm8FtK+`fv_lFt)c zJEBZrWi2vG(s;w?e>n8_vaNTX)%6?bTIm*(pY5X@YZdlbng zGjR?Mcy3^p)^xg1o|AfA8nA)Zg7*9F4`lJMjs3}>SS3S~7oR%v1rtkc3CZ(qC?&&e zHj|lXrQN;1xczvNi~*mah{Y&}@d^bwbzRylW+XU=VGolI)9|c%)Ces8lSO8DU%`X^ z%HPZ<>Rl4t<3oyvDWEXQze`B)xcshJQ5j^wz#bglsj6u%_o=REx}*7_cQG6JIVx1O1z*%UAc$G?vtIme^H zw|q07n{iDFFR;Ad;b&pd6~6!PH>dUNqS-I~bd_bo5AmP+unJo@V|tYrE7WHVblE73 zjkGHA9oGZag1lgLnQysz+)?2a7HFzRT^%bU2u4l~AP3&z*=k3mFV{~ z>!OsZZ-gQwjimhTv@qf_@|5=`<*`cT@M!}GC?Ou3ZjWkfEic}07GsJrpC}OH5QHRA z!bnyALp-Fzih|B~()oVe=bh?4+kraDqs8zk+Wr@YP#u5OgG7@5KYIA#_NwLRKdFS~PHLAy2l!Cqd!!o~67bshrLOMJ zk3z80mSD41ujlKzw3zQNIgZHlhy+pFFE}?kgVwF~b0-D^dRO}02@?N$^G?q`7?O{H zD)7BQW#h#u-5vVM>HDLtI0=XkKBq)3oGwDS1G2uoY*#< z59Z`4wRy}ing|x9)=GyiiKbT**iGO6Ut2eaz9e?RQ*rf5{CRP zJ4q(|9jbSn6stadc2c(3k-cN?9@3~XccK*5+pa9E=?4#yKcvVyiWyjbOD9}>8#C2S zL?C%x@8zzB)E~X8 zIA`n!e|3Vq&j!G}Jw;r+b6YB(?gG~ih0|Z8yf@Pyx0_=l1DHboZlK;RGc$Y2KR`oM z`}!w_c;D|DDTCsQl=B-u(I8>dYqtC`dXIGo7%Clw6M2QI<9+GXFfL$NF}{u=3bT!# z5#3cis*N5GDxp|2w%NTxX2tYh7!_6?)#CL#&U%4OE?%)FZ?dK7u5CBu@?8p!Vz0Zg znB{LC)4lO5dF)Z*!x+2_b7sUs=gtXJ%D%3UC?bv}jYSp76}vJ9DDD)`(9Ke7vT_d1W#L!zkRBSk zPTT@Jtq60g?cs#nM)Zq)X99MM6-oG;q>@k*&w*8RxEh>;4ITy`B-LRBeF!x~I83`l$)Bi;mT5;zIHb7QDRnWo4p*KgRYiptt6 zT-@B^5TB#ujnB)&=$Z}--H0OmQ1u;Ap?Ji(;tf5ibEkvp0keeIt5)ij0?%yl%f`(d+}w96f*Hvw1jvRN)#gKZ%4N|2I9|B z;@riTf?`reZffh6V{Q{L)cf17)F8AtuE0DmDGv-4v54hU$%J2mj?uNry7d&08G{$pqTz?l#79zTFlA-58RJf*x-*Z0Kc zu>~hj;N4l%q+Bq!N3RLp@|j%${00YVZ|L`Jas~$1b-%OEY41sCHnW!DHSPh8ncrLM z3{}4*w_eOO2%8moJpAzMJAmlj1$t^^W{Tv+#_$;RUh%`^{jie>`!oUmuiKu3AUDh( z-J>IWn^Sig$YhjT@qK>;Zn5Q84)nncu0VzhmyHC|ZZq%GzYZ#y zzct^3>5^}cU87^tXdaLMwd{SK9Q=SspH=CF^S?_eK16&T8&2=tG-v*f+*IPREcrxi z3W1Zahgq~0^djy8tcd1t(CxAg^WPn<1chRAjAWXfr>0ma`%29;}H-L zNGQPgBxWNjdH5h;d*y%v&|)i(7Q0cErGoTi{ebVz5;qfJEh|?7)d2h%X;ThXIN@_D zMsR#WFLI~kBF37&QqjKn&YzN87x6ah{U{Otj{9v?^n!3Nuk@;h6ytN<(~=e2<`S0mOsRXJ@BoTIss>I8{r+dRT9qe7 zVZ^~5&~fWzp^g0EysbI`-vvg~rW5o@%I;*p&5;rOvACypu%EM+Ubq*MUzQ(m?keNx zQIeCHmaa|9-B2Xe_*3mHFMnVvqxpU-)FsiY zX43nay=)$ALp!DvJ)bf-&7!Bg7#I+t1c~EWNq+_ccwfo~dp$Mpk$0QIsS@|V%wk;^ zR-yrN)jm+PZvQq?`3K1Cg~99cpfnBfyp}s7-rY+UifGJE->iA|Uz) zqyPEE{7_DDzknqg7OJ<)U7{f(fSQ2!y6YkjUjVYkX21$gF;0WFJZ5g`d(a@nPyc_ zGe!5Xr&T}a{vfRrrpakEwB6TRz>ES|_h^ZEWuK!r#6fx*Px*Oy13+tqUTqhyG;tM{ zWZ>pJHFdmczUi1A=ViHnYXH3dgdo<+J1>t5zDi2l%woJDpVr*KEF~4f-!z>JvKtRK z$A_t@n{ch}w%Lxg@3b&fw0;{^91*}ked9fq#36cZ2qV&m!ahHcew@>DE?RP#lc*m1 zFp{ep+U$0qx83jD`%58P#fFiYPIqT?fp*oSL(;sUrv$S+jfzHDF0FxR{u<%-;`l7q z5*);nh&0%irbRu}Pvkx*j3`JgRY%wlcodX2A<1#7w_bv8k6`Qw*k~GW}7!ASO z#h3DPU*EI8nOOpf%uIm3>By8LD5a#)ql@rQ%fL=t63pDmlgTG1fIq@+GD>VBg%z^n zF%#Dp$+yTWs@1I{1-OJ9CNQyJySC zCY0eMywYmr)7>qLe$L0{1WFqX)h1bNUl<&_e43!;;$r^gggaL9Ttha>uBk`aCi>=Z zaVk<(ALVapN+-U_n~L3=$SDwLii??a!s%9sWK#16SSA1U=AX#blgSoVqn`#4)@%5t zi^S86l@U4i#QS*3OZRclXGIn{Cyr7i_3FZeeI=t?3NI>#jx?A*8G)KmYMd9h(;y&SpraD0&T zJ^Gnu+eZ+c-=1%3%~%XPJXlP`nV1wow)E=&2aY%>K%uny;YaP(>ev@kVp04s>gwKt zx|%`VqjxO!pQrLiwt{?MjiwJ*2XxyOzs5)okh6Gzi#Z8{6%;M?9bqb12CI#=j8A-- zs{d1s7rdQ(e{Xd}Z_I!?XDNK1%p()~=Vj*eXEA05J8lHjdj(X`o2*HzW z_Qjl5Dod#;Pao-n>c`T}F2-Z9jR98zKi&~wH6)H@H0Q+ z;;T;bYoQ%H{Ju9NWz?XLwN0w=N<1F%tE6{9pW@3==H(uL( z-2N%rv5G6$Ns8CWIg5AqrS0*gMJI#4gt-02MU(g8{(>*K?{`iv?6N zZvSxLD;ZU~Snfwh2V!Q+4hN9`?hG`ndY(d(f~D%rRrKm4L^_7^-a65L$b-~rQf+*- zofeF(z&;AY)uwBmKF09p8IQ!YF|ei&a5-jE`F#40ePIMvY(v2lKoscR&bNLH`f6?x z#t75KzZwI?c4$O+^Ui{Yw%F6mVp1q%W!mJ6UqWdMjipJK$bt#pz8 z`liM68Mfm|5NB6Zu7Y%t&eLa10#D~7(2yAiuu26KQ`#{qBN;Hb*?+?JY(;LQqCD$; z1;tAs{ujX=qQI2`J6p(GpRSXtF}_h0%mw2DpN$^J-K6!*DtdLYtflLptUwfmf@K&P&ruJ5N=b%uPEy_ZIuq1Cv7d!ch`| z8D*_M!x{Z?6F8ogkISd07ws-p_p2@3xw!tSD4lA0z*pgU*uOWJy9u_l=@2rWuglq9 z^S(|{mggHe2BrMmMAsvktE_J|3N>D&Ujym$QxgN69tYtE4I?568Ta}p7w>g3q?uDs zs{iIDQX7KUvfjVX_WlpO^Y32MZo2(<->`-^&hjNwQf5E5epsC)Dv?~r71c!$@g%(U$_DiKVrc0tZ?8Q-=&Zmg8v?|olq)4%IT;yy^P)IaQ<3z& zO;C6we?b-aKh5Vadh!-t5#nG65MbjXU~H7^B_#)|UI0LPUR4vGM%GH&p<)ZNIwxc3 zKN|jb3aPwGFVCmmEg@knaCPQhWe-$g#u%A1tN@+z9{ije%(!T3*-Lp zsexpucraz;v>@FrS=(tMbh73zj@gRr$m3N{N8{Zh|7%GTzPWOAnO9l0@$8Ye(IlE` zp7arYM&!GNX0tMf<4Mi@6`e0`4s+<(j&8UGrAy9$=S2j*(!3;2FaP1cx&4Bm#;Yf$ zX3`^=va&EH*XVW`%c7W4arT)lS8$){KIE9&;?30^((rc9P2y!N(<;1H>Ps zW^?78fq^3;{$&{deUN3`)!rqs06gX-0<++XRB^P*h*`l!|DDsJgL?i!O||41s=(Md zmh><7E8+v*wf+;v{#wJ6JK+F2afxC|9OJa8#1e8#3^0w_WAxfAmsWy*=^e^kv2j=0 z(1?y!Kw<(L~PBB_@HOF=IduN&dsMVQ_}a26Dl+x@wDh-ubZij(uh; zaVMOwVI8L6nP`J4r%z4m`TglVF;X9R5!ase2ahUzWm4rrodwUrEi}vSPn&mPMy}CJ z@?bpf5KuFIO_b;LtYS7>8}QIvOl2xVY-b~Kjoc8(Y00OgR+0K~H)BCGnKPAULCcOS zZf|HbA`8)1qj@yE>VqZ(l%D6MiT5KE#M>zvvfZ zblb!w9wU36e5$sam)?T+xcbF$gPs5=^y;U4ov_8_`!NbJX!JAZMqlX6s*r0SP7h43>i5 z#Tfh0F1+4IpB>7)b{)W{7z4wjlND>fv&3V@;qG9KTncxSga{2P24Jmvqd+36Q8ENp zSCKGXpY9s|-GW}bYyAUj!>UYmOwi_LWCk0L4gJY`d-C=)T{arW1n}c%lwQ;3*FdA9 za`U0##HaI3J*{{?pbUse(an;$&@yJ6VQuC7elze(<@%gV&(2Oso<#;2%B>ho=2=fw zX#T+OH-7%$<7eo$eA$hhD%KzW(ck1t&{--wjO_@h*Nh)>$$7vBWISD{B!BnD z&Q8U`Jljd37O211W9tS#pb30U(6$S11%Q6#n5rr^;lHyV7sAL8f6Df<9nck|ZbS&) zMoh%T;r2Q8U9JDIjxUvsJHJXkcZ;*~|GC&Yz{H-yY4uJ2M^P}h<)Su(`AK=jbrqm3GvueXPlKXb)V z@3p%4KlCj&nc8aW*fH8uT$_C-`e&!}5H~XP76(cB!iO6|a-3q-T(sB2ujc5JSZ31LoDq)xP8fc=cxKc|^C+R}31BL!$SqdnuKJ{1IW)nUCCn=Rp$*DgQ?mNe zj>en$`)ai!3%1gnp;HB|bF#KpRKAi4VjCrrh}Ku6SF&KwGgP3*5~q1tH`ewV_$!sd z^TwQ#w0%k9TXc+|ZB|2!d&?4=!%uR_#`{Ra4FYIfTKxB;Ri8$H;Qm zXR+wAYOQIS#cygESs#s1)#cHQR*XJmyp=kyO>!T((%Rk|0J9q`BHB;mskoO;A7N87 z8!T|avidJ>W!2c2b6D~AwpWqY-~VP(qm%~`(eT%{bUGO>tk$~+{MrH)T3((XhsLu$ zkyL*9-tO@ZgXXz%q6fUgKSd@CV79gaKeRnW>+g6boIRY*t9WOX=t$e0es}1Nx@NIznnU6^{DsW#HP`3lnsw%r)TL|TRj$Yt)B!8 zc<{+WT*6%ty|oKPHgj@-1H{V5sDSc4Pz_qlm`1XmAc0BB(!>vhNG>~VR-9ofg8#^@ zw;p=3%sc7>3zt|&7E|Kdzty^>8@p%0Y2V~kyXdGQV%R3Oy~YLAZM~lMEVvQz4a=H9H90f!w=puqSbd=p=K>+npZ9W_t|&-nRI*y{?WZJ(_=;=jq18CAL-DF1S2= z|E2GhQ%U&w$=&r~z}+#+qge26&2tp4U)DLKK$q(^@v)m6@;F>Ze9)awG5Snk52_es zcnXYZC!O6DRkel6sV}mUjiB=^t$8$^0AsDZZkU>WyzM(zXK3>Ka^(_VfZ?KR1yp0r zoc(0_OKcxTM2U=D&e%CW=uSLpGbu$g01PEmcj-4>CB^2RUah}3pk|P4sjshnzF@>t z4L6_~r9!*fUIU~C^v*5$;)G*Ba)oDPG03#pqP~qCT?Ggr&B>Pcs1+582v1Xy$|C!{ zyVjb)aq)%3u)*-%Ulti*koBKsy?`Ekum9-ZN=H=f2o6lZ3mxh#DmtEZPJZ3&S)`77W+ zQKOD*`GRq{|do- zjC;YCK(NVn4L5(uROeyKo@|y+R4;$*sA1LX1V_Q+zba%uZ=M}KB1CZ}lefWRRvWKN zHak@|&@76jl*q(TL-?DP`u>at96pDy;1fb$Ir%B%hvvk*6{u=~3(3f~+@0kSKN61e7J2m8pbvJa2 zPdwJJ(aD2dY?+7MdyKDlJZWMrR;a9;RIyrv0+$E%^Rs6mNQV0p(lw}QEdZH*ew*mS zM{{DnX5~Z)NvFpKi%5K$tZ9^v@9PXs+ZpUB}c{bD7KI%Cu7DfIzpscB4Zfy zgeD{}%qIW0pNZYE&u!-BoBefe6Z_L3^F@(4^m^GnZ#dN<&0&8@@d0yQ{bo-|tJ$HL zs%gQ!-*C*Ma1p~#HUD_8kLIHmDfWo{mi>FNAZiGx=( zBZny`Ez-KEww7~ut;u?|U8!Z1vk7g)0z;|O1os3kDofXGpQ<3dS2AUQ-gpLwkp9Y< zco9?4{xfm!C)2|FE&EJro6ZM5JCTaZGU0JnP!$fwzPcAL_2ZRO*$z8=~(>PcU{U z=mT8mTfK1)%+1O2KjfhLy)MDT5hPJ+uM{qeok{Vu4hqZO9aK)IrhLY`sZX7TAF5pa zssii?s#vwE{tGyoZEJBDpQ`J}yr<|$?f1i!yclluFBY~Xwdn*~`Fz3ODEr#ZM4I*T z{rlZ%r|gZ@vd^-Nvo0NvagCkZX3aV0Jcl*!>*B8Hw5{l4l@se6CZ7;wB=5dsOy#o8 z<$f&3doYQnK6=MeHKn-!W2OEm-+3bBVzMOEcp#)qVst4fB)Ld(CNP89KI8AnF|(Ov zOrBPP+ho3RNHT>LgOCBZ=zYw4qg}{-^lW(~hx+iqVEofCj3Sd*S4l^LW!&8Aqt>Ht zySaZ^RGT|bRGJpm*9INstwB`hxV)j}Leg?U&3cbCftgBqJ5Ttb*+Jd5hhofh-#=ey z^*DbIyh$?lhA2PO`d7o;1uyG_a*FDoT2=gEYmIllEKLhOo&8}q>_Y>0hRi*RE>6sF zph^#lrwSUb+e}E{4FC>!x>b+M@VAm(D}&MewCnXr7ej-2kIylOz~#lBeKf-KLu$#U zv#&D@ZMf}AP2whWp#92Bp5z&54o@SM-r>|r@&8^3+@L9HKDQihYyj=iv3{7s^Jbvf zkA;EBU=eKx+s=cSUXB*>b|{X{_2%vmcZ~5L??7aT^p!{xuTIvFJie=Jq7-*i(Wh=x z2C0nJg7Gl&a^M0$nxnj9_N&DGJ0T2%o-9ZF4=u6Rhfr@M$!kg|Xg+uO+wY6OGyX z3-_h}gl`#dq#)jWcRrFKvU=`or@t|*?aw4c*cH{g{qlKjXfh{VUYTnTc%f;XbiBS~ zT0*OpAb>R8LxK2Hdoy#P%`I3qwL6R0F;CaLzVwAgX64l(uhuM%5&eKhs@z-&xS_qI z6fBBgZ&eTd8>*w+p@;TJ1z5SnfNSY-jbbGyTB)r z%sCvt&0ufTA#}yrix6CGB$ocfeXBs=aEDGW%44r&89MP_`Ho&WIR)>8#xR3lQhBBOwJIqZ^4eC zJo0TL=1O|`nHRTqy0kl5v=7$s*B3>=#jO6gQmugd!(KJycWMBX64JS z8#VlN6$QgRyT7}g<*WO?I|c2712deXQ(wp9i2wn+(N{$=A^*K6GNpbn#1Ji&5ku1- z5HDUYEh};=)1MfN;zA-MJn+%{V3-47#ByCFNZBoCE^gn{yOeJk*4jl_3n-CVfe>b?CYc_8tIi`l^|geUMi|}iz znS&DRQF)%0?uo%q*9W?*X67DOy(oMSmHEl?t;5L;Vs0n1qSYLBxbxCcS63Ez#SiiQ#xy-P^oxzUz2tP3Bw_zO3#woumde5E z+kz&(J4Rt7wSE(7q8Cy%0vR!ErM%@1%?u2>e6_pXngEs4*l#4@In`x8_pM7tC%Z3l z(jH4of#>J&+(^I2@?DT}I-LzXH1CsJ>PC##HCdFB9%HOMLY=y(jE)Iyb1(0S^2i2s z;E%?)L@szLw1$j841KdL-XuSH-gP)3pB*PYS@;`B^WM3$FR|8tFQL5*p(70|ZbM1$ z*GSNw}~H`0KhBX90QQQz8}RYqAy8)@22tl zQ}%Y(;9J=`B2tyy07~)Nw0T=tvJH|8yu@ET>75n~co=SDf|s%;@qKj8QA1zI_5WUk zyZFVG?4=9SXis>zQR_ifsNVUi<ab$9M! zIJ{j>!LhXcA8+x{iB@HqXlM7}8N0Yj0NJuPtFHQ^jTwh=E0S zAf9|#YP4nIyUbOq)}KJoY!{!P#GF8cu#Ef$W;sS3C%HKPEf+*ysy+9tZ@$cS$z1%w zuMYpu0+5RmnL(UY3gnJSo4%b*Sbe2zbEg$8Z*Ks#Io!p1bTY_cdUFiLTiZ>{mBkR7 zv%kaqJKwGCx9QZ^7T;_U?qn{(Ek&V=BqEjpOK#V>BhC$MIS;6m^JRbQ7H9d87MZ3q zqzxCqqIq8ZjFak%AUTAV&aJxqxxeZjGc~(!u&e0`a7O}JaE`~Q{0McNTvvlvn<<>M zb=fW!oTwL@r#sF)Bd`Kz~sJ;(>?EaaEN z5K}|{8(ay}y0u8C<&e-hwj9w3Mi2}#?CE$oU?^Y;q9f_VZE*w;sgcTjy98b_t&5Uq zVIH75K-b^Ipg+4fYTS%qn4R6nW`uz18}muefW)*d&)nwUNIZWG4<)g;lz62(FCehq zillZWR>eKZ1G}Q6ULEh}Yi10sQaFBC^aj1=V?-LcM$?8zii$Wn? z+owYy{RpKyoO`Amah6X=`GpysKXTA7bQ>9eb3E<9;@Z}F(?R7%`#Dw6LmjWEfb84N z5BLk*9`VEddCKIWWlWZDlE!qI{t%|>Uhek11}Qa~<7TSJdvb;knCo*WA`xgda>ma)KlkHjMbllJBeXp{Oh|h|@=!3zD`xR3$9Zq&4YwCe zG8}1+)aalR?XoW0xkyiCl`fs9h_{wmxZ*F*WJ*2Y5@6JMlw)q%dfx=(GD18~W4YHP zmEJv-9swzdwXRmj#agqA4OEt|T8=la04P~t#uAP=9kyHC6f+riM0A$bWkaS%6wX~< zKT| z4!azSb^D-uICL#H;5|j>34l$p?445sbe&E_G*)3Er}m#`Q$ zrj2Ptwd~fj#sAEv$)w5~S>>Fpl8&DAC%97>60e3~NsQK{l`NgYN`x-B3IsVlhQJ7rZ+t+!psyy?q=#M-7 z8;Zu)5Mv$~U=pGetMAXeXEE%a=-ps3H=1{Fn}Wn95)zXYC<@<}yT(7;K^H2}q+m=g z8w#>IUk>(Yu>2e}`fZYCza%UY;CiGD@mLsg&qm4Eezjgxk)kRrAjvXNDQ^p_n=044MLxQ3K)s0dYZ& zzQ|F8_NgXtz+^x+b$|NlC+9XLTq}^UG_YWrx07PJft@d>CIDp7IayF3toY8?Vxx)K z0vW9m>hjxfzjeMA7=*2X#YRagKbJPBny|<~>{In{0KtF-^=F@b=1#%ll)48We9$@m zuAUripDO;C@LlkNf@_aH`lxfdivVNoXZts{-LdlFNF5(fd+KR+^uIvAQQO||uiz_l z4r3T52LOj?DO49RW>Sx+|8r6z+F*8CBx8pOw@lJ#3o=(aFrb7C6P_{#3lmD4Dk0jC zZ(|y}P)|HF=lZ0H!~dQ=d%C_DazvhhfnzE>z`tfq*=_a|wT52e+fIv?ldC|HNS6+^ z<$=4>W%!+0GGkV@f>^rnY?&kvJ~>~OEZ-`(UE4uAcdq9&uyC|~aKC19)@iNfl{Xi= zg^XL^6i%#*{@Nm|*XUNg0#V$D_)Sp9zSCxG&_z&=^ypqs8q_OUY-Lv;v&Fh8+7~rF z-q&VZ6qhzN$;gbCpMPB{8F~E`Oz!TU<>l+b6DKTDNc8{Bn>QC}7uz@zg<18pRD~RH zU4k!d&opTpOGz^D8TVq9Y;I#7Mdo8|;>g>XqqK1JRZ|GyBtf+EKmGJmr;Wu4g${_* za#$I?#jt+O=4eeJ#@-DZRF2gl8}KSqHy~=O2JAc2wyKt0f(&S*8n;U3>k5zxQqf1{ zo;c~-qove2^=loQbL5o6y6Sxn|M9kb7u^4faSm|8I2JPy>z+`lWJ#cu80e&QHWpVP z&78b2KzEvPI((TPiwN=Dh|U_U!`gHIH*y@X#$okC9ilCQ))k9p8WN}6Ks$P!SwAw( z9Dmr1Ts!rqM4M&I!EX>JF(KG6?sym0-tg6NO&$?W_@J2zJ*Ott^s{^Ky|>gX5e7jG z?JK~qp+kqd@OHn{m;HxDyP9X5afSnp@Iz+K(<99u8#4zE9OzDU96frp^BHNUDUu%? z+B0Y3OOgXoXgl_v<(Imr%|k!__@g_05wMQa96PH?0;UxU7A$Z(yWVy}7@lz)w`n6< zoTQPSp6=obOph?!0}O23xUt~B3osDBZWfj{N?vEqoauPZ{RIkeQ8s%Qb0+{gfGJM9 zBs~NZRK+p7wG%e!$MAvv0ue7{X}LfCSJZK!cC92ysoPk#ZpqM1%6i&HMM0@na(_a; zTsdNc%$d7Uc2+H{COgH;+wU!vjq7vd(VBlgnGaExKrj)rT$6W0sfLT*79ARonEwECtS^W5R`rK6MUJjry9zQ1 zAQl=XPdxF2T&c2QEnRa_r-Ai@%tGJhQ&ULyIh^mSIw`81riB8$dVl6tIbo8Tc&En8 z@hfuV#vX}!`d|T10yV5S&|}Iyd&ol{%uxVK$3AV`>RGm?u*WZlasWAzP5bgtxd1E@ zG7yX3^UpuuIURoLsi&Op76fxl(Y!>bjJ1J?{8vSuO<4Exr>d_>le)6MF$0Ai89;lqbJU?1OHWf5i~T~z4Vd+)tnTceNqR5k}3@0Hh_&h_-u zPZurp%6_wQLQu~T`QXDi!C1?QM3@hxrKJhJu>gmt({A0mIUj6jGnGWV|HBVI$gEkj zBtv(gXDDbiYSbtf&u+Wz<^U$NxQGwImxByuYgo;IJG}`I>WeSFkU#$TqgWDvNNiY? zMu1mngC+P`({ALk1nTzwVsJ>qcg>9MUgY z{`Xujx&1%$<$oV8m9{%4$xhlWKKo{c`z*h$tDJUROXu8u|GgW@BlmWf%ZJZ&K-cOu zdD@;-b4tOmyC0Y*sdW?NlC#@NYKr^cyH%aRN{-*ge@y8BftYV%jhMgTKW+lpP(lvm zgI}+I{p(*&KCm*eL5>d~HfHg;ine7%&kZ#Qq<>HnJGHmzS z^3J?WIq7o+Dt~-RUb^dSIp?x_W%07L^7NxO$gXXhIX1LwRZk}C9P!=E<#JH>7Ajx) zT39uqeONz7)&#)9txLF6ZUBpfTqo6$69kP9i?)j|zSudi#gv8#m&F)DFvo_+A|Z$y zqsc#m66Bp2+&xb?3sVqs`)_Z2+dr+EJEY03JkGBfP+91)h!`|zkUaCuGw$RzQnvDX z%U3lyik(<#$4#`8`BZ%zKn)uoAS{l;j{%IZ_7 z%A2Eh13@B{_gGKI#;xj5zqj$+^1!@^wgeM1pc<#TaylY&B*H41M`$X4|NZxJ>Zzv| z6K1Ig>KRANk^A<>fu^H6Q^0dDn<*7l+1k%rVEf_c5zw4*KAO51e)xz!v_?(6B^+4>gaL zhW}gj9I$$>n_5>Lx3`rej=V@ddbhqD{;&F~5mHCu^K+zoxB7C|h+U<=I%0qR<%OzM z6)%ld6YC#cd|ZF)Z29`T)pDi!#Tu}GV;OQ%YneG`qdYrmku+=&C+pVdOZ|HBvT0+U z+<4n>(pVk9pL6QYlBa4WbyRa}l{#gA=HdlpM->n^`N0*ea3a!mjwr1taFVk|= zXX`k1dvULt_a5ltpnXO&)vH&}on!_m4Ov}8bDsUYRg43SpvVhZs%|xReWol`r`B7v zFX%Z1O1dU-Pi>rK^DJLB>htx>H%pzQcxhL!mVVSci}AQHF5Zj@qO&R?ccVq@oljyS z|C&kqD04D%A{$xE^|-&d?Y7&Tk51cwB^+}q2fp9}fjQr_l4GT;rR7t_71#dE>Tib5 z1)Un_%i{x6<=205WX`618NE^gyI&T`txHzQqxW55(HYqsVHw(ktWXd4JV8aJL6@+>!g%8t zH*TC;=<3Mqy`xf z825_*!WY`BPu0f(cCK*d$wD3q96PEnXWOSt-mG8ZjD6U!VRG}$H#=c+`_S9&;LMP5 z2a4wMZOd`=T^au(l{3ax9LO_oopQ=4MPsPd-+>1n=zI!tS|vNLj3au*)Mm(UDXO4+f`6@FV79>V!f&oZ#Qq#vDe=I|W3~_)K zfF0(BUVhj+{%`ei!0N<+l%tM+TV8r*cRAzuomBHwPu9$im-IF%^6Ud$<+vl7$rU5! z%J1{ms|*JyEArKtl``Si)iU&yw(`*3UF4n{c9osf+53Hu{GpmvqUMBos%a&%Y^BJQ zscU8VznZEe=M?#8b*8-c$#VIA+-kRrKWsy;jC#C_bnTd|ey+;q35zG3!_SThkR;F3=pf>{9z7VwD31IIY&h2&SygkhPfwhj_5APzx+2 zTVdGmwtJb{A2QXC(B#amvT|#V?A9n*QWem$I0+f`#JZ?ZhKlrjUYWcBykfpLQ^M)f zrweO8TuLAh+9nk9;^U7$?!YV7#%8@_GDKzODbjIKm1qpoX2&hdmAyM8NjEhyhrXJb zA#ZO=k}l(4lPw8Za`LtRl}Yd1r5hiq^5v_y$>1S3$>>+_(UAJSI?;-&g!QsZ*#C+; z4j@~UhbjzUS)$&~Mu#+E^-eujp3E^2@ymToA0|wg;4~?M4NwMT0jzMOJbpEJue^n{ zJo0OKz+7k5s#Qh#^oFr;lnD?kR3dH$CHxhj?ZkDkJoC&;q=RM&f;%QsEQ-AR^oHe{ zXP9uXs3(8$kayr4+HRyt7{Drglc5C%m}NWUgyTApN=q{^?2h)|=S}2moOVrHMZFrI zK`iDCY>j+w!b_3%LO%q+;M6(%7;zm)grSUpbAZs8m-$qF4n(@2GR9*DhtSCm>#eum z>VO&Yf$)mYDnQo<9(X{~($d^BN{5fEk3RZnJM##_VXDX2PMkKD^G<;n$W_fC>&x(h zIUZAXHjIA#^;aLnisC9Z*{)r?Y#DO?X!&bVCpr7H7Sf`b${o6ZW?6mMK~3b7SL(={ zA1sqM-d`@i%~|hCv{6m?9(#77Oq{a8X^fBG;f;VPNkYSX}Mf7JYI%hnl5|yt|#?V3-wWg$iRSC*_x;M zOLFDM2^;0V&o9@-;!SG4+QsE3S{=80s~&X12YhBUl6Oygk-~9SiV>n>4EwUdQyeQU$ zOO*y-m6UKWpoASfcAi6_{r1~0lwegB1}0ZFRfPajR-86KR(4tZ_SX8A+myW-VXUv>VT z04XugMmRO$l?{Z!g9potFTUtBjxcjXNJUs>*rdL6mp^7eAt;8rqu+FE(!=_N90)>_F>5NpD(Yuu+t_vSKf<|^kahWHoZ0ewW82B{6f%39`&~2j2iHPBU;#x-61@(b6Zo$RTe4(xuB^?p>@;c%qCMGsb-|na4*ZzJHMQ@BxXhNR%J&TZx8pO(66n7?rI* zsa3f;v5%84r*D?gn^L6PcQ43_#S0~|ZmKlblVJOvI#Pa7Ev%FdXX*)rEgTC9rkz}= zTj<-ZCh)~C!!`~e_m)>~uW|ruTQQ29WT=U=ZIZ$&rfcXSYssn|!SScZz zCM-Z8o5(HiwhOUl%^I08V}?5_{r}lJ4>+r;654vB$27E!HSj#0sJm5u`}(>{6Cp%K!I0Z)d$cx56&$nm3=% zd-uJ2Pn~17%X{qBt($%L;fKLlD+}?%4?jFi%xKtQJ_s=5 z?-&r^=%bGgLM*0)Xma5Ps7aw)M73ZD#T64K{YcM%%Sx zMsW0Am-5v1?i%%Z46z8n4x1RweGSIA#&`oL#bzcT7WKOE#v6n7Hu5cXL?+-F2cJNU zwauL*)@a;{0j*T$;=MDhc-;n@H${zmZLv*QT4JVy33Mt4M+};ps!YkHty?aBbjc6Z zU%qm!{iJWEop;<|rKNtsX&(vN<{V%?qTSNIw}fcrn>)8Pc>rrm7?X&b;6Q={&6fiV zj?hj!?PS0FGvKt*-r^{wuC;(qNM2*P0P!5TaS?+zgq$L;ui#FP*) z=BAr&3SbOc$2e}sf%&!afzd~f+_}ezeC*63(BkYJEtLOt!i*l3asa+96^{PbdD&R? z&yq8DXwsyG5@M6(h#hn9wRXX8C)i_8&krXK9)4&$>)19;3l90&)IvNzFUj`WHO2PY zLkknhykAZ#wrg*hD<3Is?Uh$v37U&I@<*eQ6Dl8j_)Y<3-&Zb6ZBJPj^q1rI`SaJ< zH8;-HAqVG&gATsl5tJXTGwvtL6qR70gXiEtJs2v$_uS$5iQ?Q zp%GLhGPf$(y?GpbgTt zN|f*4Uzuu8OkHL@vQljMnsqjOm#$VK0DyN6J9q{8sVyx{$nYwqJ8{|C;OII$*TQ~{ z_%vhcQfskpy{#-?XA4)a6=>B$&a%tx`uz&+&>cGl{&-nKC7Rk-1w)XR@GZGD+IvH@ zfHJ#0$8W|P7U|6OY_f48AzJw(M~)2g0-Ky1+ya0Fm=86_ddKrIHMNB;TCBqd)K)pH zgIAj=&Dh`WooQ{pe8K*mwbD8t*e3u}q0ag`<-%L+`m0W~i+_5!OvX3bb=supTcE)l zqN5~en{go8u5tN*zjEg4=>7T@Zj}y`BdyCWyR5!Zt@_ney^~5rNN^y*fdmH{hXb6% z)LbVo26q4o9C+d{X-fwC#^G-BPZePip&YH9sNh&*&7#6kMBuD3P7MzcQhB%dGUSj9 z7M%#JfBMs(f{6eoK*5yoo_oTfCY(6*$iokZ@)92HEH@pjA)5wyL^+{&Uitq9$)ui% zIHO$hlcVBAkuJ(EfVZ)6U1>eR_Dz2r>r^2ga_07x%HD0<4O;NrX<7{mj*JTn3+=SiP77LI z0|yQeIQ6K#^wLX~tOZ`#`Z8%+O-lu){Qluj z?e-UETDL*%ZS>CuTj#uV@imq^JeoSc#D4j&N%q3$i_MlugF^3CXrxxC$u@Y0c6Rmf z9(G_qdF59mKpiwEo_lYG9ecxAvqJg&k&h;P+r;K|5>}$%Lb8E-`wi))P?{~Db_wQ*LDME$2<%+XW0R$pRkx8Vya#O5RL6-esyL`KF z{~p$>U1lX%MhKEyE0ev4-qnd}Z_cvWQv^ilei^)O&uvn>4C~uDw=!?a-PFkErg`(`1+C;` zk3F`j6|u(hcpq{tPakjSzkpbmTylwk|4GK_yLaDxw+$USG`Jk_gBmayhCE)4nKgrl zIobFFrGLk|EeFzUQ2YMbPmT?rnyT>CoaI&`faAh*b`RjP`_rl^PE&g3ZDCuL$@@}# z{&i_v#Kn?;wTt^@<1I4I;m4fzd%*=4Y%zcpU$*-EPXJi;;Z`DSf&&Q-Y;_K-)C7k5 zb#1%0TfJa)%#g`~(^uc~*a=#l3)r z7D?8obqj&Rhw8Jd`kp`$QIh&0PTE%#YZ0_!Ss0hnb?8l6W+%ZBDOS2_wyiB)8R91? zy^PF!3G8dEOdn+s((o_f@$~czOGz*CTU+m1F*a?T3}Yi6wa>%nOq&$zmm%$|ih+=^ z0Zpv2vzFVSTi&+<03HD==j<)b9^GFpZLAmOtg_E0FS4ryrsVv%l?~~kQz|Rw2Q-JQ zeTog1hEex!+16T`PvMlZiuB26t+Xy$NBrG{H=%OuoRe(rWCqu}OI<6Lg)<>X@6LssC zoopE!lC7n*rF!M3i3I}1RQ@lPmf9yvSJ{)V7u(pci|vsSJ6aEcSQS;XQq3KIfBh>v z@98O)DStkFyJpz|T?BIJ{h?iT->jAPw^1`~;`C+qx1SES?K@^y0((+Wu`j0!@RAl= zn#^FADE!I$_O!w_>6We6kPF`wCf2fJm>8IxSRXzc1h&C{UKARSf;&U|U#mcC*Xfz+ zVDneoyLvB$1v!?cFz}&*?Oy#Rb#=ftIyW_`0#;KPFYl;Pqk@(-Ck`{Gao=Suv`xm2 z9c!FS_}9PwHOzH>$Pd0TaroY+PoE%Q8b?a z_VAlkxXp-^B1s8h#DPK_qVk`eP0HfLZ+4V8*up1euKagl#)T=5Q{OAV3t&`1L4p1G z&wmcPRrqy!^2sNSlX=m!!twhNM;sAO!nzI}zONnk^3b?mG&PP5(iy(dzK@AglWsnC!@ zt0=nC#JTt-k5906dNyC?aKGQJvyJ@G_X0Ck34AJiaf;pj&P;n}?n?XnbCayw>AP5` zteA;j=+R2uf-mT)W`-0XC>ZW^j!D=~qka_E)FcC$oy}g9(f6jR^~_^WKG)3UDO4 zcg)vz-ZN9Ihx}D_Z?6+H_basHhja_iR!D2C&A%qtgX0(2TkkKn+uBaBKb<|qT9Kjh zeeu(IcG2khHXtuWpqf;^vQn)`8gB(@+Ej{L=o2FrD+Xw(xKi{I56MXlRm)F-xaE3v z<#T*XRHgg2PO;lRU1Y-tb+#@_yD|~tH9?Bb*-fx`RThadGe!cC;PaBXKRYYS z-hAt=VE&8$K*9h{U2(+~h8Fequf4OE;;{aqq}tEJ1NIjH@D)D!`E zNimJgs_M8@AnyD{EA6hAX4^Yw^|L+tw$+>y1J0`oU*CK1|2Mz+P1r1XOYa34*56xw z)2b;v;0~XRjBfz)n!-24v#9KbEJa&lzNUWs_j=~beZ?kYZWWmm7nc^urVw$}xWm{< znwa;x)}W_bMKZPCV#BtlWWJDot)Vv!kN~hYsJQf3x$P+!2~W^Mh6yLGYBA#lk(7-IRE_f!z8_E z(V}3+gWv_Qg*F91QZ%1vnvL;$qb5~A8XW#}MS}x6>7dYdcl_uBlo5{O^;!kv?j-YxIJ^!y#ppFiJb%$Q++zvFMVXvyNB6-Ih*zWHX* z8pDY;!ZpAf?{RCB*9(v|K~0kbv`^akF1r9)9B&_e^os4ed!ha0$9vh)hfTM`4s2^5 zeY)JnjxV;CURz)b7Y7@xP?^H6@&T0ADzx8E6h@o#;p8oRzX0$uZt&rhu8q=S*|34W zutIG$*+#u(_V$>8`v0lYTxnlSUTydMlVs6wSZrsUU24;3X4{j`0gMTdP3mZS?4wf= z%k%7!=ku*pW?^ee$JlN|7Y9?vPd<9wp8ih{gw(z~;yFe*#yNPcwq`TgGQ z>`$i*v;kL-j*WF1L!F|fmn$B_jSh-PO5|I#5e=qjmikToRFPO%jgjB))y8^v%d;yU z{KP)fDVVPals$ZKXG@hf8T8hNb5Q%Ul>+r{JVoGKm#nbx#;;bcfU|3k|DN^em}N`) zEwh$cX*O~83frk$ZirjHQYdV3nQhEV4q8?lm#nhaCM~gnI}})+)U0v^=;6zX1)H*{ z)WoV_hx~H25>C~^$~AYbEY@aO@lt7x$p=|VcIa!E7^W)zCo-p82S65x3Pvkkal46J z7-y1IujWKP&*Dblth3Gvn=!Aw_L>bEG$?!?mdgN_OO8D9$l&)1kP9Dx9XfOfRR#D$ zKEUMw@2zHOpm}hIzLej(rTpq?yq~w;Iu|6_!sV;2L}tCI@?Te0w$5^8GFv9o*KuE! z*oPD5*`I$tAXJY2(~zcv$hvpld8c8T8eEf9un_qR;vKpiQt>Fvcm6y68TXkFUH0?% zxP;(y>Zzv&@W|z7Pp7(g(dSkDj^^WOkSFCw4rkrS1|?>yzHMY3u42rca>Z{9y!8v|_=0A>Ux2EU@m=T>Ut1Iu;yw8O3`RAY8 zh!GnZUARc%Wo(;zNU%!6gq`kRw=LW!xT|gG?XpEs5Mm_kD4wE~-A^HK}SxpB4e5;Ns8{iy5 z%O{_FVi#Y0aS&uKzx;AL@Sx%0Hvr$+XP+G$(BpU>p@(oBoPe(*gq525UsUMv{Hwa> z&uY4QUF%iHDiS0ecv(}qtB97kRc8*w7QY<|itLNe+t@OJRCzglwFp^cKRi6cjykM^ zjr(kc{rK2U_W9(MHcQUe=guj$QEy}7EA^;AGSw4(_Sxs5Uoh_My6di?pYZ36JbIhiZucwT>70t(3F#3MHlx3^wOh;WVDtu}WHBYoz71daX`n)Oc93s?5gdAOZYh?bIXN zI<(8)B&$$1-6@s!)gb&40I6oGD80onTIZU7EcMx4V#sTGdb0h0>l3rpyb z%`IN5u`#yj`f$cFo3&_l0K9%I$LoMszVHV0gHm+;`M%;;gwQP6sQdF*2*46x3$0UG z^D@ZEiV<@TF9ZirXVX>y4+UFf6&viL2PfMD zQ#Ed@Z4WNUu;G33?8_x3_RzV5?ciP7h5n5k(NN$OG6Ct~Fx&~#PUK+TSyX26HHXV1 z(Py5Hr{Q7a!|+Z4BLT38Lq3FYnZ(nH#`835x_Wr>L(HpaP9dj;_!aW2=jZk1KhGiGh>bp@E_|XChIIy`I+F7NZAE@Z)30L^2`BD# zWmF+GoDc}|Uo&dG*@r2)}efQnhXv6TZzWmx829O6p%?JT< z+SqY=Udm#OYGfu-nDjZ z+lBW=Ti}xAtkt+xE@(W(j(iY8^c22<#0?68`?Zp<%9%ci)9lVlms5~SjfTnwTOMCpiukBrFF>zX8NJ3xg zQ#h3Ml@`?TyLPS2in`)IYpuYtSxZ)1zJRzA0cPAAGi8xIJ6UZ)T4P7ZQGI55tc>@J z{@UJ?*B4? zL*wAgnM*X@ceZqaPdIG{{#CM8lp!-g6~4T9mHJkR&73&hR?S~zsVYxSo)-Tt<#0cM zSENW~D=q@@9CVA_Y`WjLU4SaVoTgjUZa z6RO!9Ey|b6+&0rr7`}sDe((D>Rfi~O z>Q)~?>#Aa)X@I&gUcf_SJLE%TDCa4%4+Lgw{mF#YZG#NPFeuvOIZkYV%Kj z`cr%Q>8FDnjuzHohaDDxQRMvUh`6M~20JpZ%UiTpPSxfEIWPE)j8?j)RHmRY-f@pn zhPtp;gGb1@nbnw2IuGdM$h~MR`uorilB*M`sU#@DfdmJdA_o{i5~P7q0~*>R40^SZ z4~#x3F^Dsuax^iM319<@`#tv9BM3&7gJvXQa$yLKPm3U)u0P{mU5E(+$DG@mC^vNg zc)9o9d+mom{GpDRzcK(8Owwph0RjPFK=23tp)0)LLzt| z5V13dU#VWbdIbOsVT<<+pLmBjG6m!Uh~$4;#X~p(u%v91llDu$K``+$QZN^!T+rqj zg3gwJMrh2zH}@Uk?T9keMwAEM@%iYZkAm^1tfDR%w#!Z$SFnLX?v*wRyYHn^#JcRt zC3e8DZg%uB$H-Sqx5~_Y$OqU6;D_1;)Usu&HYi+tjhXvaMymKxUbeV&7IFYdlpP@F8-FW` zt27D(3f=OS<_S(Bq&<)%ua=L}JZ+yI0poD?Gt-vaGoJ{w0>$AYO(YFS)-x~7 z)(cbxu8C@FkSxH{_&LS)_Bh3(>=kW@cqZ@ZM-K1034jHtm80^_U%JL_ePybg*|)Og z0`4ZQEVpTkwRxjc5v{a^?b$QmvQdW;7U#;F~IRa~!eL2rI zE?OoN#dOOnEK=?*f`(eLG}8ptDP%=KNHz9iV!Y;m6DP*(?|tuk%@V+ZwX}Wad=~ov zRk#iSWB|lsvk>$|lcWJ$=;x%1Jck)52hTv~op+wyc;k)X-6Ly}J{!F6>wV%*0$B_~ zzPswGtL(n}?hF5kce#MhJh^W0w8MkQ7j5Z}fBa*}7cF{ZB($t(E70-BA8-2!9EB(s z&uKpp>q(cTxUN%Lqu+ba9(?e@Ag{s~e6l*&hCB+WMLFRE{gDk+p7S5s*Fi@3@Sf7b zkK4x|e;o3=?Y7$jc;@ds+>HA|PD8E)$RfQ!jLUT-mjLP*JP|Cxf&V=YR5zG3HYXTl ze86e!yqbFYby*PmdtzY@&;aOwrp)!%Utd{0Ko(AiVPXL^V37{s0w}=%jqe{$42-75 zV>p#N@QKDO+VHU*3&R+?0X;n)SaaPBoJx;b?2|?TvN; z6%`c)0fGq}a1>#LozuD!lQcVCl#w>^qaXb!I27gg$dMxhKt#Fu&pTmaXAB7CS0?=C zKJS4r5dQ&|@N)q8h|o;=$m8v|-@BzpA9|`AQOOGJk6LfBiv#quPS2alC-@{ z2y(-Q4GZZ}M#_TF1mNjJIeA8&l#SnBKlsc&;!$q^Q3N3XH0sWOo{^smR-SHES8sFl zHQKSi$LQ{1G~PO;jS%L%a6Xe~ZA2ZsZu$`xp_KD~g{V94EHg7*W55yeFSX7NJ^C^` zaQ|$X5O%b}E~~6}k93Wh(twls6JS9A5Rgx4didfIcyZpDU8NPZ!G3?;L$>RXJ>@&* z>=+BRK;I~J99UaP>tg^@@$&J~YFQ9W2xLsAwyy4787THch7~gV%hd^N`07%2>tfn6 zOBT%&XeMvMB2FKK+O=-RbaZ7+|vmPV>i%P_VHNYjzSQ~-EEp)oz`jljA-KKRgJA@%B zx=tNr<1bmZF@CMath>4hR@QxGoAg>qT7W`o6kjefb z`FkiVEDT`~(6w&6_kl9gpk(F(Jff6!m3?irp|AuA-TS))OD!Ve_~S z`0yWahmR9L4T1raH~xRNgvbYtDaI?x3qMAT7!iKcZUJc#M0hv&W!|4I7w5DD`8qxMlTAprA@d&pyDZFp-kv4tWh?C{>D9KJZ;Us(IHO7qgHsYiV zJ8)>8<;!Dtsus^I;*+JV0uUm0>0Jl}5wO*zQ>;(C^Y*LKx)>y%R7Eny$kzK$Q`nfl zBf`gEjMn!2v)9|w<#Hx#fDdIhTUt~s**7HtSR2!oUl%)YFS8FO#z2DcpMR+hh|~~t z;i7WeZx7Q+gk1$X2nE|1o97EUw6Q<_=}v2V*PS}7;za8);69Z%Ih1+RJgWak`+`4w zpm>n9`k1jN-bUCtM=R0$SZ(nd`bPA2PRAzOHWyaS<02Z2Ayi?vM8c_JC z`e6Dp@jl7A>cqQlZFK4#L_&Zl0L;E8i_a(46YhHr>wI zzt9F2WZJ8f7TcdclL238xfS(jW5*7OX$iWYt7CM^;d|re+2|GP?R}XU9wPI?^}%ei zg&nhNH~HX_38gfuKGd_TUY{*{NI;F)SevP`w#fvri@|DMGRo9aK-%p2B{uQ%+17W5 zE|w|usaSt{gm9kJLlRpTQ-Z?W2~+8J%8Dx%h;rvzfAoK*#CBKLfpWFBBn@bS8d z6mPSNLEohw%n$gvy77h^f?5AfH{BEtulUJNeiH5@SGg7zb2a4S{$^#|gd#CEFh;yE zW_mbG;fO)`_SeTHXB^YtK3Y*`1A4c&zMZmd*bVQhFIL@8>suYhTx5lpUw%0NGni^3 zqoM(aOoJvm+Hnwa99m5rAhN%-m}q-!^x|@XQ=5cypT}b7wv#{blPdB1+B1) zFS$5`0GK0h-{7N-K|WtZ+fNu1HBPX>4|s)kK8L4Z833`~LEfW{vi^b4T4TL4>+4xI z?0{jR%mA*G)#o;3N(gHWKtJk=Mxj3s53BmBc+bdNUf!yLRd+7|U{%MN#Qpzi4$#pV z_|O6XwP*3OwTaGOR8$nQZKwf#Yu900 zEuiWa?0{8F=xCm?h=RBKX>#9dFRchgfCM-p=L3M^zy-n=&*1geYP$e24FUL4-s;MX z5Q@;@1q2gj5 z;}zhG127(sHpjcb$CeXq_XY{jv~RStsuD7W4jmf69)vv1ZF%SJ+rR}P;+-fkkJBIF zS16Op5q|3mF9<_;?jcM@HOD9q_oy>kdjNv8FTgYUAwVL?dw<{bfw(T=^|+^1Y<L`cz zQO|Fad{m~`3iX@2l&LzZ>RDCopt|_c``(s`fB4}?+8b|-vOE8NyFL8K6L#N2UG%|p zIwlOa?UHVtki;h-20os^Vp?G$|Z+iL_=W!Vcaf3B0S zWZI~Ax@2*&4LK!S@3)gcm}0%h7=X25W4Vo=+Rj!kPq#mmcT`!c=VdIfO}}|x@B?#e zJ^gXs3UBg*b=BzEHoSia+r4+2kbjB}8%P1b(7(68lzbuZsh1qvw{0B* zZ2`>kFsUy|cA<|4tG#7}SM%uj4aZqCBCNmL}R5cG9E^!x3l#m@&B0Zkm7Z==D&oW-UV z^Df%dKL5of)hz$g{!&^d>6AhB0cjH6cKXiN<${;3_|P5ftqb?Gd;a^S<;$nuYZvcf z|9*3d9kF-spvAbkm38@wO<0!$Sj(|#4WCg;z^wwZ8^i`JJm>wOnGF!de>9VLAApR= zb!gFDbImouWDvOx3l21$u;yR`lXVA#RuME~`gD8r(MQ7^z~(1_7h^h_TbS)3S6^_! z1vXSNm&=Mi=7$6-zWg5bac~J41epdIh5AAsjw>4T^dpG2M;Pt{no$qR&n6@%W+DsH z-!OmV<4OSJQKWS!o1+kX;{)i0e2L7DtcGbU3Ie2uHX3d`@Iy?0_A-S8s`w&|_xmay zS9>ddysS20;z5D~|4SV3PRHQH0N`N7*5m;L4q7n@sv8G*F3uGn; zw2s(GVUPu2fZX{Y!V&{A8Z`{Yn9mg!7KX{kwK$lV)@ZRC+JxdFGS)@|2M!GB)fTR? zpVOuw{2l>fpdL(p*nOhzfDSC|2Wp3fg(;Km8E2dkKp7@DRD%KZ`~*TKJbX9+im<>& z9Uk5ylmU(bI{1!aW7W;yI}oI&d3E<(2=MU%a{vh5;4~awE|HLg z&;sy-PXWME+93jJ6ky_nB_?1%a(GVKlxcGXcFN@CqVA-_P8c90JAd#7Km@Rc_l%Yj zeF9*LcL)ERC?{ub+pbJtN@n zv(HeCmz}M!u$NsUUvW_1J{@i4stp=jSL>8CbZKJmAv~=3nmB2Jeqq)<-(Gw52}@2J zr!fc(Dw)vD7-zkD(%vwG=qR`WMV>7A*~*b4?j-RHA^jUZlB7GDPtk&X`+f)L&M0e> z*V@KUAZ$zP-7DEvEMH`UcRtAS+hxi()`|*Kwh&uBm^KK&LIe66vqGNKg`AJb4jk?P zz<;rP%$<0`31Pv*!iD!pI8M`}rFh$@OZr&m_Pv>Q=SO;%@)xyxLAveOy)~VH7ZNMB)>*AGTH4be z&9z>0OM(fZ=Rpuao1xz;uui#Y-&9lgGxhm$;VNl%Ee&y4uB{gE`QfBRHbCgj_=Rij z7ag^6kS`!f1m;M#S<+&fE-j|}-<)pg(smNNV?bD|B68~`z| z;AeB6ef9|;2XjJWjr;X2|K^0pq&7twi0_P@X}k9CWM|#~iM11md&|iK1%$7&-d*zS zy4Pmd8^8L#Em*O}j{DPxwsD_60dx(AKh!f*e?RZP|9+U0(IUdSfVrAA06wh%oB#`v zQ<2q}U)gAY*sx~93k@x_k04|@-U;#(8b|;&xM)CzK%0q;ZA|%)iFkj$naU=(uQLF{ zNQW!n5u2UJj&&h;_>{_x2_NNVqYuq3%=T~x!of*wjCh-5K4)VP*@yece%{`Z*OB8L zq&oA=GsDT#0CsG&F!lp#ktTfj>&<_)G4GEFE^D%!I>2 zx+Wq$K8;CCU`*}^LrhR;p8-+;I60w44KTArzzP!@02KcLV%U{J&~kF2iUk@3e*nec zJ2vou0hp`#Lft1&kKa&N>V$9-1)6;#uPzVQvI*}|d<$s;s0QSO!`>l@4x^4m0%WNj0>*Sm9;MSDYuv1>kgm-sEB|S zCEh6j513Ui($opF}DAqVr&8uR*u z2le=3%toLDG$dVs1aGS(L%Jgb%&Ef#-~jN6U0MWCgk3;A(t!LudwU5t6~A2L3}G-6 zpr61u*Wd&+T&y%w1q8x3>V(h-Fjp4>(4%jX9*a1L9a}WUs0Z!JiF|pF&Tr=#?Tmg$ z{@nBDjE{hs^cSxa*TTX=Pja)aGQg`O+Sjk;3*!ixHn0e||Ni^Kv%Wn9*vi#KmQDw3 zC!a!X@>|*99b4O8Lt0yx&S^H~%hh)HNgvwX4~z}rd$fz;9$C7kq-nP~3;%2DWv1KN zx^ocZOpQbX%H`2>m*bq1hFlUd_{0bTS+?N^J(YyneA($-qPJ_ZElwFS7% zlqR7-wO}6?1Cu%xEU|&x>q&-$b;jjQlaFTgt=|qt(f`o$Wyk!lfBkD%_z;+Jg7

J+PN`%GPPFa_AmRM|GPXzgK#aUHqon^*Z=U;^e4nOI@Vd z_3lNvn*vw@g*xcjTN4)92h)~?;^0#4z2#ENlkYV-ev(7}Jlk)_&av{6n$k^El3kNq z*c_cgIAVr02*}2XY)jYM!2)$0z~WYxzOq$*DG64TEe2-owS9XV+EZqNGgsPe@6Wdb z1}?NCWCn@W8GY-BL7nVs<@1}Tr`mSqEA9A4CfUz+Tx{Lt?`r(~Rrc_tWx?-T&u%$( z!qA@K41;wWG-fOA9(~%{Sv$73Up+g``nOHBl~Dvs#h}RSL#zx(|pFQ-_ z1Uu=V9V-3F)u)_3*8%V#+XEQ6b`$Bafs6cu+=HwGfQ1&+#EBEb{L0+RhTX`KBZF@; z?1;sI8%Db6N2vN3t%c%AaH z0m?eXH-zKcD&Xti1a>laqp`;(J?(}yif=AP;}jJY*(36=)km9NF3ZqH$%_qod;!mr zZ^d?6o1kUIoP?zXZ&Ucanbap=Vo#Yyd8QUt~hARKINQd{uE(kyqiw7KzBN(7b z#322_`yWUfV5OaX_SvC$fG+IRIQZd02v^cVeVE0CJHTduhR`@jsiQUofDRL0O~fP< zooH$byCu)47s3z=SayS$Q~~*@Bl$DYC+P|?IiiLPz<{$y98EGYdY@oYpP`*A_t(I} zm2{y;9(g3}a=F&g)@aMri+lhTSipOpEC5_yN9Y7l^hrMyb~C@C0z)(jKn8Z{Ab>3Q zE%wG6ZvgxOr0k+_avcdz}AHN|$CEg+Lw5~)N@E!q*cnDMgSO{A2MA}jEK0$rG z&e7-9{pLL*2-C0N2Oq#&uZPE@Y}CnlMlCrBDEjV+Lw}+D(ysu$`S8A|3)*RfCk^r= zJ@;SX`PAjg4lLjn3q<%u+u+16ep3dnPUJ(M_HuKdGH{Qvg1BBDz%c+%#!}h~LhpzX zBW&NHds^4dO4mNJE(P*Nz_gDqMj-i)7RPoFc==FEEAo`2~x>7QiF9E&osP$j%m%xo1VaStqMycu!@?rucm)h-r`MC|*?Pv-3iYxzDn?vx?+Y)^+ zoHV6{Dw``f{vD|W5{oJL=cXt0NpDM=lV?N3L4&7+1}X)#MG>A-PBZ~9kDL6n!S;*C zC)kXymfNfH5B2I>^Mw3p-pEZ+8)$7$|EQl0D2n;G^0re-nrmCwsI~eb@J6aa6~(Q% za?$(J%v*6^;0^c6TG*Jiy2?b))?6|`l~J*bb(T(ZU<`fRBk_2?w~vWv|6 zVlo$)W?_xd`ZEn$U{+qDm&v=T=iuTJ-WU2<}f~x{R229$UGk}FABjDA$ zV`tfV&F@zn-p>}42zVC2T_7zi2tRK9v|F*mMX%YUdk?m^1yY~+*d*I$*Pa5h)vnb~ z>qGv&kzttokzZYo;2D65Z{Pv)ph1L8$j6EI!d&JXc--e6EfS!Ij}tN;pd)h~n-z?a z-2^=PJnC(X^#=eD@&dplKBw5kfM|<=J%xpZRmxhl*#~6)Ncl;FHp4v2CJE$u_-{5y zeyCF~eNExdY3tMhkdBWN{ROKF+FjI52C_m`M7c>5g3tJVz542_!<@!Fw99DA1uAP} zQEZjcSg9w{O>iK=fdmJ(G6xvw(V+O9j-+J(Wuc0{lVnYv5J$(NR*QpYb}*{j8s?E;vyx)TUi^ zI+`~1!{?Wqu>$xw=;Hz(f^dQ|5T3Syph6piPb}_H>E=I+SIjeMR|p8~ zR5dhFSHz2W!7Q2&^`lKAe6iC;8>QUbN1(1t+m5EUxxXnZ9|*7){`tG}YhBXzcjIxq z{M5XuNEgh6&(8FSp65vTfqMqWlw zpRz}ZGU3BR4K?tQ{|K`H@i?8Oth7t&3usB3WemimlC*gbEc7;)Fd?ALkq5kqjg#dX zJ5sIXhS`>vQzHLIv3yJ_J{wrTYiQ<#wU+W1^u^R9%g6^{mgZFd6x+UU2fdFKa=ts! zN>;^87fNJ~IDer|u2dnja#8}&ke!_=Us>Ob#jWH(JTsF8lKOhLT|;$}l41*)Y-t6} z{(6IrI8K6m?_PS=Lh}Uv=)Mt&Q(LW&r%U9@s;yvvQ zKXL5V^Zw{pIJm`WFJ>GgM~)0U;y0TKIMsz{gFcpo8(ZP4q6^9idXimua34E!??SZ= z`RK#Xl!)g5C(w3sxcuOV9c@X;8e6`qT#8Vfkk`Um>m9UhEnjDOF@RVs2wFcQWQ)wVi71UPF{imFjcL3IkEN# zUnqBOYLe|M)4*xxXIaVWGF!ezt_|-u!xd z$sBLYv4QQ=Y`L^nqQzdee2qQ#_E)yop1sviH&{vWTATavILlI>E#H_Xv&?TV^ywe% zWq$a9Om`psVPAo5*|C~dC=OD4V#q^8aeKCoA5sOgoibJ zF=hfDVA_Wa$R;Qo0mwycWJEy?(q$ZGe&IJ7VA1#K*^oEs0)pV9Y3R_QVbhh(CpHW* z-UO~dV((bMhK6SlX4wz2jIZ~~9MzT*&Et__Ae-5o`%PvRRIR` z4x{o3X+zAR@oNcxMp}@|AjpeD^`4PaapX^W#AR;aSye9|hnxa~8gx~3 zur!adWdv9V0zx!U z1N?^f@$Z*p{T=h0=aj*P9qu{tEb!ECG~(6sK@*4tM?>->T3@e1XqOdjndgiHUIyBV zw_^`q-4!0ymFgOnyb&l7W@%%LS1g>WdPjB5o3Rj639lD*Wg$n}@%*9PMu~Uf8V%g1 z9)K%oiP0|*Rsp{#FAGM%GVT%2%{Onj;r?inwf-Kv)C@zyrOOs+@t12;rshafWqlZv zZn^D#nQ#_Mpf6Ba1X^wEVAouKmo1z-Us|>W0koL;^?Li`O%K|3eZR0p3%|6!z4K(g zxYqLW+li;Vv*os4K80R?eUznV&$rT5<7C1ZGxJW*%(feDdO;?R56S5@LcQ5pXPjjF z9eiM|<*qV--k}qGVq7gOEDU1<8j@FDd1WwN1?Toy(&O-)YmW}!0C4f)e1YiqNgInnOcqZa z-p3{^SY^-2Nq%2Bv|giz$8!ijR%wznwp?&rIJ5z124nDZ~3o`Kzl#bn*$|swjZ-HU3d`X~u_Z$p0+H^|FJxC_cX8%MjXu-fgnl2n)&e z3Ll-ADGLHjco!8rT-+la?x$&&EGo$Gj5OJOaw0vzD*(y+?z_+4lg1x}j|&{4Bb2i9 zM>_@BLKr42=`;p~_onh!M?9}G?;6@gb!lzMeScSIsze3OSk# zq1@&zXlW~o7s-Ft0L{s3HO32s+Q`X-t?aIUJYgxVJ6osL(#Vp^&5GjncH8X_DKkC- zg0j0>K^wHv5PLE4*l6#4*vdw|eX~I5QcFqAur#3vUAx8{dcW~5_GdA_Y=mF5TT~m^0}>)Z?6mi zYHK4PR(s)~<35{XnbH*N*;zoF+F#4m>`F)Sm3iq$uE48Li_2_S?|eH)rjPBU`4z^2 z_(FT6W_G16d?5M|g#Vsro_Qvm3WpXceGM~xTtu*G1*qN-#2k*FLgYx=9pDFM!MIJp zj}kyM;~?@R=6}ec^l4;KC|Y*@lLli2{)oJvyY@QchS!g<&51Uu?Mm>r*J{N3%Sn@8L87w&CWKKYrwG;xvbJ!HAd!CnH>k_uAylRSH)3d7bt8e(K(x~oUG;Myz z#>H5Ftc;`k(QuXb8|X&@z-k~yCX)GXa)8cA3@gsq)#eX3LWe(UkBJ&wO$7caSArNCLA0x2x%#*c$r3O;Vv zcTgB-H_9&xo`+i4o;GCgD*N5#f3We>`pL$0u1v#^vni9_uu*S5tVMihnR3ms_eVcw z<39MSo%O59inq-6*sYtr@SKh>?|Oi}{pRB~W7^Yp$#2I?12aWqdJCC`FO^ott~%Xt zJGos-mOruuS^$r;aT80doqQH`?HU8Z=vb|!^)l+s5`Av2Gl!U4(8Oe6%5Rq+_|HNY zlTF$i3sjzC9)gK6{{gK4vslD3?{FVr3))(3RIj#HswzxX_o4~a_BVk&5 zRX_4%tGw2nR|ODPOS<0Qq8ZedE}$!P*uakVh_qJDePEnTE-AO|q=^NH1#MI!$@=~2 zCtusD;x<;0l_OVTTE>OG5`u)UTmfI>mz3IRgA44^qx)O$b``))ZH4$Ii6;GvJ^+uM z&p6NKfQ+Y~emb~^K&yt+KQYTg>nTSvp_BI?kIQ|(=V5BQdj0@a5Pg)*Ob!5I9zpX9 zK#P;EZ@lrw0A_QT4F|ap7oS1V{5?NUm;aO-u+!yM57S((0J5#r4reS}Y3sBp_0zpN z+b+L-*`E2o-E3f=3KRP(+GCdvx%SGX``Wu-F1FisXv66OU^8Tz3(*Z~us*IAYn=ap zR``*0Uxizf-(H4#6}*WCj*Yjq42*GQ_t5q2f1XSNz;aF`t_cp*zyWryze|Hn4ORV4 zJc>GA^!Mgh04_eY5fd2;izvB(f&T+1PuKr%&Xsgo*d%DnI1pVtZ|};oXE(oZ?sILO*}rdFX9m&ki0$=Ek}LgCBjzE0pIzqK1B0> z;sPTz3BRjYKx}C(>kEvf|!}_fxM}XARxOvJ~vT zzuC|K?-=Vp=p>mOrrY|`SM9F9eP+oU=UASGea8f-50)!;{P_^uWB;k4zXD95;Q??> zyP?eZN<;&M1uSE^_j3R~G(qsYh<4#TIo;+tr%_c$Xm9~``^mLMMMc4951<#OO62Jq zE!CxwxZk`uK!2pIaXp}4dwWs~{onj+qCGQliSn zZx(Tnc+v5us(W=|O*9P;&l&+s4K#TGqkkg*t7o2hW&oc#d;|I5op;^|`Qvf`t$y-H zKERh2n_N*#586Rh`FMU!`3iW2R^e!A3x2AT2QzdE=CczQ+k#>p6eMsK@-S7^BQHD6 z?tgbyI9;@@l!Vh-Y_tn+d&~ZK&fdX1yj49+39Cv!dXKR?>dd^k@;q9n=JK2NkFU?o z<#__YYA$a&QO<430T!AG`fhTdK7!rm4z$&A&eOvTg%fDp4_9^J6ZdQ80288<^Va`2 z^p*4(7Z65ozWL_hz!j&felpQF3-K6e<>m3KmxskDCaksX{JF0Pv@>x0NnLRW515J* zc6NZ{!7|Syajl&L@$aSf7>N*7ae#%FB+{xLZQi|&S=O!RSvK<869wFGO65As%5G;H zb~?az-|q_jM@vDcvt_rpHtn9YM}{z0$NbY}WwouyGuMiGpJp$-`h&30*HTsh5RTQ` zSj$$~SYm0FG}O*K=LI|UjM%A)@rBLIOcH<=(_}$&5-03YqCavZdR6U+aRShaMKv10 z^!F1^I3e^``Ze4FkYWLQ?X}m2zKuyC3t{5o6OKhLzIC9g@(2$SU(K2Wv;&t%`pHTD z)937J_l=ol*S|X5X3O*nk7Qx_Pe7c!tDkX90Mi~ubaT{_h8N}8y@&R;J$tnYY3X*L z2JDfZLB3N z#2HI$kLyR<<-?2Ym&fd6U(H%-L$+^kI~27QpJQR7Jc$4O_fvMVkADAOYu6>;-kP|; z&Od%aE##n%N8qo}CJ3JVLvBf=-B>2d%;6QTJpfB8!QknoS@ zKGUiybxrS+XLZr0@D{-$nB7~BfkxvCZ4Kh1RfsPgK4^N-Z)(anzMQ^r0z}1h7V`)< ztE*2soSH}3F1X-=_$(6t6C9{N2k0Nk1(wsgATDle`J{`x)hf$U;_p=iNlMC*W>rm2 zz0(DPb<-y<&1(L}_X+;j1Va1bn!XCS4G89hHWx>#L-_0A)j0qB^Mm#n zb8ghsadY!;sCW_|0x;_7sS1Jo>i45pWC#bArq3@no4+QUy7}j4KC=UM>TE*?#k9^m zOf+u%@BG|!8*}--cGY9!?5HF9+pS9Dpdm$e`*WY$&wtQA97X_Nt83%Y@^ca>+KFs* z)KvcXG9>;t9tT`Lr9WUIAKmb+Ypw|ZtFD|%giCNB!GUeU0T=kUN#+fhsV`unN%|m; z@G@@PxNwYm^m*cU{Wt(W8=FRr8fAChd1pAi<`3FA!r#W05W;m`%8jrFAt0Z3-g!am z>i+xh4;mW?$N1YoGYhBjXoLYkdHbqM`8KkPED3Mn!i6?MJHlv_u{fZ7^ufB)qD71B zPk;K84I4HroZN|dByHczUzc?16SgktCBh^)uq`;?pfGtmxJ6&W)DZ0|OrD;2;)$?% z0qBf237|CK7S4sCY15{K1#wmLM+lJk`c89zQ9mm++4^K@FTrNM1dA_6#2Y zFv`MVK$Hb7G3H}r28eJR%5d(v=LW5@Xc_Q(wXF%Tr);fUTF9$GxsJzFb%I)8lFR@( zkyU!XDFxJ0~$kMDw{FAn0LC;lfm z(A+rS>ne7cY4?~S0Z8G?3e#iyE1)xdmcv1!4JXB!2&Iav(+EK?*DK?>L1% zP*F!r2LZ1j4$*O<@35)ECJve$%+IdvME~NXKunbYfHAo*EG!Jb6Ze>Fn4d7sHm08TIXyELF=qS`S(*;?mdd;N{sE5FB1o@aYp{f-?!sGWVivfTcC z>OlK`|IW6$v^?+=Mx~Ps`0Pp*!6pgmV}S_^&La~oq8EDa{!!; zIuWgFX-Esl(EvvfeFT#QfKs%eAljABrQt>uUz7oXpZSb7*iu4ciX=O)<7)Z(SS2tE zK_7EPKsTNt?89^B7UG?F;)!85oGc0p3+=-XKMcFge!5Q;S#NU7r*qO8Cr7xXouC8< znkxrrzw{Fp-k2D(L4mJ2{{Qihe+=LiV~LaZxB8AL9zStSaNxVf0sKs{Da9PgoCXNQ zT*+KTf1$4cYQ6T_YhfN_(*@I8fFtHK;v#o-=+MD9ZEkbP`(}0Vz0cC;F(16;mRo{J zFj`-1-Y_>FeDJ}Q(Y@@IH}nJTXErcv%PShEA%Cwel{~R%gXQP9wC-JfHr{l(4RCtP zIRnMp4fgX}Ke4M%8DMR7VylP#V8UG6^_q9g@>2{SZZ{u2$ewzAvh8#6e{AT^o$c+Z z%WU+d#WrZWwsyV_7`W!-K|zDCRHtsFrL?lgM$fR{J@Ki%JXRal`Du2~kNVr)FHN@N zzu(zT7~0D&duXg3zkhF=ws^Je*1y26c<@6z|Ck*^8ZBEU1>h<#uZ0zMuUccArT&Nm zyV$A2y4aCde`Hz7YwVvF?-klnRqv!B`I|_q;W@Cy02TtW3*$5Zju0U?A z=-dds3_4t0)e}<(gjLdr65-jgLLm2nA^Ln<|E5j|{X6WiLr_aau&=3{JcBolsT}|x zJs-;E#65V43M{(_bYSePS#&_r_ZMy?z7ia0VjN)H$jPb6cS#c~a9nZdbM&{WOwD}U zpdTJ_#1TO~o&K8uu;Q3l!(Si6*y&{cp}jOFZLXl><)myS60|>&quAg7{`Zv*Wb0dI z>WQ$-X=Z>#93#$g;>??9UqMgGyrR4OWij@9Tf;{cJJtyHm^X|VF(QCXmKmYm9Lx0$R|NZUCE3dRp(wKDc zitx03@1I^@<_%vM`n|+8!GZr34)~f0PzZBdG!?5O<|pR8H{X2I9(?e@;GYs-kj#TW z(P@ClRIJmYYcS>-KqD>AzSZ~Q^NH`9Jder&tZC5-Lk6LofNbP@)KNzT;EH+A=d<{} zUfIu84i6Kbe`EhgYYFhGO>DE3y*}j-lY#n;uWB+<=UmNq6BkZX+zOW?=OKi`5yV_jsxt)Cf zID2LMJX>6*gK$R8va^ruZGZbFk!?TXKX&8+ z9qpyJzO;KkUSijuu)TH2OShce|7kDXv7ha>eRs>rZJ~E8BS)1VpI1<+4m+Seyy{ey z8=_9n{C2SY`tC_~%++t&a%=1GwzF@4lN10u-KfS?qBt#MW4h)R$o%oov%qJ`Z6Y8rq3GkkFeXpclSXdbN#$eBm1o==V8V3LtPC~IEr!1(dA9mPb zVG$E;JUmO#cZ&mINZ5>l0WJ`?FHII=r#-P#OFLm)b8x<~cv0W>>+jLW{+DDvbia<(e)O@&f|F+21;TWF z$=TmJgjp!LBKKm6ej?MFZQkzI7r zMPVoVlv7TzqN1X(b4;0zJ@(k3S;fva=>nLs5I^p?#q-+ zRs5&^@XW8=tB#zv+e(nHu3U-x2@cei1Kx%i$1#7$Hyv`Z6Ta!nBsv8Qt&KFVz6D;1 zrwdC;EiWt8k}LSslDS;>c@MXqyR)@8{Z;FcS7w9TrdjX2H2ZjAsXcnycGkbJjg<<7 z8Z~8!o$<J2otk93ch(^v0Jtt_dX;p%P zKiiTk0~O_RLBG1RTu=ZcMisj`6M{@l0u25a#>#ApIwM$dv@@D02yKlG$TVlZI)Cs< zh0}4s`ldv_NTp0AB_+YE0N)q`1`f1h{bz?5P36rYZ%+VE2msTk>!@{|o^IqdGYlijL$8sM+v@*{5}ebw2;fyza|tKw$pU+R{rVK!yD;BI4D1q$7bQo z^M?(vcCC|xYa(|7KX&1opvBcLBgHyNDWAH%r&AF#1$yn?DZ^UmMv{Q8ThH!qKe^>o zD_*(Qasi<6~fhKl_e3WNpQd?P1*wsM~)7LyicL-%wiIa74D(E;%$dEf_5Z3#Fznd--)nH z_JrX+U;*QShe^W$r6z2Mu#|&^FpFNs2Et;>>PHKEIFBD9R`ErC85tQ!4plsD`nQ1~6lu%)8o(M{m6GMiAb72d}9MLD4#Gt=|q_(S9+D!*rmj@QL=% zhqg-}L<1I+hN053L+Hh{2+)ZgT>1&ux{~)7`YD7U9z7rc9$b9!#fD}SWk7((oQV^f z$RBM+ObC}QTN=gz77CQneRRM}!lTWG;Qz`iuh=1n91^qwX_uUmhN&Td8BVEhyzxdm z{`li7c~6}o@`i4`_0}Mq0JPE8>msOl9YX?%uLK9083&lDX#ezW+A@98`^DzYVvKhQ zN^syi&w+}0gT7POHXoh$nPcdC+(QEk!XG2rS zoO8|zroqgWzV;<8%HaJyWY9#vVD)+pD=noA1$q6r4|ugyAksQHZa?kW$<{wF!*=eL zze$*s$(!^CS0o2Yd8s7e0~OI&pJ@{pyZ)t#3i54gFq$&69KZ0|$1rJ6@b@&z`@R^_1gx z2>;qkmaVd9K3^y^ZE0(@&kE%&UL${ms@%F2t8B`uQd_z($&T2ubBL3gl5Fj>TG=%Z zePY+0wVOb&mC6|QzEK-PH*TH7#sjNrtjhK62OFs z_U^my4q6iU5rA0mVn+1VTWr1hCth;|%}UH< zX+O|FX_>HthVZ~*8^Hl5(3mR$Vo@I28wWZt@l;1>C!r3GhNq7|)rD<}_c!RBuv>T9 zX{Uwa#rM&c6=UU2JMCnj$(O@S9retv#9jaRhyC=7GXk$2tZ9n%+uCLF_5u)sIly(- zT^CH)0D;gn+ScQj3$MIqz*`(PGhbq60El(lZMTI}k0>u1PJovHO6m+XveN6tZWX?% z=uZ%B3vDX^Du5_7`RFV3)00j*DG2d^dy^+m4g&i-@4ORCRXGg_)56Cee>|9%1E|qI zImro->X~Pr3Bd8NVZ%Z`^g-8fJMh2*t+22#$1x(3wiK7zU_M) zzxwm~0D2xbaZPZbDRF?d9G$OxUaW5!iHCM1^I(D!9Qa@10R6}3gZk9j`y5gzauf3r zU=tsRH4t+eeTofP{`;I20@U)w9E$?SlTSVwHhse8s^V~;yyN?QEy+~^Ld^g5DMxMT z*5x^;4ra>dReCz>zHda3a>=YIt&;50|4y+SncLlcP;c8sV9|P+*-<}ut4p9&Yd2Vd zHm`^6T3`k8N%h6drFPS~dvB6Grg<%ck100xkP)b>KHt>7GCZIb;1)z){7#iw-yP@f z749uqw%TSdU1O~_l-p1C=xS*?aAxU>Qu)J*ZRq{^sZZ_d8^+sj{(h*fR5BSYHrV4| zt+f>b;y!<2j7`I`7T8f;AK z;vZUAu`qu4;fL*n6HW+u`at1>M|}&D2;cY|V7$SGJgBj&vCv=uGaeU!!NQ8gOI41h z8^2`DBZtZk<|B6#E4G6iXrw$P=gpW^0aD@P<`RLFEG7|#(ZFLd%hACQ8k7)|CvoS^ zn`dVTIH525qS>$H;e5bsC`@KblPeQA02c(fP*hYDG)rhxq|M?Njg4oYebyE))={`p z~2#Q76pKIm}x)&clf)6Y7@}Cy#l6ftM?cRn)pg^powsRe#d4IeGy9Z zizf1762)lJ96+vO4rES2zKYJzL#2rfalf`XDFlpsv0>x$8Mo=5%=z&fIguze{tRnt zLa3a`eOr@Meff+sOIM~9;V~xZA+QQ+%&VDbFEz3W0sU>rE>C>wPKmIlBQYvlyXbUO|?|{%KG?= z1-A2cvGvOGm3r7BwuxRhDXcQP#E&iH02{s#=DhW#Is;hiv^b@Ma?Cpm-j`l_sZxmc z#aDeZPlVrg9AMD?fA;PK-u9{dANVr+$&DgY$+_uQl0^AcQlT^|(nuOaKT#=DDnq|Z zku({qpQNJXPcwxOiKrA2DvgATM1Dva`>*#p=Q;O#x4qBVXWD0R`(58P zJZpN^XFboed<>XWUk}Poc%5KnabicoU|$8L1@)ti58KKcMPsyI5M)YzMIDS*AJGSC zF3^nRH_AeGuf6svbxJd;Ndp4|&A&bS9r>_W<~!h;tZlT%9($BYuCDeX9U}^4i*ldX@jC))g@4^wUo-+E4nQ)B0#g z0eYOy7e1tDC>ekC0bnz8e04n2L-|k8;JoT!HK^>p~06+jqL_t*ampkrBCuLt& z+pfD>I^@6BOk14whxG3!uC`G208EcxZMk&C?`|vifmZ;p$s~CU-&XP%d42NQE2mHG zymAKJvMD4vDD%+zAlpBtUtDrcy6M(C(|@D{$#;#uo8!nNKX{y!AY@c+h-X8zjo$#~)Z z`s=S>eB4$rGs9za3Y1-_$=bocx3pnK+rXm0Z z0Dv~fv={N61jGVP0QH!kjYdE}fI_L$92h?>g!m$xsNgkP+LQU5<{n@OxIt?PF^{8R z#GDN&W3-oYqX%Eo)TV(uh}IQzyx|NZF*P{R6v3q5yd#f598$M5H6n`jIxOtu;X_?J&yZ-2m@}XPzi5XSqN*WdJNU z{cvetw6ubxIog~Oe59d?Gj13JB^qnpLRsGImbwD4Vj;%Vz~UzsV}W>fn^+CkpByk4`#5cC z_uY3d3&H}Jho4I+R*FG!uxuwd?|KJx0d08p)I$HI@9(g!~9fuhY3 z;FbPrVw~uN-J0fj|Hs6yT!5JU_S>(REHN{jJ9loGyd#V>qWv+&3gAi`rD+mnYla_h zz9#EEi+ziH;E)CQMY&F#%Enx>8}gp_)ipRH-jFCo_40}!H3 zL|X}fC;v^ysZ&{w)_nP|JknG){-~IHI(3ZCr!VG{`hYy@!Nw0WIv_05T*L%lS)$xd z2kb_LT6)qA5USMtQ2j^w@hr?sBY!2)p|9px5cML?_;_EwgCfm{OM3#Ijs&1K^H}ne z;~b45|N0BF=U$DoqvfkUuN;_84(RLV%jh4|nU~ShZq7CG8=ukMtIsP3f&%*LxZb9fQoqXI6Kr+HS3_hrCmsqBz_NR&v6XO*U#@Ay~xLDs<9`1ZTgslU7~ zZMET=Y4azonqGPSHR&g-JUl&X)j1`X_hpT%Z(o0BI%sb1ee?m#mR@@BMDqZic6`~B z(vSc6_jKLA9!O8fPTcQaYT0zlb-yisvNBGUmb-tk^yB&eNGmKmD3jiotq0G#17ETcN=_UrEs#<>ys1-Jh|%siFZ;^~pH~)-b%(SA(`MLh4Tpfl zR}0M;<~jTh#Y8g0iZkii#RvVs0<)Jqk){RaXbal_u=LS55eyUD@7F#G5G0sz@(xS^ z&d})47EK^A8AW9>t>X+{SY}r=ZYb2*1ZdkxsnlYT@srE;!TXE5X&X z2tQgH0V*&l1l0JTO*Yvi?T}5g^i|K8dj*iOP@K8Fl0<0!0R8|#g|9>9i~denqCGQ} zq};R@o9Rb-%M1a*_##a3F`m00U2E8>W7u4B!ker#OeuPfSo@T*haC-Z*WzBHK$)VL}K#=wD?&-4kh*;Fm$^)YQ< zpEXVw6P%&BqB*a2ll69eKE?`h$N11qTXyvMr|ML- z`^m}ex_UR(ITCE?+Eb{txbv6Quu<9b>J&^b+albyrFEWi6#$esfFu z<=_9AuKH&Nd6ru${YTcw3KGZ5S6w{4?WOCMTl!Zo>0!$)me)B_%+>`NczeQH*{tww z&rHua?1FU7pKeQYfBMICGdC zZcwYN*LBcKUiDL2!Dk10af?Cz8!L4MU`fcyM4Y;YlkesZ?N}47;rovR0d(-6Vh7yf zw(fur)>8;gaqV^OMMrO!t~{wbS{`SdaYlOgyWd@cLA7T}A9W(kSHJqzvT&tUlV{UB z7lrt~pbm^Ze0$N(V9LCIb~e4y)1RK6^rR=XW6iYOquoRk?zGcROGh4gWV$ee7Y66C zk~&{1gC7Piy~?Bu@LFCu+-$Yy7@&0RcXo54pTRFjS-L$C+gU6`$5e#eRWim@BjBkOLvzbUBXC#0UwZ%7Nxt91}PaZxryUV7VB6rQ6 zVoP5Wf(*@IfcoJRYkk`3f&XNMpTf4)xif7f~_(8Qh)`p56Ect%H$QR)JI zWu^Jo%FxGL!!V%V=4#*rIIaTd(*IY-&x9XcANAiuKk9;g=@70#xon^JeujzYliVjH z2fuQGC+oC$lnNu&_-P~x|)VhSasiX5}k70dKTu(=BqAcKlL~4r3#HuNTNj)1+?Z42c`CbvxG-JE@`_obN4612H`!EME ztwCq2y`x<_>b%K*Zh+y#0)J+Mp~U`1)^_%|pnjc-X#-}4frd;EG7OTrn@Yqb26W>B z%M5nc!!PsvPg@R9^ty)Yghu<+MoY&d&(#o@`n?iK{)v999WRD{6wJh7a)>pU{^e`x zE`h{=u?7G+I(q0ME|f^PCTt^@(QOCg$G`44@xBD{!Q>1aBekDnV_AJ$WbXpgo=(`8 z?}hh3UBo}jZ(rqRd}(hLzM8u;c8TnP{m^r6Ug#{ZHkWux99}eE6sW@u?{?0&?iuFI z=~7{aXP|IMlavBsg#rDi|cy6HqP`rWjj~1B?@Zga2k)p>O1|kNB{h7?;Ok={vL2 z&~}XaSDW1tuLWW=e4IR2j75Kb&Beq~IL+>p{hM@4e|s}D{ymcM2ytd|HG~T7H@U3; z$XMWv-$J$H$fx-r(?PE#7Rn-9F!WN|eDEXeHDI#i_EFN2rJlL_N{*PvCrj>e&!aY7 zlh&B808HI?OMca1rGd2i1V0E;3zFOne=W+O*@b#;{*6TTX9u-S*7I~cdt^izhV5~J ziT-1((>}MKG+M1Pq-;}fPC^e<63zH*VtdIGc3oQ@%pB&=%u*gdoewO8y)>%c)^xoY z956auxLOT0;OEyBowkN|l@JBK&(r5xOZ>Dr#U6W%n$qGuB}ya)+?EzXzmpA9*k ze{Z(Qa#sE{u`r(tdg=$0Au&%(ttpvs2b25Od2+;mP!WR!ay!st<(>L|8qs24@xl`x z$Nh4P=cr1>*w()&tbgl$mRS&w6=6c^~R=^AJoAYIK`q zszwCmSBr7TkAb2O#-OH@vH_wQIm7ThJLdVY(`^~|s2RH-q@8?OiN{!^LA#&=uH7Tb z4o=Fv4$d~1y^oM~%{z%nSNhBg46KRjd-1RoG>g|z=@1HSIkF`S zc=&0&%bYaHf6_;nZAtibJWBqmWr|N&xjD4}_w{rOb zqd;W;w26-mvEwkrd~}+*>R8k!il$^fPLE=ow+pBT#%8=d5 ze15l`Lb(udE0FJUchoiW;oEmpYK;21z%jHF_?OMp`fjf1uI9TR*YQjb5-T6i8)k|b zTb7%HZ=#w^2M2?rs~iR2Rwc7n|Yao?{G8zj6wzF8EYj|xt4ecr87dawh1HXcGv zJgu@$h6RLX1{PH)!SBlq`r?Ky7oSl^(8+^M=?ZDVY9sU2| z?w2quU^Jr^pjd;#w;%B2#A7=m4f>4GiGf^lmWfO$ifk#TK?zpSh zbTlMNo4LEGO2#@^9Fkm%!2W42>g68h!h|mmfU9a)z`?90^jh%s7os%I#8-zEmpSzB|p?CADcMtYtSCM(tUJ=1ocGw^^u5_I}G+CK7`fD`Jp zIoO0gV!!+`^~-UDMKxXNJG-v;Zs3wLQJGTJkibF61$V0iq5WmByLPHrcEX}mo)98Of02)Q}Dz8l{y%QL-4 zz&>EshDd-;7b&1NXfMynCV0jJ35mBQ#@OBBUet{RYN!Q0%L&nUbdhS%|IuJyuNGUZ z@;r5lmn8~UwIQ@DJ>>)-g+euKHt27RBmO!N?^Np`uQw?HWXuzW8BQS$ws+ATQ^@;=rku z<#Wc$l=s&XHzX1Wyogn;n;eSqF|2+5^?)p}Eg{l!nHt{4+7YVftB`+gWDtz1tF3asC%mS^)*V{x7IjdZ(K=;cgYvkr*i%c6|59lhD9s#P zy-aT+H%r~K$-p2ic&UM^aQ}1=tqHL_s9m{M%~=^E9%jQ~R+r2Q(&z>=SFLP3hqlJO zNc(u*KZl%o`ZcdFz2Y&-$7X6}o6_!@v_hXEEra6xVjYT8YT_y*8NBt8zF6(wYWyT_DZuHUoHJ}W7@0e#aB2gk4n zRGqByQ_Ebh>YwYj1GHL0B#6T?d8JJaym43Xq|s!@e6(tu@b`4@guCtiu#J(egRV_% zsk4{~NIFnG(&IrXU1d3kAt$u@KJ(DicawkHZuk3+#Yf{%W5=MpcBnkQ;k~W_+lpH+L21{~ z5kwEIHn-<%!OR?KXI#N1L(EbKN|;yURPri*qO#W-BL`S7(XWt$EUvfedibOIj{zP< zjFv(#`h7>7rk)L=a)Ad`z4HhiL5XAX6F9_G6AO7C8dlFJuwinA0yLXkntpG9p2afP z+(a!)JCDNzDu2k*0xbF3@JF#$F4z2c_1W8`^{W8-JXPoCLgd?OgpRg1+=<1d4Ii;K z(?>en_m`G41W&P<=#AlS6P*f*Qt2arIb0rmLu#6duQPw9#235ZsMq|FH>6p!6xBk; z{`BTyJXn99gxDd_jWa&GhgzvL)()8YXMk{F);Ur74`(Mo&Nmq2@c>#kisY_kk%|Ca zMyVEXsEKZ+MlcXq9?PHZ(fsT{;4GC8yRe>J#K|z@8KHfUjQ@%=f4Z0*>1}b?F%n$4 z4Hflub!l6JYy5szfSSaeRa}~#lT(E*;EL!!d+HXj!^r-V@zw`azvwweky!bJL*>qo zC~Xp#ZG@~{;9PsGas00V03KZb^MR74Zx7i>fWrVwefune`(RhRM@?`uo_Q4n#>dBh zfv55m{Ne6vc;oFd84(}Bi-gY<%NbyR#wv7w)gn0z@QHqT5VB=*_UfXan$-H+}@B~?OA7sEQ?42(QRK-hf@WlQ$N&PS&)~O9G(hNA% z8Rq&`oOA&QIQ1-0RbmSGcyIQ90IvNyE9rZBno4^G{;UNB z$>Gt+L$zQnotJ5~B{?pwm59Ud5>2QvF%=@{wXJl9LQu>XaLmzEYDf~{-xq@K=Zu`w zp!&e=*Y`Jz<@7@*mfBfFsbqB!l&p)~`FO6SIYT5hGIvw&gIqQqeE9XFKoezW7xVq~ zBg=ohX(IiUyjD~bcEdpvMQ&-f{MaZI38&Xv#!}nJF<@8MS4@JVPi=2~L7mYt6O$fi zCPW`&fGkwlV%oFq)ohADCt}lbvk%$Q&-$Gi(~U%{>*2&T!ynbxL;dz)Z#grF7ViU? z7jg6~$QQ#HF7Y+JQzcWBT88Z1?Ma;>l+_k9OttLom=2$HC`=;umN;-LutS&dS`r)W zV+*?@{cq&cb-9emclz5#PiLdGtEtB&?V-1Ke8ls6cxH&6m&(>tWnF^W_Y3eqU11u_ zsySCgYlHm^{HAfflEdMO2A^nXh@fx%cr^ zzd)ei$!D>hxM0~%(1dvL5Y;w8Ap+IXqpwDpM)fix&c^p3Z?4`Yb#r#GW3!w1YvGDx z5I7diVJXM?hXJaR^7b;KYWfU$oZ6L@|D>^FID0C|uN&l4Y|+$CwuRo<4YogbDn&euUBnem}?5 z?{F?}vMK)tSN$iviv5wkqK~JCRyOk!YI?OhuKG`MVsLHhG0Yaai)Aq2JCn1PnlSp| z$=i#emrurmn=OcP-rL6Px`;};MAhZet+jl~oTse%X(yja-s0X#sRs#5 zZstV*tn^I0-b*0K>YSf5Y{W}%w3Q(P<5W(c(Z%*RGoAVK=j{gFqAvD#g!mq^1%h@d z!;`)lR{-l>FDNV1qMC4KXU?7no%+1>I1wQIk#k&}jRIlqRNS%?)^m$mEqc3W%n7L; z6THys4xL^7jc5d*D{45-=D_dJHv>aBs{Dl=Bbt;F!3)=tYsd+-%~&OHIMJ0H7uv6Q zy3M+0N+8uo=L(4_3OB{rxLy0WrV$F-N@GgMiKCh7)0NDV&R}8qH#ftldfK71?8oy+ z8FHi`Aw%}Kr@~mUs+|OiyF`K5KJ?Ti-7y+Ao9&k)Jh4x?r$yCYY=y9KFb@B{Ou((g zv@7->*l5~thgV4AqiQqj7=!9Pf27X?kJ!7BVMuxuST7B`5_-w47y`ywf-F=iL_M#uR( z+wC9;;bv5|YT{A-d|f6w5K#l;eWbyOD=^xCgj3^fU^@&*^ z4iw!0FN~Oz9w-L66AxLMTE73IwP5FBzDG^MdlDlg;qg7YIU7$8?-7-}cl27B0(^Pu z^ACO4hW`1McS+J~aN&q@@0_$!7{>(KYK!Xlq~ruNG94A|R(hEKaF;Cr4NmubtV1t^ zu)JWb0oa4GavZEo(uv*zl&@6mtcKekljLQRf1s}DO-G{1{`O2t@(h3)G{^AE_&p#L zksdSix$AyMfJzMG>0#~3Ku*O`ReEEF&z?hdk27HTd&t6&m8g0{ZaUT1 zEh7v0Ajpn);%s!|UeP3cgYA zX616Rgy$}~TMXb0^~Zw3QJL_WFsbqoBI+xPvi)loJntt|E7RHL(+EFT1C-dhpCR^B z7f*Zvy2;7(&K(0Ne)e{d%nXFi+P{LI5hO;i^oGl0@!`efBHs#MfxXx3IbDTD6DrZ! z+L7RqV3loX{0nwZvgcdu&g=;jk|e~jwH;;;hkE_)QVYS5b=jzRu~lDvQTy9YbsqMR z)pEZ6cZrHre4LLkdbOWojFz3sDI?`7u&>1Hxb{kTd4HM{2^vvqNnc;j{{fzcTic9u zIQa`|=wlM&6vVQs^wizY9%&t`v^79|N_4gm5}{Is2jTKme7+Gg`AbZLQ~{6&JF}fA zyn5>Knj$-r?Bpocgbw$5(FsXb<7{Ytq141^3wm7R>?ag4rHY3m2cPPv7S8i zJ>5RV=>Bw~`d%IuEi(P@*=tLflW6kf{~$g25*;TZ=*^$HKS{sKsP1HFXvMa;AY9&B zY6~879Vftr11yc4>R%%ja!&`1Ii)-m!kA!v_ErnlTWcRQu+T+EccWza)+x|FqToBZ zou*n_NUzvYqQj2ttZNd@mi3auxb_)oxw|cEmISZ!Pg|w~6aEix($G7$x_|jOE01QJ zIiEVE)Ala99qJ}^@C2!&ewT*g<{3$6;n7++T55=Y*1w`l4U<&`MNT~2bFQAjfO+hO zh2#}Q@|3Q%i-~k27XxouqX#)0d7P4tn>EQNp9Q}775=vD>CLBz(D!~MKA^{&Iip&f z(0^4-cp-$jMRQEn8D9UWcSw?i9R1IgIM$woTUs&*Xo9_Brd?)LxHUjv+Bu`#Wh=@} z8=e#dwd6IFiWSC@JmEZCl?3#n4(=P7)5cB#HI82aa9E?pSq#tgd7DU{x4YOfYJFqrdmVFCD({5&PvWzH>ab+JFjH&7WBEBEMPi zV7Kl-Tdq->1I?~6_8|C-i*zdr|dTdVusk$o~cg-gQ}VHlo2X zy!^Nasx%7@w6`neKNXCkRT#TSbH7aM7unLFQ9U1vB$M3ek*YJ1F=6Q%ZF(;mvk`Lg z@am{a=gtqxG#%3^lY2`1nT>xJ3rsT5p=?IK3iLjAu1EDhrIUq(MI#>aaa4FN+49Ta zqA7tT0z??-da|K^^Rbn!tnt869G4uA>}J2@0I^A;Zc&Fk#99HoOBJjZebRGsEnIct z@hU6!R2rcvLCBOcAqg9m78dxW0R9ZoN&jCN#80ASMs|&RhhOmu!e?)?gbG4Rl&TA` zR_*(!aKVvleYObO<(@v0z44d69MTWwr^_uib`O`M0s_pnI#u*dHOqJT;zi(WceVaA z<-Koa4<7{6+|Pf$b0pVzEn(}QdwB3iivJKVjT_sR8#~U~-rU>;g?$x&+GQ1*X5d*H zF&0bRtM6RmhitRfpy)7Zi);S^_LenBP}P+?sOHZ9gM^j_^VlUpR(i`(wquO}sngur z?5_DID~^nMWAVSo4Ksd7S#-R=pOIx^b|N<;!NXg!LsN4d-G?t3m>h9YXAz~6j>9Er zPEAu7qUIJmQYfw)NN`p9nwZ*AszLgms`EVDGFxdppvYP;O%t1J9#t7WzUh<(=X#{4 zH&ZWNtCZ`RN}gt-g5yC-I{>r+HoCV~o=Ff{A52<2o!Q*r{Gl>NkNky$ zHW6h6K{&dh7DCT0_ZBp>RH}Uz21951EznwOq@9h-b4Tk2EQ?IBkE4H;!|FCovfryI zB519?4kSj}0px9oWf^vY0=<2gwk5zLR61lf=T&Ni$Pt4lR!={@;X)!AHd+34TmFwP zWFYTRTE71vd5dj!vTVp+s;AP>+Huk8IHN%MVp#;UjrpH1s{p-`Z!+s5@iMoQY)H2x zRE1a<$CVCD(#9jZ%xX=$!D&bFAzPGCflf&A@l_6qga7}%T$tU~W3np0_PvNif9+d4 z!gakkV4nOhj)x)OrF9j^c~0VHk=<}e;URM`dOKQ*L?~Poul>{Cr8QGymnbofi@p>b zv83z6wlwsIZlJTood3U>-CdyJ?sN35{@`g1fH#&j3Zc(>hHcxp`-Kprn%-|eNGhiP zOuk@F@+WW|+qCg*n}4-M#M551sxMw(C#}6)$2z6}ry;1Dw;Un3QS2Zedl%i#8gpKb`&bfxSdCpHp?cDTGSh-8W+VZ008D~%)N2$) z080^X0G7T?>%j#ZS3QKf68uu3va+%V=vq=4b0Q1FRZpASJY#Jd_M{WiN�V7IJDi zI6=Yh5A&fv`$&oVvYHSR(tEjRuZHo;8)bPp;L&22E0vHBffWw{K7Q&} zw*<(>kk&RcZ!^a@#lfoabw`?(gWF+6bYR;nHiT}P(_;DyHMXdK1w)(A|%-@Bb*>P^WvGheV*yxng zAmQ)zrgI)MHXa_2n zHW!I`W1aFUle522X397FY*Y`H=(5@LvWV@!(ekubzo0KZLS3O)LgpJ+spV3ox$O&1n;=AsHLl#(E(WGJHcuSucM+ac) zrvDHiV~DOJwrdHf@ZK01zo+n0IGNqlRj-Ne(FN5 zxn49z#J_ZRzHRrbMM|z7e2o7=>Z~emMLIc&J*7N@o$GtEK@`-yd=w$Wwocty;ux}G zl_))(vwB^IX8H?ro0|G}aiR9@p<+&y=xp(}qfi<;g9P!GzWuw8_`ffi7wg6|_qJ#5 zAI-qRmQUz}oLBw&12gINM{6U&og1=oOWUsuPa&1bq1v#gOFOI}E-R*Ti-LQpe9T9DTfOl+-YsYg_Yiwr{DD~KMSO3K z&3}J|dOmud;C9FKJE?WAC`i4xzz1BCp|9#sNB`UkIi~-&3+|M_CVyvE$qCUbOjt|$ z8!|&q^)ZB>!8xX}+9erwMGqN7mg@Zvjq{gpB^PDIY%ol}0E|Zp?%7tA1y18X{;F)G zSd>~#`9UCpL&+V(#Q1P+4kzc55XI@M%}gQNHd z_5~cdZvVBO_u$2P8((K!3>d8C7!ul}k6)O7(D_!cM;JE-yb7%!y!II5)J45doDsAXJcKWAj1 zrgu3~b2KiN82o1ViyAw_EY!(dawDhxDGRcAtF6RnfRNLmAN?(8S$5bv63pn{&7B-1 z=}C=8m2=&_@J~f^F-&LGt=}+10Tr23E*Fx*ifgK?AHpyVK#`IvwU8a2Nc9AEt0~q; zjqdj)(liHZY(f2ewa~1&_Pe!Du2p`5p9d}5bA0eLJ@e3QpinL9rE;t$i~nwJV5s zkvz#aKKb#|<(vY3q46kA=t?>RX3C|mJQ<4o=vlaXM1(vcXV2CfYX?U7 z$Pd9LQu&B4*7sZngJ9=JioIC83UhGD`4_g%w}0*#)%DIQ{TEpw(i=$M!M1d3+wnDf zAK!Q};XOn#IwdJ!$>DWE6AI=UJGqRYb5Z`s!Ms8l2j4s>GNQ7y18V=F6H};3bv5&% ztxDVRcENbhv}xPq3JqllWANt8(FgEenVQUckM`5n6o?l-h}npFinrXX%M+^bY=8S> zYTXk!a!q|z6gTpvAo(l{XAy%HkA-eRSbOLf>_qIhdFW-XaVX(Xb~(aaWeHAQ~* ztCH>$OO`S*tuaeO59mg@Hj$&)ZYV%!kdipo)s#KVzWo6*0MuuH^88+qHQ)ZH(V>)( zP*3O(Z_)4S=6vOQaoyC}2lq#8H%OS9GtYa^pB22Gx- zb~EnLR&&uVSZTp$V1M7G$Y1Sk8bOv$@?lh>2KRQblU7p8|s~Ej5#mjgqr5m zL2+xFT2w@$BMY>O#xV0A%O0barc}<^sm z-~XcvP=o;ycH9B@WE(oQ$-13$H6*x;TVDUK)6pi5Me?%_YJ(WVAp9IZRt*-7w0|xL zmpOsWOn)_)fN9fx|D(pK*(QYrP*Th)u({OHl(5m~m-&aoy~4<}c>s1B`mA`32Fouw zHK<>7e2A+{a9c$BuU_RB#lzMAyY{@0472kno%d#|yhSO}27u0Db9VKA8WPwFjEdiO z`Z^D|*)tC4zx+XtB@d?t;DLB?9W)qFsFKZdXoN1`hfu zs&z=4d%qi<$e2?D5cm3neO;Vb^}#32)nMve4?6J^%0B)gJKE=rhEX!COxOcpC|pg zUz(|e!Zv6Gnr+O+Y!|wLzhY}1*yWUL4!t7kbSN2Is*E# zt}SseO>q}muV^%>!Qgv->eJ<~KSYNdPbxTEdm1^J)^|bmGL6{Jbt=Wvn*|t3Xi3!B zdv=RcY&|H5%dv}ZMzKz}ONpLztp)Z9r67)S1xY66e9B|c7t4qiI4B!6{C?DVQO9~Q zfL@U^kH6co;ogdWtY%8CN7i|K7hebqk6fFhF#EeWKeAhQgNG~{TI0xjr^Z9Z7Oe!mH4W_REVYzV(99?$Y|IxxjuO7J>M((BuA5_@^Yi(xN3RQ}eq+Mu zHQ9CWizTIwq1d11kySel3${6FQXAMYh>?PUjP`Re*{wZ-g|%+B4CA+#3)jzVE~Q_5 z^UQ&EC2_3mS^Mt@1RSXLh7zfZ`*~$!#Mi|~XwW*nn7KiT3veTd@{~=!5}l~EoPS$W zk^HPTft$tKgr=YWtQedc8&MfpJg!K0b}RHp^%2mb;ZnjUq74)mrnSG<_UmLnOU9Ch zVGpLDbU^=3t;|=1nO9?Yl(p8no(OCdM6gLVeb6uH+SktHF*LhVF<>k)nT5Vl6pUZ% z95JRiu`!yKsOd~oA@9A@^!_)_yu|J!gj6W!CRFEpwR)~%ynRN}Otn}hF=NpE#ZKqA zdXDwB5BYbzg;&Ad z`Frg)!Ohz(BAP_cj9fpC2k$O^ckUp#2z4TgS0pK6T%lV#-RAoWI#d!2wBAb7-l{H|wf9`z5ag-s_ml0%*B;GTC`1?aMw~ zIuXYOYc*HsUjQ@GN8E*^zo=Wft9&a)CLr*;Vb+@uQSb)pD*Nzz6dTEB>jLxFI-WB+ zzBj{itQfmcQSgsCV{CWYU6Ldzz!JP)3RRmf;D1ytKo&|l_*RGrT<(?N-#+M?A=-7Z zigm{bD<2YWkH6#0l=xytebN5b1sD$N<>{u{i{McU|H)U%O4a6^rt3R9}SA%`1h9Y61; zHm`qX>NkK*RxMhXCx=b6GDAr?L955H{fN_SIsI*H6os{>g()K5DjiA=WP8g2^+zh; ziVsbA^RBnEz!UL@weC}&Ze>7H z3vrGmYtsks4Ow@J!E`d%ipk?Fx*qCDl94smzl|?!OkV$Gp={*E8d~q3Z?azB6{Lw1 z-1|}E&$623)?c7M1N@e${QM2x(jLJW!PO#wA&cQVdacvkqHxofiK>|Xev}W^q>3X9 z4@pE6Dj+&!kfQ$$ayL`JPj>}%JfKD2WH`G8t)K!MhLsi~2FtaDoW^1W`+g1YEGBug zCQP4R2F#|`w#5gfZgbo>XO+(lr4HWNx#->l_P@ei4oO$*EmzOW7UmrdIfLYnJe?8w z#tA_?H~4COtqa*Ja=LS6Zp25*CoORVWMdyuI3tF74C#po4LN!0pV$OK`w%DWD`U;! zAN|}<25^Y3;Z=R;*NjyQOU&hNn4PtqA5e53Cl`}e7GDU06}}ruXd$ODgPGYgtdGw~ zn-z@p8Exnp86Pe8Jv*k{?*%^eckSr(b>~aye{AZ{2yXvxwC0tY2fB3>={D89^q%}BFs8_XvR}7ix+VOvC^5z;%;Gh z<}@fz&&~_X0|yRa+mKn!wpYowE}V;Pbt5?EfJL-@amNhC$*5ZHfaF(YkeNac4~7pv z<~@qX=Qmq45Y^M2e?A4A9|gi_N*Zo+8MZ(dr-Pm8QHRxemE)vKF-RU*Zv$ct4_^$c1V55mu~BQ zSmpeb#N@S#*r)x-h^R-U6+e^(neT*NIWEouRJ|e%3%k)kaj@NLdw>>>U1Y%RN)vB# z%eDJYPY;TmBlpj)x`pDlPvMu-^Gwu-y;l#Tp{F-qM5oq+oC@d^Fin0mt=B% z;cKF6B&7^~@2DU$f9*AmFSUfOHtzS+Yx=BM6NjG4Lwxt)8^x+Kj;cpD1;&(Q)R}VvH}{Fa%KM5^887) zM|SR{~04B8E6C4_ZaxJRIn$_TNS*8~z8n#gJi@hXW`(bD8HfW{Buf z{~0?CzT%cqyMVEH)jWTr%88S?c_AQ6mM63+7-uLNpAgo7gKH0iy~)hw*lFAfwL@%^ zi*e9QE6=Q^87$*yy++i+BI&D>6LbTXN zSO54iQ+Fu7ld+I1GvUe_O7ECTp#SKF0NQ9c%Q-oL2D7h{`UR0g5ieRSV1BLb!q+0{ z{gJ*_Ke;;IfHKgn)p9NtlfCD;zqM$V{ithPZv@O^4JU}oIDbk`bB;1po1ujL5VmvF zX|M>yWagCYlBp8>Pf_nkG^<+WBGh&SNi&OX`kZbS5gb{;AW)PTQAPc*N{d-*pzd2> zb*t)Wc&KEWY4`Rk4%jHxaCAMOM2~dbZ0Et4O7tX+ZEoSB{f1j}ldWyk(-A@f?g0wM zNh78SWn<6Bzln#(-voOYMCxZyfHw>#lz3hnKy3yJlP>4)PCwocvU+We^eFn}Fx$3C zaiz321?uP>qB+o=lV)p)j)apHh#0kfx=ncL8{>LO>~+xOjRAid$(%X%&sEZ+2hSyh zE73X2kBA?Rwp)_1-9bN*Wqf3hMX9V_9L_ktrQs%a!}v**YOp7y{1s=NBA$enE9mj6 zU?iz{_NzlWpcX${jWC#&S|wD`;22;o#8*KSurF^jy|=_k$#R%Rc zW*yoKKor;D$K(j56a`g#OyUT-0BbjUt*mR^V!mDBkpp++T`E5AI-l$?bLqH3-wU8$D8weYGrh{5Ph}uyp3@AxOqn*t--q46m>5vPPfY=@vku!Q z`^N-vrBgrWUa5&HOu&<>0H?h8?(d1UA)6tY0n;V-_osc?Xj1|0VRS+y? zmSQ%<14X06w8mUVZ`-YjL4vn{LC@J@W4g=kgR zGaRryiwp0oJ%&L_Bk=CQT5S47>V-5U@ zcko(Tk#T5L9c)0IqR#h~56YKW?KcEpy(!~f|n%0<(0ZA`tsciGjZj2ic4hcoA#-tLyGZpfgdwnt&*#*X?)BYjtVt zm}kX_!UgKrKB#%+=jeYTLQ+!-ZGJMZz>PZSXr5Bc=;k9YAPQKJ9}~M_L!j>2!n<8} zOzRk7L}O8-C5ssJT+j+2a?XNJRW4*$2TCcn3GKW&aZOX9>c(id{M!6W*C_2LHguO2 z9NMB{*nSzmIU)dT)Pe)ByanHJ>WE0%4s{bQZ~8~e0@a|CGegrV4mc`ng|!#2L%9TQ z5j@c1xeM+~VTjC?7dO}R-^68yW0oTJ=3q@A`zJ-mXPg+BUsTEn!}g-?b9u*BQkG`X zv0CWU3o(e3v4Rv(qA&iFP&n6#I||IB78S^|FORz`aqb_?ky2c$Hv$%J@yMDMk5!|8 zKCgyW3i+^{q2Er7W;?2J?b!xXW$7>M&iq$`76X8EgL*j3WKsSBU~|lUMBfozK8bZI zW+sk4BQ^E@*YU4J1xd=tUYf2?*{IH&+FUAv0U$s(1&0mcN>zPh05X1%7NMVv#91A% zdzge!E$U4vm9y{AI{p)X?Q|m~%uTNaHITeXR8*olmO5z=dbsg;1Au?_grzP^(zDI? zO*ZnUvZp&hWcr0?+b5#_hRgEajE_V5+jQC2D^4ELFTT<%84&OK_KBsHI6d!0@}12+ z8~12ZH4_(NSNF~-t_M3%iJnHAH7jkm{usD;)Zf51CygJk{q%*?}Hnlzwt}or+~mL;;Nck zE9lIBjmI_!fZH9KhYuyud71EpsscYuKCb+-dar2kZV;mz*eC~NwsxavI%=FZHtGQ+ z?r!g4(Ksvyl;0umA@3ZHNi(U52^bGEA?(_}us**$X|w+Y^ZWl<052t7g1je~k}1)^ z#wo!>*%q^QC%RE*K6+0zfOqt{Q^$gCChq+dJ0+5bjX5?1ynkknE7BI)hH4B-|Ikkw z;P0albBx7%O+e8HI$DMNaKO!wF?Kci85+B~!Xn^2`L6b^oI)uxn>Sg;(9*j;b2vK_ zpf|cF%J5%~)a!-pgAbAH3eIhGq}U*p1Hl&OX-5BJ?M2bDVyG**z(1kjS zjJN!yy)$WH*d4xBQHu38!=oCKsWJGs>osxbo{$&o-i?vOKr)k!?!a&I2GPcRI%;ap zOKG0Z%(mg%L`RdKC0f%mSQFOw+~|fKWWvr!noW7I%QDdytzg4k8eiLVaQxF~!hdBE ztiQ|U+s;Ctt|>~lbt8I`a(e0TehTBu0ZB?kxX6p8|NQ4oeE7;D@0AxA0Jh;OLnKO! zNq<&c*DQZ{+F@}>`Fur}F@U`lA8H82)G93X>;_`v3vBb<E=1Od#ZTP_#XPq(4gmH(0W5*G!99r6)c**hvQp*4{ae3TduPsD9-RQ z#eEfAJLG80`gJ#m`T04tK-fAt3fQ$`&Xn?591_NX^EUcV)yl-%63KMOZSA(Hf%|#N z(l1P5=UYML(duiHU~veQ=5{~{M9T|&w_iM?t;8Ub86mVwRHt^Q?EfRv70qg~K2Iw^ ztqscsm{v-3jLtGjl;+Lsfn&wG(+{#qNoYs8C()Sk&g*r42X<&yIH=0s#G0jj8U#Ay z!W(&%9*QQRGqF#d5*8o(&cBBhy!6-d*X2`0)sQcO2lMRU|aFuyMx<9tcfG+ zl1-^9Y{$vLCX{>Zw6V_Y(%l4jH0ZegHoI@>FWeV3sxx%SwZ_oaSXQk+FehWF(;($G zcW`nwdAiBes{!No#wuipp9%;ifmM&mi#C;`}Z;K45yacm&N%sD0EAk_C+O;Tw!HgDzbHj>V)>2mg zbNHcBlJ`TlY>a{>Cb)fxd^Vx$g16?b%J;mfF_0yWN5(hYV{HiZuzp5! zU7g>yF^JDS9XIi{{-?f6yB&C9Ikc=6EKKnLQ3QRn?fgdzDS8xol~s9axqcJ;u~Woo z2<(sEEt}g@kP!Dfi&V3S7js>0Cl$PXITvQ}i>&flC|YiOqoP#p{J#-s?Ey+qSmZ8BXVvZLq-V~UaUl5a@>u7%Br;ahOGG?q*y_}Peh2}M$zK%PtfaS z>j_npOGj7J{@Q#!ts3iUwf_Dptcep* zkqxONjp?riLet6Pi63K#C!en#cyAKJ^|4c;{H4y)-VSX(XoXMy-qTNqdL3Ww zGcG-MXWt+n;rQD!;2edS4|)6^q&iK`ed67a9K{Wk%GemfG;;gyChuTVMZM{NoZ&jO zetGb>;Xb5)C8PoUeO@wyW}G zs2dnb`w(+OBpb74(^sud){eUhK^Z`KU2N#*-dFznHple)Mu2ZQMN$t3ngSCQaG9rj zA>~)E2?hN>4Xc{I)u8wo#>o*WX#bJSl>pQnMVCXU735jH7FIO8OZhSa`hhf$i<}HB zxAX-IvZ~Hb^6JJvTYW49KKj^>IAeK9#(5rRAkh8V6tc;HMzxV&AZZ($AU$9f!q#8| z&koUVLjKW^42CV;a_|_(MN$$%Fc?w6vTPqF6de*pe=4Afko^)*JL{~%^ic;F_MASS zObwC z4;lc~zNwS6*I|CzkV`s#l=d-*yN9bBa?Ur>ABTZ zstociW>}By^#X{_nFdNu`;O0-EmlTnf~X!y%!R=1e_^?pw53w9KE_3 z<=cnulk7;8Sa?@fEk?~$B5s>1p`ba7nRtlxZj0$D(|d^BI5FVZ3NK1nyG}4HT%3Lb z6^F~UC_<=ao_&uZqS`W3(8h7f!fYeu#rYy+6ORiS{I-)0P_YU;N(qwhfagFQ3p51E zE8U4>$#$;a`H2~jn4Y(um}2uxb7X9A?KPFjyWF@}meP=dN>*$tH@8z?x9n0?mw$lp!^ZFLxK1XOjql7bq3a)ax@VBqu zlO?oGA_Qr>U@+pXcLZ5R4ysp84O zCqys0Kx4*PFfZBOtSe0FnYBwpFLnor|4INDh_+gWe#Q&;IDZeeSaG-%9o4d5d)?+j zrPsCU_(y5|f&+XJo%~*?Xk2i%#(4Sv(R9{fQGH+Eh5-abQd$}%q`O0rkXE`I3F+>k zq(MYF1?leYlx}2*p;J0%sQ1kG_q>}{$5i&{d%RK)=J5OOe(3eTx=Qu}3J`EFcyNFl*Tn9|MLY?Ogzz(1@*2A!`pVQ3$>g&sVsHqd|0i&Hsv#OM=u_PZ}{B0F5 z=Hq{pMpC}N@PzmdJn4YS<&~hZ%esme>k?nK>CWXpQnQdlUYvEXBfVJmqJBv;(5fge z@ldl*!pGAED$NJG^&Grs z(j#}wc%s9mWPvjG0zrd6qB|!#4h5*-JP1#bVSsXM*g8Vr*iee<_RayX^DrD}Pnk3r zLOpmn{d>_gitQU3=;1ZP*jZA&+Y3?$UwoCVR2U=yV2508F-kNeIUy6&jWBj%t*CBc zH-Vav7{BY#1W@zxA#?GTTUpXhAO~%9Jt6i6TAlD37-g85yJu3g$H*-Dls=HG z=jJ6^msz3XN&5{uxrGalXDI)l)?9~xyQL_1jV&EdBqA4)G=*V>nYd>!AJ16^`tZ>9 z&}cTIA`8gHubp2!)0SR}=kgDF2rn;6kW-Xu=ggGVyeywG zQd#2R@^W7#pe6d#j(}Ns^_!w}uKaXPd?TQ(&UHi5I-U~ZJmfRAgHqQF$m53F4!4ZX z(ToO8ye53USD;7AAlvFs7C*0O)us<-aEnr(N^3a2V~W*0MI+}{P>4Q zJ)-KJbkm2nwqA*~)?JB`EsvZd+blDr>{X||VeF=a@(+-QU#EB{^lI>9DM^MRBmd%eA%p80SZ*6{N zRPq1Bn4|JbQwU_fv-ji*S%#i)uo@Ek$pMxpp&kF~+=8aINoa1j%q#3$Obo0U3?vdq z7Pwj4Zi&MO>aRy4D-ETVzE>WIi2ljBi&T{#KxNfuI@|dI+JI%{+7t#(vhKf@8CC)f zu920il@xnu=%$a6EHEWKe(H{k%oElMQY)+Y!mq^0b#-Rz@OCi~yl_!jL)R4Sa{CSm zTmTPI2TTmd9ihm)r4Ze}YyN3_lm*WLQB$#oAK9n9|Z*^wOnV3+Fcf4g7^5RT&)mGRB0S{LIgGk~lX@Ag zqWrIlOX;;EGCPLFr-Ad&VU@Y%vw?$@hN;s_qjc8tZqvTN_w3L_67Qe@t1DBPX0@CR zyh`?E=E?bXosCgm`9tlvKc{1?C; z@MvJ5L$L252M&_8Y^Bf2VWbcp^3oluR!9ABJCgZ;-(ux02POM0hZQXT*0*CF>7ZAa^Eaoo&nbhYK~&v>HPub;*U{E4Y{_qGx9-bMgFPIum+r4~X@92jeg4kuM+{ps4#pF5Bhgn? z6huydP;|2>htg@kW$ZYs&OOG+e8f@#s|9qcr8ZWX+7I3kDsiOQx6|pGt_$)_M|00~ zY`%&Qc)pexY7WlDXITA|tmqDzBT}kAy&+oPgntug`(;%!)i&J;8Z}KTnm`r8p29i4 zxY$>`wM`fb;veO_n%b*8Q@)72DfhbWyfVDVgtsERTyLSAfHV~Wa?o9CD$k{xS+Lqu zbSJN5OykbVb3-Kuzxi1JN0L(d-reyQiHs;kT4rlGgH@R7yN9pP!6Ta=Ke*}Qyf{0I z+TYXrA|0BsGnTH!FLE{V+|X@eg1pArxQ$}(c^CKJSsWol;su28YB2DORVGyg-!oOl zPY?xz_^7TLqaEpWfeW0w{%C}wv&|}o0)Va4TTWVMBT>BMq2pD%asS{fQWK7wt+u+( zn(hc9Jo^U_u9!J^Cm?e70$$3FH3~R; zImKv-7>@5mkvKq6lmg<_R^BlUr18}Ej44Vae}b=(1-GhJ--kOefg|D7y_JdwBwf8a zVvM_B_MGGvU!7~}P^$#W8*&`%!iH4-2sw?hzR~zg4Un??Z=5+a~|!mWMX- zfX*`>C1Z?|2 zAd<0NQQAVrvoXd(Y_*P0C~Zf{z5*7ajGsFRjU6a&8 zA`VQMwBvk-_2Fi*D@ga7CJyOuHrH2&+4D440l0#A33u8Fbf?o)UNjsbn}nnt85r#c z^MgU;RwmUizMm>9j*o>__>^P0!%Xg|kEHb8y~(hpVnmYIh#%)$&`c0cm>qy1nuCsSiq3?#MD4J213U4T_N)72%3Y_F;B)SuhzHq;&V`-{W*zaJi6 zJM{Zf)NoaNLky#$KaG~YW6)c!mEZRVltax3JtBCA2D^gtqIQG_!+>IE_fGl4vP)(AbMRYsJMt!r zPcMwo-pkzCE3xc2+~k$F44llL@|*{-<9=qin(g4L<5>LBYy=pJ6dRm9D18j=*ri!U zO%8%lGK1(AIQM*g+79!7Fc6wNp`a z+}X_RN{vymSI!Gi zv(}Ou=Hz`%xk%77L6rs0$#rUXeJJ-h=Mv%W^i&P>JD~%}UeBl^gK>q#7f;h*jNqyE zF#%{>N+{?cdfkbL!>$KOS8t}fcHKSoPPU@z9NNBqSI%*D7~newdq%ja(g@R{uDfi( zhF11h!qUVjCK=3XyGbos_Xf87uN`xe%s;Im>C91>qkN+P<;hw>t+U;;AR(QpMq+!Wtj4PL{vkggQK%pNK&cvahmj>Y& ztEIUSjoQrwr(>MhOgV(bsP_NFsZ#h0(%q*YCOG4V9$xA}GDE6|^a?!~@BHhqK&Hic z8IAXORts$LhnP=k{^@l3R=1vm=p4SfA8!qlG4~h(rlm*iXD?(g{sc=7?nB-rX0Pe7 zWS`w4d8VIVbwr~j58b+>60gzm|8>`vgMeK%U<|ei0@3J(pv+W#QN(L zt9oZ0wg~3xc{N^B;uE@&Nd5Y}?=?JN3)grs8um#0o~^jgN$va#TQObQ zZFR3Ot;9=5U2t+v{`24)JiXCHjT;Hne(BIpQSXK3SWF2%1m}XZXY>x?ufC%y)pdFW zNxgEW7WFtGYJCe5^pd+LdU$fUjOhwG2I`fg<`zhOF z**PWWmeW4u@f~`ArMjzzgBBhJ8>S!}dOq0`CUV@yvyvIePW}-AZV+mqsOlBnM5GMJ z^(5tUlCH7C+nyfLxx4D$PbvRH<(Hr_g3+J6^H-70l!aHEBe^D=Og7kxpIk2a@M4=$ z>=9NV992H_%P(OnqHK3HA7)kty$v1>3rOgXf|kEyqn{g1anM>u{v&Oyn>ZoB}9Kd*Aj z`(-n|9<@4KM1%EOiLu!(kl8rBke_o=vTc5TgvxJ7mmhVvmfz??Gs~XhaVFXQJzfeN zwJC!Q1WpzJVO+u=^JU5`qs|_LiYE*7%BG*_G!hdG+hzKEJu2 z*Z$qu_W5m`C$5&w3(n9CPV>58RZe@wmrI^QkA8|Td7DT5z?A6^-`{WYm8Y9@;m1sa zC|NhcAo9a;x|PSsx!?FmSFG@iEHeh8^r$bE=87|n?l(wA=z>((kZ@nR=<1E8!$eRl z_tWcIa_t0isUDd-e%$=>9duYI&Z!YNYd@T-31=qJoYNb1u~^;#ZPNn+IhxN{fi+Pj^#W z#S8kF>jdD{KSnB*7rd*NDF$d53*_k7wAu2TyWmi&(*p$~_~$d&C48Z?{SAPeIk+N{dIMu$!7{Q4_<*+LW;?wezSs?pSn~5^T0c#T3&2agq;$L1hytUvv zMDo=3)P1>VG^S#^r7`7xD8V51y}lxu>6KPOwq;DU82XUJ&M9MMqanO!vUpI%cjM(q z+nlg$U7U&YiTuodwu9G*-j7~Q9HFi6&iQ|m8DI`vQ{y-3jw&OJpU>Hzfrs%6 zMQTn6&I9@HfqhK#aJuXgRkpUuHAtc6UY<(LeYRyg@{2S5Yt6^OP_XoS)-8pOZ|9XK z9&cqawK{=}FDCls8}c(J&@C&orV$anc?1;tI8k@gAMjDd$Yik&LZ7UXE@9b!3FhLA z6G`+Q2}Hm6dr~LyST)(hqvmjKD&`76lzt=HPF8#`#`b}v*{f%j@9ApMMy$yFi|xan&>Zhksur6d1ESO(9l!F;q1E>bp*WfEP(jMDv!Iju-?0B6Wr4crvaa zf1U*)_gMw)V~ye?8)V95yqnUJc`FrC4wP6@`;aX=(;Ygzgi=fxwtWh=__o4ZkJ}~u zn1Z#^9|wQz0~xq1T-4ot*tKdX%fYNrSWI!3WI$ob~Z8hm56 z^d^}CP^)N)bfTQr@7sy3h__5Zo#)E-80O{Gd$x!+;Im+~zR1VFTF zNK%>7tG2I+9xbWJCN{QWavef>Y_QEc#_Y{bS5c*q>6Tao`n9wr-{xpoedWU5aX9pW z-WfXSj`2Df8`E(8atA z?&s)`Gl63_@4>xZ7k*ngiwpH@FS!ACwS#k z$=ZNiM&`ST7m$z#06JKif(yas@K>@_C9~1|B7x4T2u#TAOb+^X@TB;*ehiGo(N;w< zvoh+T^KUowA)~G8$*0%6j;!GEcog}*SMPDk3c~}WB0Q*?t;mmVN;}QIs(iIc31K+0 zO84~~MQ}AFnYxHggKh!(TU&}2;@~rj%liY3Myy5lePoY3=e+-gk`G#Zq=x3q3 z7oru*yC3=J$5;LtT#kLB&Ve-LfzoNMM@I$tpuv+ouAVSQ7S#rJ7;G3j!Y??Kf@qzc2IeZMD^uS89TcB;>a~cL zF6~CtxfoJWR*7OVJwzt3bk} za1T=WxH}8(_T{c#cGU^n{qU#ulQ^DH^!h5kulg5(U$bbWYLY%YK=e)m?K^fYzzo@8 zr+>t66kV;Y#13^P-eM7@JC3SP5^z<#_#;fe%{7gv{2Dq+-q~mJc*h^-ejlKIJUPBB zR@f=5uSuF~Po2nUfM$nL#V%#nY;Ek%1hlY;olUV6#Av3fQMTMvy_=Lju}EJS{2C+_ z_z+GXE2n2yuK~XulrZcnO-b<9zE4v}K5QNGv;uop;!C@R_Z~wV< z?~7v-V1*)JZ2Lf0zOCs1{+3Sp(wa=WvH*I<`~Rw8=4J);N-(th3ywkb?`J=|LUQBE zlW_-;9EJQC>y&yZN>Iy;?1b>lQZ{!MbZyH`IvTL`9j^J%L8CF^Q_Qv1Md9MJ5S0{t zCsY*~H<^8^NA=_-F?%`jIp!A$MYweKyE}wP#OcQ!;N=zF5>(g5cnbwy=8p2kq>dp+ z4RiReW98%Vl-r%}UrfdM^CD`BxW%XzTn!c9A12xXB$SGB+j?1+QJHXqc!&YkN|%7y zUrfdfRivIqfE-BFr5i3)dodvw^?c`~uKu!g67sy$stF(c9a2JGzzR?;vj+SI7V+y0 zup4~8un+-I+y^8P%#6Vx8DxE1ODHRR?*P^(?WemGRUIBp*!hKV!M3;{7{$CEQo)dY z@x_u_i^S+Az&gLo+RaTo)Ui~SEgNU${%jxK3C*0QPa1i+d%RrUPEz!FI!>y4v~iin z;%U-2G(A5w5jI1Wd-TC*%k{bfxURWYM?cza3{<|1&DH^2z+J!%NGigx4Uu|es_1-# zU#3p@G61~g%6~TVCH3P-=&ATXW~QS6-<%HDE8x#p zg0@>#30%qYRR0T)BdC)7EVMfiV5rsJdg%fA{Y}kwWf1!DAO8xR2bi1nI6;&}wia%I zkOEsgKe^`zU53GS_h*&kO8Hw{p!UGU7<>CF59QV4t4&qYyl>yQHBhaYke{}DL_&<9YBKfXcjuxni5_gh1u zX2MxX?dy9WCA8V4_Q+I(M_Hs$LHcdk)3G73EMJ5@5pEnX1eU%W0Ob+2rCpMf8igp8 z2cGqF-mPwQeye1nNs@X~x9hKe&v#*N;yXy`L0D^>R$3lqR&U<+TWKq#0f9`g&Wquf zN8k=;EOic}CBn5alF#_U?5;PfXPu+b%)Gw5?T!7C1&B3UN98>hEs|?e32q9wC-{z2 zdjsv4r1rI%wac~M%zgQKx|dwu_hlGJuc!F~zpwyyu}hK~Udh(cOm)*VNECKXESE7n zW4N1#m16N?B(vkv7N3Qw1w0{A`$HIH8V3P!p}#Nlw&wPQ|HcD^b0o!o} zY6gkIzw1u{{w6oj)M)h9690&J;pNBH4iSGGBrYpqP2DbJtk%3(;S#Hlq#hLY6qfHF zxL5ZaEHXy1lpgFl{ce;Yk1Eh^$;q)GVkd<@pE~AiU?*hA$rF(`ZM$L?>@~lpBN`vnE7Foz z9dqheq6OxVqVmLKL?18ew|hCPSPX!s_I&6_*bLrS^C%u*$c}RBr8iS4`d?3pL5}#@ z>P*Z6p%qT#TFC9rbe}5b@8u-2>UZ)Vvll6>v=(?;KeSKyf$0NUF0wA$OC81~XHECB z$*OdPY&F{NKL&Tr1~*rp$6W2b&^>B=wAp74fcfasciu)@B+1E|~ zyRK(0>K$De{aZukxSyY#)GNdwk~p30OumOm(yYw^FQ$Q0scYY?+n6wRLTr^T2>>kW3FS2Tw4Uya}*Q3bLPV9>v1mOHrhOjMIoN5lVrX9$NwLB5|J-L zu0hl^NsOpxEqk*SJ*wwb)y9yoK85}cZT|YtYoT>jqmB#*C@Z?cwv&?qqq0Nh0uKat z@m7qB-~FQlpEbJxM1rIYt_3mwncV3NTTxNHt(ZL_MJX*!=(cGB;+w*dn}4bQy+*xH?o0v>iBJ% zp*c(J-qp79dE!|iU*-Aav0-c5oX8;TJXouI`3k;Chc!f+o8>$@_IE|=`@WKosM6M82q93zlI`>v~0 zIm5HwyD@`>>teU2bm#;=VW1WUjP^DQVB-8&p6~|r)#--UHOYx+PalDPamg|uIBCb| zA!x8&rd#aIpbK!I8Vyp#;49eWFTFX#~(fp~+Go|I6WAk9wQ( zCzSE<6JYcQd)gPpgrA?50Rr;ABh=z^F>WLM_CY_gO~`YQS=W zgR$C0*4_EO|LrwFvI>hzsNE0G)9=JiU1Xb2rvz$OHMs9EZXFJZ==lz`P70o!fr-F(of=X3)KJ$iEgKw-jE-v$WI?)q`){UssP2Yb7anj6wrhDjezsZ- zPkQK@MD)*$>{k4Yeb}_yANM&{;L!HpzZ{3IW@MeE5KYUJiurQpbf{ZT3rz~UD{F!% z8eUPS`cebU)^Btx)P^|Ruz7X#E6#wR{2R*Vvno0tRA$P6+P3efi#Da@!}c2p8oo23 z=$t2(JVZy@Gm_+jwfhgVNlfz#$qOB?{_GF%#Lzbcy&m5;wcM9WZIQaHWQwOSDZ{*BCUZu-uJ5N z30O{%KRlcS7!ms&4G@XAfOE0fo7QOCIFoclr)zY93GNEQyg`~Hozz1QA9e3eIs+g; z@c0dC2d^gY5bD9T_{s6mT>J-l6b4Ay40#1I+6psz2)YE*<}GtY{Wo7KvH7h8m<91$ z_?|*RPnZBSyb6g?;aEgQH!EUyBjMZGn8QS{l~i-wLU_c_xLa9zC$cBKkN3CMXBED6 zlhGzfn3naD9F}_LgqwDhox081Pqx&})yl8v>ozTotSw)gXYQ3&Y`sqRSDJ?60vR># z*t3aT4kN#-IccmjwQRV$^^>H5-(=Hu5J?-2(=D678E8`1$d!+sC`&FUlp{Xd5)#}k zocxRL>w^T0g0mui9Y`lg5b>N_aU^6<|BH?9n3dph_u0x{$ZzFNtM|2kx&hJ;Hvucz zt4@#9)33NyuQpyNmM1HrBI`t1;9*r}|5RbfNUpDHcsk3Tj@n^O$a1~$+b`96;wyg` z3d@%@KefdpO&K5c1etWc<4<2zKBb8xE9? zK*_7*Zdv;ltAcbMktwZw#>%?FgWvZ)k53q7u1a+d&iQom8mR$OxV(`;tqQ|1wp7x~ z)R22Y7H?Izj-Mp;Ofhrzok-JEfR>xtuh;?yviSht_&ib`paG_?7e|Ek=ck-=KA17c zcvu(qn=>d%+(y*N4?raA^+fFgp(WF%dNf}aI_DNR*mD94{z8f7Ap(YLjOIx%CaDU5 z$rR|%mwd$`4zTzWQmJjTt2r%>wF|2gcW?HIx+>PB`07BN4MqC4qRj=x(t7J52Ybeq zAN+GHHUd;~ov^8!OduKcOgX+7GPVHrOb5eJ(9e+} zZo$=3*M~=l6bhBI@nr7dI;Qw#t{ZlJfcE z6f{5A)S^Y+!&TINEp)x0-Fvs09cnNh_Bdz%3)0Mcl+C-T$`2;?9uIQ#%_^(!0jjMre|9a_D9n?sHaEQg;)+c!n$39Q6a8% zCwrTaCZESAK(xyAC_T10kh1)1B(YPJ`a`YoUL;F}93nW;X__iBkCZE&av(3A0EC|- zbn#vnh?In+*Cw+P+m?VtA+oz{+n8;1Yy64lJVot>VS8*G%79w7yxA^N1#+eRzd0Si_gu^>eQ1Wl821vBu+X69yZ{eg-?%v>i18-lqb%6 zZ8`6+qEjiNxFjh#`1ZWrzkPMA7&o?_mHesv=HdL?<6$Yoy3w(q3XS znkNz~^b<-PLD%0uM<~MV)W5sUPZ3O$vwU48bXgNaKY*h#NRh&e#4`L>Up)kmc;dhf zE-2X*O&PvFBCX_AMqy<@8h!O3O)E;pH%Mvm`1CImJ%vzuL1rW*hlvgZiZlih5lKs< z67g8}M)?3bdw0HkODaGI_lcjEX%LBak@3Pe4)>#mUyOyVtruMIPyY<9jh0(`JF8!$q#J(K zZX89WZ3ULEa=rCTK}(6&v9ms*F4J>`kAG6!;bP+w5v@CEwvZ7l2+)FY#6oNUl<&Yg z-?@)ctRoc{Ep++RV(|0v{+zz-Rk!iYpffdbJLtH$-NGFny{kx;&h&KZOxvYOtv))t zVg_lv2UFj8R!66weQByP8>v!0WH6bs;^%|iq?#PmN%WL;T86=~o?az|2WdtTD}1Ab z@L`=q--$(Ma?hZap>4wCwOcLn91kL6dh?;m7$Y@_nF_FTkT z4vCCie$jI}_$kBB`;=DAew;O|I^^i6|m!!JyWalBc&rs@(gdN z6`c6B1uFRC?$QADgCuH6n&eJbamffv(tUrvwzvFM87p!brviaC_l0Fn@AGz6*Uw*R zmt2TZY#>`f#C97Z9QP*c3hX=-zNg*%2JyuU;o6&naRQy*;d&|(4|OmT({&6~j#z+0 z=-(iSwn6;JhhINL{VS?Ca`8Yqbt{wVlpe2-SNkU|@lRg7v9i(zX&Mqx25=>?l*Ep( zQt1vRqK3tiQrlAt35ik^5_}_b3Pj&Q+88S7rZbsHQuyYlX+^;wu7;u~!xaTy8PA^T z_L&i!a~#x23zglF4^ex(XZ%|4*zn$8!oE00V#5p59Y7m zcOW&C>WYny1TscuLoq;X_v_0<)-R8fRR-gaZNpNG%vktECLoQs#&eEK>TMX|;Qt;; zO1d=^GSo8tCj-VY;7D7c{DOcZk@=Dm7eT>d&pfF9GTa)+TL-89(yBCM(67$6J`#-m zB2T|TTWM8<4 zM1cxEQxww19Gw$i_k{n0nEo|v!bDa7QOTQk2OhWf?ZPlY|591Pg z*+-Nx(933Dm#yZ0vLf*_F{zgRTcnXU>5ypAGesSTyaGY>BkI6~jInO0MxUYf6wf`o5``bq(pdFA+|tVLPBOcno}2-q)? z$zas7jU$mceZ=lTJTzFy&$!sf&4omv>=$f&&{aqy3N2H*McRct_om1hex9cVmd{Ch zWBCt3K)2#J%v?x%fh&GXLjeG#i77|IRK#JgjTifQWg75FoG9W>`)P;*qtFzpqKAOQ zF5Wn?HVynVqtu}}N8?AnR)cSeOLjy^b&WW7x$ual&eSxAt|qOaAD@Z$-}`=t-YsZB z*ra*2qxMpMFFfW|$beIQx5OswVSh_!y=N~)?4C>H>&rs)M}8;EJ=n}_)x>#B&+!yp z)C_jB#~EqBY#*6PDwF$IWY5s~)bqn1g88RS^?GUGH#;!;wVTWz&X=3i186K+(IBEE zjUz~XI-{`DHylcv4I=b$!OiF@U}((JgUrU9Xi*(4Ncy8BNb{3>`db47R8P5U^zDPU zYTN(YOX^wRK8#b03eNznWB3nlrN<5YmiCH zI{b$Rh_$FjwJrnhK3rA>AAyESU*}ySt;ftXE4bG#YC(%|9bT^{1Rs&d#lcZ zOj2d&w@u_S1kl&Q3P)~i9W>jZw?9vTnT4$X0OFNtufS@-J9mj)QXxVaWIv*Z<9_6pB=zHK=t*6`y&CP2%}iS7ri$`vRLX0 z_;|>Or$`v|0#XZun9}@VA@fa7?5`&7HMQIngcB4+b#gi>BUagtuR{tHztzy)k7(T_4eyYRh{$p0Y%^P323 z+B!1VQ1Cte*KtSnOC@{NTEh4ihehS|emJ3ky+0I}13r;GCCIV3Q2b}ZiF}>l8$A(G zQNS*RT$EDZE^j}XARUo-rDkm9RbFRl{AOX8z=rKmoztjCj9FQnIi{Ni(<2S;{qq0)$uB#$#=?+J+N zMhad~0f~8cif+9PjgBD}YJ8>_AcCsrdDQArAo?yTbO}#U!rGS4=g?(PX-Nq^h>iyF zphSQQregiZg3RJ6r1&GlZt}TN#;@G>3GCai zG}E~wUhV^_w>Mw$D3>=&85edjA>o}=Da_`KNc4uN5{+%R#5zMh1i$#;{`Aw!O}=j= z-PiY6qKwsWH_zr3W*Zk;SXf`;WJK^d6@;+k5y#D-B}Y~Y9Yfh7wv*)*$t+F+^2cNH z;@a@n$fbYFbkR`&t8}5C&idAAEfD=1`x)Hq46~1=;4Fz3O~`>rO$PY+bMnBoRKKBV z(k}<5Rt%7QI{+^o|KG~csbZl12$HH1{}?~8mDZe#ZHmf5W{IW!Us8rapK~P}Y9nqG z(cyI3dUby>k>1qL_{%Px-z2ZXV8*}Iks>aeZMy0`U9jOz%ugMv04py!aa{9;Fgr-i zhu&}3JrWZ@OiO&+e$sKemlyZV1M0Vc%6ER=-L#ty``Etfx0_i791csW>1%zET7e=a zP=B!17CpBoD?b)As`Brhj!`SJ7$I&LLc&6?nr0yOrO3<11PM0GE(L1fD3hUun@Xl4}E0`T2x_@88fMm#X^6T$Nm) z-W&A^Lw zdffT+c=}1d{e@HDM6#hbb0!W{Kb%5auDod@az_CDFs6Q9m!+&3s3?1;8l?nX8y#^C zr_!5pt;q;>t1MU-W|6~hP8>c~kLI!-e4u#W0QCv^?$Gfte_ku+ogRmz>4{G5TXlt7 zeG-5D|5*U@c8-2W^$R_K_SbB=BC_KwcVy~rA9IP>Ky>v*koX&M(?Mi(3S*Wp z(>EAYw_4t7-RQl5{ECR2&&K@Wt~-o6w01&Vgo4EH?Rq*kJ{rkOMI8&Lpgf z6w=(FQhO@rA=>%DMYu=DfR$`9>b*F)e!J*XURidNOjB%%@{pHg*9p-Ead-2b7`<0^ zHHP;~WPB!h;K)WIY(k49m?-_yOpnA{*N*VN8;&VLa_7YO^V<14JMMBl{aZ#5Dza@l zFhGjFofFFxO;cHst#2PLg)2tre}5*Mo)tn}fj{*`jI4lDR?CF-?)eB?Ncl5t@(`t5 zTy~OCR_Bc|w71U|XVIoMfp)R_8ax4D>=lm-rxJmKY)$!}lZdZZeD9EKOM#;NUCAVSY|+K=p7buj zT~t;-<9_F`vNwvO4Uae`NfR;oY&*iy?Wz6dTkFOw-*}~p_yg}(pi%AD#{>porgdXo zS_$<_O3m&{eQJG`%aes_Y79X7l6G_lWDln|<^Y@`fwJ)a>z5UBC_W!dU)JnAA32D7 z?jH05Aco+72Wv=X{wfIvzg$Cau+2C)-A*=wv-?H<&u0Z=yvF#+PwMt*DDNdnb)=T1 zWDX)vsU-K_(R5P5x8NR|jJvPz`L!5OHrt`0dbb6@evI~}^1@*CJsc!6c=+tvMA(5L z%&}ivi7?dW{i9~)jw6)|Qr+EwieBWW<(R>{rQcOX*PcqsfI`tvopl z6STFbl2PxC@!^;IFO_4GVo^JEK8H#l*o)zO)1;T@WK3LLsdcd=V27JX;M7;aL&+Z>+pL`m27b_ z(d)lvzYyNn=k>dA>+o>$7r+E?m0Mt*wxZy>dt+}15P4Bl0O!~dC@C!Yzf?8#BXp)m zx)ZT)rB}@271HoL^A5pl%?4xnEsicGBm^XM6lL24bPZ^~BZhSDU`6@2 z?iZvhzV0Ox-5qRnf&(oP@3%3V_AUW1b;<;Hg+_%#0k{LutC=KfXmUBl{}5$|p2fKX zJl}7?)Srn0Y$E_v%>PD^EX?ycU=RAG0XyMzm29IIhCyR`hL``J@kAI3RWDF#l2QXD zde0lQXMeBiO1Mno1gV|zaBYkZ{ z6Xu>vDb}Fd7>AjH7)%{SmcANvELzRaX5Y)dXCk$&#|BhWEnRc|I*sD!;55=!vBY*$toR$$bG~mfkaQmlP4=^9x zWPyA)R!@3I7rh^GnX!V%DM>T~iumb_Q9U^~W$zwtpaRfWJ#_!)d(f#0qTXi8gq*KI z@imqscu{Onv18lu9vIiH$P1y0Z1+Gj?3gfCcRmo1T0_MZ^*+3?#@CvouYzRuw4oZS zbjgd&d=J`b$=2w zq${P}P|WE1*@%qtEJda8>kPSAjJN{h+Ok6AwrL{u#PaonZ<71y=9`g`HywpDU+ht< zeCavo5{k1W?oXwf4oXU;I{gQi>QsH_)={FYTf)--5kUp4DB+Eny)TghYGecuf0`b! z&DD?Ne@}JEJap#MM*WaOfa;6)r}OHuonVo<7~^M@`0is8U8= z-(VQ-cyTBEfTgz0QlnRPbJ(lTWcz(V33Irl!JDqjcV#8d#BOsiJA1!j^JJU*<+JC)-AWx36-TS-t$E(^_jryKvsKgjHU=q-*iMZAeS^D zk_GDY$j?|5=X%ibXDS62%@b@`4WuKw0_HWxLA%Gd9X-?pyIVh*)q1)DQS1(ASauPg zbVuDADA(6nJ5UO{p8j4R+yiHoNn`4) zYuQghyd)tx@^L~4A|?SMg~dpdP)*sn!fS#x6-J8S+8)Z_uc$>Ht6q#%&bq*g9QAuG z=E=rY0rDqHv_SgX+h7M2w|nEaz~4c=APEchyB~D&TmSw5cy>6z|N8P52+B}OC0m}j z8ePkdzqnr|0GIF;!d&^4X?%_+Sfi$Ryn+GeP*JX=!jd!W3CM$%S8S-qBHnEFIf&Jv zb6TDy^i7g>4nFH zJTXa;pu_`(0h72x6W-z7^M>`36byA_(l>r@>d0g&I?Yv00euMh(%IK9(>nWtpn6SW;--@vC@g<%zexYY|c6U8) z)f-(R!>3}IP$_VtfY+SEBu(4RaqIeNFRqGKtONsI_zlFCW9bH0Q|pYp*ZC6=!sh_< zpTHU(a9uVwM&y8@Eh8Gk02$e`tIfEQ6OJ$L+p)^2g0z5>HaON?D#FC>AtV2vHL1@4 zV`<-hG?5l4Z~#uB8-ZhWBsc8gx~sVuVP}Mq0~!XNSa_%kWCNTGTHUb=v;A9OKbF2^ z`~6ELmCz3FX(Nv9sR{yi)(ZV8l;i7)ZL2OTD4!r@6UQ?a#m%4TA=|g(4%e=jY-`Zm~k(}8apjZspWNI6%wfSx1n!j5GT%zod_-^cy> zPj63dL9CJw8sN;_w?st9s78jjbv-utQKdjhTLj^z&*Q*a-j*DzTS(`~V+`%o$XC5p z!ym#81mqio_f0QIIlG>5D=Mh|FKEAw14Bu|uwM}a`Jxwk@z2qZ#bXXTQ~Iy4!}`OW z38W)Sq3U|T3k~BJ0a?T~>8_3EXU`?H*lqT+w1qn4*+BiufLX4-$vH4O#JoIQ;CMkL z`Zwv@AP2F;8p3J=Pxo=yG6|-N*ws%=?J_xO$84#FPahn_1#JO~`w-Y2MAcClp7yH* z;UP*kV>(Ju^H?h00(Odx&A*~*sAoc>dUa+5`yD2*z2y-XiMpsT6Npa*oTvr#Jf}MX zn-c9A_dWPRUn;`toHK6~2psLeJHd^*(~5jjh*Rjy0PYIT`$|#?zdm|p-@Khv zG z&e>HA7l#$*S(eUfuFu+j#(DqLh>g|C=mNPU@JHl607~R^aT1oSywAYW&1wR+(Z+V7 z#GpR1tE9Ao9Iy>SF3O|9%Rmr?cd$_D@HEY+B~qdhqiKe4GiidsIv#z2)zBM+tj-Y- z5QPGP0rP)pbh$S;!5V1zOg5lTA9|jnjU4(uBT2%`)b~8M9SCK};z?=guVaI1eWt`q zF!G04q@ULV)S)noBprr2o4L2Vb`ajoz6+HTrrqS41657;)$y8*ea|ucQ*oKPZHkSy z&;RZ4uc)t$E%I{x=u{<9J-=^-wadm)S>qo9>5bY%9mof2W4jP?Npr{zgaHM~-TdZFSnM9KSRYm%)5+??!pXTPl;>x|ibxN;xP{`q^4 zTUoH`_hA#T^f@wJje01w?ed$cf>X`d;+vsZaUc>5Yem~~!J#LBCy}gH@}5yK*m+o_mxhAxqMp|ChAV3E#6*XGWJ^M3aZK_hjoNwt zt_ZQv#-raZLy<6dw|gXi(%475; zLP1!^L$ex6e(%hdc2pu4$x}0Mr;MpJ7-6io4r1bF>CWVy#rB`M1!ey zH4WN7VZ=Hs%|MG01>(}K^;w|d;gWaTS05{jpRGPV)%uDm&iHvkPcZBvky=MCqRRNL z*r90Q{vgN6`NgbVRYt574mEi7)dggb4?7Scs;v3&iNJ*9udDzWaJ)_=doCrNLt-1# zpQ;a#9}ZaeDSBJhGSXVcfBu#?6@sy=BF)m6ewCqTbqIaDC@^91GAWVUeQ3h+xAIGt z)_0z_oV(D+PH{GIi(z{UH!0tf&eI)a_+P$WID03_S8=eH$h%tFG(uZ!=lOz%ZS9Lh zUhUUh(Rgbg`J>P2(>l;0YR5|x;s}mAT_q|(34Ni2j2&Phz62Q-^FfjLqud5TXsuJk1NFzgF3bXJb%Od2tZu~ zh@8J+$scE@TCduQS;mR_f);shz+^eF4m5&v3^{t#0_)43e+f4F>Y+zD4K>Ge_L7>f z$|r8BWpcW{pcOH*NEWxO(#vozeYa%P8PSxNGpj0geY0wFj+Y+=wu)P_sEci~h_ls; ziG0;x<9QL*pVa=!w;33~85}CAtX;Z*wnOjmyYp`TK45?OrQE*Vp8m#31Tc*1(H!}R zQF>92Ju+IV>*dm_$t%-}s`JUIW59Q~wRe0@7~LP@1g}iL`($8d!sDK#k5Z<}2k~J( zwcfL#*xls4BTuDOOQu27*537ZR(;}$jnAK5*&pzqoeSNrttMFpk?QmfJzYSTqC4_z zzqbbFc2^+tpspdn_wRrsv$2OAvU@`LfS2|WuG^7ID+dy8kqj48n^%FfxV1B0+JrK9 zqs9E9LD^5*nx4hjvY;G6l(137 zsa`fL2X9cKU=(rE?jcv-0vtu6=GEfQp$&JXRbyq(XYkH~Uarj#53cO`1865m*+xDe zx!tSg4kt*rL_b_HO~H74CwpzM#Sqnnr3KbWg0Y1{;4^z-E-s}BJ4}3U_dM=^^$0v^ zOoxTKT0e!uOw$z&$*m9lTqz+fy0pj_5l8;grdFAbk$!|Q{AnytYh!KnVLT(Wtq%52 zX@l{bX1EL0D_Q%%yrECot1opswop2)c$m(eT08&iTCbP?)U(0W&cHXgxq1cum#CWg z4X`CeSV{G86?-~xEJ9Y+ypvUfEFB@0dPQR*cFc@|uF_WCXjs(eK3p5~swhE6zwp@^ zV+Mu}m?~3UbdTRcsqM)CS1jbTaZxpep9$geuvwH2M6N=lyk9>>@_195QTJ!stoB6V zylOem*K_{?ntU4=+rVX}+By+<-f~@Ynw@nd*GE7B;FOdnD<>w*JyfB8Jkos~_zq_o zJQ<<)cpATl1V#{zuSPwcM?i$Xp_1WYk{w-pS7%EYEo2=K_efq4pW)YGc@xy(eI-Rh z{=M1!2e!$&503dK44% zt=5IKFB}9A=V1c-E|@d~R#@V{wm$9{eb6TM=GQ2%(~8`s5gBCs13@u>vJP1}VuKIs zBX(E)K0FZb^S(Zr(RrAU!kZo^B*425pZKiya4@n;nYH-NT#RNCh{;R{I8Vrh*hicx zozZb6Pn`nIs>|8x3Vv{uHew{OmfN}_xCZ;UH*&@TW1F!(yaxvd{BD{8bR})WlYs^% zHeuxH2u*2Hr#a{M?Nr3WPn=V{fpE&3N=q z-)QHPWFOAKSh>J>1q15Pp2FQ-4!Mn>XDk;Pi^8|mt7+vgOxWr07b=y&M3D%4 z{Y8+b*l@VJ-Nw<}B(3#Y(9F{-51ajcY zwY9gMcn_1O;K2CIdS(xSA!ow-o30KAQOyLs>)}!BMvef@n|l<&<_pRrOwioTS`Tpx zN}%u|de36^P@(}c+J{4k(~j-rS^o7DJBsq5kZEa&>o2!&L`(H_w5@Li7qAn!LAsX! zaujW*=j+coFzF=30cdWz)aHjiMdJfK_DeIkuih`=brA?MRiBoNMA)LSnhLNfx!j@u z=yf1=e(rj;@_6Gm@<1eG$ofW!X^(BwG#CtUSyL;JyX(x)x+tbemV7Gp=!MtohTjWR z??4pgd@8?S+@jCsYKf`kLhHnBt{rqDE7Tr(w6Vx))*X>IFl;^icz$zcN_<8`T&ECn zq-un*-XC$H&a<`3kTWxObkL;hBI)ifc=A{b3yR9HGIM%5O#+kQoGJGMzH*Z#5&LQh zF>TjY^#HE{GL%o(=Uqj9m|}0`F*Xh4IGEAQz7w0weVcp8rLSTP#0ShordYvKIyUVi z8dpwcrhw#SoPs1z1LiXaVJ+;)3ZE55z|U6B2#dGq145FkT?0RRYq(wT>Z_YNg%w0Q zK62fG5|=~zDoBp5Ro=-p=RBIcUJjwfH}*UT3OoLdlnE1FdG~RAKHFSSwry|sSYX-( zp(6gKC6@;hS`u+&EGYV-e33~uWFJ{Rd$MT#e;IK0K3)>GMO)kok=O_bvY*4NH{iSa_W5{f;p44vIu&c!wAZ58sI*(8Sl~&?HRZ?_6tN1Rkc;+>E3e9)+Co0j%!mFL9v_m6I~Kx zaIjbrCxJ%VbOp$GN6g2fT&BPvO0wje5GH}?DMcE03+S7&9oPOeDI)pxyT=S-Zm^hCH#vcy**PIsq6eAUh;B>%)`tNBEqp`(QMbmFQSAZ1*o~;u#;|RU%5&ysFrCrr zZZIUM+FeK55FDml*F!Jzx+gni@kQ?Q?!He*cKYB5kRxbDi~@4xX0*ztVp_+o4!~@< zjD5^g3xg&JBX|S(gF5cFHNwnZm`mPK z*|Hc%6TL3&4|MYRn17JmFOHx{G@Lsr1JLb$&^kp+4ItJFq}~fl&J>j=E&-^nbkm4XcZ&`MpIs@{9JrQ`x#d;h zx)AlH6q;>35~|Ui{+Gz=pAs~dL0D|lV$W|iMGn%myFzIUdcvTeVtoDmRE?;?WRT_Z zKrQID$lh<2M)F8_+K?sF^YzAW_qq@3hd=5csipN?`m;h+=%<;`zgT#-tS*Kl;apjK z1$DzR82%@KBOu3*#@)qRkNe3%*!@h3#;9&@!ulIwmL#MIYIz6_==!Pqc4b2BdH@(2 z1xyC^W145ze|<~vvnUdYLAw6skD)M^1k#*H`7A0lh%Ut1(P+3aY>~Mng#Dgj$-&5w z=j^L(Bjzrqm5N2<6q~q($=cl56ba|lHxia{=4fbmMl^8}??cUdS6D93qq~QJqIzSj z+S`M=Iw4cg&sB#K`Qi>TjK#us&~0p3iS#~7k30M$~)&284e6OON>O^ z_Du{-d~>FJrY4cu70|+C!75-kyo$Aa^y#k+u84w@5pEMAs+W3zShaV>yllq zt=oS9mP;i^=Yy8a9ons(#QogfV+uf&wm0cyMsCYbL|jLV@e)2@#%CfikivMED|=~h z$PFE(pZiz!nHcl1O({ zY8hZJo_+G}(YsSzpdwKG_2bLid0H|j61gl2%!W=c^1_!uW5WC9mwGk#@n&H6DE<`q zds!QpfN~>pideLN-O4N^OgaV}XW6t5k;$Uu0C7ObIXhcYsUt zkZ>DdBlr=VRee`jwTfuip-9t#X}y?bps$XFDC{W2TEXRwb;UpMCI6qjOd{)19)BN zU~7@}^)=<{FY)21LNjdwIU>vQ(rzTda?1XTw^EG1ZOjZV)vL6{XM=IyOH$hK<#l_q4cL|>2_l4pN_ z<7DjcLYDrub~@tQW^oglRK!TtuEi+>qMc}jb8{W|}phRZ`4gzd#m7^6`O<8R@#Gk#KnRAAmHL1Mvmoo{1B(K0^s* zy;;ewgUHcutvBfA##O6pz#_5rRk&(}9i<=<#^ zNXm+~McC#PyvPBJ_7bEA;a_gGF4^xdxk}$KrZ23@k?Itr=^ZU~pyCi<^kbkc>JIN5 z68#vwIsLNN5rv8o%or`q07_;}shGB*aYvv*YT0}US z_r&&0IW3~A`{uRxCucUX^sIQEZQvX8;%*RG1|z>uZ^0P=F?6476^R9A9Rcmlh_2Kp zg{Yy-IZuq&vz4F`9D=AETVTxyE(x%e8n~dc`y58hl9;lK)u) zcFa%`_dfp&vtQLYC}dmDgg}C#hg*gdabHpi`4{AR&h(WrCnh=l3NH5;PZ^p)pIiM+h4Ml~OxUI$8BeWEQ{ zs>P=p<|E8CJi!BL5BtSW0Kq-$C4%Ra9$FA>PcFvLUC#7if@!?F?>U=fyJ(h&9!u1)%Z;PRAPtw}#P>*HC zx^AoJ(*PmZ%>Op>Z5i5HH4u9K{Bd@pNE`yNtlZsz0}d(*TC~6=Akegc_V|&QZ>B%! z$A~n8s_Q6Bu|LioPS=nX?;d&G%)feG*Q=Hfqgc2)3TutBZ$Gk_r&i`*%&3?rnxMp& zfG_ca93|BiG4&`fDA7VfN5VyU=S_CF5B4-}JIj~hTR+4v6JuIe6tR!&@Ysi@lBZAB*Ny`ukvw?R$m+YWfRFY9N`B1Y>R z7T&bnAHhvr_};PsttxHVwTW{%@VfY$a3+a-`6J@Yk${^H{%G!OTNCZ@Dc!+$OW1QW zu{}#RSrD^C)l8yglSN~sk>s*0P;lG4?wR&i+MZ&%$eEY~C57C5bn-onWU-Uypv3whdCst zc5VGSIreJw7RB+5rlYG;d2n(f)$-|6!!bC7^Yt8uw#1X_BODyvf!ze`AWZrOF0#2$ zUykKiA{4>?ui}DR%$kv+s$-})h5+%)stn_%^rH&}!dn^QS6*yqc zp-h{4W&%LHEqs7$!LZ=~KAc(@EPN+#xa;f9#r8LbUlpDBIDm%*)mO^)M&Kh@inLzG z@57AXzGm#SGDIu*W)y=Wi{Ml0^oIeR65k|)}ty-J!Xu2I{P`ueJA zlet*2-015NL%3JTVGW2tAmyBIU>ouKE#$r0Ej`b`5%{$2e8V`g({=T zkg6e2BfR2TV=k_%F)PmGC3j$5@)0hZ`5pAI zOo98oP&OvLZQB&8S!z#e9;qb*-SUxb0G<%Iz4-Tu7l4x>UC~AQc0}5GYHg?GjCVVc0rn>_iWe>(;j{H0tBuTd# zL3?!ef!3Ci$YQrPQ|0S`yOxeWPaajhLUaRF(eg(2x8>7EL||0g98d&SZ=u1RJxcUT z6|Klu${lT+oz+MWv0eUo#Y=;_;jZr+(3uPKC1`^LgHE8hfx=}@1UGh!_+hB(Ls|Q_ zCpe~9$XM;D?V+nstYw05Hdh|SVs<&YE<8`J^TWZHF3vzvkP06IZNDR?pdfrRTn3DZ zql!csx#jLRb%<_B$!F+vY63c8G2}7y?g8Nb;vfdjw-v#n9{%Ko!BjdK>992v^JP5R zM6t4Qmp1@fDL96C5VQwXqobOkqD*BbtrzM%9WfOc?61#7ST_I-`FGPPgtg6s;z85Dm4HfC7 za*nJtxh#8ecN*wCCaH*n_A`T&Lf{z27g*VNEXOr5ErC1^#pDZ0&#@*$E*O^Z!X!E~ zB6XZp1ms?HeT>=+=3K>-+=^tydN|PTf%CJ*eeA+nhxADmgHJuSa+zxo0XN)SqnqVK z;v)IeHya5Z_fWGmHR!qqzRru|hzPT_VmcH-Ro^3{eJvw-UpA$VmkNR$bSQaP z6hZ2$mH}4|h?ZTM%N9u^L2LF9HkX7!!5hl*{#R-m&9CDaV zU?9jpXXJ=mzPS0lkdZ>_C?`YdMo0{r-tN78+C%7D(bCS2?G!Fv*&)p-GbAl1%W4+B zf*>EdG(wCBMl`m3|Jjxz;Brrxpm>hz%f*z16gU?+?he@vvfkYNO)ZqiZ9e|(CTtmmrhrZ*F!_A9~ z32<1B&d)@>6>VnhcXjmZ6)^VAKb;V_L_&I-sp?Oim{s&FL>V;U$ciCFjCz8)= z_CAocS4$e)>dcDoGFP@gnrc;#-W`uAa;1yae#S@05!Pd*LX$&;gBwD@^gna2pd+}| z82|9s#$AxFxIR_)8AjH5k-J!c`dz=)))IwEM}HFQXMB`o)o%?TUS>7B4@0Ngq`dII zp!=@TVo*buMK7~L;ayrg%^!jAM&pi(e*ic=6KGdAIuj@A=YJIjfBl@pA~`J(f0tgX zi&m6?l*s%c2mLWU208Y=Y~VV7;u`BN-E z=Y^q6vx&YQY&g9$^o2%1LBKCL5lnTX3`@vj^6%g&g5o7U!S{Z2{&+bPfRJN*llZ%N z^=Wlo2aJsD3A|tC%8_L^*GVI^&{IX;VvBD0Cqj`;tl)?rGujm|Ol|}5sqmJ`YJc(q zP%$4?hHtv%b{9pyf$GxhfDDc!0Uim~KVlp^#d8>~ZTR^g}Z`*N2uF@(lTt7wZoRhXIP4msqIYRx?ThwtJhF%6PGqXq6z{cg1ZimJ$u=4w|Vks=*8(f2xk_So%9+Z(t``K`Y01u zODa-tY-h^SYmqs)16ertlZ%v6AsChF=0(i@qzuayrkyC>CGhCkX~4T2#L>Wq9bx)} zdW1$Bkw?U??*jfIc7!6|n@v_YuC@0JpeCS=h|q#iolSRb9qJ$m2?ONJ@l#eA7UFOq!Up~@)5soH^MuuuHA z%WM4ptI~`b0fDjayB3CWVri?4>U>BFkgl94wnzyjZu-=7S1}$5wg9PXH+P`dIcn%S zunQ{;Eb;EcPKGa=4bXSoBh%}L=V~D8RP|7lL}LL?H@~)T446a}Mca{muSCG*_$C;a z4XFM>VL|ymO489Zw`c2uk7kHs<~UKT%Tp~}WuX@sNmW%`$C?+$$}MD}bLW7r2y%9d?B@knAFPaIRD7uk1F0%J6abneRScG~% ze6sy$yh7UCMVU>aV}E^Bw15B*U!OR81Vd>5#=J) zhz?or-F*-7a-4Is${?Q7At|Gm^1Wvkb4DXRx0bg)IiL1m0@dE$Lq`V=15Df}ozJ3l zt>{$wy*Hd3A`onI!JQec=ayNn5&vX$K)@60yMF(e4J8lK*2GG^Vj>n@H<4PqtX#Gt z{rF&Zgl-VSppO?~)*8Xal*bj8`I%$Wjr&qy&_Ybp?ZETN*4U7^c#BV>)`H1! z8v#$Jq;j!xo$LEoO~z)-KNU{j83A!QPp>4oVJyjQSV~K4%}Cr~jyJT*^hXj&Nt_EC zmoYUEF+lvxnA>RY%>AC|{?0u0$0Yz{Iz_2qIc_nR9frqAJq|d`u~#A0ZhAnU8y>Dz zCj!Yb#L4B z771W4W*UBA&A^OLMQSPUiM(sh@iS4ZlIY(godST&qJ(l;xF}cV7bi2uw=50gWH zzboah-~ZD<`0qFU_jkX02*O-&TQT%M*YWTFMO@Q&(CHb7`e26W2mW8LNeH;_f@ntF z|E;?I`_45)=jLS^2`E$)cQp@7M&ECFNkiu#}YMSk}q-{QBRuS3r$e2+`hJ97VBhJFF(Pb~D_ZfD7UL zWK42JTggE0uS;(DF@*r(?>%@yhhvgX^uM(P0pQZO4IPRl#Dwm^`prs)Q4sI?ZB){G z^^m`NlRz~DnnOf@tbMz+rpKQzUS=m&dOd&2gn7+3D*CMF&d+`tFO>W&Rm%wgCJ6t*C-^KDj zDB{&Kr|3A!v{}O1$BaxgOE1J49Fh; z@9xVZq>36YN$mfg8L_WWpnH?2zK+#^JQD(Rwn61B7Xu$b^)&IOS|mgDnBE^pk||6w zBk}lUzmWUknj=8=6WHUJCKBT7bz_^cz}sybHscP&fXe)uZJDMmYbH0q$6+uT(r6kr#N< z%fP?7;tgt9i>OMP@YsqK_H*aA^B9^K2DP>cWCPwHjQSS*k#Paabr6sYLJa_WaA_a3 zVf62zRe`pf0Pej9tcH}O^^mUxgPLn<{v2TZ541LBQvj8Q-qX&Obz|ZM#u|Zp}NRM?{ z%@!~Yvq5Wg8F1|h-u-U0ZV<&*MRf7P>2bZl*JYOZ0kFu8q%8y~9MSH@k^{P2OabfR z2OK3}DNF7EC*tK9W1rd@crhq6o=bHh$Zx;Muc40sKg4yYv^%c!?ZGx6F+Oa!oh;BW zg;y*yKdS{qDwG@F^xLCE^ z+?IM`$0*_^7tt)_1-$Cwu2_0m830OsV+gZxhl*0WOlmB@SERv0SgAH}>DwS&PZ6X< z0>B!K&=dR$zS1(-7Y{MEz@pCAGg-E_doNFh1$KuU9NazCtDI&&^PWI}OF9Pr zq3w$5LY)IDnG#fi3w(d%_K?E2GzjXvIsor05c-vI-rfE0N5cYRuwqs@dLzQ*r?bF+ zX0*aomN=dpuxEWK9Sk!L-&U)`D5La*zA7C9|A7BxNYoqq>0b(r{7+U&4PtgJF z)I_o|uDi`#D%YK#r}I-^+PWAz)J_fUO+8cQvRol|05l_%iMxmf1o3>u{deHxcED!p zP1bK1Q=ml87kPFDveXc8-0(cGn~v=KLtFS0fzc7(9e%EnMgPg366k&9Cy3R=O*LiY;^3ZNa;(NlSD50J9mURygb)lK02_5epY zE<$*m*Z$gezQ$(5klK#T)6Dcb;clW$lZx<8MP~;#M@g(i;1mg&Rv=I>4F>XY7DL*( zoA+g$Sx2#aDktVbDe<)DtXCQ?yYo|=WqmQ5p(Xc2Sp$z;)AA6GKSF{B026ysPNvkd=G0B93w;F6 zGTPL--c!&#u(vt**3BLaI@D=-)?0KZr&WYyjxA5&w8u)hAfRB^xGK=WJ9_*vmod_)?BfStigl}BO^>QX8{QW#9*}$&c(t2B5 z`ERn+Er%3wKad<{uQ8MkQzF+iLDGo(XT`njb>bMlA2Gg5UGZOhZf?XliApmd1(+Ea zHiYkI_sobc9mk_j5@AvD_pGI+P3 zxL-du-r$g3|HmuH1wpVAo!zv3eU3ScYEMIJHmKY5C$*6}AvW26+6O~wxiHYUVq$1% zP-EMUdHo=)MHySPKbK0j#`Ggi2l4Rqfo|>h*MoW8%p5#DN6u}MUadHMJFs=j^Tt(% zXZ+MQ*TR`9Y-k@7 z#?@8F`tQz&k7;WjY>wwCHH(l+r`vyfr8`giVp}d?3IKd;(jmo8H=B#IAk&2-j9nL4bUG>n!))UMVc8 zv?T6SLulQCqcztfDPFhivNBis)Y4V7aUuTDl5#de18+zhbV(^w1GJNJcI=DP>G(2+xbKl4u_7O?cxzmf; zSi^ERY44_pojLRb3>cPbnA79>yk1TI%LQf!p7g5Y026owK$W#nSD;d z+063Ud2B>?9DY_cAG_ONpwGlrE3USsxW8+^8R3%27cqoz%KPiZjwlto8`!j|8QQiK z{`5VIOJ%?5Xv&*Npyc6+dc~%XNV5cd9g?mhX!_cG&YrT0fB4P%`B&&+ZZu90o0f2j z73E8AU*dA{b>`Kk5rDfeuyXJmCC_jyu$&xAc|8JU17kC2rey;We@k&krjgGUddR@3 zK@i8YT&&93#~ES?Hg~+1U>MhNLrD&M#c=~m!o{Soe**33fai;`%=OKQp>1;50T`DP zQ7Sy!?Cxx!x(^OW0mVIVlw;pIa%m`(eEB2w{cG9vVI&R%PDy~T`sw8Aom|aTTM$bmC%&GF@r}6kjZ;-bG>Hq>bYL4fYshK`H zPL_9b0Y|{0C;FCLhH#Hn8y-ttVKoj~e7D4%-yo&ptbVbwA_fDJZ0r&7qsZDz&~CvQ zd32V>$AWtJws*AA@iTW<^_$!%o;74^=@j9H(>7OmRGZ!yTH0Td7kIzZc#L%9kG{Qa zGHh7?GBkeZya7cLs4eGj7T*QjcH*DlWIS#hd0S=>t(lROqF3&->a^n6f)J>g_QleZ zbq^oSeqP5)vrEH3m{Yg`-=9u^`_LlHavi~lZapE8DB>Zw(};Wr7B7AEI?ZQtV!=&*BrdQ{4=F$EZ^!e}+>ZxSFu5|-sls_(e!y>RWUdJUVViJ(C zIosj5I0k2^9exlm#5sWQLKp!-js)cIs0)gQ^D?bMFGkzqJRZfJTX>JGb=TFO{Lu^C z!L9HVd8oo0;nK%sJV0-BW6|So_xqfIh%`P_P`9Ri91G(&X({E-uRTQBh7fBYid&Ur z-gYCz;9_AwmiV^C_Bg2Oml!!EVQgQ(k)K_N^DcN}t@EAiO~8VvSc{j}_hC9Z-b=#) zIJhx_4?7EZyPRrWvj}}=Jb)oM#X+INtREA8mI|yD1tRYXFsI+zZwcwDtMO;smua^S z$IWng4b|0yAhMnJmlP9$9j(v-a7p;$aczk-3huF6?2UDqZ+AeIKjtJU3N9u{+07~6 z(7ajlV@{)cNtZ)~NSVW!5_uySyRx!nLxKULV<8J6j~0qxgRKS3+XwWnEeRBnCZZ|h z-uaG6o9TV3*8pnF9ntU1*dhpl3mlM}9I_%qyO$=7Q_ers6Bq)!%*W&PuNDzrG;>0U z&v7HaM$^RDEq0SdKszWcA!mr%g8mhNMzKHzW}Hk_S`sb15{fXVOg$mVp+dnwfK(`b zNNY+;d9(vehhpkRzzs^ZBw3+Hrbof41(mnNo1NLQ0jmn*kKN%gcd+ASBc3 zz~Nfdsep?I$n`6|dh!AcXf;qGD2MY9enN6;D{aGCv^UzR#FXzp?T~=C&zV)dV0h zZPe3Ce>46#8;~xAo}Yl4iXf)D8H{0@azya#g&gJsRUiI%&;#oW z;vawAg?^7o-bnno^9i5N|9S6E=J7di_UBRUmmMRGww(I%e}f!SrSx>P(As>Csn{Zh zwoYy`T5UeJq9RcaW5Pai9}rpwSI!S4SDeaT=64GX%r2Z*7-$(;)Su}GWs%j zp!GRCAW<%a zN-l+puC4El)vBglukaZ(`QcI@y>QJwSXL9*QEsW-ML>3QpIbE~5cF%SJb7x09cvsT zpF6|2`(PIj>?86<(?zc{Je6BfAhdLdZJl^376b~_A9vQEMP7&fEYo+kVD9&i&b)a2 z#M&1&CoUm}6+`zZ<%4WANX+j|15&^7fGhl||-NqSh+Es$~4s()5IHU#d1uI|W0b@$MNLaSt6B z5nzJcix%_qkR@DQ|K4TdI+Y*tFDz1>GezkJJZ8b4@9@(YG6Gf*J55mA!n_^G)xNR3 zG0)fI@of1Bvbs!9;UqFJe}y<>TTz+2>?^>cQua8|)lGQ^_aD< z?ksF^mWQW)+W7}+R6roYggfjt$(Z}2VcMvuTz>+im6N3 z#HqZF$E?0h9LbVzf2z_pomoT1#hibstRIz5V9|4)JwpNQ!Y{9Lo ztm_cX2Y3zhed5K^$Dy4jl(fog-C1OWJ*WY1IK|+GmI>M_a;8$M&pqd7m#i^DvS|rw z+j*y-A>?LZBJgyx0pjV$p*-Bb+IRuGPo%)x*KrKJ?eAV)K6 zg(I;9I~r4@t@RT@30@2^bTdJc;KDYn&OwrT2O%RIgHS0GRX}iDBhBt5^8B$PWa*&m zAis$W@lqDA zcLy9{Y&2|M+2b-uq8rG{Nx2CF=1&s#743Iy+TVMv1qjl>SQ$2e%E-zRlJpL%gX{qC z`z?5i-*|S~?f^wR@1T6yV-y#u(C&G{TSrgD*mcoG08$JkkRaLIgPu*L+1ymHpz&6L~^rQjFBX7p3pR{nfTcW${%Jvwn`G_U%~l#Wg{yl8}cW zGHPn5Kk1E-(PGC>-h~43$}d!n3`i)SjkT&@7|DMhf|ROrdZ+bRhcsk%Z~9MX z?%$83;^VJvzo5*`akE&^h*uhW=uKc@oi=Xcnf8k0=GmC>`-5 zG;k~VKQX;u?@e7)+*BMfj}n}&aMR$e5#0r3tvWBWR$|RDSdv!^O8UBnj!!Pm!17?G z&$T5?58w%$LS8x(?F$dZZ*Js_oeO_qlz!Jf6#o?i2(uidrHs4H5e$iyk<)&K0A$D? znbY{B`1w~at5ALpU3_?s8hcq=DuR4JZmIZwd^qUWHrLk}9*wRV@lT}M?AR|0S()ct z(>81mnp@|Vn$|K1OG&wg!HP-_Cvzm@(Razufv1E|Yt>N)@m)sd6m&<)yk^IqW}EL4 z>2n9BoFtcx!Jm{YN*eG*Xho7jJ6(na?|E(IHTcN2FC4lk)M?1SGQ16BozKf*HIUC^ z1?ED3z_9{KGz&k{Onac{r9{4oiYE9`C-cm+?_sNi3hJRxadqC91YtoG+6vut38nC@ zUO#T{mP)@V`=aP}iMftOX`-EF%bWf5wc*yr;EKr5oCMXN+D%lz=`?_i0DBvXrB0{k*@qSUDYHu3SDIK^?}oM9Eo;W9Jgn&lO85i9T4`xhjuZ?Pt*vqErV$nMpwq=E z5&23w#DHDQQO3p@U(wvLN0RtdcrQe7rYmcf9Qw{F+LRFM9cQlN)zd{yj1r_q1=Fq? zxw6M)sS7W^a5;K8f8$S>Ed3%pQ(49!G5uVFqHZ%tg>S71cGbQWV}}vZcK0AHi*taU z*E05XWfK^;*HSGy^hh_QM~T`v{G7iOAtuHh}h(KVF&dy zqaJuv)P%CzCYm5ciL)tQ%zcMl@t85FygSnOV<%x!J)7>0Lx#HFggk+6eMhl57( zM?i{dL z3f4qs)2rzphcfp~E27lC2x(|#|HPfW`>QI`7uxZ}Usz9atKgG;6RGAr1?I$!y`L!m z*Y|8s5fdL&{hWQCz=_SC&`VS%D}+EhvF*RDJ_?H6^_EG696`7tGs!fMFH>B5Cp*JF zFn=8Fe85SL^WlVDbuBBST+!HYEd-vVx|4Hd)2b$L8cK@WylG8-xP)+gNG<0{Yh%3l z0hZlzUyGuC%h%Ig%i@8t7~JvxfTbKXGi*-CW?87$6Df!+L^`*lm+1Th4@0u^TVv&} z5!e##a?Bx5;~G&FcR^btI{B~*j;BJtCHiG)xJ_%6cV>=d?(<>}g|XnaaG$k^*(QKc z&c12?n!_4p5-J?!JJi@)OU89&UqrXw8TJkt#)#Z%S`ysbe%&=zf*Lo*2DQ?35#0$Z zfs%a|pUe8#&Lt5KJi`gAj~%xoQ=A+l7n$!&hO)w~I-cNu9>dgw?`Io9-Igfs@r6nA zlQF0G6QgQvg<}Tul75z*lS**7a{;QwYP|V__SL6lgw_R=HyIQK<_G>#Pd!S;SqUP@ z($pD$9(0JJSK4+CQ+|Zk+ecXbzqYP3tch)lhR~bT0E$Qv1WAyNAP5nHfJhS(Ktfjm z0g)1sDk!~(D7_0(r8n_jdKWG=AiX3ANGH-F1bD;s`rh}w_wp;>WM-c;Is5E0v-aBS zV6&;npDlOTIgq=rs;ls^N@m@D(0+j`Qkv_s=3r$nG>@5d$T!Wj8hS~| z*a`ykVi5A*6m6&o1yzw=yU-ODEqP&Px)u5~$F|rLH4*h74;8zxHx#eg>eFjuL$8nX z=h>`K_|PlRAiP9wpSjv+OW_$)3t~{@-BOi;SF2&t? zMK$rWTe;SA+?smFF<5pHNCCN^($*Im+hoIQIL1V>We>=d6%wbVh6Y$p+gDj|JRhZ* zrd{e1q|5A7GRpMWSZx(d(%fc_YTlHQ;$wcp#0p@-d$ltqK?*RTW#;w#vheK1VPz}4 zngi|CYBP$*oPnOR{)&(ir*+cug?R{dWuZOtp??Iv`Rj#s|JQgB7LTat+V8scuZ1EG(;&UH>4wm{Mk{|%ZQDb)-k1{iL;8TJ|?~Da?PU_05^;7hH%S2OV=o4_JiTMiM&mN`bt0IT()0%1h(QY zUITW@zEeH)q68$}Fh9H7wx*!_nC6Ai&`*r${hAMJ%pKrfrY0m&H1gbvGxwZH-11&t zS&on?9WQ!2LbuA$pjxxT!R{*QA#RId4n3sWIjlj(59wg&Wovp=-jUCy9Nx{AX_3r- z-(f=X@PjD%vUuZ{Z9T-ck3TXY*>Tq2d5F`1x%$E4hT~+LSF0Cd+wz)=upgVc*^V_N zSW{6rha98Gv~ntPColW*vmL0$p*m&*XK^pYA8_r3l=+0*-9tqX^#| zO-HAEjV7<%n^V5G1EmhbQr(#dH*if6YY96PDyYWHdVr?StKjR1do@7*5b;rRu3C?% z{(ihaXU20%3?jC^4k_W^AVq!6U3tDE_}4<`ocmVHS;lt8cwR%qMP;etPS2fx7MO^X zX7GrzGW_fzISvkJ#7s3+lRjQdr<>a4JGJ{1BseJsu89Dlj9r@_qBzoy>H!^R@4#b| zbt_kQpn@kELj|3-ZfU8`eu3YUzN#D8`Xivw>IookS6};wFo&KA%8IhR$BQHwtMb!^ z3~@tmp$p$U0;^4W*6z6)N=Y?UAGS3n;)RIqfoUh5`)Ruy|4KzPPllyH5n3&&L+{AG z+FcZK*Z4sZ_fa_=?yxH*NG_hkp7Vgg--Pq-(;qSy+lpu$S@Pr!EPLAVVR8VUSC$KP zN|NEbppIze(B@&w;5X| z)7GYKyOOZYq}5RX2x8{h<`y5ueWZcPxh#ocC>nV znk!;s^f{j4D`GcX&tWl5Bxt_rra`$3eRsNPy5x^C`}wrL5-vbBn)>Hs)Y-At$*dPS z^(9nX8>K{+x%cpYEZi4~Mtb%4J($t#QfRYVs&C2{0!rCVs)UdWz`D*hqPqqgAA&nkiaJnul9Uv#Mq>!vo0oLEH>KIbndGa!Cc(z-Cf1ZqVCG12i^%%IE`O+G z+7z*r8sp)AjSpRE1&#y>=ASO+L^*jiHxKx2iE{>3b%=-f`QJ1&o;2bai#_1QN)bxB zj5_U0Ls0cVsBD}Y}lI$mnfK^GpN5{AD{HK*+RPifV+P!KpVw1`6+{Xnh0 zbv0T=$Rg(T2EA7O%iu9KOIs60+(-uU3MArHfSKi42AN=??S0h+G zOJe;SePsCK%itZB?HAun)58}#k~aZ?*|Wd$4KFwhb<(5vdcxh65TGKI;eY55*}ki< zilnzgX)4SXFzvO6D`VYEpTpnF)9{7Zg!k;)r_EMHWd9q=hcBL`ZGJGstMTy9pY|WD zjSY};=RXBQYJK5G8WIJ`XYJ_6QIIU6w@SZfO~X=5Ah8f4$>hE4Uu(g7l>nSO?JI)2 zuKWHeZIYOEwFn|0ivG^{YABJTtt(SSgZ;?{Qu=qE(+OPJEV_PTut66VaH{y?`}@$z zw>v;3alJzFqZjCjFH~?!BXi_0J_8>soI>DfS!dxXfaE;FBhf6<7k$<%R&4ikn`hS# zgzoROBTrS)j1tbWD63H;3@gPpQlklHRsukN{H5KGBRQCZJ6QiC?=GqfZ~6eS%a6lE zftWrp1U;3yn3&4sTDTPu)2ApOmA7+O{arjwd`T#MT36#^fm{O;Ya1KNV zlJ@tlvz}4OKI9nnwermia42grSKNMm(LuW{hWmOy$p#07UVY#YV8C^pcb4%UVFXXR zC+De``?yZuF18V>>rYqIaJ9yrQ8d^p0hUsFHgh1!({3@|_}sCsJbxO1=Ue03!*@tu zN1|uNrcrLQgdY$SUSMMsi3i_}F#8D$g z^{l5fM5}CI9el5ycx``h5}?{iJnV@ZEHo6JDz$8CWV{v@$!4L=mimUhdOZJ^{{bvO ze{l>hKyS0~M4*e7?aDaaD*$fJq@@K$N+YdjROw8KQZDyn5{YH#+G=m6ux}Hv5$E~= zm7`Scv1iB@-Ka)0kaF0GxLQO`eU1Tm=Rk7-O0@PsX^`pZ=W?W$tlKRVr(rub%mhH&YO+lftE8($K%`0y0@A+o=RBQFVkPUgZ)uEvl{=U|HS2DS zlX1=B(zu(8C?F3lO1{ODZ%(aS(CUyqM=rliw6I^k^b*)@O23!gsXSzHAPA81P)-A$ z8*qT?DI<_lXwX?)oo(>(maVg4EwdL6e72KX0pO++nf!nxr^9nVJO8Aq!F@R#P2E;r zLrvxC0H=|s39OznfNM~)YqD#4XzJhN6Z9t&nlUA~*cZZ8!MDgLDfKTI=nuNaa-3VP z^lvzdcFVR-f+eGqx}aZ^+bdL5YTpr~H{@sRc7^Q@*j>Y`Xvv7;5O9%7=eT@kdiv+S zzP|Ft$&HN?tafQ)BInrbY{s1w1elBl4gxa=1iaYaVS2S+Psc<9=i?{4*ija=+39Ht%NVJ# zu`%lp@fKI?2~M=e$H&LkWF9)4aYs}1@INe4zrRshBn?OqoIO3q#9*)>*@P3~;V2{H zIaF}*8{?)+?-1hd_u)z|w|*G&D(t|-t4yc+9MYkmySlobTFy;O&V@7GR^kJB7@ zwnKA*nSR}XW()|7@`i;M2GiQ(P#E^&sx0c6hK7d0is2hvJUl$OBt%Ja^7*l;skP3| z&Q`V3*`(!-5d~t`6lrY7y-@L)>+99$MoLHfwY8@@b5WttO_x&~=(G$^sfS4_zKlfs zm|-gi{e2bvy}d;-M+V9sVuoF2Q$;SUQ?kc}A{f1m`I8gnsOLNvF1Q`OozFbUZP{vT z!;y2Fo10r(Ka2K1&&5@cpi$RC#lge2e{o9UuJOVz!?xZx8)FELw_8qR2nw^mXL z3g>HkCqo{5YKN9AT$Z$OE~u)KMh`>?SQWnRmec9G#KXfV4vu<3`or<1&hDb>`L%IJ zxwT0vd=G(8p{SrhYuSk;?(Jbd)z)$#*v*dw*TaoxfhIRc*OFE(t2a#i*4Ng0N1bN= zM0WKL45-au*+&8Q*~W3`%%IU|4=fh@9=3k+M7+vPHm#li%=Lixoi`xxS8AA1Wt<8b zCT*7wn)#{xXF})=uR}W1*xRv!LfPIQno{~b4$VJJ1Vj%@ zvLG6r?Ob%^t`=>6Nq&Rz@L)%1MxPt@zPqrnu!+YD_y)Rqc}+MxNN9(I`;}Q3;`4l7 zXO2+adwM|<1vQUliY2ogG)R1f_v_duO4xFiHzud`x&#Fc4 zrJoDNsQ8to#MHu4!JoOL!S`YBaGq5(Ndd#f#Hxr--aOkI`Q6|*PT0)#nvG&Eh#1f4 z&2STx1*@ST`pc)iRv9)GXtw&*#kSH+R#zE_g=%!PM3JXOgKf%Jd%64a$Nu}+ zvCw0g!SL|#axuc4mEIRWx-9DJASwg>E+d~vnjZI*S3B|-F8s`8vBl?Z3|KbSos#_1 z%UGP3@axVh%7Cx_J$@@}@a8+2S0zN8(0tgvqdjeT(f1y*Lw zq{1ys_90)EH?fSf_-=s~_7SZ@beM%puBkT2@Xy%6D}gX&#@!O)j~mx~gi5~*ZSIIt z4hR355Q4yjh{)B=_{U+GiRBb`>x`*tjxPxZcDAP+v}3zJt6%6S@a0x?)(9^8{_OHk z0fv8fj~VD*=;eyPgXG`#n6#RM(jM8Jvo4qj3g-nvCva5%3D|j2j(6>M6Tq<)Es%9# zvPU@guQNb_w`oa9&2#qlt+&qN?U|`xk?Z$YK3g!2$$4^Ng{-0qG_zU1BLJR8l|KD;HEuYmxhO}@0-JxIIe9J;SmjxQoh8jD1 QfPjydx-Ozr)#~NH0Pe%?BLDyZ literal 0 HcmV?d00001