From 2164a2a8898da176799bab3815244b234746eb45 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 25 Nov 2021 10:54:13 +0100 Subject: [PATCH 1/3] Datacite: Code Refactor generated a general SparkApplication Scala where all the spark scala have to inherit Commented a little the Datacite transformation code --- .../AbstractScalaApplication.scala | 37 +++++ .../application/SparkScalaApplication.scala | 35 +++++ .../dhp/datacite/DataciteModelConstants.scala | 134 ++++++++++++++++++ .../DataciteToOAFTransformation.scala | 127 +++-------------- .../GenerateDataciteDatasetSpark.scala | 86 +++++++---- .../src/main/resources/log4j.properties | 3 + .../dhp/datacite/DataciteToOAFTest.scala | 57 +++++++- 7 files changed, 337 insertions(+), 142 deletions(-) create mode 100644 dhp-common/src/main/java/eu/dnetlib/dhp/application/AbstractScalaApplication.scala create mode 100644 dhp-common/src/main/java/eu/dnetlib/dhp/application/SparkScalaApplication.scala create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteModelConstants.scala diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/application/AbstractScalaApplication.scala b/dhp-common/src/main/java/eu/dnetlib/dhp/application/AbstractScalaApplication.scala new file mode 100644 index 000000000..44dad93eb --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/application/AbstractScalaApplication.scala @@ -0,0 +1,37 @@ +package eu.dnetlib.dhp.application + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.slf4j.Logger + +abstract class AbstractScalaApplication (val propertyPath:String, val args:Array[String], log:Logger) extends SparkScalaApplication { + + var parser: ArgumentApplicationParser = null + + var spark:SparkSession = null + + + def initialize():SparkScalaApplication = { + parser = parseArguments(args) + spark = createSparkSession() + this + } + + /** + * Utility for creating a spark session starting from parser + * + * @return a spark Session + */ + private def createSparkSession():SparkSession = { + require(parser!= null) + + val conf:SparkConf = new SparkConf() + val master = parser.get("master") + log.info(s"Creating Spark session: Master: $master") + SparkSession.builder().config(conf) + .appName(getClass.getSimpleName) + .master(master) + .getOrCreate() + } + +} \ No newline at end of file diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/application/SparkScalaApplication.scala b/dhp-common/src/main/java/eu/dnetlib/dhp/application/SparkScalaApplication.scala new file mode 100644 index 000000000..247bacac0 --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/application/SparkScalaApplication.scala @@ -0,0 +1,35 @@ +package eu.dnetlib.dhp.application + +import scala.io.Source + +/** + * This is the main Interface SparkApplication + * where all the Spark Scala class should inherit + * + */ +trait SparkScalaApplication { + /** + * This is the path in the classpath of the json + * describes all the argument needed to run + */ + val propertyPath: String + + /** + * Utility to parse the arguments using the + * property json in the classpath identified from + * the variable propertyPath + * + * @param args the list of arguments + */ + def parseArguments(args: Array[String]): ArgumentApplicationParser = { + val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream(propertyPath)).mkString) + parser.parseArgument(args) + parser + } + + /** + * Here all the spark applications runs this method + * where the whole logic of the spark node is defined + */ + def run(): Unit +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteModelConstants.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteModelConstants.scala new file mode 100644 index 000000000..0685a04a3 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteModelConstants.scala @@ -0,0 +1,134 @@ +package eu.dnetlib.dhp.datacite + +import eu.dnetlib.dhp.schema.common.ModelConstants +import eu.dnetlib.dhp.schema.oaf.{DataInfo, KeyValue} +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils + +import java.io.InputStream +import java.time.format.DateTimeFormatter +import java.util.Locale +import java.util.regex.Pattern +import scala.io.Source + +/** + * This class represent the dataModel of the input Dataset of Datacite + * @param doi THE DOI + * @param timestamp timestamp of last update date + * @param isActive the record is active or deleted + * @param json the json native records + */ +case class DataciteType(doi: String, timestamp: Long, isActive: Boolean, json: String) {} + +/* + The following class are utility class used for the mapping from + json datacite to OAF Shema + */ +case class RelatedIdentifierType(relationType: String, relatedIdentifier: String, relatedIdentifierType: String) {} + +case class NameIdentifiersType(nameIdentifierScheme: Option[String], schemeUri: Option[String], nameIdentifier: Option[String]) {} + +case class CreatorType(nameType: Option[String], nameIdentifiers: Option[List[NameIdentifiersType]], name: Option[String], familyName: Option[String], givenName: Option[String], affiliation: Option[List[String]]) {} + +case class TitleType(title: Option[String], titleType: Option[String], lang: Option[String]) {} + +case class SubjectType(subject: Option[String], subjectScheme: Option[String]) {} + +case class DescriptionType(descriptionType: Option[String], description: Option[String]) {} + +case class FundingReferenceType(funderIdentifierType: Option[String], awardTitle: Option[String], awardUri: Option[String], funderName: Option[String], funderIdentifier: Option[String], awardNumber: Option[String]) {} + +case class DateType(date: Option[String], dateType: Option[String]) {} + +case class OAFRelations(relation:String, inverse:String, relType:String) + + +class DataciteModelConstants extends Serializable { + +} + +object DataciteModelConstants { + + val REL_TYPE_VALUE:String = "resultResult" + val DATE_RELATION_KEY = "RelationDate" + val DATACITE_FILTER_PATH = "/eu/dnetlib/dhp/datacite/datacite_filter" + val DOI_CLASS = "doi" + val SUBJ_CLASS = "keywords" + val DATACITE_NAME = "Datacite" + val dataInfo: DataInfo = dataciteDataInfo("0.9") + val DATACITE_COLLECTED_FROM: KeyValue = OafMapperUtils.keyValue(ModelConstants.DATACITE_ID, DATACITE_NAME) + + val subRelTypeMapping: Map[String,OAFRelations] = Map( + ModelConstants.REFERENCES -> OAFRelations(ModelConstants.REFERENCES, ModelConstants.IS_REFERENCED_BY, ModelConstants.RELATIONSHIP), + ModelConstants.IS_REFERENCED_BY -> OAFRelations(ModelConstants.IS_REFERENCED_BY,ModelConstants.REFERENCES, ModelConstants.RELATIONSHIP), + + ModelConstants.IS_SUPPLEMENTED_BY -> OAFRelations(ModelConstants.IS_SUPPLEMENTED_BY,ModelConstants.IS_SUPPLEMENT_TO,ModelConstants.SUPPLEMENT), + ModelConstants.IS_SUPPLEMENT_TO -> OAFRelations(ModelConstants.IS_SUPPLEMENT_TO,ModelConstants.IS_SUPPLEMENTED_BY,ModelConstants.SUPPLEMENT), + + ModelConstants.HAS_PART -> OAFRelations(ModelConstants.HAS_PART,ModelConstants.IS_PART_OF, ModelConstants.PART), + ModelConstants.IS_PART_OF -> OAFRelations(ModelConstants.IS_PART_OF,ModelConstants.HAS_PART, ModelConstants.PART), + + ModelConstants.IS_VERSION_OF-> OAFRelations(ModelConstants.IS_VERSION_OF,ModelConstants.HAS_VERSION,ModelConstants.VERSION), + ModelConstants.HAS_VERSION-> OAFRelations(ModelConstants.HAS_VERSION,ModelConstants.IS_VERSION_OF,ModelConstants.VERSION), + + ModelConstants.IS_IDENTICAL_TO -> OAFRelations(ModelConstants.IS_IDENTICAL_TO,ModelConstants.IS_IDENTICAL_TO, ModelConstants.RELATIONSHIP), + + ModelConstants.IS_CONTINUED_BY -> OAFRelations(ModelConstants.IS_CONTINUED_BY,ModelConstants.CONTINUES, ModelConstants.RELATIONSHIP), + ModelConstants.CONTINUES -> OAFRelations(ModelConstants.CONTINUES,ModelConstants.IS_CONTINUED_BY, ModelConstants.RELATIONSHIP), + + ModelConstants.IS_NEW_VERSION_OF-> OAFRelations(ModelConstants.IS_NEW_VERSION_OF,ModelConstants.IS_PREVIOUS_VERSION_OF, ModelConstants.VERSION), + ModelConstants.IS_PREVIOUS_VERSION_OF ->OAFRelations(ModelConstants.IS_PREVIOUS_VERSION_OF,ModelConstants.IS_NEW_VERSION_OF, ModelConstants.VERSION), + + ModelConstants.IS_DOCUMENTED_BY -> OAFRelations(ModelConstants.IS_DOCUMENTED_BY,ModelConstants.DOCUMENTS, ModelConstants.RELATIONSHIP), + ModelConstants.DOCUMENTS -> OAFRelations(ModelConstants.DOCUMENTS,ModelConstants.IS_DOCUMENTED_BY, ModelConstants.RELATIONSHIP), + + ModelConstants.IS_SOURCE_OF -> OAFRelations(ModelConstants.IS_SOURCE_OF,ModelConstants.IS_DERIVED_FROM, ModelConstants.VERSION), + ModelConstants.IS_DERIVED_FROM -> OAFRelations(ModelConstants.IS_DERIVED_FROM,ModelConstants.IS_SOURCE_OF, ModelConstants.VERSION), + + ModelConstants.CITES -> OAFRelations(ModelConstants.CITES,ModelConstants.IS_CITED_BY, ModelConstants.CITATION), + ModelConstants.IS_CITED_BY -> OAFRelations(ModelConstants.IS_CITED_BY,ModelConstants.CITES, ModelConstants.CITATION), + + ModelConstants.IS_VARIANT_FORM_OF -> OAFRelations(ModelConstants.IS_VARIANT_FORM_OF,ModelConstants.IS_DERIVED_FROM, ModelConstants.VERSION), + ModelConstants.IS_OBSOLETED_BY -> OAFRelations(ModelConstants.IS_OBSOLETED_BY,ModelConstants.IS_NEW_VERSION_OF, ModelConstants.VERSION), + + ModelConstants.REVIEWS -> OAFRelations(ModelConstants.REVIEWS,ModelConstants.IS_REVIEWED_BY, ModelConstants.REVIEW), + ModelConstants.IS_REVIEWED_BY -> OAFRelations(ModelConstants.IS_REVIEWED_BY,ModelConstants.REVIEWS, ModelConstants.REVIEW), + + ModelConstants.DOCUMENTS -> OAFRelations(ModelConstants.DOCUMENTS,ModelConstants.IS_DOCUMENTED_BY, ModelConstants.RELATIONSHIP), + ModelConstants.IS_DOCUMENTED_BY -> OAFRelations(ModelConstants.IS_DOCUMENTED_BY,ModelConstants.DOCUMENTS, ModelConstants.RELATIONSHIP), + + ModelConstants.COMPILES -> OAFRelations(ModelConstants.COMPILES,ModelConstants.IS_COMPILED_BY, ModelConstants.RELATIONSHIP), + ModelConstants.IS_COMPILED_BY -> OAFRelations(ModelConstants.IS_COMPILED_BY,ModelConstants.COMPILES, ModelConstants.RELATIONSHIP) + ) + + + val datacite_filter: List[String] = { + val stream: InputStream = getClass.getResourceAsStream(DATACITE_FILTER_PATH) + require(stream!= null) + Source.fromInputStream(stream).getLines().toList + } + + + def dataciteDataInfo(trust: String): DataInfo = OafMapperUtils.dataInfo(false,null, false, false, ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER, trust) + + val df_en: DateTimeFormatter = DateTimeFormatter.ofPattern("[MM-dd-yyyy][MM/dd/yyyy][dd-MM-yy][dd-MMM-yyyy][dd/MMM/yyyy][dd-MMM-yy][dd/MMM/yy][dd-MM-yy][dd/MM/yy][dd-MM-yyyy][dd/MM/yyyy][yyyy-MM-dd][yyyy/MM/dd]", Locale.ENGLISH) + val df_it: DateTimeFormatter = DateTimeFormatter.ofPattern("[dd-MM-yyyy][dd/MM/yyyy]", Locale.ITALIAN) + + val funder_regex: List[(Pattern, String)] = List( + (Pattern.compile("(info:eu-repo/grantagreement/ec/h2020/)(\\d\\d\\d\\d\\d\\d)(.*)", Pattern.MULTILINE | Pattern.CASE_INSENSITIVE), "40|corda__h2020::"), + (Pattern.compile("(info:eu-repo/grantagreement/ec/fp7/)(\\d\\d\\d\\d\\d\\d)(.*)", Pattern.MULTILINE | Pattern.CASE_INSENSITIVE), "40|corda_______::") + + ) + + val Date_regex: List[Pattern] = List( + //Y-M-D + Pattern.compile("(18|19|20)\\d\\d([- /.])(0[1-9]|1[012])\\2(0[1-9]|[12][0-9]|3[01])", Pattern.MULTILINE), + //M-D-Y + Pattern.compile("((0[1-9]|1[012])|([1-9]))([- /.])(0[1-9]|[12][0-9]|3[01])([- /.])(18|19|20)?\\d\\d", Pattern.MULTILINE), + //D-M-Y + Pattern.compile("(?:(?:31(/|-|\\.)(?:0?[13578]|1[02]|(?:Jan|Mar|May|Jul|Aug|Oct|Dec)))\\1|(?:(?:29|30)(/|-|\\.)(?:0?[1,3-9]|1[0-2]|(?:Jan|Mar|Apr|May|Jun|Jul|Aug|Sep|Oct|Nov|Dec))\\2))(?:(?:1[6-9]|[2-9]\\d)?\\d{2})|(?:29(/|-|\\.)(?:0?2|(?:Feb))\\3(?:(?:(?:1[6-9]|[2-9]\\d)?(?:0[48]|[2468][048]|[13579][26])|(?:(?:16|[2468][048]|[3579][26])00))))|(?:0?[1-9]|1\\d|2[0-8])(/|-|\\.)(?:(?:0?[1-9]|(?:Jan|Feb|Mar|Apr|May|Jun|Jul|Aug|Sep))|(?:1[0-2]|(?:Oct|Nov|Dec)))\\4(?:(?:1[6-9]|[2-9]\\d)?\\d{2})", Pattern.MULTILINE), + //Y + Pattern.compile("(19|20)\\d\\d", Pattern.MULTILINE) + ) + + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala index 1af72e8d3..6b4deef0a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.datacite import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup +import eu.dnetlib.dhp.datacite.DataciteModelConstants._ import eu.dnetlib.dhp.schema.action.AtomicAction import eu.dnetlib.dhp.schema.common.ModelConstants import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, OafMapperUtils} @@ -12,115 +13,30 @@ import org.json4s.DefaultFormats import org.json4s.JsonAST.{JField, JObject, JString} import org.json4s.jackson.JsonMethods.parse -import java.nio.charset.CodingErrorAction import java.text.SimpleDateFormat import java.time.LocalDate import java.time.chrono.ThaiBuddhistDate import java.time.format.DateTimeFormatter -import java.util.regex.Pattern import java.util.{Date, Locale} import scala.collection.JavaConverters._ -import scala.io.{Codec, Source} -import scala.language.postfixOps -case class DataciteType(doi: String, timestamp: Long, isActive: Boolean, json: String) {} - -case class RelatedIdentifierType(relationType: String, relatedIdentifier: String, relatedIdentifierType: String) {} - -case class NameIdentifiersType(nameIdentifierScheme: Option[String], schemeUri: Option[String], nameIdentifier: Option[String]) {} - -case class CreatorType(nameType: Option[String], nameIdentifiers: Option[List[NameIdentifiersType]], name: Option[String], familyName: Option[String], givenName: Option[String], affiliation: Option[List[String]]) {} - -case class TitleType(title: Option[String], titleType: Option[String], lang: Option[String]) {} - -case class SubjectType(subject: Option[String], subjectScheme: Option[String]) {} - -case class DescriptionType(descriptionType: Option[String], description: Option[String]) {} - -case class FundingReferenceType(funderIdentifierType: Option[String], awardTitle: Option[String], awardUri: Option[String], funderName: Option[String], funderIdentifier: Option[String], awardNumber: Option[String]) {} - -case class DateType(date: Option[String], dateType: Option[String]) {} - -//case class HostedByMapType(openaire_id: String, datacite_name: String, official_name: String, similarity: Option[Float]) {} object DataciteToOAFTransformation { - val REL_TYPE_VALUE:String = "resultResult" - val DATE_RELATION_KEY = "RelationDate" - - val subRelTypeMapping: Map[String,(String,String)] = Map( - "References" ->("IsReferencedBy","relationship"), - "IsSupplementTo" ->("IsSupplementedBy","supplement"), - "IsPartOf" ->("HasPart","part"), - "HasPart" ->("IsPartOf","part"), - "IsVersionOf" ->("HasVersion","version"), - "HasVersion" ->("IsVersionOf","version"), - "IsIdenticalTo" ->("IsIdenticalTo","relationship"), - "IsPreviousVersionOf" ->("IsNewVersionOf","version"), - "IsContinuedBy" ->("Continues","relationship"), - "Continues" ->("IsContinuedBy","relationship"), - "IsNewVersionOf" ->("IsPreviousVersionOf","version"), - "IsSupplementedBy" ->("IsSupplementTo","supplement"), - "IsDocumentedBy" ->("Documents","relationship"), - "IsSourceOf" ->("IsDerivedFrom","relationship"), - "Cites" ->("IsCitedBy","citation"), - "IsCitedBy" ->("Cites","citation"), - "IsDerivedFrom" ->("IsSourceOf","relationship"), - "IsVariantFormOf" ->("IsDerivedFrom","version"), - "IsReferencedBy" ->("References","relationship"), - "IsObsoletedBy" ->("IsNewVersionOf","version"), - "Reviews" ->("IsReviewedBy","review"), - "Documents" ->("IsDocumentedBy","relationship"), - "IsCompiledBy" ->("Compiles","relationship"), - "Compiles" ->("IsCompiledBy","relationship"), - "IsReviewedBy" ->("Reviews","review") - ) - - implicit val codec: Codec = Codec("UTF-8") - codec.onMalformedInput(CodingErrorAction.REPLACE) - codec.onUnmappableCharacter(CodingErrorAction.REPLACE) - - val DOI_CLASS = "doi" - val SUBJ_CLASS = "keywords" - - - val j_filter: List[String] = { - val s = Source.fromInputStream(getClass.getResourceAsStream("datacite_filter")).mkString - s.lines.toList - } - val mapper = new ObjectMapper() - val dataInfo: DataInfo = generateDataInfo("0.9") - val DATACITE_COLLECTED_FROM: KeyValue = OafMapperUtils.keyValue(ModelConstants.DATACITE_ID, "Datacite") - - - val df_en: DateTimeFormatter = DateTimeFormatter.ofPattern("[MM-dd-yyyy][MM/dd/yyyy][dd-MM-yy][dd-MMM-yyyy][dd/MMM/yyyy][dd-MMM-yy][dd/MMM/yy][dd-MM-yy][dd/MM/yy][dd-MM-yyyy][dd/MM/yyyy][yyyy-MM-dd][yyyy/MM/dd]", Locale.ENGLISH) - val df_it: DateTimeFormatter = DateTimeFormatter.ofPattern("[dd-MM-yyyy][dd/MM/yyyy]", Locale.ITALIAN) - - val funder_regex: List[(Pattern, String)] = List( - (Pattern.compile("(info:eu-repo/grantagreement/ec/h2020/)(\\d\\d\\d\\d\\d\\d)(.*)", Pattern.MULTILINE | Pattern.CASE_INSENSITIVE), "40|corda__h2020::"), - (Pattern.compile("(info:eu-repo/grantagreement/ec/fp7/)(\\d\\d\\d\\d\\d\\d)(.*)", Pattern.MULTILINE | Pattern.CASE_INSENSITIVE), "40|corda_______::") - - ) - - val Date_regex: List[Pattern] = List( - //Y-M-D - Pattern.compile("(18|19|20)\\d\\d([- /.])(0[1-9]|1[012])\\2(0[1-9]|[12][0-9]|3[01])", Pattern.MULTILINE), - //M-D-Y - Pattern.compile("((0[1-9]|1[012])|([1-9]))([- /.])(0[1-9]|[12][0-9]|3[01])([- /.])(18|19|20)?\\d\\d", Pattern.MULTILINE), - //D-M-Y - Pattern.compile("(?:(?:31(/|-|\\.)(?:0?[13578]|1[02]|(?:Jan|Mar|May|Jul|Aug|Oct|Dec)))\\1|(?:(?:29|30)(/|-|\\.)(?:0?[1,3-9]|1[0-2]|(?:Jan|Mar|Apr|May|Jun|Jul|Aug|Sep|Oct|Nov|Dec))\\2))(?:(?:1[6-9]|[2-9]\\d)?\\d{2})|(?:29(/|-|\\.)(?:0?2|(?:Feb))\\3(?:(?:(?:1[6-9]|[2-9]\\d)?(?:0[48]|[2468][048]|[13579][26])|(?:(?:16|[2468][048]|[3579][26])00))))|(?:0?[1-9]|1\\d|2[0-8])(/|-|\\.)(?:(?:0?[1-9]|(?:Jan|Feb|Mar|Apr|May|Jun|Jul|Aug|Sep))|(?:1[0-2]|(?:Oct|Nov|Dec)))\\4(?:(?:1[6-9]|[2-9]\\d)?\\d{2})", Pattern.MULTILINE), - //Y - Pattern.compile("(19|20)\\d\\d", Pattern.MULTILINE) - ) - - - def filter_json(json: String): Boolean = { - j_filter.exists(f => json.contains(f)) + /** + * This method should skip record if json contains invalid text + * defined in gile datacite_filter + * @param json + * @return True if the record should be skipped + */ + def skip_record(json: String): Boolean = { + datacite_filter.exists(f => json.contains(f)) } + @deprecated("this method will be removed", "dhp") def toActionSet(item: Oaf): (String, String) = { val mapper = new ObjectMapper() @@ -200,6 +116,8 @@ object DataciteToOAFTransformation { case _: Throwable => "" } } + + def getTypeQualifier(resourceType: String, resourceTypeGeneral: String, schemaOrg: String, vocabularies: VocabularyGroup): (Qualifier, Qualifier) = { if (resourceType != null && resourceType.nonEmpty) { val typeQualifier = vocabularies.getSynonymAsQualifier(ModelConstants.DNET_PUBLICATION_RESOURCE, resourceType) @@ -318,11 +236,7 @@ object DataciteToOAFTransformation { val p = match_pattern.get._2 val grantId = m.matcher(awardUri).replaceAll("$2") val targetId = s"$p${DHPUtils.md5(grantId)}" - List( - generateRelation(sourceId, targetId, "isProducedBy", DATACITE_COLLECTED_FROM, dataInfo) -// REMOVED INVERSE RELATION since there is a specific method that should generate later -// generateRelation(targetId, sourceId, "produces", DATACITE_COLLECTED_FROM, dataInfo) - ) + List( generateRelation(sourceId, targetId, "isProducedBy", DATACITE_COLLECTED_FROM, dataInfo) ) } else List() @@ -331,7 +245,7 @@ object DataciteToOAFTransformation { def generateOAF(input: String, ts: Long, dateOfCollection: Long, vocabularies: VocabularyGroup, exportLinks: Boolean): List[Oaf] = { - if (filter_json(input)) + if (skip_record(input)) return List() implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats @@ -565,7 +479,7 @@ object DataciteToOAFTransformation { rel.setCollectedfrom(List(DATACITE_COLLECTED_FROM).asJava) rel.setDataInfo(dataInfo) - val subRelType = subRelTypeMapping(r.relationType)._2 + val subRelType = subRelTypeMapping(r.relationType).relType rel.setRelType(REL_TYPE_VALUE) rel.setSubRelType(subRelType) rel.setRelClass(r.relationType) @@ -579,18 +493,9 @@ object DataciteToOAFTransformation { rel.setCollectedfrom(List(DATACITE_COLLECTED_FROM).asJava) rel.getCollectedfrom.asScala.map(c => c.getValue).toList rel - }).toList + }) } - def generateDataInfo(trust: String): DataInfo = { - val di = new DataInfo - di.setDeletedbyinference(false) - di.setInferred(false) - di.setInvisible(false) - di.setTrust(trust) - di.setProvenanceaction(ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER) - di - } def generateDSId(input: String): String = { val b = StringUtils.substringBefore(input, "::") diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/GenerateDataciteDatasetSpark.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/GenerateDataciteDatasetSpark.scala index a63627d1c..e1607ee9c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/GenerateDataciteDatasetSpark.scala +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/datacite/GenerateDataciteDatasetSpark.scala @@ -1,64 +1,94 @@ package eu.dnetlib.dhp.datacite import com.fasterxml.jackson.databind.ObjectMapper -import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.application.AbstractScalaApplication import eu.dnetlib.dhp.collection.CollectionUtils.fixRelations -import eu.dnetlib.dhp.common.Constants.MDSTORE_DATA_PATH -import eu.dnetlib.dhp.common.Constants.MDSTORE_SIZE_PATH +import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH, MDSTORE_SIZE_PATH} import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup import eu.dnetlib.dhp.schema.mdstore.{MDStoreVersion, MetadataRecord} import eu.dnetlib.dhp.schema.oaf.Oaf import eu.dnetlib.dhp.utils.DHPUtils.writeHdfsFile import eu.dnetlib.dhp.utils.ISLookupClientFactory -import org.apache.spark.SparkConf import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession} import org.slf4j.{Logger, LoggerFactory} -import scala.io.Source -object GenerateDataciteDatasetSpark { +class GenerateDataciteDatasetSpark (propertyPath:String, args:Array[String], log:Logger) extends AbstractScalaApplication(propertyPath, args, log:Logger) { + /** + * Here all the spark applications runs this method + * where the whole logic of the spark node is defined + */ + override def run(): Unit = { - val log: Logger = LoggerFactory.getLogger(GenerateDataciteDatasetSpark.getClass) - - def main(args: Array[String]): Unit = { - val conf = new SparkConf - val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/datacite/generate_dataset_params.json")).mkString) - parser.parseArgument(args) - val master = parser.get("master") val sourcePath = parser.get("sourcePath") + log.info(s"SourcePath is '$sourcePath'") val exportLinks = "true".equalsIgnoreCase(parser.get("exportLinks")) + log.info(s"exportLinks is '$exportLinks'") val isLookupUrl: String = parser.get("isLookupUrl") log.info("isLookupUrl: {}", isLookupUrl) val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl) val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService) - val spark: SparkSession = SparkSession.builder().config(conf) - .appName(GenerateDataciteDatasetSpark.getClass.getSimpleName) - .master(master) - .getOrCreate() + require(vocabularies != null) + val mdstoreOutputVersion = parser.get("mdstoreOutputVersion") + log.info(s"mdstoreOutputVersion is '$mdstoreOutputVersion'") + + val mapper = new ObjectMapper() + val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion]) + val outputBasePath = cleanedMdStoreVersion.getHdfsPath + log.info(s"outputBasePath is '$outputBasePath'") + val targetPath = s"$outputBasePath/$MDSTORE_DATA_PATH" + log.info(s"targetPath is '$targetPath'") + + generateDataciteDataset(sourcePath, exportLinks, vocabularies, targetPath, spark) + + reportTotalSize(targetPath, outputBasePath) + } + + + /** + * For working with MDStore we need to store in a file on hdfs the size of + * the current dataset + * @param targetPath + * @param outputBasePath + */ + def reportTotalSize( targetPath: String, outputBasePath: String ):Unit = { + val total_items = spark.read.load(targetPath).count() + writeHdfsFile(spark.sparkContext.hadoopConfiguration, s"$total_items", outputBasePath + MDSTORE_SIZE_PATH) + } + + /** + * Generate the transformed and cleaned OAF Dataset from the native one + + * @param sourcePath sourcePath of the native Dataset in format JSON/Datacite + * @param exportLinks If true it generates unresolved links + * @param vocabularies vocabularies for cleaning + * @param targetPath the targetPath of the result Dataset + */ + def generateDataciteDataset(sourcePath: String, exportLinks: Boolean, vocabularies: VocabularyGroup, targetPath: String, spark:SparkSession):Unit = { + require(spark!= null) import spark.implicits._ implicit val mrEncoder: Encoder[MetadataRecord] = Encoders.kryo[MetadataRecord] implicit val resEncoder: Encoder[Oaf] = Encoders.kryo[Oaf] - - val mdstoreOutputVersion = parser.get("mdstoreOutputVersion") - val mapper = new ObjectMapper() - val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion]) - val outputBasePath = cleanedMdStoreVersion.getHdfsPath - - log.info("outputBasePath: {}", outputBasePath) - val targetPath = s"$outputBasePath/$MDSTORE_DATA_PATH" - spark.read.load(sourcePath).as[DataciteType] .filter(d => d.isActive) .flatMap(d => DataciteToOAFTransformation.generateOAF(d.json, d.timestamp, d.timestamp, vocabularies, exportLinks)) .filter(d => d != null) .flatMap(i => fixRelations(i)).filter(i => i != null) .write.mode(SaveMode.Overwrite).save(targetPath) + } - val total_items = spark.read.load(targetPath).as[Oaf].count() - writeHdfsFile(spark.sparkContext.hadoopConfiguration, s"$total_items", outputBasePath + MDSTORE_SIZE_PATH) +} + + +object GenerateDataciteDatasetSpark { + + val log: Logger = LoggerFactory.getLogger(GenerateDataciteDatasetSpark.getClass) + + def main(args: Array[String]): Unit = { + new GenerateDataciteDatasetSpark("/eu/dnetlib/dhp/datacite/generate_dataset_params.json", args, log).initialize().run() } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/log4j.properties b/dhp-workflows/dhp-aggregation/src/main/resources/log4j.properties index 63cba917e..81458d1f7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/log4j.properties +++ b/dhp-workflows/dhp-aggregation/src/main/resources/log4j.properties @@ -7,3 +7,6 @@ log4j.appender.A1=org.apache.log4j.ConsoleAppender # A1 uses PatternLayout. log4j.appender.A1.layout=org.apache.log4j.PatternLayout log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +log4j.logger.org.apache.spark=FATAL +log4j.logger.org.spark_project=FATAL diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala index c7c6c6a92..50fe73d9a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala @@ -4,21 +4,29 @@ package eu.dnetlib.dhp.datacite import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest import eu.dnetlib.dhp.schema.oaf.Oaf +import org.apache.spark.SparkConf +import org.apache.spark.sql.functions.{col, count} +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession} import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.junit.jupiter.MockitoExtension +import org.slf4j.{Logger, LoggerFactory} +import java.nio.file.{Files, Path} import java.text.SimpleDateFormat import java.util.Locale import scala.io.Source - +import org.junit.jupiter.api.Assertions._ @ExtendWith(Array(classOf[MockitoExtension])) class DataciteToOAFTest extends AbstractVocabularyTest{ + var workingDir:Path= null + val log: Logger = LoggerFactory.getLogger(getClass) @BeforeEach def setUp() :Unit = { + workingDir= Files.createTempDirectory(getClass.getSimpleName) super.setUpVocabulary() } @@ -31,6 +39,51 @@ class DataciteToOAFTest extends AbstractVocabularyTest{ println(dt.getTime) + } + + + @Test + def testConvert(): Unit = { + + + val path = getClass.getResource("/eu/dnetlib/dhp/actionmanager/datacite/dataset").getPath + + val conf = new SparkConf() + val spark:SparkSession = SparkSession.builder().config(conf) + .appName(getClass.getSimpleName) + .master("local[*]") + .getOrCreate() + + + + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] + val instance = new GenerateDataciteDatasetSpark(null, null, log) + val targetPath = s"$workingDir/result" + + instance.generateDataciteDataset(path, exportLinks = true, vocabularies,targetPath, spark) + + import spark.implicits._ + + val nativeSize =spark.read.load(path).count() + + + assertEquals(100, nativeSize) + + val result:Dataset[Oaf] = spark.read.load(targetPath).as[Oaf] + + + result.map(s => s.getClass.getSimpleName).groupBy(col("value").alias("class")).agg(count("value").alias("Total")).show(false) + + val t = spark.read.load(targetPath).count() + + assertTrue(t >0) + + + spark.stop() + + + + } @@ -38,8 +91,6 @@ class DataciteToOAFTest extends AbstractVocabularyTest{ def testMapping() :Unit = { val record =Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/record.json")).mkString - - val mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT) val res:List[Oaf] =DataciteToOAFTransformation.generateOAF(record, 0L,0L, vocabularies, true ) From 028a8acad8b392673c9c94e49f15dca1c51b279a Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 25 Nov 2021 10:54:47 +0100 Subject: [PATCH 2/3] add test resources --- .../datacite/dataset/part-00000.parquet | Bin 0 -> 132914 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/dataset/part-00000.parquet diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/dataset/part-00000.parquet b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/datacite/dataset/part-00000.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ee59c325ea304b3ba8511b196a9c180faba78de7 GIT binary patch literal 132914 zcmb5Wd0-QD-aq~sNa&$R?;sJP2|U#z-o{l267Jl}o3zvrJHj_G7F z^Euz2_wjnaKGPNM=1s`-RITYPTdnDN0?wzHNUOPw=U9J`=iIT8q=0c-C0?RXI%j=#|MNp!^rFp26)jg>SnCs1up33grLvmRBG53~Q-V^Wz{XEMB2J-y;#(*DYD!o3-#T9piOU0Ck?%*8W5C6G&7w>WTf?jWh z=1~}_|99H#FID~9thtUghY7|T@WJSg@TI!eTA$bFb#F|rk4j0aZt5xzlsKHqXOdE! zO{bDF3o{-NMW3s;7s~d#v(OPQ=i;DF4#vbK>dJe4KHd*Q8jG^gY>bUY`FM6T>v5^; z_aJkS^Lm0_kM3~Con|STlM+%c?6iU~1zFDL)g7+#2{4{K7w`rAWtTO*O{U{9f@5HI zpzoZI(Y#onM#d=X5jlVO9n}cmcHU&8=yQ9ch%Gt@VV0e39Tdn5ip)Gr4Dg|Yoj z=W3|g?-N9xHO#rw57HxLbCO%~_@WUxCItmv_68+C7v()(*)PTXGAD@wXF!R&NYUe# zw&;DL!23<6Do%h^kYl0OzQvCQJf0v(XqM+WkrO?F>yi4a5W{(vz$j1Tt2V3u`1}V<~WfTT%KB;?`F;~xMf}veBPkP<@E?Lmsbb` zU6K%!T!I+jA`y8Y;PrCl8jh`1aoJp2&ahsV<2j!9WvU+_hb5` zN&#WOD~Mtu3c?%^0&b6Y*y~y!5&gQMMwlnwJtikpu@oB+pbqc)d#pae>mFN|QNkG` ztbIWYy0*53CY$*H@A3G|nn$~!sR5WDWiYPfSPuuI11dtd!^`t9>lxlHiXc^i#qU~S zSq9val&P$NxffXv%X@sGKcZSO%16B3s2~i8e$#CgK`%^Ya!`^b^^3@VB~`h_m<8>@ zxyDabRB>E2jEz@t-RG%)da^zp%X#?~%aAs}3FW%C?XL&2pn_nv`Gr7LFj&ov2UISu zd70WMctHMqhVQ-p0L*?=O0Xc&fuLy-Jn9zgPm`WN(5@7O((3`C^6P$V5kPl&-O?Z{ zM^i~w05#y}FRE{tWaT})A0*zJD1wFJ6$Ll62xd3vx}i$|6~(!SKq0WnTwY>%(JKf# zD-4p~?}p+)9|S~CCw|+kdj|bh+A-5OCO>{Zg^Cp=+VvzT7!r>XcFvhTOGaGV4 zE@;7sOMn)6Ji0$3@mGn8VAL0h`3IstuH!NxG)eFEabmFEiz=i*(8CL!NHC!Li&@}d zn%z8czl!tdKurWaAd$JDF_(1UVWUU%fLOO3MP@HNU6|TO=YaC_i|(O}#KJ(q>g0XG zH6Z_>t6{#lsU7+`epnUmvBbul$0KYU9GIb-7M@i#%x0rXE|rs$Bh`O4a(>q9^|~s0 z|99)tb`V)Tg6Iu+MXXy<5%7CB_dq5Q^cc2-JrO}y_{5;MKMlgv)_}~QJ^U~fy1!}* z+V<0bs6^rTS1z#^nhJeOrR5~lEhnQg43NjG+3=JHN(OC}il$)RK>2wB1Af^BHwRo$ zWY87#$2=}cjQ9l~7xe~Wflsx9fiwnTJUju)KpJL@zYMx1;0k(Or`K9xPI*_bUs&%` z|C#Z?`u9Q8#D^xG(P@XJXw<$8i3e7we8Dfb2pMDqT?@jY&eXR}st_Z=nCJ}(UXN>U z{Zy59EE*gzUBdZA-sAR1Gb667M>Hjt!RY(l>k>*X&gO>Xg30u=AZxB}xn~Zn5g2gK zMljE?)ct|Bn+Y?B4)5^?2G**LVAP<~!sJPMek3yz1Izz=t?6OSzk31XO&=wtf+aVT zr5|nUEu8&q*zcIq@-M@FY~Qc@3zzSjSh0p6wfh#oypkZv-%e>&>&e0TxoU>|inOaq z@=q-hwV6zu(ZTin37eYKUN`I7wM2mYc-cbu**}|v&#EgHzyq(S7OMl~fvV&1z>ez(CUc-r>cZZ${Jp;CBY{`v;ENg4Qux{Cbh#~7UNf4mv4&%2=o3VQg$ zC)LZywy}1#Ue!CUxtUyl+jZ(H?FBzBczCv~3Hcb`jcq9(3JF+~aW>k;dr)eQ6W}(`w%0CCc9Xxt(<#noC zU#-4V^`lqQqPq4WXw&5(%~Ry*5vO`BIe+DLb(det9T+$X=fBS$f*SMF3ez$FRWo$)^}96hk+02sRs9~heb%^|Q$5K; z^Ok>8{RR2dhY!GmI}J(o5%QL_MLk3=xMaDyMYScS36Q0??tx)4U-(;S-F5els~;wd z?i^6hCvVpN1ZAJ|&4sde2-RPx=8d&JJSXc;wGm4ENUQ!Ri@g1mqLOt6Tirv zvHf3S_m7``diUPX-`(==Pd|NL3Qz2Q+HLyiOzFQp1tsn{J?+hZ-9NVmp-qmA9G7xR zDw!Qp(pg7nlLNd>IhiY*#iX26_?sI|%Z`vkQ+9FIQR`4*a4u(av6RAsSL)cjd2_8p z%H=XjB%hPBMDzSKnE!@UzEB8P>i|`&Dd$odD17B6M^Z}2y_%&hU2-Oy0*?s_90a#B z*(oK6q;a8~&5k;R^N@}=H>8p|aDOxb2JRV<5=wkj=Wh&0z}HN!OO3!qX<$H!E9z4L zD0%hf)wPZhC=|L;%}X4cYjqag7P2QFQF~R6S`&{P z#0qmv&PFpzx^kiVgK~rJyozuno6A509JOpY`l1<1S<>0s+r*?ZsSyQkW+Wz`R0b3| z#%z#AnSoS>iOF#V)Q*xItYzSqWKJ0v{rO8Yl}NzNQ7Nakevy!p@JK4N4t`=bD7hh~ zu`}$7NLeVUCoO~S%`z>c=@jU$ESy&Lv{bW9I38yP@?gf}Olxlsla<%!3o1IxC`krx zgg-eMJkY{S!Y#>sG%lxd&;}-}tI;!snIz<`(B_QHMC17^+zhqlQjC;_{%lmh$CTrv z46z7hbr)OX@`xN~BBR7i6enKZ2tCcpIVLs0#71HMlqeHZz&C{InGq?TCyrFj;aDHz zV!*4a)EzT)s)j~QuM-zGwN|&(LOW|2xH}S;neYf=bZ=;3s1iiNC^3zTT5Bsqwa|Vg z!{nrmsbq@SYVVu_4njON2*VX;q+~LcYwFaTC1G-_Z#U+1^?vm5Iq2C^Q#ktL+I@PHM8Ml%)OFs%TU? zuz0bq4`uBZBO}2CWnJ%~EcnMgEihqFZa$ZSfwdc}U}!-+VB+%3a)uVDeU%YGJL)O^;krHI&Pxvmv(| zL<%C!e2$f6H$0olg3FK@Wz$3HYDRLB`K+w|P!Gb9jgCKOpN*KrOk-{oM5R{!PL;e- zN~Gg3H}RDHP1D`xEJ$t~Mq#!Q!Y>G7iQOECX;R*pa}CN#Ia4`d;3c_50`VHnLZj14 z8l*ibXIbK9D+AF>PEIgH9dWtjXxc~S)3M4KVk4oFvTCO>nNLLIO!!yEbUqSSqJ7s+ zHV$S|8?x+1lIPkl45QDiLh7K*<9Ja`$9<-FJ^YrsAiG-YqDzMC}8XB1T<*v?#>dGVfYys(5 zev3Lsf{cg%14YN1iMXbMajydvC&yjh(3`^3sY*!+|BnbR1N{z~MgQOgETKw`sYyzIBmX#2Wlh4LorH8T`8{s+=_~ zKY4m3l6Gw89MIgHd!uWh5QO8s8Zw=Z>;9l}4~Jc7HSw}>Lo%L{VrBE2%(vGRYnNma z627MR^r`vK#f6_g`8XW$VJQw~$WSg+7PB)EaZRYI7}fsf=3?E5V`H(Zma#ojEM$?1 zIh96L-7M20Co{@AN39{pbSm*I%y})tT9Ghq!J0c5ehd=KlwhP?*^qN<7>722gaC` zfS*m%#34kKzyAf&W^(yV`MYF%L?%w_=)z^|sUgI)fo4n(G0kjwoJ=GX{00dv2YW1L zk#6Rx5mXyynWnfhCPg3$D<6gWQ*kg0AsZ6Mkh;)3b4O}ON+cv=zUyggy#$e1qLwMQ z2x+7#bY0s9KZn0IB(!pJFelX*U0qT6~fyJf|G zq3dDPh-VvAL(=hjjTweYS11xS;Oh*;Aw$xf2RjI5n#^MIU$u2enL+%BPDmjnSN9)x zu?z}rKqe>j4P+;qXa9`AdD) z-c&>Z%emkkuc|pyz1)}uuL0)9jzXiTY}TnQsG>(1RI&r$XrxB%%mOJ>aDZwVp&SXD zewjVgra7h^%gO`g8}xD_{&Qc&JW|OC%#tVLq`3ohPbwkb_(5R7P%0wDT%GdpkOUK4 zZioBWS?JUki0oC8(pch7W&7_=~xrpXDc% zN1B5C=$pZaZ4AC>ZUJjpmtR4gmGJwn zqpF2N_(3BY3fJh$V7N6iP9+fuPqKeP`qWS=pHsg{(4z87k*aVKKaLH5)TiR{R0994 zgfN8$5HbDK03w^>amrMx8$#{2lUf+%b^ofVHKQa)@thJ=N+9BAA?nG= zBLY+;OCw4S&(avm#Kgav0IOB1F}#Ts{LwU(q)?h--E?SQSz%6{20Z#QYRA7bl0)&@ z8Wz_XEk7?NrcsWhGSU#!cH=|l_Ec;T21J;S#Ls?_zSxE|S9n0%J(^lKx52mwJWH5BlQ8_taKi*jnc*K* zt^3KevhMU05ctF|0Mj>ksC?!G)HGu3&V#JRSNu!*kC(9L>s8&O^n=)B7T$)0LP1Zo3babw1O3No z&5jxY1kiFShRB6-f_l(#_+fpIl#DC*pVKsSXb`EAar>W7TClba;$0mB{4meLb`%zYg=R`7Jj+o3S%$m{KAZb#zl$uZ_$zA=AN2liM3Z6ji)xqY5|c&?AnVV{1#ITevhvXH3|Vs_F3ZY8m?KSAF@sD$8`B<8rZR3e=yJwB z2z4TPyw7N|Kqn0aY0-Bm(IJVQW;HLBl`PoFGBc>|pACn(VJHL6m%{d0^h@)ZvB-f# zB=m*_3CSZ_ElejlETXzdKlr%sJ)jT1n(f9p?YUq8OMME zTCa;Lap)O`C%_T2v?Hl_9uLgcG7hgm=v8qE+$2Hb=L8> z%8~u3?UPE){F+mh>KoYaoFhF5rSZ90^xrFqY84kGzQE5`HhJZa%625!vw&fL<6T0(JhA^v?l;x#bXk%-( z3pD?t0ir`~WxdG$rC^>^??9#pJ9@iUR>Dv=fc=xB#76vDC3HAri8}eG=sqVcJIB=; zJv9JbaJ*l!Y{P~PtQ^nE5bR`FKv3MdSi+spVsZu*cWT{`?9%kpw^cgcsM0iS15b^S zN92RMxxrAPESH-Cwr` zjqOE-I@FbTU7KNA8(Rlo`J3e{=MJaCJFD7(Vxno5dZu&QEaGqYbJ7%bH1M!68hPkt zpOVb4OF>|5cM*b}CKseeuy>kXk-$w7?1*h75mP>km@7Pn3k!WSs2b-i#NY!4%WwJ6 zg-G8sBq{4^vNI85+NJH3@>%K~Yx7JbUV?Ph=PpLI3qf5C0ce!ZI>y>8sC3z~I1z%2C5LgH0O=Q?)<(uUW`ar_KZ zcH#VPDK;cA-H;jupJhY=4LMWXibB<>`(iLS2P?xD(D`E9Pnx|~O-4EA1;t*-xC9D0 zUBAb_S*QMNYq4c(u^#f~nZX~j#bBKe3YKL8*O*FWLQHRF6dboKwYC@&TwsA*2yd9B zkiG^VHf?Z$?~s%R@f1wPnJ!D1+cmRhu6M^g=CY*`c$4CzmACFYi0J zOsxz3y-3B2;@^w9{#RlFaqW+vQ%M#vkL31zq{vtcI z_s>P@_RgD8Sv&u?Gk>(-9okqV*P@gmd)#3VBC#Se98>W5)yK|L_2lr%V!YpPnl5_O z20LgyyEBH2->OV864;xT);>yyxD@&~%vXsqdAVMeyhiU+(Kbjhn zssHd2rKky?E-_~S&|4~vSkgXypOVc>Sg6I{mk<>P>Y2rPa5wS^R|BQ;$ZP67aNdK;zO;Eu%-KZ%boMILdwR4|`tZO9mo*V3_Njp8aDLic zYQKk|*Z=hu6@IUnFl$&9AEd&*ia;+1kWnBch!f4F*Eg36wn_C@O)))qdvhuAhPtKZ zccnE)#N*xgZ>4p=FQxC$Gy*9BGQd7aph7Mgl3zYw)6seX8Lm4x@L2{`0cjyJ0GES) zfj&k}tV3JYAxJ`U0nyiFg0GQvU={u^!6e2CkSX`kyUB_5D6$^e7tAkY`#dfY()gUu zBLrOj?vIIyV+4PU_yElHNFo&@&gxmekM-jd0esvL>Zc|$h|M4(?)@*y3eZphPNVuX z6U&H!Wd!F5xB=jXx#u8rJmH&ki*J(VJw2pG1A&PX#Lg20+e?r^j-@81A%0@cq-}F1 z=|-#)>+=B@!3#|rS&yU{I|fKdM^6l>t{G72A@l4B;(c=_43n=hOhyqIlfcZNyXgu2 zVj~!eKa*1E1RbH<0RLy(XQf}DtpS16&t!pl^mbe8Fgc01L71tssrM28qTxPKE z?EEcFbS2`=M^M?k`u?J_>BI^Z5Ia67GaEcRAErsZmZ9z~sb60bT2gA;SHfPXdr>L1 zH1jn|DdwBBkfj-8$z6_uLuCy0r(C6%~woRn8qqhDDC`f zD}B)1cE=3Mjpgi51VD5z+B!Z}Yl*Ka!m${jep!I>sjo`4ivX9$hg)rv^k2;JaS;56 zJ3CwZn!0S%2Uw=+$h3tBsynz~y%7t}IK zG@`1tS1mG4jjf&aOS%`d+!g3dm`()s!ZuUnNGS>EGky|R_TOT(3p_ZpjK)qPajF%yEv zW@hq%2Wd?cri?^MIDlU+HB7(q3EOW=rJ?$9vT0Ofp&w}EbM$=Nx zF*xy(2@cCkrn=9kP%Di1c@wZner9*siX1jyBVtZIZ>nF6kRmQ%?29LUwBG)smF!Lp zzF@rXDANPj5P;keMOnlL)H)`-u=$8ude{D1czyE?>b@>Qyjl%3oB$9Sc;Dvfn$PYq zS?(}Zj71W1jE!XzV||ZXq#T=3OP^3Twp&io_Dd5ZfXp1cEPRaZskC;cqJSzXDQbVG zDx0P@w>7_30T6ntdi}-DZ#o4Y*_*A9;b1^q2B{(%?^|GI0L4iq(x+FUEd5ivF_i%D zC4rSnyt9LZQ~WiF6rC1j@ri}>j)f?O=XMz30R@oKRqe*)hypeMucuW~95{9{%1Ohm zLkqPVAc`Z%szeH$&pfzxOhnVAdu`iyYP88Vk0zcm_dw*KBwhG8qle^}lBB;Usb6<6 z5%2~WpvUFtcc*I$xjWp`t_3lvzix90mJYpK)#fAG4VVJuz2{K^MQqt_9>A(BA!0kK85kSgkUe{ z4+;26>uWbmv`-mppMv;-Rh7CAsP9XTM;5=hvwI5uX(rv#Q|1M43f!MmK5Mx~H_TV! zsLA?YWWKNsRP=vy`FLIk0uyTYJ6L*sWEg|^yf$U58ZSS)I3Dh@xG+6wA^t?)dcECN z^GSsx4E<<(&&qyMFRXL0wX^3YQ>68 z1ZH%IRsUF{#yiW4T+bf7nPFqH8?rX?MtrENVjg6ta0@YQe4Z8n&jh}*R@=@tu=el7r9~PWVXo^ds+(P* zo=@>+Vnuisv4$NhC(OgU0C5CSlb8l66FJ_5YN-d#t2bhy0`0jLsoL1Deud0ksTf4T z8}K1N3b)N5Of&Xe0uaTyOAxk9*Ft_S*=HdPwmrL$-MG`J>3mtW{6uJmrUIBd98lIa zUqXI1Lt~kt0dM4lt?q^pPHO3amF|Byc%<`s-nQX{nh zxcd)CO^jMDuCe{D*z&q->^ZYw-=6gk*sipKOEQkP z)1=GG;rqMs0heVqi+K9)ddK4xXaL_>g(9?THsz#nQ4atBHLMomI}g6F9C2;ul}>zY z`{84o%Ht~A@4Q?Sd5Jtei9Nrh?pP@p25P$7^krEOO27OOM2Jgqaj*SJe3-O6QbLp% zrc+BRD<30h-!1kFnq90}gp8Pr=dsNRgsKqFpP9!7(1eTabFqSDQvJQ9;H3P;c6{Xo zsrfyrNpw!|)8I(*(@-mYyVlAqSqPXJAO-!X)?#(m&D4;Pk;R|ja2rTB`~MJAcg?7w zHI|2|HPfdRtnO}i6(p%+11Z3>k}`Ff8{4O8rAXGjMnd}N6jGdPb6{Bx=_i$+=AXL z6Ehmk4Eq~(8BJ!jh7woPE&UARSPp5yF>pvq6>1;JZDL zx|H!f{q(U}+AO3#E$`~Osualh z#T0sQEt0652H@$HA$(ZV2>C$ZI8-$*YFNeyUS?gSnqI3p`msUZ0e44nRfpVQX{eGe z#VISbmlA+E-Gi_7%l8o;L)XH)J|XdRodo7Jt3;7TrI7+4yhJ(OQCYV zUJSxN+!L^UwN`guRh9up0w97}c>_LHO|M!@{bTK#m1yE?^w!sqzEOWkJy`9Ujk3Zm z$bK(mYBokxsdV@&dVY`uQiC{?Ae_1%j7(Pwa-<#4;rFQ*Z=0(1xdadH?pJxlzIOuG z-!}ErHP_rawGYxh$KSejY6b7@<%1#c@x9I+#gR9Q$6pd(EUWulv1OAH90!yRy;e-E z(&F=r&n#+wH9()LB_+G5oEJiNK-h}`*uU01m)7xHl;J^5(mvsZ5_v26c)!$OV zwbBmiWWS!7%``Mj+#K9;a}ZaWXsN`=Femzpq+8 zA1^=8a$881bvwsN)Q`tWV?W=Y8t50cl7xYjlKnMX;X1XIRQ2a9g;U#BlI%}s99v0! zzto?RY#ZR>_#f!PA7|l%f)97ceh&Cr`X*z5P8a3gbpy zSa0J>%u7@80Z57CWvzIR$;60aiQq)UypJlh_{(;@uibgoOm7%~g=8w5OQlo!c*!?dw0B^q&QlOE$&|b)DwB!e$yMS;V=mYqF zV*xjZqyh=2-ElP1p&P56jPES5%xNs(C_%3uIJyGA`RbyHr8+pCUaA8GCD>@q0KW+` zycs-qm;|8rI8QxpqkFY=XSLxu+lI76<@oQOSX~?dED_iSfN}sJ0tlzAUYU3)h+hg0 zEUbWpwDX@ciKpILRm=$fUf^^HTxicX!HHGHmQ}@6zXpGxdD%deNRUIMk_uwf`t^U< zIiay1A4wY9rfNX#Z=*DN;DNaF2d%70L%QmhwhO z+RE|aa{BD-iO0&{f2@4lmn8Dp?{HjQU%x|T{AK&XnV*I~BkhmWoF)rY1c#wSc!d=H zF`4X-r`G=iWaw%ea+k-Q7SbM3ubrUC+bI$?IJvez#lXLgVIV9+`s|#Vu-m_?`}-7X zny$`-%9nG?<$;{U^Z;EhHY^<~Dgxw!OM_FGvAlA1A)TI)VvtEFw~7ZL`5w6w5pC=5 zi;NfHk0SVbiD)MH@Yni!P#^0Ys;I#xZbpU*A^>7g=_ER~Z2=`jJzA`8A~<5RcA*lF z;A2fBr1HuOcbY1?w;*Dx{0dOLV$>fN)~pOC>B*t%iWuNf1J~NO`6cReqit$?%m1=% zgKE&~vbqd%i2ux^m~;&m%#k`II49$)>m~VVcdh6LsCFxx9nkA=3?|jt}BCecTHV_b5?-16>{9jpN>4 zZ8(`rIS%%XT!~Dxg)0$oXii*?jE&&sJ##Z&F+>7I;2JZbs}Xi0&4T?G0Exx&{fH{- zu-`**Rdb-c?63>`P8aV28k!3~>xcn+yr1{OZV`c7CcxGN9$3gh=KZZ#F-|k6f;fXM zBa~|fKAu%I=W&N0Yc`-^e2501j`Ev8pX0GA>Qojt&(cO=z2f-|WEzroA7&w9+1fD_ z1(F&hyBYXNP5~AUIh$O8G6nr1Two$Kr$u^hDQ&MXI(otjni$Ii)3Ig|evY*4&|r5K zoa`$pcBoV@K+IKZ6HL{}G0)WEfQEv1&n$uhQ)#JS8HpYpV=JYR1CN-(i*;roGE!s@ zWHessz$a*GkVds<-=#}w>Mxck-Lp)^zVDA0X z+8!$yz0Q5e*aGoQZpeAoG{F;a@O;~KXu?fYyAdyi&0WA0NJvrYC@q)j3@SNKUAbZR z1JhPWgoPb(&a6FfEuL1=`d6YDT2?ut5_lY#huZ>b<7MZZVZ6Zq~-|?HF?@|W7J*eQ@0dbZk_4~RXM9V@J5a8*{o$B<#cu^tLbo6b43UKFo&{^ zkFzRA$o9sJ&@3ZzbWej1iZ-Inge~;Roa#1ejmhzVslJNVFzgl+?%kl`*aM4^`|x74 z@u9`)yCJvfx{9u#@rNZSPF*ow2Og*Bpx2~O+oMHW;t0rAwtvpBu5^e#pXe1G%|@qr zM!3&1F)dRw0u{IR78p+yl|QaUVdsx|Ez_Aws?C;(ydI)DrWJoV9VOMDP}i@dy*g^q z^n>$vUq~aDB|8G_N?>tDK|MSjr3Q45Uqmd>X4A2a)ZXcqtwEG^9vpGbC5<4q(Wr}h zegwHF$)(#{=6EQq?dfV<_S}APz8-&TMzUk+7QB^o^q7A<53Kw3n~Nr*MZ2R#BrrEE zcW4k(|D=YV41a}h$Q{q1HvB69^XZ-B>BG2xs!B@lIH_gAu@O9Z3w7c7+KzlSiu;UO zz}RIwK{hH`>^Ofs6m)j{#j%kKUyBCChA|kG&c`goG^zwIaZ!IS?$cvSYB9Z?#^<(b z!HIK+E+SCB^gy#pgZkO$;M|5Z`hK8e_R~o7voS>)JZlsOM6 zTaEn%K5V}Q!Q*1#1OJ^QTojzSZO@codK&d9X}jNW*9zOCF3UYE%meiX7qu;UCJY=PoQOIHWWEOHbKc3~7XVC1f;jD=3bZo5anTn&^>Mz;|h|&!K2zGXC z0>wMQL4-B=0_wJHI)meCE75BUb-eR(MK7eg0E_o}R~*y}ub%(Pmxc=DBPyuoV!+Mu zJ+n0LIHbO|KBNk)crAo~Djv9cJ9R*}Z1_z3m%c529>+PsuPAvHS(B9{NL z;|=ehccv4c8KXjY;CWr0AMHJ^t7?Ei61?WvEICJi0>O{W3;;Cp%pH=Fl~*NUmY%r_ zUo%d80_X;wx3v}+y})M$b}nES^s^2y{^?XYAID$YD}Xu-i8v)&?Wliyd*3l#-`!e$ zn=B<=)Cbdc-a;$l$vx#H28bFRf@~m z^O$xCJBC%c6u#D>jm?6Yduujms#`U{6C{48kIjN~tj=bZnfNRP{G8oeh)swg3_u3~ zw1xUXt9y?;k7?CGrilKn!wL-IsFIE=^bd}@C{o4p)IL%xC(CvC+7qM`K{t^l54~kq6#Ur7}3^ zM2>@hOuW&ZAn}&-k^Jpi9sibVN-U;87$YYoRt8FKmL15zrWRl^XOeE>Yi~broU?%8 zf=w6zQ20+V`we3ks1e6kK6K@-FnvQY-hL5&M(?B?6RXv5oLj9%9{iHo3OEn&Pkn&{ zLAKO&cs5&sP>`rte-y>|_v%)wLjqbH-o09_SCWuduKs=BN;NtttW?+j`(8)t%0*?q zk=5$XAllon*49H@M!(E!JHU)MK9($m#;G<5WmCcaO`;47&n;J*S2-F%HWuPd9x}-! z=>0TZDI9^2e)#3Fe^voSU zXC(0QVxXMzi}m3VNr_YYIP5eKWt6_OZWg-z5ll~Bg&ek5du#_W@hVa#9>MjGV8>o| zMMRx>2(Rf^)9;R}hH~_GBQ#Pi<1Vd?xeLluo zLZcl4t#f=bb-A8uzFfUi%cSEH?!VN?tV^agB=J*bE$my!(%MUbX8{Crq7>hZalLl# zTlmvy)oZY|`DWa^vwNFmOsWCq{0sbvU;U>yVO z?x9o+2&}0MklXmF66J_Z8d}9fsmJy}-4KpgOWHP~xs7EXqMW_in2KcO%!vAS1~v#R z0ss!2xp^A(Hf@T@n8rVO35pA5b4vWovbB&Q}1I24^;!gUNw} zMm|xwh>Xc;T(KH6Zg|4JN;jVbavUDnqe|s)Zf4mgV6O}oux4dUeSP%{_FItj~m{l!Jx3pzRhdq?836vG$q33t=?t^#wvU=JQJq70r)k}P-<`+`UbzYF90Koq}l zxr&L=UKUUpVAMo1N(?`Eg+7x4eW5$41&azu_Z|{G(e^sh~=>B!?ghnOX7LFIHYP;hC}xgLKl=FW@0b@{$75|O&oZ| z_uqo(Rh9$yQ|I_CF+^0O4#z(mHEi=woUv2R1EYyf3Tv7Ov}j_UFfdQ3UkS+?>Ux3f z08jq4%l4nOO#Dc|A^u2|J&v-UU5_-^hrT6tuR_f|gc&xVf-ByE!;4Vd@{skjeMqwp zvou{>Mhonr2W^fYDwO^_?x~JFH;5)5&e+hnvToJhnXFcpER=g;Nk6z`x z#VTD&KWC+$wl>?*pB|*1wT8c`Q>lnuoqV~WgNRY{8N5?NHlz~upOiPhQ(j4_6{pqo ztMxc-!u1B~K)I$8xt$+gMYFSrJf1#-UTE?qY%g6!+yMFYSLlgHTV8y$h5mX(#W<<{ zUKGW5WLMlaZ|pmd{^xfA(7#?aTr}J00OSq~)V(vH$C~o$Zu^rU3lGg(QBq!WHwNGF z#7rtx0xsy?nBH&2g-edRv2KRWvj4T0Znz9`D|VFVN!e zWHV$9Kw0H1>oNXTr%KE8`AL|(AM!v;E09p9++=&Q)Y>dXfuoyRx65DiSH16tx-MJO zu*FT9Do-zWSV-IVRY1jS>S_mT1p=O?xB-;4RSlqO`-^y;JKO_7z(=$3HIC2~d2Qjv z3C|JE!3Bud7)DDMr)@tqLe^k~nHK`0%Nw9~M**LGy@{Za5WZLC$nM>ehwRnJI*|R4 zCm7<}O5+o~3x4Qb(6=8&t|9T4Cd;ccDxYI-Fxt+NeLV$|*=N=CoikwN@ziV{ZzP)t z+dWRrblkguUd0i1-vRp;NFj^~ECP8xvA_(74RX7^%@C@mN7bD2!P##eV!3G^5qsT2CZ2LMmpSYtYMXdI4~OIVyuNi&7t_`Hti5p! zM}mttqXZvt(=X*4|GzJv_|;Xz9C@0hYAby!A&LQDF6%oj_8_A&7hZxeSH0XgZ=F0k zH~PwqjS*fB`hx>L5%fkxkojmVFu+5q+82b~LOu`GBk%h>v8&85{%KoYxJnS<2Z|g8g6_pZm4NtSXNMQ)! zv-{{@lf+Pgot3Jke%x0-bunxg_+c2SR=N#Z&pfoHdvOyV18qN)XQW7M9j%l<^R zD|!4%7i_;s04r9>cQ+FD=5P1Y2WHd+Jjwb=Vn9;`c)92g`aGM0^#( ze5!V-yY^|wYHrXBh+D-Eocc{+ld{79z17d z5KEVkjR}Rie|-1$0ur?y!H?S!z2cHeweyNgMlK^z2cE}=v%)r{X+1HH=Jz~vAPIoz zmq-5N#+;7Zp=|mGr9L;H6XXyt}1mYo5oa zOf|~s5pmMCm(;|?mYtQlwz{|O-J4X^JW(kesG$z7Jo7;f>s2*|smtzgK=p3#R%0muZ(rPR6GQe9y{{? z>$v{0BLaHp@IRCH`NVJT?Y`|^@wfM4x|iB{l+e_CcQ4}+;Z+`mS7JDfM_A7hqaWUq z;^zq)+Kd~tHNNt%a`0XgHt6L&;))!CcYg##Sclw14%O$7Run})T-FvSLIeR9fyi(6 z!fSp4Jin&!JG@i_i(~MEm_tUM72&lGD^?NK01vO`fOmonA?y>N_HpP`rFlaGlIwl1$b?a80jPc z;LnZR{wHLF_kFJ zLze~UCmx*OJii<^&O@EpVFu*f3p~O#mj&u8kj|t^d74^d^H0}UDc}XHm`d4)s$hpFn^$r!@Ec$MAKh+5@>qWCozi3@n8o{ggn->lP%OW9B`xdd^=n!L=*_JhG9aT$`wr|#9dkw0` z4_~cHR#VZQ`eGd(EfdZ>7V5pkH15#SZU2FUI-TalHU*(B>Ua#v>fhNBe%w@k2zEW{ z4q>epvg9{tQ3EZWM@6PnJF(@4X}|!gk3oUjh0)> zYMQjVIpq{RmCW44>O=XYG@`ls#mCE?jV9tSidi-&RX@KgsB3W#*WPMBzIX?ITZg(G zsYeJ)**wD`s>zIhbnMNP!(LM%0Ppk(K{AdP9bd=kOC7|Ok*O;%VRzWD?VZYfudMlx zMYuJ_=8*HE?P0@7m$PpXYAc`8I*QViYcK1cRCiao?hVHUbJd^RcrP9;r{$}0wtTUd zI_!guUi?+IJ>~cw%Z`!I)unhUuQod_Z2$`Aw%7OKT@6jMw1xK>;n!}&(<{2SHXx?e zsM(TamlF)aOezuufggl-?P;(Zmz1@#6k z3wcw#ZkdYRh%Xn>JWM&jPndfLnQM(XmL}BemG{id4RJ#Xk5ZR_pKV&&=uaL zi@MtznFZagjg3u>E~)*!o~B-=si7s@;e$7oa9n*C)7#q7~XHRQ=dy}iBDctL7ZeK!KnuWT&F3~0wp&n|N$MV5o_?fV!yvqKesv(XK-A&36 zJ+yl*2d*Faq+)dDeXEauLGk5Cw&)%smLDiL|La9nbluE>`l3oN;jQ~WjJ*k1Q)e3f z{T@u@B%Hh?hvWbO;z@u=kc1E*K-6HuqM(2RZm4L00AULzfP%YKTdmsK#V&TkR%hx| zTf0u{)D~?!wVf_@TD$7hw%X~U-JP!KcOS5w&iDPl>-t|;L=2eq&AU9$egEzqWW(}} zr4Z;A@f%VJ8q%r6P9uGXN~2vr!LWK0^MdKn(_fVtKRcEbt+-W|W7nsw%6zD^DJj~$ zH%(DeeHuHmgUhkuDxwK}%;RUr%~5{`@s2Jj=Wv8ZH_>RvSIoKDw)SYvI*3dR z;#aIA*3J9*>U5Ze4CXxv*`Df>s_Y;??in>uG`V^F_uYd}89h6HZLGYXB$jz!XIx3q zP+0QUicDAK7H*{b?F;f7&JHl~o_7pCm#Y0@If{BW-^rrZK=f69Aq4oQG15p7P< zbc*AjZI|Ws(YHAnd>n6c8wPgWC){kbai+N+&o`CJ8u5L<6b)8rOFFWkZA#`>EFO5HK$>04#qw0~^HTSe z$ev7GdiG?R?%7I32R)uG^Mv^W*~Y_ah)t_Jd{#E;^)xxAM|);@ROb6LISHIbsp!#(C^o4d}%lv7H*5(WMt}LdvucBv^JJ*q0 zHdO6_2=#u$jVda?#dO16+fI?ha_{=+lbN=q8rKcc=kKSE$5mR`bbG{7$I;9L>}h+t z;27N807I2lr!ozncK2TD3DhDKp^ti^4*Uu zr@xx`C6xYBd@k4W94b$x)AY}#22vf~Uaq2~-a;=r%A#bUK_PdVx=x&7t)MH{3PUmO z)6qvaEl-|;k`zQeZo|DPtZPcPipTnvL|?l?GFZ+($*=XQB`wUL-r}YGuh6Jf^nqL% zy6)RTFMLS!TRrrYhhCheRjcq23@f020IU z{s3+LCmk2ZyFJDRHLzj+F5^P_?$oqEOAG3LWpJab<};_6ONvQ+c7!{QDl}Q{JIumKZ)bV6mZS`H}~x=K?SDwe41h1cZ(<;ur2NRUKz;WXka4 zM>i9H;x2w~8U4bO1zj{CC0*el`ib^VA9d({rWiu-1zwhTSJTI&ADmdtXYG!Ftb=jpbn0uzr@RU_RChRFK zo9?OB-}ZW8HLjgiCYE_?Ju@n*vVy#p)9JhtC_{BK zFv)x|lZCD{vx~p|`ej)n>|-`DzaR^8BAW<}b&ee^!~sA3;59A0H&EZd_D+ectXa72 z4ZKJj1&CB1C33v@_lx^@v7V~lRzyyQacMjmHoycDA5;D# z=PPjh$gvO;xgO{*8{~JiVJhwThT}|j(cAcsux69(E69H5y|>ubF`kp8J5TZNr{XDsD!RFZ7P2+8Irs?Dc4|FG{!LPEfxRdPD*oJ}6Av*rKKsIyCYTTdaFAB}B=?V}t1Qh~uTrIXrn>7@ z?krPc`B0B)pgGFfrlNImT8TTi=7mUq675eCI_KlV7V^y-NE*4P%!30hzdhw_GH+3t zS_R`2*Ga~xa3wQpQ-7!}z;{d`)0l1+y{NtGQW>A|b~Tjd6(TL3V0rk$H@4deAJLZm z9i$82&5RSB%Bzy(dM0*CMRl#mH6L8OK3DyMJV%aVR%BSkI&<>qhU;ZBO6h~>%ZT!2 z`ZG%ErD?69V7EBEr?;Z+wIPMRzDg$6>hsDJ4;!}~ZBvj!dJl^Vjk=iHD$F}f+u9Uo zY!%qSvd}MIY*Xml6tV&HKQ^mRY6>x_oAtRYt6 z8qhV^Nanc(zG>dq*|N73dVo2CnV%_b`y9eTW#m(B`KMa%6;bAcCOtW5OV%+q7CjJO zFZTSmmOy(^!$XHzU|{mhmrOQKD*e)_pLn(6Ieo;$Kcpmq#@d6vQ_N!0>wR8JpRISi z9GwQL-(?`DpkBElLpby<5qD(iZF!cN{E|sRXR+Zs(>@#D+Ni(JM2c&_G0EGt3##Xo zc&eE_uJ&m6YY$i7rj>gW4)djs*7cy=(>c7V@;x~dZ#=#E zpKIS6@BTY+y*xfG4~3h&T>F}g_=_bbeTj)Q8_rD2(KH-O+I?z#PMhsqvF0cBZ2w?r zF}-fHSY)k7!^Vdm!@r(Np5+IoDby02*K;lG@E(2TN@n=I3$2>X^d|!e@{=+h@dC^+ zn+u2HqrX#RptT{k1Tj$z1oR1;F^8_N*M>ly>guHO_tB-*OXLJAM3!T3Zwr+k64TJ6jAb^A&fyd!juZ7e7x8ly*WDab~68Vak(9xU+oCUQ!(k}$mD>H4de;%}g}w83Mq?X0 z^PKsC%;lbj(SO7#)bPJS3psHMo9AfoD4B(t!iX$oo-D;t8s+^wTC;l2`i3RZHdzYv z-E&cV=ig4!6yMQ!C~M5LWOsn%m+RLz?IO>HE_RXysCL_77y>6WA57jNhYf;Sh`ac7 z5tjL;0kXi6R|JZ0*k*6Soa@icDYWPMeXTzJup-h=Aj>xb@WgyhSLL&Ir<1qEMS94s zfIQ?E6nXxR?^}BazvbxgSq}K6K3Z3o04o%%sX5!8-SVkBiiOgMK3_*O3e^R0Tsi2; z8-y*|$fsogr-ZfV+hE~{yuu8;!hmJu$nNOD9;zrW-$q3}uHN_xGfU!c!bp`{m^;eq z#8WA-MvUVIjN^HV`e#!tZ3a_y=qb$T!b@6>&!GcSd<~ z&-UUw)P;OE*~6o7OeY|#8TMS`^Q<$^$}id!xj>pOkQw>FwT6K`%?02wKOgIm!%pvg zn64O#JQ@{xG>Qf7)&M%PjsCh+i>K+#cSLSEy!MvEGjhPyh2Owh2xOHGUb`JgeL6W=e?kHi4s%H^jIWcKl zoUB59hWhCR!~@T z1L>`P6w2*}$_p~{^EC4~uYWMR@ip{9;VtKhdaiRG?l%LM;e3;)| zLkz>5sfJ7JFut|_W}>M`Z=;LW6S3#hztbxwdCTarj5s=u*R9n5x?sapCc$Nmi!?f4 zXmlonKAB^M$4P%AkGauUN2=)dS2CJ?z}i9gR|%NM!@ch9wJB-LtCctNj#a#(oPRZj zX|0Z99CnHHvfv$g1C!0%$jyc^j2f@hCCXZT=Gi|IQ8A%<8wsIjv?OvRgA10ne5`6# zFn*m|f5w*UYxK9e1a3A>{-S(6L%M4JVsXz;#!iqaj>s8Yw#gZ>2+BAsIGF`bIiqgy zwL@!i!#-eM=p2}M{QM)u&Yu}oT6KhR$1;iKp*Y65J&M0!t0auOw~Az$U#XW>`)Q~V zMOqio1bc`gohfje#y$J-zLzs!YJfiCCezJYwDm3y+CFmhIONW(_>4m6unQgb!Zk0O zBU$kWvf?w*WEyW=&zB20ov$q7D#GGye*JX-HJtiVhMm#_$=X0mKRT2A>+M{(q+crR z(u&@$AcT=u*=Xl4H2W85z(+uIC8PHvR*1Nn;4~5vzW1)3CFotxn5PT;8I|o%`$!J` z{y*rsT&Dd&?ATI2N=t+d9^QZO6aMfcMi3qFv|-vT$d@OvqZoiR2dH4;DC0~Xn>@-m z11XP5OK1*Zard{#r!0`L!;*^v7GmfWP}P$#&-MmpqIwi_CZ8(t`o^dL!UBnP)&kn! zyfG9a-~gZ=m#$pS=G*jQMg;UT^^^F&}_NP3rw5bNmj{Ytg%|~ z3}6LKpM|E+Dga(vjP+>xOaWK`RAtzmoGyuVkdqF6IU#dZFGI%nE=XQo%)6u_|0(GV z%i;kgc55$_1@_C6i9aPz{%6}ajD!}LUX=|joaIF(K|5W7)%lE9B>avUP{0D4vSsQb z3`T{)eHCPn605xk{9uNW<>z^3BEAW>MHzS4O@i zp>IjvDD)VpYT)Eo7CL>iIIx@TkuVtulk$t`p~)|@<(TA{n|!Tt~)ng@yJ)>j%UU zIUpN2AglZq!X){(u&n0d_SSw&)T2x0hI=@$rwb$RFi*V0$Z(!J&*s^kk@IBZdD2ys zn;l#U`CiLqz;tFrN9Vpz(un5>u;vK&YE zBk1hki{gJ?p)(@NkBV|qw4p$F0Ur5? zOBQm;Iy#xiE%Jd|(!FN78FrQ_yit4v;$#_&bMFjP^Q3Ll!mG4u?ik$pYtSqlEGAMsy zP;R5L$dl~QldS8b@x)%#9rzSeO*1a z;BPyY9Dh?wOS3OhHEw^^I50fU7@lE;adDBUil(Uwch$HwywLfWG%y|_*~*RC$`k!~ z&Y`><42+=zGSeaz3x0NFk!oO($}^SDiPN{l$qMsFPV=`)FQ-v9V3fJ$N8^c^{>)-H zeLjBZ^YJq1!yS7J8sMelPxmy_KQsgvq-D$=( zFA9U~@y!uVzLt{*_9Zx-Fec*O-(m*dV(3I6t;5zKB_5#ouwk1drp>iQo@duS&$>@B zERJc`Q<2xXq1QS2pe)D6e7-q1{mo>s?t#CywMI&GYfE&NxA5$i=j4wW*FUKt_U~%6 zg$`e9Zd0TtdSgvA3vs%pKiL6$Rep0zP9!CEASHI!&4yJPX>2&?%}N#(<49HX+N$Wv zRfH5UtEAIyL?_>=lb0ROf1h)<z7`k=*)7kp&QCEDz#$*W)ve9dVKqqlt z6Sy(bigpaah^{4QE6mlLL~TrohalRP$mA)8B@81YihqEv&{-x!nUUWpk~(Y5O8jIL zNe&xZi|Fl#LQXP#OYShAIBQl3(;zPt&KX(Y8W}g;XjEg(YYC?N_%e}x5kqh)P6oH0 zUSVXhzZRaKknkI?G2VJmf;FZWkVVE9NIzd~$_a58e4WmHOlUDNPfs;28KbYrjXxwv z%vG3G(~sW@QqDp>bB`8NBM7hZW+s+jn<_CbSg=m`n~lu!gxOeuamq`-CiyqOu{<%^ z=7#+Lqz5c@gJs229rS|H`cBs%uw0)%rOoRFwB@qfZvuDYGm5~0Dgjbr@#qES7Bbx znnyW8d~VVuf0vWU&&Jk-%j9R@KluAn)!;sht8@d3!k^uMIR|2>LemU|@xGsk?jvZSQ6M9jgF8Rj9E2rVTv zozNpnWT=wRU>GfUxUVI>AGFV;8s@DD5m(yHgRUkCv6H&6O^Y!*a5smtdgku)#Y$D`163@;f;oU}wE-3qz zVB;iF=vuJp4u;$N<&(cG!R{EEN^sery6zMK#w^D>);#2g<*F7am_ZR}y}m%FZ#DKf z1*N0Ki4}80aNztGH95vrM!p#%ae8!CNC_m4#?Y&}AS`BI&~)?(c%%7>w)|zh!<^-MJNH)rP&>UbnE==hACLQ6d%R6}s{qRH-Hn z8+1YeYu;cfe;E>~-7i}ltrl8m0X2g9G>S0uH1Qw(-Xd{rrALrg@3d&H=jU3a5z_II620ZG73SwB@n5m@OBQPH z#=k20r#SOpm9{!1w5v@LWtWLn+@{~D)D(Gs zA-&f7`MrwA-*N66hYrTR_hn+qh?4&%OJ3mKq$CB7O-l7>GLAvTCjXX-BmXLMlTv2# z^K%p(o0K$zQFo_(p*z<^Y_7^?7=Ph|7f5w$st?~Q~- z;a`4Y=wYS4ku{biX3j|-G3=<9!OXpYwHd}`{z<7el~k0QJgLbokSI%NU0Viu;c2Cz zlVxr)Jgqbhuo853hEfH^>iJ_z!y7D9u34>rpOxlckY8m`cs+%oz$d zAXuA-&ce9QLrhd@IH+KzYc?DHssR7gxeoMECjb0u0#9qa5;wKVY*N}lj$J)!X1GrTUj{u-kzS<`T9oyY*t$#N z!-J>?<$WKf8Zh?wM{*c1yvUcW;klSz?ap$t^-$&qF(c{ozUMmPxV-W}n>2~n2GL>?fmoPvNCuar z7!EDBE$kR~EL>R#mBIR_Tl!PcF)!;ZX@Plp^Ujz9)*{qV}L{!GUW@I!*914UZu9 z2{_4+-(53q-j(=>^+ckhhN7dU!CPNBd%6f0cadbShn{GM!n-UM`^~cjjmu(LiFU8z+nXVSHc!-w#JV zmTmZW-;m{@7-*7OL)WD@^l?>ypyhjN1PdXEnSyYh?J~)M~*8WbtLo0x=6ZiEq#%Nm2>-(nhtCF{5qZnf-D5eLJZP~=VfJCm)4PBwT%m#22!pTiDJ!&|@ zbFA|BypQP6_S>MVggK+jz%Pjt@ma*tq8nxeO;X_>k?`Qb3)(dPu;_!|O#;7Rgz zsQycxyqMJRg`n{XPwLPveKVQ3=sKnFotjw+8L~=npThHWoPuKDa%>$y=i0HLHU2hD z=36@VVlmBxXuATy!9*{Oi{&|8W~XR8VBk!$9v{6wC2<)|GU;&=20$QBV_z!GregpB z&r(CKhAb5ZRQMu&M@PThlmtRRu7+FM7hI~Bt}*9o7;i#-&eGham^X`|W=eU-{IkxJ zP8RJ;9r#Aaq@KU1qfK#eqMq_HbI@;kIAKlz6z`g5f%R+s7kUt&gyaIUJotucor88ljCes?w5A~uE-=YyVq{SP);v@ zhcL#O#wdBWbRS7OuaIs-|8;5QeW1*DR79nf<6#W;<8kzV{VWLEcmiJeea>T8?5?BZ zWZocdn{7bvl9CyoZFnlKjYOWp#8)OWj>;%`@x>DiSx5^_GJg;5Vm$t_?=&>O@+lWL z5Z2qD)gWhx_qrT486xR@)k;MlD!&IF1JRzEm!^-fyMbQ%Q>(z%40GMFReWg?Cvun=mrb ztBa%i()E{AZ~u%tF;CzW?~mN1Mf8MEtWvd*z_Us8Jp+@Z-)QkEv}34QdXnYfxkhf`x|{Q{m!yfTFjNj z@H8WRDSYUtYYH93j3IxoRJJxacqYNP4Qc`)D@|3XBLe0?+f=gL9KJG@5jBOTLtMpm zq$_f6rf_a1#79lN+?;t0SI5}S%~afhePz6_4)p=wvO^ap#dI3xJRtYnw(#F) zt4qWgrmXZy6`su4SlhiZ#H7fuO=dKuNL6y)&oE)*D#x!Jwb}!iMO~(Hx?hC0MZ$ag zWOCcUT1iEZ5B7}b))5X(mlbq9<;uDOXhksx2i6fyrep0G(E+}5AxCMv!8AoV!{n6! zDA-2Aq!ED5r>kpyF_g`A|~!`TVpdh98#*5Eu)-BL8#B+*dhdf0z<8$e?gOgRO(2W@!eV z8UJ#OK04m;l|9*n>yXxDQavZ)7l9@Z`&YVy#GJu0S%&v?kdLS8)xZ? zmZ>8LIftXjWb+a0R}hNKGOe*D+M={$pomO{^SnWSjWM6&7@j_R!hr)(GGB|Y5nAA#xq0E5R%qWmJuRMQ90U2?)O908 z&51t>!iXo6wa%^c)S2(j-MMC*h&3}Ge~PSujubb>G)dF4K*T!j~=0}_&0LC{OzSdjb^|mDu`|@gu}mlHphQ`4Xwf6sX{F}v#DFe(FzL7!na-#KM8@(40?Sk6&P zC}{A!i0?@t8nL<@==b`FmN0i%PvGmTux4 zFY>b7Tz+QqSD&$17WwDBwy|r@@a9{TTdSJ3HlhnfuQ3R5ai$k}td*-{_%oUGD2Jcu z-_aza&uDk$7rA(QrQpqryvXB{`692ogE&orxXQ_LdUqBR58<3)`()YNo}4|Nf(6qd zIc`IZc{(m279}@&s17d>mf0~yTH*z?`nllEAEO6yV(&Fw&i*tmd zx;xBIbAaP1mIk)s%BHk9%4sq;u%^EJeV?^{KS7IFC2LO$ADJNED`ZdjE?zf%_)gk1 zLEmU$aomZ7r;S}Q!_#)Q#W{bO0Qlcqu^gg;ZS?OtTHZ+P!bp6iIj*`n&a*wk;mj|} z0U;aP2!0~;{HE~<`#!#|c^bc6UHLFb&U4;sTpBm94J+W5rEz#l6Ox@rr=72iqlaP# z_gG_{I%1x1S;If_P}!%pBWAi)Ii}zDWf56ug0DJ%??f?$!Abj(W=h>#CIo?_*I;C zHfC^ti19%SaVPeSNz%p`V5}Pf#$xR5U1UE3D8RcaB124>raukQCEdNF$wnzDsG4h1 znF7>clD!}_hEO(!Iv2&84QeSRC_pU}oRBVDLFDO-^V9W23%9;M+>YB>#ZC8h_-ULc zB?)C2g^|fc=?B3Sz?viZC!-o=PGmtyWqlBrR)M&5X)RIL2H_RuADFE9PMjI)_k&4> zN1B9%TxGn345dnB)`k3o;w*ooW_5yBik#^iqW_9>B3xx7(M_z~L_A-bzE?J$*Ht#i zT_5Ps(u1G~s>3$@#cdPx+a^Hd{D3*KuVDASf{Yrt&VrJH9Zs`Xxdar#d&1oT;bW&% zTbU01w9f^)`Ly?=5q`#2tI6ME`P$S_H46ldkt>pgA)S{`g?X+C;A}&1e%ZLk{{w^jEJ8 zrtmr)m1WVS5;CB&XoE=ef}0(}05M~px6~rS;oaiz7HV^ZMgWzNaxg`IRHgVO1Et`u zFaiQQG3nf)OHW`?iebK&bE=8yX$cVUdMhkh%ogHHm#I;^@9E2#s5&SM`6%wpA5!f} znTySzhjib zD*z(TAf4o5G1W3RF*{(ChixfL55!LWDH(-xlsGZ6`ZZ+=(~G6Dz>orW+vq>K&VPXS5@_gTY!KV1O(sL{qkRVeIo!kSOT>HX2OGY~)+?iZ0* z6|{&MVOy8YBCg=qJA}oZ=nDw9mE|Fl!QUeeKwu8zZP>$-R?kcRYq$Ug6FsV;Urd5O z#JAF?@S2vSQkp<7mdScr44cbQ$)RtR85)!va1CDna}_7}|CA?VEn}qYe~4`>KPdI zCC{2D8D+ZhRS&5`Y)opBKGs2d9m{ta@ zpJ`EEgxZJsMur%(EU};DQ4gUK+mMJ_M)kZ=rlXLjai}{hD-Te#Mc)V z>l(C7mUrmZkTUrWcF@-@zsEymbvEy+aTwR#=5D};Y?f^1K5Zb>gpeKgT|_n%?AQYt zM;^0BmY*wc$r(=B;BxCqmdbN{&$&-LDE^tL! zpiN3@MTxtwtk5$Fby0c#hPutq8K2~OkVKgNClDORTTgouP+Gna=etFbB1dCG-1n~? z?JOiYjHt#t3;eGj%kJenIfx$<5=Xrk-?>TH`QzQZpX6X22PU^0EQwwMo)gq866l@YMu|hx9$6Cydv7|h5d@?Q#$0svSL6LaP zOyaG43SoZq%Vh7KDNN$PjgwP;^sXqtK5Vutp=^xGt-n+dnd`#k=vqjupaq)%sZ$kogRO&Kt6P3Gq&kb=fAVV*YpCq=WuiH+qWlT7=QnHu_1 zLRNuQg0iBN>U8BMq8}Hb)kCzJubfGL#U0S1xeiYjMA1T)FtLp>8cxMEep1)~5bNYD zs!w2McsSzF&l+IVngxcHi5_|Pd2h82ZqM^ZA|C3hmZCsDDTLaofF@Wah%(lC>pQ&x zSw&cJ3x{57J+I4<-vkZZ{me&j`jw~TXqzE|u@~sC*L$5LN6B9jEYUCgIg$1&Xs_~# zrXrH}=?7VScOktd#ZyKfm={>7dDXlsoxZ{A&tysYAfK+}}~wMf6Ckg@Pp@rKr- zloj5EHE2;i!ZZ0OquSAvOGoa_odf=&{1bq$>gO5^-t^_)HhOF5zv4}~adcLZzA~0B z0Lnmi|!%6Nf{y^u6I! zEJ~=aC^+mgva5FBvw56$D}}x_#^-jvkVu^FRW7%{B-YJ|nftE$ldQkBy3Cvu`$KQQtQD49qe)ZGA^$kG6jFTbtX1_2=_sN;#w&{ zv1t?OxI0Q#=i?iaIkBcMIP?#f<31Vir6I>5DdslZw;@|0;Gr|gFYm#l_JusLh5i1<}_L<-h? zcP5r(x_=pq&L7X#x|Z9W+cN0Jm`rLFPG<)W+&;lJ2z!i_1x_bVN|~(6!Ti!tBx~;5 zIvy20{;ES7$f$fJ7p zKat%se=`^lbOEPE#k?mdET1%%*ex{1|2ML`>^#2*=W8T(YsV71eHwzmZtYlLH*L}3 zbyCbm3lpcGX8Lp@hYg-@EVUchi?FE)+UuBv-{ZQ0xeC(`C5+S=v^A3*&FhBn3zae& zV8L-~{$pM@(~P`s>TjBJIj~!Lj*H#OsQ*t^w+I;v_L(>t36~OO*ifLhy3MsPMFt?| zH9FA2dYDrv`=c)&rcWhvqXhL|bOb@&9Aq}(g>x*il5&}DtJJoLQc^d6PmHV|u*BAc z%R$}3!^@HZO&g8rmOny6*`%qem9FLKi0BPUV*M z^waS%o*MxsASy(1lQw$P!T-9KCZZL_w>Tc<$zf<$b9pru6Y9G64a3b*33 z8m1eMS++IPlx==xso9ea+H?8=5IOU`!|!3ILV{W>~OnD$>K!7-{>^dsf8Q!^>ft5KRX|FTuztX@fVUxUk>mu z1mu_FrJGqg2U#PV4`I*uNi0+2+8&FFV&-t(|0`blV|FZFIw89Q#Ss(or5$BUsntPM zRs5)&)qodhXsnvYifUnNb;>TQYVkl@+b8Nm7{*E;+}A!U~mPf81-g0Z>4 zXS8vmJl@epbSWQnjHOB^o@6~;19VwCvVFV8f_UL3mtv)PwtMdG(O7AA0$%VJmm!2 zxfe$M5Fm~It?|&4gioh}HJ=5Yo9@-+{0R-J5;bY38Y?1CFXu?hJZ&^ysp}?HiVMm9 zPRRPC{fGf%b_A&)Pg{!hZKG#6##41ITiWketcB{>r$`jht4Wa(rPWfRbXUsY9}=aj z{*Wlm&a%~Qs8oGYh&E1Ju*KJhy;UbRB&rm|pJE&#WB2!H>6BGcv~=@-qop0|Mnj*0 zI4?_C!OiY(_tAO_213W!K(NhYkR2SR_KEIIarl&d7CQCgxZgsgjq9yX*#9R++Otn* zbi6?vh>?B|<&r+lO2lJz^7ka-77@oS4Xxh;q>WC^hvFX-r17qehP;j?NZ-<@9!-#@ zbL$kpM@UP#;mj1(+Z^fsDG)%KEWJ zEzY)=q0x*Wb5zu!1*H?(lLvxfZ2WqI8=FbPc1{YIHl9=UCd?d-n9ib8k{H49B2h@W z)iO^HT|{+l+-S!1_{9HaOry2d6gU_9IM7$ zgUQ0+c!9Xn$i{z?h9fd5V0y!UtWR=Ph*F~yI$6|eRf!HH=>C>6ec$DjX{O_UL#91S z?P$n!57Y7Alxfxz=<<0~Z;(36qsiD_`Fndu$Mf>b<9RG?IunhV6(DF>bx8=%&>7Do z&mE5~C4?yB?rud;C58Md3?FoXT*cKR_M9I{mg^9hJeqLr?%*!xN>h0fqJo2CQNbAG z`>8evtp$3V&rW@NJq*XdqR>|-{~=delnx|aQm!=pDTgVqdRRSM3YHcsQyn-%Tuzo| zkt}WVtI6z&9sd{6(uOf2pb3%GM!RdJBT3f6HGN=LG;Y^&?msr8Z4>0+OWzc$@wZ zEiLm5(DSUOn?<-ZeNZDG3zt6B0*VxJvJoIa`&d<*Cib-}SQ=$#`@=)G+=SN}iPH5_ zqBKU)ghNM&ZF=10P-)7tSX*fK0$I49E}EwL-&E-*`$zUtyHasEQQEDNdCG17QUCu1 zN?&9Bfvxwq(9#0Bs!8#CptP(C@q~%GE^N5z_mVyGzk$;HW?t2S2y+P1`FK5EqV}z5iQ?^yhK^4Usn8Y0)g_qyTC9l#T@SuUQc3 zF`6Nbl76V;_OH}4vDxbT}Ye_u*4M}V|!$r{s9DI7vrD7;fc*RJP{6RL+M zWFD77ZV|$mTVzLaba!m!v5ceJugrqR-EV7wCI%e~{)G(C=3m4Y+q8=PS7uLh7G}5f zf_GR@*j(T*EOZv|r*idwn}|ZC{vY_Q%(cMg_6S>y{5=zdU3q*UL)9ne=mS|<^>7%r zNO+Mv(aJ=#g*K7)t*3=Ud|k2r8ZR|lNJyVPB%2+gNgDdTF&SD@*d^P;kgJSD(L?U! zx;{ijqD>&LL&;5E&sxMq^xOGrNYa+LG`Ym;b`Z82klu?Pwglz#j&yp3SGHQ@r=*IZ zL6?ZOu?U%7hTkPt`(YPG%sXQwKKkc$)OHo~WiFBaeWF~;&o7qW$%(7Q>9lY%W;cUE ziluvhfx0M9O8fhU-nCw4ot^%O793{q$>bgS$dm;52rvjmQ`LRUMXn^+11Jql!@~WW zld-F*=xMfz@(R@0`>YkEaBm`d*`7}ytsqX`hvJLtl!acA<9*WtG=}*tiv#(22lwB~ zB#1E6<>Xr;!_4oO@?nN7O8<<$*hQ95^ET2~dx+r{_bl#VGSs+vmne`$?)r(OW$q88 zxoo4)DqQ8EA5Ape;DN4=l*+`pv@?UWl+{BHz+1E{&^!~n`DsZq#A9?6qash=%b+WZ zNl=j|_kp)OND==wt?ULEYxTcmP**WMx0PlbmapNjvS^{_N3V`Qe-&o~#hxA)=m%^4 zNt`vvjcQ&!^D z!jTLXpd7=r9;=)FX_9e{JV0&8okQIm@#5|4Xp)7l&#JCw5+ItzEFuca7v=Oqwdrvq z3))yun)o-TksuvEm+qe=e+Oql{o(1xskHTKd9SU9!Erlvy}XtT$j=y;sm`!y%}~{5 z=22lVGk+C39}|mm30A~ad2DHk%@UMb6h8hrFV3y-WQo*{#_%@MHk@wkBNeofHR-LS zME~mwp~1{_9=sak2EbQ!;jODEDJ`-L2T0D2SHO(vzB+xB@VAe^*FfQ+EjfleA(L?Z}o-7iMJ0#P&d|e$i5AkxYi7MLtAe z;3u&l_0U&iWbFZA`CMcx(x2AQBl>PL>HThhdM>YFWx+0Hp}x6N__v21%_bp@m4DOb znzVz%X}p(iUtDpob3c(G9Qcru^iZ!CwQ-L#o}th0AT@pq(?{xP(R5ZKBW=iLv&6E@ zH!B5ewp1;gm_RpJr#g&{H?~;*yxC~=-@c9Bbro~+-u?EubrT_jhqVQ3Y*$XLZ(sM* zX_)kSng^k~?xU|*aT+~7Pd7ngWCw1ys8O|O>!)|wF-N2;Dm8vjEA{ljDbCe9GQC!&%U^6Z6?R}LYkIp$d_D<0YRgK+mGBGMEE|6OtUM5|_rV6kdO%s;dfXTy#d*UMQ% z-1bV=(7dU`DQsQ4pT31FgnRL2Xzo-}V{l!sz`H?TkgiR%X)z;1hs9remAS}@bz{*~ z(qszX#3x(5wNsgE4&HY$EymMLU$9l3Vq$Wh)+7n1CIR0iyaaQAOIT_X?z7RN7^KD@ zwB>iICHe`hi7FP3Df*Y7lugZVoXY>WjI|<}{6LYy#q^y`F(o`9?F$z%G7`2c?Nw%?tgpJ>*m!cH5 zbYu$5_kG0`rBickKdYdgg86?cQy|YG=I-5ODt~^ez8t7e;VYK80E=On{J5T{g0sLF zeQyK0*yXgO%Jrp+gMLXgf2rCsrEuQ8G_yCBx_77LA#}A6k*kN}E-sJNFONlo|C8d# z9ksjfs6}ad+iVu*1=y)qt0sd8pR#7=SWeqPcL{$W-A9C*`Wqw4GK!>*mna1tfS@P6qi8YuG@z zoBvWlqqYk1g=r|1BNwSV7!Fc*4L-48aC|ruGMKY_MF|?;4Nq7fYHMhgp;5=b!k(>R zQ7%0faKJy~<6o>a$ITXwu+XNQ3{wj$gIwX9lGntXUC%+M9OO%O)B9>M)j>lwJ>8og z&<_{NdVLnO%K4gYP~ya*{wer~3qXvn90Li(7a%;5|Ka}M~?%nl(mYhj(JEoOTMeRiv3uRV?9)Hu7IO@pyd44InsiY0Yy*tT83mz0o1YrzLIB`5wxO}fJiz}v$G8}Ctkrb(X3*YzUDgeK znvaaD84vO`t%m93uXa1}}%{K#}n?4%(BD zF`Ivuh-lj57y9PX8yIqkgyiQsF%4WK|hO#jv=;@=AsC*XF zkMS6!?Ftgszf^cj1(hQhk`^|RL1vccMySmNr7yEBvA5|AOX$$@HAn zBK^XLD*J2|8xmU$SKg{GP^mFau}C$bs}IcI_Dv9XsGMb+W!*mdWD(GHO#bG_<@@{( z!EzMxWaaNGYf<}G;=D>9ik0FG0-7iPGpLpSKvvKz_uz}2GhZrWcC$Ial}0maV>}!7 zb$iPAdcQm%6~!S7=jYUKXvK$NO&tkZ01LGSC|8arF=H-j&Z4<{o>~fH5SQy6hh_Ip z;<|!pp-Wlm){~ty3c3I^;fDc45}!duHet5M;@Jx%lSuV zH{CLul%0H=o?IYItJdFCruqdd_5NnQQdvGw9l34x*yV%UW_$R^-8E0#UDIOoA-c{3g>5+^43zWqnx?Y>q17!)$%FZ>T%@GF zQKG-TT|b4I?CR%=HH}xz=F5u{!=%6zpv=?9+BN2@W@~e$Xm5w%#$qD2wrl9;TGKu5 zed99!?pCOwEut%9MlF}Lf12>3o6m{m-~RL28xFdd7~@HrJ(d2f5k9(?5e&yT(@D4S z9J=D!1oxvG)GUZxHT$}&W&>$NQ`PiMp3BQGu<}XkNXQMl%q~-2mEoD7VSpF@qVD)k zz3uK|$X&m$;XS-?7aod!`BvytKT``?yy?nf(-AdYhFn)(W(#_Mxp(MsKJ(@p`JnOZ zp(l7}E^%GW1GxkNwHhV%8z?>>c@ z-}m>w{?`?Yq#?`6S)TWK?)!5~SkoQm33ZvW!MsQfm&Zn`t-ddA9q%yN!dxxBfe8A;RyA*i_r!bQqTB`07GQ*fhC${`8>nx5rYX3u>@KqLC1$siwIZl&&K7#v9Pgq! zc?Hs%FV2T*a2Tq=Mp&hWSXadaJYj6kEPBZs3~hOKY2XkIrh}Tj?|%CALOTAKeBHqG zQ;%inAIs1|Bw@pU&8AtC1GZS&W}>PauGz)21bVSdrl zsz&2GD(+S>hnWW6P!<=tJbMxGZJ5t^>4W_Mt=L{p5jM-) zH_KrwHvUibN2jysW;wq0V*@njUQ&GR>8zm6+a^DXJ3XNSx>#OU$fO15P_14wbKvex zbL7a@d?JN0AHMHQ+iPH%&0scFH(qdJow7mH{5^L0nMzWrzv!Npr5BJ&?hP~J94b#? zR)Q#utkK0^+mmiwztPc;GZV#(dWDr3y z3u%{LiEL!MDx|hYdb;f(b+y8r#k7`fyI@I=e!(S3owL;1j!A`CDYjWD-e#W6FW7e} zy(dTiNqX7A=>@khvuS0UGW7d?VM-Ff4FiGZDE6$o=e$#i>8~ZktdGX4${Vl3C4v6= zH3a-z%o$>xeMO;>Xk7nL9s5;=@j7JGbDdgL9TDykEvk#+dL{g$Z8X@RsKVL1j~+-R zC6n8=5MJYwNvHhN;WSWd=Do$h_{QG*km9ALfz>|aN*yZxBW4B%lKH`4R7W$N&C>5~hH zYjOd1{#Cfw%%}gHD*W;Ow|~4}wu@|epD%l7df8VzY(u**+Z9V!0#N0$?4SJi_6=`+ zk6Fs2`{)Ig`UB}Qn-ZZOe!731)a4b*uG&?0l?`s7x#{=oM;UOL3PP*R@6DP zx~zh$Rtqtb8S?{5gSr`>eOO(Kyk1%x+Ol8$?H9xkucaMo_Bf_0VEx;KZ;o#G=4hU` z7b45V44ojQrn81Ha~a#QsHr?{c)y-SNn>Ky#b+?N!Bt_r+0ku z#ASINSDw$29{`cZA>_Q0%z345hZ2lxzpz8HX@|snf*6YNTNrh)ucQeFnf8NBZGy;w zTolGX8XEto&ZG1qp+LCr{jvMr_kIehe-L;=SJBCrgr!XTQYNBRgG}l1z-%u>$%ZKY zslzWMrVk~i&z-OII`aI%TcF}DvD24Qg=3`s7%@Ff|9KoxDxXi4kV9<54yb`YM7A?= zY;RY5D|NU9rzGi=7{K_m{A$EJ7f z;!;e13$%xBz~7xhJCkh3@qx5xd^SYQ7O}_olbB#i4VzNI!6Cl!ajDNCyer%Hu1xCp zGgB{mQ)=?2RD;9ewc~s5r++@8hOWcq64K?#bUC`Z^(w;kg2-B;x6V+-zdC8+TS7UtS>jO%EMN1 z6zAhB@jAFaWrOe4K13A9)n4G^y~3|4+kQ>ax*)ptI)&d-lE0s5w+fVeyk_Z-0} z88S+|2h!;lmr#hfWEEhIF8gq6PaH@uA@urK=ym$_AzdthuN;Tl$$i3QlJ?6axy62a zvBv=vSh4WhwB&2kf>}CH5}iJGNVw~7V|V>6xRNZs;4D!uO`Tv8sm2Eow3A+<+XJ1x0( z8aH`N>jH%;;1r%^lFu^F>|~B>9X9L>ej$$=%i|&x2MW-jI0%x*h$1TW`t>>Y3&*F8 z9iOK3`h0Hg*&Bo|(%wZ}H^`vpo28K>`Abl-V=C6NK{)6G2+2UwB)Yd|OGkz6Z<$&}v0EEfobkPQRd+jOitC=|ddfiXgn!PK>|l zq5;}8TMDG+zcPitrVRZx1*ma9^F1npfE#WLwqsbhSlWKERO)fk>vrhe0E>E^c6!IX z;FLOrdnCzwBs(sW;#~!P7z_Nc`6Z<{0LH7uEG%H!7ckfj-1M@)2^rIp8PjMDa_Nha z6on7s$)m5Tl-RR;g3*vP8hD6i?friAz=hM(#!gSOEKsAo#rbNEY-oF;r4+I=5sU_5bP0|UAaD|J<^M% zhOb!YBtxCV?+VyO@?A%_Xrn213Oyv*LvnF3!FdAt(r)2=CV4)SzXvutfYIJ2$+yXJ z@C9+Iz{L{*VR?R!c-9B+)sGE6w1ANv)8Sko$~y6iL)ZsDCAk5oc-;`;yJyeO^Uj?b zMYt_ajZVKy=p#dYWC5NTBsH9YFc>JFV$wQ<6J+QFL4wJDFNGll%-9@Wm&f4|PLd5L z$%^}=K5y_o964XGAqE(Eyl`K`1ygcZCft#%+Y z2b@Y_zGP^=g!u>a71Z9i)B>jG!8_X2N9JCn2BQs^!g9$_xnyHP>jsI?835!p`G9y% z34oebEZ8MUy99$U6j1>Cl`&h&7#wB}kQ{KbPQF7rMLsQHjjke`!bK<_=(X5pxF=8~ zBvQ5|Qjp#5^|*b4S&}qM-ns#KW*|5E{F67xf%wKo+k}f+e0L>}=fwSie7|Lx5R#9D zssHt|Kdo~zf z1Pmtbx{xvFVN4AX@)Unlmfv7L2V8^DA@sj+YW#n2YVOe0ILC{4s<9~3Pygrfa6>?eb=HcPWeCbKs#$2h`lRGIuxer0QztC%I| zrrss(i^>)xD~a<)HmqW9)NI6HEYTa3=fX5#J_lLrWBwPOXBJQ=XkiWZX|5BIYn~tf z%Uj#R=&Z+7vL$4#2#je8#!Virc`9ATbgAW_m@Z@K)*M%h*)na+XH(fSwiG6Fb^{JX zoT>32kh#8M{J`1GEc;P(r@5PwqB5-zO@%dZH!&9&&eGcILyBt@fG?cmwOJkr75q?P zuIU93W-x)Q0lahM>UKxlcyS&X;TuqZMqA+qK)-i!KfYN^{4Rhw0Sp4sM+hWMCqESc ztvF9*zX;BzL~n=amhR<)tzg0QK$_UjH&K&fJhc=d`|Pc>La!`o%zN2hwUx~LqkJlpO}${spij&(=w}pV{`EmP76*Z7i}nW5nNfJKWxzRN z+T@FB8f5yyQ8Zey6IKAj7T^3z9tdr%u_*SxzFt$U>>c3M_zc@01za7U*UmTu3yyI?}*3h&D-b8SJLM3$vLwAcrw>|$Qx=nn6y=kh! z$o*O21gBhKa$4qzNM+L}TAY#rm1Dcp8RN3|0l&Nj-aE&&7XVGazq2K31mw+BN={&Evp()NV}IAzTx13Zv!g=yM~ zK1jw|nUS*y<%`Yqmo#alJGjz%gMztY_hEK5GS%8z>FNv}z8P^1$R10>a$R$yv~uz9 z_s;M`y4oA*>WPK0NOT9hf)4IJT8ioxFN>5rp!QMkIE7kuvR19?6WIn{VLp?b&mhGI zQ2rhiH4e&U%0bPp`bBtxX@7zd7gjjY0UhlucIMHYx_*Z<-|a4OmI7>45{kAJg?eM| zGQ1gZP0w=)kr`u=8HFfUdLamjMG$T(Y|2V*$^ym6WuNlCLg;$0#vr zdml5;RcsF-Q5-E2;9L-1+tgiT0Edo)B*%#7;<EtfYM8l;$M$BP3Aia1h zA=@yPZBXOu>1qw$M7%;q>R3jq>H`E30tAfDyh^%hl{8ljA8-@}uzWnRoEgG9zfrg0 zM%~HRX@|bh?RMlj9r=hda5%649~&ZC(6hT*!=@E#v8cxykV&>Mp;2YjiGsB=E!mky zA2YhUY#3Gbb%8GpRJTjeDw0}-SfRGIwN2b9NAW8trzMr*+SqUJF+GCLjT5ACH&pC? zx65aFK^RWiGMqvpHFmn0($CdEYRBS4>&6GRcb)u(@9VY!0q#NtKR{vwh>5m6BGX|Z zhCR@?ts#xxtED|!(pRv7^*%mRr0o{ivh*swXg7_Fig;ZvLtlvv1IX^GZa7qX;~+=b zocba)D>DED<>>1L$OKMZF_>U9yT*aGWltPKVW@*JU<#}CM>Oo-tb&pZ_OUGHa=~Xf zd+YTX$OWd=p)7Vg%e|0cg^Jo&Vl(KEDV5k<9@a+p*$i+KRHKXisyeg5dS0gfN;V6M zDUGpuCR0lfWrsS-uP0f>t~ILQMq}R$+pY}3YA{<3u0QF)o+?_B2=$s)7&qv33-Et{ zOJ(c5tFvT0<1AaN*VT&vmEj0H*`(L%*3TOSEGpV7Y67Q7zJ1;FdF^u1MhMbg-~6#n>U;I|T=92Jmi0aUQx# zISrCo#quRqG&W7a$)T^^k<+CT_;mN0z%GsIk#!}jpX zMi2^i7wkzBXYTYqULJ{^m#ts&#Y2CSkVb0DQNqVu=W3S9;eZ=oCQ2k9CycrNaUy&o zS@OxmduH&^+9tln2eG|pEZdt>@}4ml6_%(kSjU-!_l&q5d(Wr?P+7EyvoDx+=cW_f z{WF(}YiI17JgmnH7B%Jb<&Blqd_B#`1q?6AE;ewonPuyZQxbvGbN++2eermq(Jj{XD2(x=>K^AryfkXR?E}T33tdTs+F)9>6w43v{eK z1WarU!_~Bz^lcO4zX(OmAY4u&gdmLi73`kAFt|}evX@FlJ}ne#%bMml+3SN%LCpqf zk>4U$Rzb>_eN|p8($k0M7tQU=f_WIcpcVq&`Yatc9?514q0s}g9$se{u9-qPicHfq zHP*HmcV_20MA;6U_ORT9V%^MbTGaTk6WeYj{MnW~0KT#-^SUv8g}f*s(u<#(qNC`y z^JvYWjPB^1V*AE%_)l5XF}DT;LZ%(^5ln*zWmZ&aq1ZVcL|g_7No@loild$qk(JCl z^fb?-))Vbpk$6=lvl~jXHM5mGu(R|xUX7!(JHn+|)OZV#nCmu%2JruoUq!q&0sv%9 zFJ2xMBHU`wLl2u?c3{ELD-N}%Sf4=&I0&K?LbgX8jg3o%RNwpYw1WuDp);M54~pDKN0edl}l)XYJy`73(S;fqdpKawos0{x1Y$ z>#Ap&KdGB*%lV$o)0BBwn^zq(V7(IRw!w*gOkW5){W3rgfB=9b@03|L1%`w`?OUY* z2!u!nNY&i)c_g1!q?ugcSg^mDmG~41#*JaR>2htT10M7a<`gqx8n?29#@sj-{&_c7PWF7^JK z30{S5-_EvpA>BMAaUdKn#t#nni5X9QH3@U<#`p&mU|eB%_^Jx0Q4U0F%fn{t|EuD$r{G* z$iw=h9hWeyX~L7armLiG3-goWC0U+sO(fEfWpC(9IbYF?c!y$(D&KS=(?r@V8EPDZ zVAPhVQCN46Mm|MeSbSoVv1}e^<}sA2%&u%H;G)ef!F=Wt9oE8lfsm#^Ssn`|5vD2a zFecWjN9r0w`iU zTiw?Rx~(T42l+Xf9}hwE#M;TXS6(B{hh;Dv_vj_Ijs`xEX9+?)}vV=;PHDxwTRD9!9hZEq4pAJt%L^0fCIG4|B zTGm+I7_7{%IzrkfFL2evG6FNJoOFkt(=JX{cPBout>J7yCvch^j7`)uUK)v?w4;Sb8q($kp2fHk$N$n6TSU(du2iG@CguDL>c zbK~qnC<$FlovtbVHRW@;53NGu>@AJ6n@7oK__>&1za`PK#@Y1W?8FK0CTsAt?Vn8J zpRLA#{XbwrHY;D&Kv!7x_hmq1v56L^s*5nKa4Oge8zPx{8H; z)?NFowo1v1x7gmXWKK<$|15&4L7Y9Rr(HhE34Nwg7jSj8p*H9CrV8&f;rE$kPa$c;?>$i{ zPW;xIlWWZebk75RJc`~vN2p?ks+g)OZmk@%NE=%^8y|S+V`8eau9bsVyI3x^{NIvo zdP_#)q3ixJOXtA$gyCE`ZcQGy=3;2zw#VCT?qaq4l(Wv>jw)yA>M9#*kR3XK_+EOur;N@5ftJpy z%Lb4F(ee7D>U+{>nYJ3lD2|@C{wRAa*Swu-?Fa($+Eq)eXVm)6bkyp->9y)vwL3|c z+xP)x=h0wQy=E5Gs>{Di7p$|It+SNnXeO}N8jUg|sirTN%3+hr%<|S}D{@SCl)PZ1 zDcLuFVpdk6OV5;R;(Ct`g&^EZusY@dTRsSO6)N^9>*X~=x~4UuHHqFrI{!TI3P88w zpC2O3Dy29<;jXB!bA$9H+&^!_{qsQ0$1)HNc&E>^c@eWd_z;)@9rVqaxwVl{Z&5{e zD1n}@_ctQTt^ZYIC3r&?TQwYrYNmb3@$g5v>M7}J-tT#3Ef}&k9q}fJv^Md33<0)r%f*_L3z~6`4WXHph=+>mq09v0M>7lT zgYtax%N0b86;upcCReisek5c~^w0?%DWZR^RCk21B*?OQx032Y(zuNT4ZHtB;?~9I zkw7rBJ_2wX2@D@je^Uo{`oC4sV3$!HzDR)3o%=&d7NfFnD@DhX#_H9viCRrL^aq8<(!19 zTfz8)VLWSx$d(9V9@$nh!Qef9Ml{7QDI^nVxDM2cug3eO_PvncHJCh zOJ{U6f0u19gI{zIrue1TiF8GRo&qniBP2N5v{&q8eXxDNLxCOyhP-S6Qh=G+65pT$2+TEBiMahy+jds=^ap2 z`{+e^$`y%`=*dm4Kg8EvBff5wkPBoV;GdTrmMB>9>F2x@BJkGX>E|cgkPhsPJ(9gG&@Y@FADkq}CN50z7`6j{P_A+)iK z33^tAN?f*yqz8J9Wf7J+m@4pVP)P%*0j^U=Z-{NMGB#!XqDEfv7dPKOipm5Tpgriz z@|4XcAr2SIV*0cRpf2F7;-V<%ha3k=%68OkD7t~Hr*q~r2aJq-!gP8UuH5W}*|elc zJ7wiBR9J2AtIlylzco%=!*y0Kz<%b#S=5c-&?wMVe0>7s{%X<}YGRG)&paa96JArf zL*ck+Pq2j&c2rOFld311%9su*b(L8ZO*7Z~Hcg=OCa$Osy2%~(Q&Y4A8nbR#`jr2uY+2KXp!%G*nM(j~w zCG5q1^wSU%XhV~>9+BVg^X#>}{?I~M>@0yms_-0Q?ymCA!2#lGAjTOv*c%GU)NJ#7c1h|KTh-tx6xph0 z5$526)-|ZjVme0f3*amw+=2$Jc`Ap59)}r+D{BBG^t##d$l@@)v;g0i>op^TC}E^2 z1WQP){)Q?=snooV@s$iznd~A|HK!r(LbKC_L>+%I@LkvAKC4^J%+|N?#H{+5`-_Kr zY>8R$5OdKg`ka}sK!*{REk3Ac7X~YMT(Yn##IlQ5Ai#b&9Ak2V$>KEP>G;mMj>y zVBJe!D58~hG(Vq2+ZPdOv8_m9*<*@8#@=7hRcF4{!lDj(e75zgd~iB$XlTC(C8O%X zKr4I{{So%XIk~XsbtBB9yKj&?Qz-BEx-0|IMHQy)a%|4=`>eOtZT&Y18Wr~Oy`;nS z%fir?wQF;k8Q7qOixRh=zMcI|i*t&48HqET5`A(IJLDiV{r3l5oG*{UY9iiY?3XK7 zVT7qrDbYq*r5Eqx&OK7Ts(h8XUk=?LPF?@K_uTI%HF%gB`1@!to@fdSHT5zD{c@?@ zY5h5!2^dv%dmbk}WgF^!Wq-BX!2_tOcd@X01-K zg&xiXALtDBa^^DQOO`Bl*@~=;5;X_Z1?#qE@#ArGxfOre`(Bpm!LKlw+P#f_z37kp zTu}y&A&JA+GT{S4cWr$whE`JuU3$$7?zl$riFsvxHGHB@SIfJ5rO`g)-54tCFJE$E z$#^Ftr~h7Jwx!B#p_r+I%WY5=7$*g?KM!?MeAyM+~LVk&in9X;sSaCp$s}at2 z5OwzTCIXszZ-Z3vmgj5PO$c44OQ7guZFx#>iOXxIOKL}@ih0>zDJn{^tng%Fdnzn` z}tI(uO!hM z8p;ovWI9`kO391UlAJP-{k=YWk7Ibi;f%4a8TX{mbRdN-l8m<5THDYuXNMJqgXjhV z9odVkL>v8Kt{h_Y@doCzP$B*9W#R@@^szdJwhMOMBq~k{nV#h7uNpDmD#o5p(_wrfrUZlz(if_5z7XQ(v$r4P=KmWE5Spyo=xR?1 z!oaF1cqGJLTGaDv?nqw1hZQ1%vN@yto_Gp9sTPx5=mY8UvzY**LF@gMhM|bPKN__+ zG2P=~L{7`1>?Zk58_#9})ZS!fy;htb7mYt$CWZqnx$WfTq5A(74bbyC+HW7 zZ_16vlFw2~W~pSej{K5Bg09?odBFOv0liI3ms4bUzfZ}`G5y;`^*E}p%ii^#S#crV zAvamHRhTn$MwpzdmN5+t7WIw%k}H0Xx?4) zJ8fg`s>zmugj|1Gc(=v3(ccU!hVxR9>ZOzLnHBr>M$Z3sO|t z$*+0@_yk6o--8!2>VYdy>kn7dI#sI0u~pMmx!luA>nAHWMmN-IYTjl3OrHvIU->k< z>a@&b`q?U_7z+jwQ$fMo9of$6E|vAl86eSuT*53=JfV-PG(oy8efJN>=R1_OHBA)- z$7`5?jCZN0o5J{UBP4e&d&C)XdE9|wtVdjJ=&E1<8*%|l=kod;;c%qg@8%@&_MGWO zA@=@^{0k;cp<$&BvFMTY7$|@o7DdI)T~kNbDA=ax9oSHIEb;@AffqA#6cy87CEs)vq~4+}lVrHgB#|JeU`)pwD-`ummK z?`DlV{H97;+qe7Py&YQQQFnC2=|YQaM#+aVS@wokR1X~^-ZojD8^JjqpToOuljc;L z?7=qDDtw_T`$DB_IGd5h=IIRoiRq$a94D{f-Ylh8mtiPCf8eCBA6e$+LP^w%$prL3 z#=CX&^L~9#CHpUw6d1mpDb0Vou^^~~i?aND^%MjdcoV(35|LD6aoHyz( zoo;z}yi4E8S>AAMovv?JHH)VYeM&0(t_o1hlO>b4Z5=~%Z-AL8hmPT%XsehL?Gc-Y z2#GEZfCMHyQF`+erLHWA@PUfr7D`2iM6?xwN&!*M5AT3eSU-?l4?BZ{7PR=0Dd&Y~ zw6(7T(|6Y@;qRX0-#u5L!IK9yRx}^&JtQNE`OY|dL8h8t>@EqE2HZf!=Z2Mok&w@t z8zzZyxuv1LKmEn4zKt>5c$y)A#1%KpQu@*JlrCq{2qxCCAyPC3ai%xm7-@~-3Rb*I zSnVEK?XGFjqDzQE9RvJrT_`DhPHugN@Dvk%icuWZVleIXdWHRxq5YBq-AG}4us;&B zLBoJ_UE$O7hdw>uT@0;~J=~7jd%y=Rf0}nF&5O#!YkU!)U8JVNkKh4pt+*I(1Ig6U zwWt$58A1+aah~5twLu=d1fWz$7Ir{WK!Z*{9UM%CF^9hVMdPU zBg2s81EcmR@2ie11JkkOB<>hU8{-pYLz};iI$pOS^p`=G|-1A&ih+e z+x~*Hxv@=cR;G$)^Ais!KF`u=iiLN$Q}1v*4n}m&*6`)7k2x)fOXx9qF=3sqVaz9H zefbfmt?vtU@|CV(lwb&s#R3yHpHGXkd9dHJt&JNH#?8gvWV~B6(AevTv>umh-F*sS ziw3v)Et=9s(TIzw9L7W8M64~s=I0|16SNFEycFuQ;wAEBO+3{tW&f7XY8D&7uZJKo z9;Js^8q->pc@Zt6J+&%FZ_1&ud^)*!hqs_4DXomq$+hfF`KIS5{8+q`;2s&4pwa4oGuv zF1Lvb{Zn`@!9JJx438^r+(mx8QZsezdau-Q11h>+Q&n$Wox`0TWPfLI(f`9(t)u6s zBPpnaM&{GggN9BdhoT55H+(}Rj#PhBaech1+SYp0VvaB0m>@AaLbpZ@+Wp8}7;p?h6ii@5)g_ z_Z$kbKX*!fHo9?$6mPukrTubHpEul*ocp z-E?>vQ)Tq&F+xX6GKTJAwm}SSG$ZCQN#wy;0y{_{s>|MvzHS)o(Wn%EpJo09ziq-E z^HDQt^ggCgV$Rb;)7o5+D~uCWZsQn9KCEbeR#NhSA{S!;x5w#Q$LLNypwK@6;JX9s zRd$Z8`r{(%vfG~4o!C?GFi$R=IHUL2;XZ6U$g@}I4LE!{N88|Y94Kb$xF;NWd^PlZ z*o`nvrr!nq81P5=MaT_~blXs}vQMv8B9gX~Hm;^SR%$W7LQ9FeBA`U?DMn{+eDSif zFBPsk-2(^BGS~Po))}eeD>O6Dr2yRKLVGLo7WT0fN+=ieR1f6BFP<-g?)gFb+42+X z*`q6z;RuG-ATYNvHE@(aZOp#AdavW)Gy&C8-9?py4h@ z891ZO&?_3Vv3iEp9k@?`L79;3l6mdW$F@Np+m0C|=B;SDjd(mZ~S!$-Ji7{nTO*?CaR;i-B#M6G}UFlliS+uTBDwyA|4$&n_yxpJdg z3^VRfxJ^5m99px=^dz*QO;q7kbs|EC>Ne2@YfS%?_O{YXTD5Hu2=+1N{ z(C!HHyz5-pCu0_agsb~vnH$aUVcr%_Ty09%+@@c*(m3}bZlaA^uCS=+w?PT9rS5l2Gib*->F%6_!G4x+_}N^G&;Lxs5H;w=|}^3TV)KVQr~pR>!2tta@Y=Yb<)i2cjg{&GwW;$I#XWG<}-j2aeS-EEE=GjQOaXK=e93ZM1RAI7jFrtVe>flaP z377{NSF+Eo)!P3kv zZ;u$Y_$bg7Gnc{*s(O@w?L>xm?~r4ovRnNwo=MDfy-n&TiSdA6$2 zUZ(kB(??1RUv0AmWP4TG$Vg{29F45ep|*~r_eMWmjj=p?V!(7jvB!W|5A3JiQS+IO z@wc%nvyxW2Kfrte;&I?I#;xo`=XY`?>89Uq#aDZ6tD>6ya|jF#`mPzWh|Um~8wsXc z?$x^^sI%gOb3ZlZ5}R$7g9&B2xu4p`XJYdW(#zZBp|&AvtfBGC0VDZnHN7*J{ibbW zc(frnjV=!{`$sR8M#J=2Yaw)KYs3I~8^1U~A84;Rh7L&?H%$tg?WXqKr=WxG?+0mO zz7k5K&hks7vwxCD<3UUNsF#J0k!^GhE5x24T=t7)ZinZh+J@oq`+=1YMyn$N90f4rg~_dJ^GO>WO6YmK+( z(s>Fo!SzK8a3ir%+QuSc*m!K999dn9H_M@?o3z(@(N2dx=)K5FGto7m$pOn)^Z8^d zOx65j5F}CKr)8ku-&>AOlkF7(c()jxdXC$V0Hzcjx4WFg+tbR~`_%deE!it~US7_O zn@YLE=bInQq9)BYq1^QxVcRR@RiW5}68&cyDSwvhuX=Ok6JjQ-?0VzkN`yBRTQZS# z7vW@bLCxUyteVN(-*|=up9K{XiR^v-ML(?U!?1^LQj(a<%P@1L4c)Y?yWT^DG|8zn zNllt!CwrwL>koQBwG|l6fOqwIVJ$~Xtt7yfTBYt1`b*gQVu_e<5o={X(y~9!Mas^q z^TrG|c4pal!rC;KnrFAe?mv0-&}`zPJFkMGy2R-%p^t_2;k$9&k zue9mkXX$sD!MVhh`FUs%VuJ@M`x!htRUqC)IBB5BvZ-t;FV%D<{w&KU#_e2i`c~fh zdxdJ3Oy^q8S?kLYwcR@?Guuiz<~92Mh(T2O#+_h5?8q}u5)dZE>^)_aRx>UEUZ8V#*B zH0KlRGO-PKt=5OToKg&1{%Mi6P);mex(8d zr^`X^=fq#>8v*$@&cgq(bMOoO90iqCQ2gDV{j62XnQq*6*gixV;2z$T2v2!pPkHE$ z%#tUkg^n^af?sx4hK3MI6fJ7RPmC4YB2g$StI=%kiLhCm_n@}cMuKz74rP|T+N^2V zILIWO(`=g^NI~5LHA_3)zLY+<1A>~3E#8}fdEB-YiceYa)RjcLG!%|SlYu;*Ul8hw z^5Dp`Z*krmiK=khO{B=0U4Lpf@mF;A_4R|9356)4++*xdH+XL$8uPYWiK_C{cA{R2 z&Si)Npk+RsO$qPSU~z7{n#js!TWi-!k~I=XHHs^|?<6cTp;6<#3dVh8ha|5Fh@Kui zX1{rpp5}AKGxAf8BkmQ!<@irh~XBY~$KDKuB}dJmK&S zEr)NIw2xm)&ny8ti`NxhtAEqY7V8)@xq$u71SRNb!hn+yE`Fg7_TqGDMK^meLD!a( zUWRqVcmVsus01&-vCodJ#q1B(3c66C<%?i88KoZ?NZ^}K3%;81;18;6RO_;N)lOnI zrs+&4vU8X_>B(C98U=dh1$D`d;MeLu9zjEMl&6J0B@~3cbjS-Wna|C0v(hln$fLL4 zC@^6W3T2#B{Nm^vN2fOPkqHTvzPJ0hda|C zLh&9?(7QF5Ft1VLUDs-;jl~-T=T_F|edZE5oDI}5jHsVw^*D3a_0g~51@d}z4~n2L z9l4EOn1$H}D37mZV_|C47O6#oTPvOLDHb}w`$*W)aY1xT+)meem;->b__F{wVRP`a zgDM*h@GaTI>_*lv&Y)!X)AKZJtbTRX2rkcjo^7N%K9XO^S2Q)_FRk&Re7Yb52oTR; zWwf_wFfua8H&z8J>#BI944dv!+XhDl^LK?}INkr29^}2OWb=`N1?k==B;`4zs->JG z()vhULCstG8H#83tCZd%my`aYq-T`%ug%J97x($r5Q?{+;BV!sMN;0GuF9I6``~Z%`q`5jVZ~hBVk_oy(>BnE&N!H{`-XzU&7@(sm#Xx;O_1_c;E0S+ zsa`r1u@2rUD<_Rc28mp~EyuM5A5O6AjE}vqEf-UkBJ8F#(!Z&U*4%Nimwhtt#IQ6G zV(-|kPm`~_V>d;%NSyQCBek{K(xsg=LWAxj;o~R z$cv_2h@XJ#jCV#NC;utOH}#@|ee8Z0_+P>0y*KCe4XWn|v3 zRiqV6TZz#JGY>G|pES&X@df$)z|vKQkY!&h4|RaKy&6NGEr_kT_Hv>UPfhB&)?jxx z`;`%knnj6L*@#!iWSO^JPh^$9Y{&N-qI0)^Bn&-rEhwXPGJe~YL{)xjD=DoHwI+(1 z-~x}fK{X%jVaJu;9L6?_Y}-kgs#DjIyv2RJ5ZB!;MW!6z8A1oJboY zG(~TKykv1V0KK*MF)GuxZA4X(d{ou^xwHhQdoO+QA*I)|?hLynauQ@3^LtG+Wr>j7 zjoVmqH@uoX{lVY4JD0P6HK9UZsG|m@ylznu%}|myZhJO~zt}34jvwR#oB2_WyrN+l zd&Nd&@%)Bm+|INqkQf`^ZJPbftpzJpC$8H4D2a^?8+Lz)vWyzEoD8|C)qlz`PGVre z9+fNMh%Tb@(t`@h$eB$28+KU+J+PF~GG|DFc{zB)%{IHz1s4?k$K9Z6l(_5lS%S^Z z+U)d+^QKKF?MoiF{~F%{skt$;M4T&Di({049T3`+Rq+A-I6jsN`;Js>|+#jsKRx zEw>EH2P1>B0iDO@a(ltgxM^_cra|_%E_1zF*BW;t``y6}G+O-i>lt>rp6V4c^N|*Y zeKv?ZWv{o0ef1ut#|aY0@Jv|)yf(ElqK2@|Z>P`KOS^1z0?IWybR*hz%X~1bAgKt# zGPlQ0pS%li;2<+}kYQgNQbzmR`q}xHLYC;_?zmrgpR~UZnP5QCXBa-~40l0b0b^4P z#-^Sgrdx{mFyu>75VnSNbPrNYF2WreYhlRnezdS9z7PoP^MDb$B& z$`>uFfa<-!c0eokrYDOFp72YN^7Qx^s4TuBI+&|=z)0`lF0L&Y((2D-t{l?h0>t$N zUH35bt=#sDiFGB_zan-qt0_IMr>|$bPujuE)BO)<+UVPyNXrJg2=DTIV-Cw zreR!{!lWDO?CDFep$yISNW|>G@-T@8O(zXO6YqU+vOwltGxjFt*a9jte^{4XwQ9Id6vTB$7dGv~1-?vMvLv)6dkMoOT(GF2L^>P;U9Nj$o;M4NNV5ikx zjwR2uo9MwD16)VoN274@T`UER|1DlxoKG_yiiKo$<|4X+r)e!yaJ^nS*T?4bHILLU z&d`q;EnM(w=8!R?#rxZwf{R$X)Mk8p&V#;cjd9{KJyT-c$_ma7v$I2vOC|gF6@U?x z;7VIeAH1A9rU~D{W*@A271$L7p%$o%L1rkStwHOK4D2CI)1}U0`anL3)OQ3;K6I+R zpU?Yn-O2@$BOjzlT{d==n%=B6{YA*{*p}acf~J@I-c;B$iuSYUtA@Rp{dZ~O?HL?{ zVEKV;*LOzLQt?`{sNk*Vt1nGqAD&};e`(oW41MG3{4Wa>HFAvA>0c_@XKsP-gT7mg zqH79#N{TjTuuBw-T9HodkzD%5%+e|JDMX_~=h&yNe@r7*Qztn`KCOFII@ekCZxSsi zeZJ&^6xj=|^(%q{CACa+53_M>k#1#B;#z|77Yv|Yx6c_^M|Uq_nQl(W=fa>@?Cx?G z4TNcVu&Ruh$6g0H%2VbeH~k9Nb;*Tv%^>W`;phLcRgAW#vnQ1(LMymS=F^MzL*X8y z3!l}WT3eF|%YUpR)T@7B7kzY|?ak>M7ZbB>&veN8oN{LUptlkRRc7c^F{rw_kbu<>I zn+Bf=uDofOI9xXkn-;kO(-}5Y0ZVnE>}4)1hgq9HSx}ZBmF#;Lo3%E*VGY%$>MPeU zi$WPxK1}DkIkim&D=|H4Y+jE;V~aWQP(}hmu9C~~Or+zz{sAiSgzjLAYI1GxKG{O8 zwqz*!oes{sP`4ee6sy0#v)|r|VK3&4cCb+Bg+m4R^yFVqJ~^GZPJP-D<_P2&UJc7lbOtoR^iwmZc)Z+VTtJSx2}m3nHYuCiOq}n;&$2^i@Bu zXuo!xa4Ru-^UxRiZ7WjfV=csaV!5=~fkco28)+NZZhk7Ha?zVryckOjqqft(@2h@w zTc|y1(-&q(jr7?ewiIOmRi(<;?N1eybEt9-96Y@?xR?6c`j};cx?)IOpo*WQRS+%wW^5UKzS`?KICYeP{FS8yx>u2%}&~_rZtv5t-Z$r&C z3@gm;x$<`J8u~iMq2ovJaZd$FT+zt*pkE7((2wqr_jE*Lo2aamX4Wg$gz(OPFOY|k z*YMIXeeX610j!ruheE8;Dy?Z_Z>$i9yG__aDEgR2Ll569E`?*3L>_SW;4jQauPtWs z=_2k{SatfCe6R+v6;4S*y|T->p>j32wm2*Ni$`m|ud|0fm4=F89?b=J2TL9pX*!uj z+IBA@lzGk94n7H)YhPz?golyL5>)np>%!#VURav|a_xnl3~lS~c?1y&^U?3@?sh(j z#^W8Oxp+}*zGQ+QrXvS+y?|L?)LZG|2wdpXE89LZoqoXOcQBr66`3q&Ej; z_iMfF_(+36*d}arJ8k^81W4*O6@5Ta@T-z;R^-jWh$J3|Vl-01CDu-KO5-IZyW!o+ z4P{Ar-sT{NMWIpH_DcBZgzTG!Dx7*c@G^Oc;Rx1s3!yge>+PwCK3I02jMb}bBP!Rd zLdAuO_f!_g;R>O8CysQ1@W8V@)JlT}q_OdO~W%hOrOxt{y@X zJEDm0PQ(=tpP1P!q3yp1Kv%$M$TcyEY8YVdZK_SP=`nv{IF7u^qd)L!2=o0**`j{i z^Ca9rPqh5J$ns9G<{!zk*i!X8PzC--rHz`Ty+MwPJNuK0s#Cf3soVA>=wk&?YbxX# zHw>_G}xqPhpL;hys{*cD59TrDD7Cx(|NUgYut_{U2m|J&aL!@iFz^qf)p0n zhm#$`9|7+lfgL;bgENMH3rsA*N-rEWcD!Z$*N|;{H2hyQH$Rs7$M4Wu=v+rRQX2 zsVBSDvZB&cr*HRZ_dCDWg`M;Jet*CJn2f`j8k&E12tMEh#W-)x2e*0bxOr|9F7{bkf3 zrBfA5N>Q46&P>LBW*m@lR)^D#2I|VUqoTm0qQKCbmSduk86Un?$u5K~jUl?qz%sO@ zu#CNRI-NNa5d@FT;H0ZmK==0td+DD2NtmjeLXd>k6+@dE>T9ChWUexuji6Hfl!a5y zq`^ucEaES!9#ZpJm6*0jF&Edr_WTn4Wc-HL>(Pz*x*~imuOytj_>!%#kzD5u%4rU^iDZLhNY` zjXMcfujT&E7Mzsq`Y8iOwhhUE8Ugucb>siof~cg0%&P~?bZSQu z{^`thxvB4Zk;BRZsqcsmWmF7haGUG-=RVNc9d6X}1hm;1us%ogfSd{m$SF;PN}7da z*MA)wX}>BKNo}{F9fnnu#ZEW8M!9>SfB2F_z6Af#<;Vecz`o{VAQFBX9r$gOZFGxF5SlL!&sDDB7#9u^X zCk@0-qB4W1v4!~<&_o+Ic>=aXjh~RTIx6o>#S$NBvsTEQqN_6pS7#2bP=;h~Y*F=1 z?_Twh(Zk)co^ZI)_@WFU(~jt-*nv&4<<7AywI_k;6*(>R@*~)KemW`g>7*irM!C4= zMbXi~z-Yj@fbI144E1BP;fgG^ah3ggw$f2WH*>)o{+Dr@S_)0en5@F{d9bO zir&&U0W}+SDs5p4e^+q3q}yRJh!NrdRS$2-E zjrozx;r5x7OI2lUz5!oI{G;7K5U{Ei^dHR6rf+D`d&-7C82dRTandXKfJfM^E7zt* zhjnyVM?&u$ThC9;sJb@BmLr?#UyAOxm!HsKq^=AGtcAqFWVq8dYx6i|lgQgl|5?gd z?RUi}7q9tV zrL=4UH+2nq8ATRioKN@Z8prY*ejB-P^s}X`lxmit)E^nt6td0#m;GYF$S8q`_rg=3N5eafS`$RoBdF| zeMg^7MDQ1EXyX}vwo`vJoo-Nq>tk`d$FeXTbyn+GR7b!fV11%zxH)Gy`{%KZZL)2t z_*di%#{!R|=R@?jE|J|%;~g7ck%IE{+{vD6xxMSA0c4Vmd)<`|i#)1Q`F8$Qo479e zgL2UiLTcq(Kc#28VFF6GAUZGvoKT*M94Hp9?Yu_yFK0Jz)$Rut6e`ia{XkE zmCM@ty87s;_xa`?5C$7Z!8}A?3%)?achaU$S;*x4cY^`g*iC3yF6wHg#7*4Mr56U4 zUPwhQQjXPTSsLv0CE%(vo!BnjiI9bBB2Vk45!-7_n(JyE;su6cZ08g zm!6-S)7I6R)7-qo8a+EWcy>_a%%|h_$?SQc2yH4H*i@(k%ryu4PzRqJjy}q?JPJcD zwfHEJ zf)v?)73XIWj477KF@a#FasHc8`Ts^#icKuo3rbDu|K?NH2x!V)0Zj=HrZor0-xowi zD0YDT7PkvD9ta1c%SdDyFp$Ejz%Cmd0Vjwx)@)~f%|4<$LrQ-AoTT&^9JsFVL$~`X z+osnFzHd8lH!fiX31|@7L^g{H3{}hxmSxf9B(j{iap*A>HToktUEwa2HXE2o15<)o zG6%Uh-2#fz4eTS(w--p{0?Lhc&*Mpbcq&l6qa7sDL8ifj+}hjKD3l;!n1<;X*GI6M z%*=<>Hm9+@t1ASSrp=iH=#Vpde@x{5n1%n>VH7Z%5H3b9k>DkQ=W*r>2tVN(bw@K} z1~OyDPM5=qOO?b9lGs#mr$Az|;3A7U#RE>U$Z8v^llj`)T8C!o+#ng_xYwJwG}wE;268_m1$OMx&SHMH6EN5@T?7bNtR?OwG-)1I@9_IUUR?@PXsPD`q0a zOzG)Z1$348{YoZM2~m;FR(L5c$&F76$#V3wxWUij{0XWSIQ_7@lk31{!cnj}qVm{? zJa)Q3Tf!p+S}eHQ&?V<)V_qMC7vn!1E4v*^Pftw56JxBMj>qD>;%ueopz#pi=8*BQMzE_8DoJnMq59G{1IAA*vh zRuG`ZXi2VG@laso4B%3EYyBb+G@p{E{||zu`+pELmzyLCwmyWGr+nX^?94IHBU*h> zMGF2kOLwR>2m5*fqx5g@wFy~mSh+%6GzY!LfVf%d(fu2_DK3ztCH=|U-0-mOPuiw| z+UUP=n~ic(!Aqq{o~y7;@uzTU{(rDGnZy52)F#Q+hNb_1P@4xRsb=Jpo8TH&rFSGP?c4zDxY!JAcN5&cFUgd87OI-`*X4;VlXA#?R zQ|dTPJ2U@8IZ10X_Q~L3FnPOKU^~-luW~qF`0pIf*Ns|Tb2kR;ZX`xdB;GTmTluX3 z;j9n-8^Y<0(~WT_t9)I?UY)51g&LK__*rT#IH5-jXUWoxm2M=(Uf zQQ6vupDML5xGN1;U?Ybg86hsk$}e=4V5Wotxr99)aPJ9>?b1JqtO^Vwaj~)&1%*Du zO?KhOS~$(+W50{&)_$M3M7cDnE`ZGWu~&^;UHGAzo_3FNi2|ReyNd}ULbr#=^;bfO z$n9?P6`W;M^L=gozHU!>N{VYaQBFLLcI$cbR@IvLKiQf8p{- zDG)7i4-Z`)1C#oO4rtYrs7E2PW>bqaHQR~F4cIwH{)T-5msEJmZQXsi@EcFZs7*|7 z;TthnD!F!wkl?p+sO(}9(=-ObT%aC?{8%Cghj|y)ki79em!X zssjlc=tTns1}uvWbCo|mn> z44Lkn)?hP=nR)1mSmiC6sl*hql0|uB_A=H~R(0$}-OB5-On|I*8g1-?<)+b36nNog zzp;Ty6zuZz=2$>n?ZaZ-GoAOcql}PSjVQ+|7zp9u?1hUODHx*VE9E9jE1QKvv@^>T z?rQ{+YE*5MF^;rHF2*vPsM(kHG~S(9YJ66d=Qe)ABxPTxDi4SI{LSKZ%)|W6to%_S z?)qA3s^IjksjjKREA&gu%Os*CEkz3A`zREMP97M~QR9Z)L4b zsBn$h!O;`$U^a^NYbw0;L*-duz$^`P1bQH%^$k@c#sZOLNYfHEEtX6ri^Wqt5bBT#suYV1Wn)D#vW3Yd8Bb=Z9KhqhYiHJu643 zi*}-nx40Fw+tA`aYX|{@==Ut_P(m(L=L)s%U$wt8zwp9C`1V!w^ ze@cjzv1q%rnzRDDGzn#G!B4C)Fcd&djiNH4NJ-%Q+jnu&M~r z^0o$#D(O!~Xb?Wo9_p1LQl6P8xghH6@eNAmlt6F_>NL{mQR7DS!r^GBe|IugX5qF# z56-@TjSct_IOTsCXa(+AT)GMD)<77?udR7#0e4@r{2HAAj$OI(AJUPp^$#b@tB65^ z6fk8qte*RSL`cmv+`QsutHIRN)n`FMcrV;(Og+A7SH|HnX${$B+`a_YS`q)U%oSwB zllCNh^|DxY>|J+lKM07J6fq(|=jZdSR0IP(NS$s-FrYFGndhufvvG+GV!$BX2~`I$ zMD$i_KpokK5wk1o>x3MkE0#uOD_3p!)nG7r;HX61mYYQ(|Jg9uLP`wjvx-Ve!Wn{p z3NlZWPt{M#Km=<_JG$Ngc{iY8AiAO2vwoqWPd0?cKB#5zVi1CAJ!FZdx(`9NQS2o*qvkx~|)U_=bB zBv#G7R=@Ajbd;<;;CXl9>SnewH;VTQNut3DG2_Pj1t&_NvcZ{&A~-GzLt0!a&|$6E zVE49li6~zz!NarwKwOfo<02L(m+2SrOynlz@h~HLl%@jK~Mq=Yc|SL((z#h-?$KaPiqv`7PyvhkRWtN z8JYg1W}Ez3X|r@FC5_3e(;f!|t~JnPJf#EfYD|75ZUQbl(C7~v%f!zh(&;TCDboS4 z7xDLp4VJ@%`102nF=r}V(P(0U76jU`_$KZOhcQ?Gk+P^IKJCu^M$M#O9K>+ZNpCft z>#7Pv9H5qjnHRr%U%3?C`cC$t;kP$3vnC7hVwBbaFNPHkhB(dfik+-vn!PSw^71dw zN)-GO$>n-6{LjB|Oh4+6HhxrfilPS4^+g|xts3_t&+rGHCB^h<)$B!##sYb1F1I+5 z;gvb0{;Lo-tYkFG9OJ!=LYZNfB^bL)7fK~R>Hoso7VugrUncdwRxvRxt{d$k0n%e zs!8?GF3B8mEq^kB`K)}mk(d@GK!mtB0V%4cXp?$llRBw10MZKNd;B>}w&8Jy$?Wj6NMGt58S3rrvYV{2+CLhUYWqAlaN@`MfL^u+|y zA|F%;@YI}<%L*Xy!#Vmjg3xpgzyWApnS_7PvF}X4fAPY9jXeN=SWw|rLMNwD8b8tp z6_)kt(o5hc8UMn#xj{AaR>7}Qf73$;(!|WfXqMWPrAC_<_mkJ+NumAYnDwfnP(M02 zG*g|DLRwr|YVkzIS@?@$@KZ&pgmIly2wKftyWsx*gpGbFrC&;Qs3Ia;sKsKd<;oID zzn6|&Z*Cj6LCyVDnzn>2m&lAdRV_9$@V&gis<7Y(t!*>~E{qX%D){Q5zE%uMJ}yg| zD&Q?4;RkG;vQlU({mtAWR(G{^VF+_Suz}w*_esS;<43E%ibv$v!MESm+`kplG0e=M zp?hF2>GHt_UW4u++{UoE_G6TXZ5pVF=Y|$BZ)u;=FysH^ck@S*L?#=bqerW)vyZ=N zJa;s;^qH8e$nGidhtJVrH;uW{VcYXO+HX4)0J`OlJh^c&{@C+EHY`wUR?MCS&fL<$ zc!R0aA3>cARp%9-+!Sy0o^6OnZYrIj`{JR-hWNti@sYOSF6h^XIaX)3$F?KPTyR5P$%CVO+&{fMV1|G%890@T>ZL|o|1$~$uI*h zyT`|=uEO=XtdtG2G1qQxV#H~FPG#lbENlM@;MHI=m5-|^BFJ8yH(n)J0u7!u zJ>Cx@M%Lf}uRh0zNr!qfz=w0exw-ic{q+)aQ-Ud-^*mjmWewr}AmpNcO9!+!U4Fh( z=0FAsHfL=;QxQ~Vv$l7VJ3grVL!1h24j(RMgTjP;x>PGXZGCgJ#V@~tY}7nTZI&RmNmf+ z_@s;uC5]CI_^Ia6e)9PPFO>?%k_+I#|=fAoHtFtO59gOIbFjRlhmGgo&otNZ}5 zaPy@0TD|7>fsfdh%Z%$1jOZY^!N=-YIbQblC02Wh{U2HI(X->%o*kcF>8UP226wpM z;Gv6^=|HN$$=TA289EI+OiNaSp11|Aip7xb_0*Q5FmsBi4N_?olb7*-99ssK-riX59%kXmLLApYc zgxjD1Aqrg5L@er$O@lt8Lx(OpWD|z!w8DSqvp1tn#VcT5vsAP7!7>2+{+hL=Ocyhp*{2;mULQKX?Qp+vo7B$6M zys;bGVvS9RD*iW~S>L4UdJH(+AVgh$tl92n-lV4$vwv2Mv1}QQFuHZYZ}^;w1#y!W zsafEMxs7VlB>sj)kHbEN(SE2_1cE(%z5FCK3vw{yar+UFfHxJ!TgO=3XPm{bo)pDQ zAC&ywb>hKn-^fUkuM4}aEW?n)fF8q`RV6IBQ~o*$EV;m~RHbZ`8JgNqw@JdHLs_SO zTzGkIXL81#tVTc9e*qj+NJWoLC1LN>bRSmc=p(@T*y??;-o*-pupAB=mN7UIxZP=a zG90ib`hF@2K<(N+iN%}aHYLeVDFFIu>IhUL*H`m?#=Xo%XfsR+iBrO<*n0Q*z)>~y zXyf3N;~FG5jvc)5S{94L{><_rxC*lj{7gOzOQB|j5P(sR)cT)z$ZXci(lVyoU0GQ6 z05S`43z$NvYZ~a^SONZZv$~3lB{>^Tt=>gVDg5X=4Hw1UFC^9U2KmT~NOax3OAWNl zd#Y4B2mtWuV~<4P<{gytzhi|1=Or;*N~DGN$gkRNJn#;nIa~+ZjT0yk^FP&%9!fuF zT@khMlx1%+_bE%d2kumx2Gm+2rB~!e8|CSuLQ1O=iXM}Tr%ANH&~QbWoIC}QsGqAM zQ`9mc0i0&3iRIV_B%43Xjx5#XU~AeH&hFz!6r54bw9%gu>5ZB_3w)FIOf2^tzg9i^ z2B{sL%6y}}FH;0c&2dEoESa-<>C{a7RJBYXb}?MVJ(*0RWT*DFOzs2zKlGY^DL}7z z-SeY(%auWvSt73eBFWs2gJ5r0|3{kWYO*yt98Gq6i;G*Im02doe)nO1^eN*~)apJe zER)k$bbPYJ_&kRID~aVLhEZg^g_Q9w?l0{Bt+ZQ53|B+OTbK4S1>zj$EpxhDy;=YA zcnetMT6IB*jJb#tS&%GOCi+t5d@Z$XiXYU-7Og~3n=BRVeTO^GnYZ#d_fDF>*X%MJ z{d@vODB~2j=+DMMU3a+Em z7g+3e2hJC|6O7xnL^oe)u1_nN8h7M)(fi3sLY5TJuJrw61+Wh`XT^1@Mo%1@n`JoA zZthoPevquViZ=>DC4M3)j^1h_`NxuyaN*^`$7TLW)tD5Q@;O(oqKn2Mjl+tS3Y0SV z%2!_)`7$k^OEY=TLEmiU-c~co`i-)+xyd!ZjfY4Cw`pHkJ3qBOPDQ^+2Tj$HQ+X#; zFX&F?a)(vkZ!H0pa)-&Fqyv(c&N#2pvpcgQnY&ZD`r}l7XNEC_`QX*X>Xq0{4gJGY zddl*@c_;U&Qf%g)Z*9#ttZDC@UOU>+0CC`gSHLE)SlXTO6{Ailzu$L)37fb^sicc z)%;b&83`ock0+MMt=;6EQb+(}HB*^;J8? zt5q^4e?(#W0tF%Nx@2RM>S{s~S1IUt%K2kiBOeyG5W?NtXem!aM59Q)*b;CTYo^yg6HC2Pk&~`2O4cA{GNKFFQ3&}WK_ZyakLX>_fMqd6H${gNYZtS&mF z1wo7cB-aU?VY`4DKBV1sNNZ1uX}mvfc3p!i?(0KZ?vQpw!tLjhz~K@oM!ZFybQNM0 z(vIfQh-t%>S7&{7v**Id%RNI{KyhjYZeGNioQpsuF*201r{QItin#X(vqQ+lNaF+0fIB z%gc(~x%3}X6nXh8`xCx4X|*P;uBizQfu?Zt9G21roTX;m=p%(mRC^rwXg5h^#gI&KJEY)aBxJD$tZu(mk^p~a;)cV8*RwU* z8q+pqiiKGTPuz9`tV@Q?4|Wy+pW1<9K9rBWSb5+|L}peX)ZGGDKS))+E^eZ(bB1rA zuNw~x`h+a3`Q51Pbp)`g?rZN}L4Uhew4#!}PmyE4TUXr$JA4SWWXlTM3M9^P;_l6T!G z6SZJ(id6%o{eW34vj9eQocFpgQm|;4(eyM?XTNrFVmbBc#~Kl4>VgY9JZkkxu?SE$J^ZkY(wP zEXSnmW^~ZNO3Aim*(PN-26}))=t49>+kegNw*|naz?fz z%Z1;|?#)IQqrX$)mStdpMHf3GUm919JuK8EJ)PWFSuh5V7-0;4OzHXKfLzv;x1?*2oZCO8=o+Hj z$n9b!>xeIUURiox$vronmQ&^}vYZz3+PB?60%Q(7IC`St$prmcWzElt1yYXhNf>-V z`Rp66yJG1sH@(R~ibpaVA5j+ERHQ&#J}>9H_jHz~CR9R%=2IpRMJyd2}^)QkT^8PNs*|q_|X4 z^+d*f=|-vkUY03U4vOl0`bStHgKia3B7^SIAmI`uOtEG;&r8&rLcpknu3ay9z$l+F zUS)(^Au(-|FaoSvkV-*!78wG?C59=tuS^(IXbMrawv{hO*K6?AuwEk*Qut^(jIZ55 zlLWu|ME@T9=oXDCUl11A=`FNyiv~kL0Hy84`S!q#ne7SwlOs=Qe`M8)QM>8W@pQV{ zIXlHrw$dJqR}5nnzLCWpZm)V{=c8x ztSKrd-@R(ODEZ?lk7o^w-!tQfH`rej<0)0Pd`IreL-^*Sn}tcvvY#E%epsVap)3^Y zZld40=r3kWr+;$`rT2d)%o&(CX2RG=b3RSgg|Ij@2{NZBA_4qP%Te*yhrssd>5w_8 z3KFR9K$rfHoRN=dC}HG++{}!}oG{;kDS3`A^Lfqe^XBI@n3s3Wnm;_dQXfdfY}}!P zzQc)7MLRcL58W4%tS-}YYeUazkSCjYE&Z=+iG9vZana{A+;f^g*q8pG#TrXLIcMX_+F?$8<}1ko?+SnSYT69fSv za|@wWK(%*+VRQeoqT3o`l29Dm!_u!S0K2Co!GvacNBl}ZG0B$gxL9E_3bWj zM1%aUAP-d_|Kp<3!<*k=5`5E`WRiXHC7+(*muwJuBHAk&u;sm9Ywf({zv7vXv}dJypN@GJ6>U-x{cIk-OrOnT%e#7LIK_?CjiJH56PS643dUv_t9f++Gt@)W4(vdETGLTA;#z zxk}a;Xw0ECsn)4I7}e%lfG8X7rrFM74f!0wV>8gWw~zf!|WO-<2o%)&v)fI5eIEsxvYx*eFFD&CC$&BV))RIcdc~wFGKsIRQ4SEGuD>M7(O?BdL z0uNd4tj!c%3&wMXMG{{$3k2%(r)%jxh3U@vMSrcer;*GT>kXj9K|kcSI*Rg%{?ugc z>yt?%;6Y-%~xtROs?rhj%RD1 zAIrL#zP^wTPGvk~9Z^@DSVI)0f!={uy5%}ia~3@{fkoeIcs&8NIc26e%f1l_iYGP^ zM+2k)u-HuZ1^aO+ZpShv%f6O~D~5+isc()w7$aY6HrlZx&J*5t{f<=Z`5;crhGCZO ztflj%n*8977^?tc^`j?N-PQ*sT<^e^809tgUx=hU`YSp6D>1lXN6m(Sd0tU26Z{pp ztza|vV1i7jhtfy$McqBL%Pgw3$h{E}Qkqe2jQcCLe7us+Pj%$$yT zhd-D-7oFEIDlbBkL6El3^Hf9<81*&wSO%o;D9fB=8TzeMmG8&~dzXr>+Tsn7F=(;iVK|m7Z##uJ5yLz?AXP#U`=;~x0>f2)$_T9n< zci}Rs#36{XVmk{sC=$+dosh=RpC+W04vR!+D21A{XZpK)>Eiixn~eB9udI~{pXXmB zrRwVE7v<%{MJvj69%99*fUpHn2=DWUX2;Ps$9LHExvD1tDy5Ku% z#WJ@B!#2wOhLYjkvBF0N0iRO_+BGqXK^yE6u|*<<>fMOVJI-^9w+@;q2Z2eyp-K8poWCW6%8-4$g#wu%}&xc9Ez2s1B!=#@9K!PTp=)220Q3C^JHyEP29ajPCO|V`)a~e zp7@z4W_050&^-!KV@TURZf$QYsnulTr&lljpJVJlgCDF?Kn7{I9}36o<^t5%>W3`( z$qAyW0C!V($i9|g@E#eQ&#OlhHU_7a5fl3p(O9+=TQd!`$v*m8GkTgcXk9Z4Yav~e zET2BODo)hfmSVHR+PsiNRm@2hla8Z-l>2OmQf>0rhdFT)ByCl`_I7%xM+W?h&v<4n zFWlNOjJiTNyn2zN+TYO~#$bxkDdb-&e=%~iFsR&wVGP4z^p05j9kDpbxpvxn69n{c z6r~jnuVPRS-z77JpgNjViA9>vFk?6_79Z}$Kaa<9#=e}v zJ%KJ|UbZ9u=yE2t&9^jL0MIo0HpoPw{*gD=u4abZxB~%$s1`2p4UDL!n@X-nl%s|c z@9<_uuc6``T-)**4CPs4F&GWlU}et_?7FBGE}b<5HTyo{oMN1PK%$y~F+RJLnYwX1 zgY!K|nDXIOB)ToezAXkLQ=Z+Ohr2ZY&?AcUYF`K-i#Cik2*@|F+ZCdPfR z)Yon5m?mmN7+RxvYGh{|m!2oL;C_==V_vgtW&lc|Pj50{GAR?VDE?iq39tR<2#e3L z$1Fp5V;h7lj@Z_)ieK_mbxxKtt;c4#fY#H7jgUje<;+jo{w~^@GV*A}5)p4}TKsY54S~dDv_mr*JIyk^`-p z5_n%b?we}B`yv?q{v3886Uzmf+U7o-rJ*O-$;k8*QlEwVcp>Qp7pHXXzl6~!hDoUz zW*N(>SW#(+MpjYe;rP1zox>XlP7|)m3Sn+v6I-w*HVHrrOs4L`Fn8SDf@mUoE>%&4 z=27$Lfwk*}!Qo#-Qz=~JYZzRqzEBT0CtZdbQ}5`MThZ$G4bFk8p2g1tfA(D878#iUruBJ(Wi?gYlmaW3_M^pjwW7ERt*&xH$#Ig z)d0O)kB^Awy#dcFj8%8W72FvIPH&A-Fq4RPiH{C%p@7(Cg-{NxK~AB?P;J@0jpC>j zh!}WzxEAN(N5*5^7{X8t1R(MpcRj4c)o>$WW<=L8@2p`|;F}<5m^R<_@ET@vS$72V zu10HiAoBu}wZM%j=y^zk6tR*pHL&1iB4@~#O+wX~2;6B2xe7=7BxLlWP zoA#IMj=fE7Edm#H0IV#97cJ#hM#2{3iWR4crsvYW^;X2A1_yk&(?V!q(ymr*{5EaI zwsa2wUY*@woz(|TkN2}s;F$B~(sv?&IcUGRMH{TCson+&3jSBUEpu0hHP!g}8;49M zJ#;u(`}dkL+Qjl+I&FZa&9OZrbsh{>~_JKRFsoo3E!x>Q24;1jl7>n+MA z^ojch=Ms9fmkuu^CVZRNYaSt_Uf;a|Pmn1q;c{niU(Mp)+{#J3*syI_Z~CQEUwEtU zadcP)7bw-A)ZVK7*vCD-jo)A9`M`8z7yr%rXw|JPRk!Nj2y=(p6$0JOqn&*leYAo_ z_K<%4Afuoocj38S?d7Iz-drt&5Shm5#!`sr+VX-g=Xzc>e|{^}d-Sr0&i5gBh3;AgEak`- z+=i`5xCp?`_fy}N17C70e4K!~PSplc36 zGIDbxkVey8#SisUEpV=WsI+D@$}op_ADd0BE2wp=LWngE@YW#dSGS_jzIPy9pvwU< z>xT|j66Aj>J@CqOtb#hARY%7ky#Qz@(e8iy0TM`gjznI45O$E_@;TuQ{+TV9T*0&3 zqJm_#AHs$y{UTiRS8h>-0K9+(^x?2Ej)XG1*MZU8;^!3WIs0-5|69c@bI!hEWh#ax zOrw5$T=|nJZnBi7?qF_`+{j(K6)~_M2`vGmc&L`6&4AUZpF-gtBsH5@G0+VW4c)s+ z@~fy^z+NDoYnuR3er$l*LaHhsC58icqmTKz<;xpMx%_n*H1V~bgIfdf)T@%fBsJaF$WCqd5)@ejt&(iB4CQ71h!o%?VA3AAYU8|ZJ-srS~5UziR2Z%sNZ zhl631-Iy$qza{Nf5@8wyjcpgC>ezOlEaZMz;k?<=>`uWzRs`?^AyB9QMY^db?ZwNTWn>R!@lY-9X=4Aqq748i{|6$nyO*5>U2z zm(rMgk*~?LRk~)caisr~u8&awqaR9SLfAEP3uecR+nFD^`&(pK4}?cPO@c5z6!bUK zknG5~V@q*Yn%`|5H9|9c`!*^zqjBF;eSp5LiuKJH7>uO{S^DEnQB@y( zZs$zYuclA0t+MCYAsB>FALh4W?p-OfDjFaW=#!oJdpG`e{rPv^pMSfz2gshA$#ibL ztmrb6UhS}zRobxm&Ubik0N}JU^WN!nbLBA$pC?CyuYD$escAB*SIwRN9GieU1TQ|1 z);-MqAk|0a@gGTw0FyZ-ZKDZWc@Ns!e+3rRm&EOItbp9HB=@9Z@Hksxy+M~5o! z64~>aRBk&fh{=rMj8@aLI~hBazd5q3+&V~v9newQ8%+AwrL>71D85V7*lE0IF0bO- z@8Yi1vMUo+bVU_cJ6lshHLcoP?&5EsO?=#k59wc8AklFUAIZX9SlQhS?2e(jTkp7w%NkcY?k;+8*4Jkj60hE}UNHvj=5-H~ z5`M`|$MRU%sf6mBHgh0TR>O>UeYg;&m`JzYxeZF>4_hp^Mfj}&?eo&ijnC?R4Jh@u z0-~FTGO*V17t{HLC>xj7?mE9vST}ENXVC$nUvDYBSQkCNkUzhWW{=xeps$?olE(92 zx7U9`44!|&=hU#99D540nmunsGL073K>r+AgKLFL+Gz>KMue~s`bQ1!Zy+Ju#(i{3 z0@qQapEpjD%-lxojbPL^&tvj4?xB?p^s@?jdqn0$IeJJ;3}OS)e~?WzbsW9#HhO$9 zX*LhZ<*SwqE1jf#b=5YD2EhZCIZ%;4ShbCL7jVBT9pFqx?(+U7)>~SLF@~O~aU;$! z#|O+nzRlu<|2ArSdcgMdOq&ZfW@~P)(+MmR(lzqY`=yJN71uGgc#~m)_^&5M3FV&C zEot}XENSnx(rL+X!nSu1+e~C8+kmOFIy`&Dhp@AczDJbrF|#TrS#~8J*6wPiLO}FVQk9 zzx>|lTV&B&WZP~)MreOU^i9(KCK=j2dM~);c8dd^G%NQ93%m8AcZu^65xlpr9~*f6 z*gU7*2{A5A$j&^AGanDM+I={d`1k`jgEz2QI0sGmICdNuXY?C(<2S6hTcGjv9M;)$ zEeMbS1S2m#m=`bp6l$Pw?_%F#`t)(`Em<@tW>HRzxJy3DS4uq(h+Oulh7F{yGF2SWZ(<|g#QN8%+L5c^w9#}n`lR%}C#9mWmH%l=X# zD(ZaPaYz{&Uwc4X=QF*POpCTsl{{MO!pHPKu!z%d7N%TsFRrM6~=|zKkD99O^PgdRjx(Q2~nA>H-3~UZ*G3wB-Z>N(h1;cc2vZyc2eXJop%0qaD2}1~42zMLyKR|!f>gPSk zKd$3uKSr&|m;gTBOdn0=+mf{x^m=qZ?|O`TCj|{;b%zSUkN+Jhu$d@w_Yb`n%CZyx}-?K?e- zr7ZVaipbDoygLbNMEJnMOjwB5bj+~4Nz!3(XhFRT6I#E1*Bz?9kdTx>SLj6kEdBBn zDo0-e*$;kYU8sk&JlNy z&UmN7hwz#(_vM}R%L%~S`swkTWQJB~YWCud2P4L3Qd=Po#$tM@!8awB#V1E1gU_O7 zLZdNznYz7Bw@k{#p_)=dcF0ycI*g*kPPd-_A@?=VmB9=Y%&;&mACP=1EasBlnFF z$NT#yCbvLSje3sY^uo>5;Wut!yw}YY;RECjFs<0I|8}Dwbq`?4hs{GD-8ZGMo5-4h z|4;h{`-rq&?sy{YGuW2XU=k>?t$^S!MJPd=mQIl0&l>uIezmF@vL$~Lec^GgV|mv) z?Un=Dtn|&#$m!5w;_bhOHaw!8o5Zg*tho!<^K*%rMd`TtzMm=TGVnVlF`t4zgYLlJ zx9=IR>hSktP0jVZkkV~$XPzh^Qx-qrCg_KJLVEiaD`0K&H4Sa#JDz-E23gb+`_O@7 z+1k+zV%Ysw7o;s#H+Ov(=eddGnydS8Mdt1j6&PA7R}kw!sIyD^hP36U z1pSqlxIDaltvj8@Dl;Bt?lrwJDSGVjCyqTXst>-@V9ZFHU-(IaYtEeptof{t9A7I? zL#)bT&j-pOtvNHd3#?|kV0z?20`HXY@0bjhzOLXHrkl@xl;mjY&1DJM=V1r>m|omM z7o^i4m$PsuQ?+c-Jw%gl3@}qTlZJ}u>OISqcFX5dT9HujrWUZjMCLkO5gdq6LJx3@ ziy)#7P-7jul3z=esmA`o=7hqE1W~0=4vmYJX*s!*h${Q&x=?C!OWPP1dR8axg%ZuR zCc$FHpf`xb+38;+c?QGNjCut`bAd3|)Z^NQpWU*7$fte1f!KCLpJx91*XgI3$Y=6J z@5Q=4l&8B~ZuGPr)|FFf#fN~eX+JR1!K1LzeKwgC=$%L98tOCCAB`eYi+;{QAs?vR zM6&GmqjuX-V&%Q5=8JNQkl@Td?K22@%Hltl^X44w4|3MzY33i<3|9$FxCS%#a;a#h z-#$2rFUx{%%p|*ypNkbO_vj@0P70TvHu8c9K_l)h+?5P|&*L&f4|q7KM?g4hID!@0 z_H_EL5xAg3`()VKX7lCgxWD+s1W|vs_Q4~KDo=vY&v3Q((?JDE(TcpcwIdH)f18w;cK`BR z(Tf)Cij6|tG{5OFL|pvzT*^b-<>#W?iE=x?W`0@{*l06N~k%SzIn5-=DlBIl->*Ncq2G-E;yb$YY1-&uc55Xr zOVM)E-{smrB_e#bLrck z$np@UMQNe=6?u8S_C8J00+Y)L(3@$)(Sk^}DBLr8H@*2OE|yo%w+|mB?Y#AEvUun> z&MFaXq&enz1Y6K}iE6QJad*pNt?*Cn8Y{ksl#<1nRjev3gd{J{jF&J@lkijgmuPX1 zUidRS_KhOpyZl3H{H_j}$cwQ>&lb1Xh413$EY_XoIN=d-5;g~{o9X5ybcl?6=Xz@! z&a1hfB6_CTC4Y;~mC{%hvK6LUdNi5qC?}!)Lrg%DsZm|-*M58`z!l8RJAd;4`aFm~ zgA_hLk6U|>_!gW3nzaDH_6I8<5A-n1^fd;n8%IlrYQp z_nDIwVt&y!^b36v`fe$NM=x~)Keiy(u42UcK?S3nY?PyV@9=hJSD<>2+Po0>!TmZD zg>;Q!5DkICch(SHt$!&Hp_sQftq>Y{jZjqk1hGllApR((C3Mdby8rLoYY*~gP9WRzKk15cNZ|dM*`I}`L zv54d7zW2s#WXgWxY2i$#+HYMxb=Tcytwi7g(t6!yH(`Ornzg;iGJ!y8DfB)r~jHLLSo2IPeSPW(SC*9 zU0}0oD`naTKd{`-(wQb4H$`Gqj@6h~xN1jc#8jw`<=>Q;ev;gx-p?Jfw;WA`DxlX# z$5n8TYni?DM+u^UO77F{m+UC8-zgZ0={hN@P5ieP=xtK1VLxd!={_9u-_W;Jrf+5b z(|p}CG?()Xu%&@8U4BL-FsK7fT(Md!l@HZHVpf9$(O)kyiY1xmXJmA3Dpkq&iL$^Q zW*Hn0iQ-4;tt_wyT%Nt)L&n^h%k9X^!1b6wPfpw;&1&tATrUPDa^l26(e!82ceg@^#QCQcNFJipwRd_`yT8 zU1ze4mwaSS;}n3{z#+J2s>t8D1_#h2E;Fy;1fWY;huO( z)(k5xmA*)KPef+evNl({AVX;3C@*Wl2+H|)R;7{kr|&ww|MWAbUpW2L=@-tdHk`iy z%$n0j@Sj7cpFMrx^x-qNoIb*+-8bO@&z@O#`rzru&TKe+q@bO2p1v24dExYv`18Q& z=kdP-XKun{Zan?`nVXsVq+R&;(3zVgm1GC=zX*F1=%&iG?f=;&f$TJSlI$kCY1)RJ zv`Gh=v`L$^4N#i4Ne4PW2SCP?CTSb!MA8;o2CWDPC<+SBNL4t1f}-Mtc&v(w103}@ zk9ZC$4mg5y{a?H2`@Y}$fA9L%%7Q|Zon#NsexCcjuHR) zJ*U>4TF0zY3;5SDsYG_<)K+Q2w8u|AEZ{AUojiz*K5+5~V=eI!*`AXRO1F~-r6-x| zS3i34v6F`cyxBia9>d!m!KVIk>N5Q6-jfgGtsgFd@%YxoSwy-`acUFZ;ot1jL|{6Z z_kEY1{)zjEdiG5)D`uEyyjrpaB;!R}%Hou%bG!bDz>&T?zN;ErL(0~WJASAli}>3r zBlgi9{NA?^lyz0nh?&tUUmZ>-y`zKP6O6t)hmZm5;MLh5B+R*l{LH+6j_YOjjAWm! zvP|fh%P=wvol`b=F!f@m#*+ z(>!>*ZK#an8M*CIuHzr}XX9-e|6;gS{2sH(%yr4QcM=ezl^z=RDcGTgIop%c9?e|| zicQ2!N^=m>3o;)-W(7qquC^vnYW&EY^@|J+f^(Jc87kkS?dyy{r{A7PB4d^BU3}2W z_kdf5Ux2>04Ws3Ee}_QpkE!-Y6|^R?1SlEzc1bCuDgeh&`F0*{%d!Oa| zIq{{E>n773Eayt3Z%Y|bvq-^WrK&t3tkQF%3eAfA6xj$4B+5e47GQb|z}r&n7IrBHZ> zlurCz%%>GuPs#UxLxlM$r2=Sj5M;aa=P`&SR7r2ac{N#SC`$(Ap*)#B?!{#-s`*L6mS(3V zTVBkbq4qZs0KPEN$m4A=epwEjp68qNt(&<);c(gVD62L_8Ix9!ZqTmDUiwksGENX{*05Kf&7X{!qyMmJY~BXGx)bge%yNM2!j9Sddo zfNkzAh3Q=e6W`cEW&oA;tzron$>aE|5cs|Vn(h#=ip2iHODDXvvd1DW-0}rsKXSfnJ6BZ93UT|I^(58gN`;@TTOW3+Y zdW4IE!dW=BK(=)OiqztH>I6{~%db3qyS2*+f3sX)SOBsel=fVFibw>-(Xfx*EFZX8 z&J1D#1?f{!etw?mBr=AEBm+Z|S$G$5Es9N{p@35cSBkB8>@{ihH7WD49thK7u*U-y z@57MusNyAg`C|*1oeLPNOeSJGr2`?U3bHb0LmR-q;06^nzsly5&0nJ;657 zei^$=GH{t>_)X^G97r;X)hI7DCK>tv2sj4x5+>k$R|A%Is0?YVXP60lrYwQZi)+;2ZAKbP< z>|lJSR)7cV8v9)`^t;6LpV`leyB?peaBP=qXqUx%vvlzPV0B)mL6Zz_mA>#e+4H|)o%}zj{!dyb*P;Cj)~V2v8uo_2 zSe@Y){ufr~8~+bhr;YsIQJt9>^vw8D&{THti*(v?0LUz~J%a(7hJ~9>DgsI`K{&8l zCt^8WczUSEGC)&x!}rXds|RMa$W`E8@A)q{=djK;Iu7SVFt_Og5zhH>_alr#?7noj zid0S`Z>qgW<@|wJAOp3NskGd`b0%>-!2LtV>>melYA%8}htk1JGvf>a>q|Por2S72 zr*@o^+M~I7oW-g9i^XZ#^WQ8^t6HQpcAFk!Y|LmTOdTL2U^M|+X7XPgP6t{!?!P#k zcL#(5b7Kd3#zHmFm~eD^$*i7E5m;LBC#I~W$!lJ;W|3FseT3;1S(BCvQ=0Z7IL|UovdSE&2&ZZK*_70k1+ATS zRHmWy3Uq>*$Lvn|3q0wYj#r3yKNPcJ{bvPNwo{L_UjnT!5ycu_5GK$sL^ ze1RiDv?XBaXAaL0k)1`2L*ZVC|Hi!*uq-k2V2Ok?Eb*L-vB-2G==pwuo)4-#w0`kw zi~tIz3d(#SbCB*=19fOjbUe+f(J)HuSOQ~70F`ZwXMW*DQM%aOg_5<<>E@Y=y8dVs z5!sMt;+3<2bBHp(x-D_3=}hXmDRI6jZ2K6;oZ}b^CjeeuXoH&C?dBMShW6vvogu(8m; zzK|ZQD7OnbfCXBQg2@Zk!dml}5}x@12N32>{ke<@;S6PP&nskr*T>s0pM*f0zRok^ zK4%pp-=@?zl|uttSWx$`zYv!E0y*)B?98a~(#iHZg6L)u;>j-pjYmV@-ifAPup`2} zuFPbXeKQZ#vySTNj;7plM_!d9ue{S)?`+5of`1KRtqsR2)8Wi2#(k_X3zN!mS~k$j zKrMmd26T3A2cDGUs>;FtCj{F?((6fG1F7;eNB+J6#0c9$)v7@nU&g{DC z`Av2H@`}m^TT7MS@Ao!b(&V+(XH7kQN(Q$!w>-Y&kQ{Tkig-mXFy8!Vig7ZlKuIIQ z?-H!%YO8D{QMn{gTSSqA>3VDcdaw@!{){L3r=77pg~Op#jWaCiz}$K#eK zkolHs?rw#(LwS+8`FGaJC}jmXg5$@e41ENEO~{j&VgxUua5SR)i+hO(LD(JtlOTVnk(58VWo=B4MPKqYlXZAUe_1+l zEZ(*>aQ(>}H0w7D|3$iH-wcO~f}wHBYwi%JYC!h+zTn@|ctLEU6)H8JDU`EChTUq3 zP$6|GFerx%x2YqnrSZ0heJ3fS27_erBO2~m(?}=NOOHY%p!wA0nEm_V@`y z$s8IgGgX;Tp>^i2v<165&4H4!DV$*n$94-}6{#cIs;)CX> zolOa2@#>3@?eS{ocVz5y)y2o&&sESkkDX9md<>mXIlm-h`&1VnYxk*~Uy-qms*8_- zjVk9C#Mq{yPw8qNP>t*#{ib$D`s}uZAnbT zv00D~^C^kq*24*K&|Wm04FJnsZZ{+E2CXq?3i;)MO>}~EU4-8OqSw=f$s@B|-3obyujFGw8j}*8%P;=Apz&gdpeH84-ZI20Q#wU6 zDSU1$MM$)_WCJNlAA2Fpt4Ay|@lz7K z%gpYC%gl8m*90Sb03?lG;9&bP?m)p%2JLqb46``nDll1rofG@C55ZcPsevW*hr*tr zkRav=ir^eBqA^4{vTFHF#wKn}cm4Gfv_(bm!I;sf<*jnwkgDvp_b3po~VQL-`Q_FlwV`m5cqEo2@oqt;xPt z%^KX-sQrx-XRiZZ)3ch?)DYVoZd9}<{7Z!zif*zp(W=!nUO_0609+UbB59Et^!dOL zUnX1`3FUgR2rP2B%KZK!pS#@c@cS!#et&VX-|xf7oadKrv-;F`93uRa+&hPe9^&Ix zG@BSD`YwI7iD}(K3{!Zzlxv&= zuVh|LjtqsO98Xc4ZSN$rRbW7bm3g2W&pg4GiWJT^8d!JG~Qzl zJkQN#*S~^)SSKY$cJude{PpQ%L0~VvQw$IGKgQ+E$7n1JV4Vs9$Q~Jm6?+Evlh!xH z9J?bYLCD1fG|(CB361t~UXCx*>1xK+>lUfu188z{nV+S1TECF#aoGlgc{Xl!g7Vs= zRO11i+T1j&V%{DFJ0lOj8&5>@1_9xqAIjK7_QX69jV|V>K?2>AiySpkXN(**FxQlM zrhr>*pe>13lR=9YPm7ey>5?j@Pvf@^I&Zv;-R3UCZ# zXvE+dMfM^rjve{@0hy#;IK4wCLM04!6o-@533D<3a2Gp;F8X9@5#V3Y!Q)O1*-x2W zus$tE)eSKuxP=7~_6`r)5=#Xl5AbyGonupl2#T2Y9BL9I6%pPb%fYXTFdGK!E;`6| z2pDLbf*+rkNnpGR&@CKNudY<>z&7*5B5biVJt@puFi%~@JN<;%O?B`z#ZG)6_Hyke zZxz-Jm}7~~UQEts0RRLm=s|Y+@vawA{zPH-`ASkJi#`yp+e+J$5dSE`l@TboJ6(po zU2HVI5RcmsUmc%)x%h+OOF4Z&&Zm$6yj9L5@XyNe-}8C7r$eq6tQfhb3WjG`l4JaZ z_123Sxbq}oA-hJ#>|=kZ73Nh}SA#LskAVcpLbP5W8_%?Mf&+OG1`B2naDu@=CXGzza28tEgMz(NI+!dN=?_S)gGD-s3qN~9&$L9doBAbwfM z#2;DVL>Vv&^vg_oU$PnnK#x$iLi+9O=~vqRP8ygei8*i+#M)OTV?J)zz(~~@2+}m! zS`)n7@ky1kSv?UtW|3ia22W0HppU*KXDXNbG1e-RvEg=23b8G3keJsnJ8S-(JaP@^ z&)o6tv7fTmqP@S9(%q~vma_kG;}eFLk6|}EjM*$0@$(1D^ojyd_0N7IwiLQTBTB0w zmBZz7TWa>Lsl;sbC9#{ccch>tUXsR-bKw78Vt7paH~og?cFB^sfY*&n^z{^y|N0|I zChn?u(qsKOg*XgdDtbP}a6p~|qft>a;;B%YcG5?%Rr~4G#p%R&NG}PQmG7IMNfm|K zl*H%c0ays7w#WsVZ=NofM0$*lOlw}Im~R@Of5^G0S{)RVO|$+3XG?PY=rL|#rf8hV zl&vu+=`OXgU;fTznJFT3FS??ygK5M$;e1j}U(ED$Wui&h_=k{FhLj*2WrV39?(+Zy zVKKU0Q;ISR+#@0sTy&=Zy-C$VDegBzP>QX@jSo-9SkAqZozs1h`Q92v@r5SRX&8(U z7bt+g7VGVR5N}u|fvsUm7QHK8kDFtl6DF8Xr{n?39d7Hx8}|kxQN~&j0GqP46j?f|0` z%l}MA7QU!EU&zFgr4XQ3)CxRYYc|Sr>S=UvS|;vlScYNtW6S4nN~*brY})HzN8YZf z0tgj@O&z`^_?*~!XO3X}ILmamVSq}%Wz?oq20?|k;RY4mJcX;YuYHXG00#nStZx?o z8N{yx9}^}Qn7AZ6^iN-ytmnD?SE&VabyE`;N=Y)9I#N&wg{`%ykb5MVKA6JjvPMjf zZS6$3tpgeWZ$jW|?vS8|tR3~+;=t<);3}P5yMxM7ur>Q{!e}=72EFhY&fBy{YCWsc~Amr^{@A45t&A;hs(`98}Po_N@u!WG|$+v2}8(I&Oq*})l zzDq}*Ke1aHWYAfok8<3m1SUq`ON7wxW219T5>jD6DS*4uG?#IBZ-AB}Na5;e*0KB6;Sz%Lua-Q18_vr^xT zd@S5aG>Q0?H1$F+fgxj1mVLjF>Fu<6V{JVMlSA2U{Dr_N|BX?5JHsmjre=vSlPj@l zzDb8=R^>QN?YgKt~v*Q~&7 z9gd_9vCvSX(oJ@+Av(*2%YbcxJ!%=kaMq+(i8LTYDGzlrKWmKc)dY2+sh|ABkXc4= zo@T9nCQLh>$OK%FHy(wfD8s1cESCtiKn@}KcswTO8^q)^4v=C0@uS06fem?^t2I;V zd^}a+D&*d(R2LVlS)w^-R2qy*IQ!87Oy#G<)eG|TvHOT)O;sJ=4Y+HAk^z9 zXe|!9T0@0xK^MnnCe<8%;>b6&O>XT@I{RDWMrBTO6yoJ^lbm3q$y%8cVA>>w=oPw2 zqrFHtr_RY*j1Z_$(&rN-v!WC@(h3$L!N1boQ@tdMh7=Oy!sls9Gq46hrEq~42@P0F zFY+k$A_*FGcQ!SpzNb%ULN$RtTdxQSs|D_I6$sNFI$WqxH4h@d6&6r%ah8RJmcBp- zo=_QHe*_$7I;z!of}85?>{RNsG5n&gM>#BtqS+LdllD7JI7GmU6r2M5L2 zT}eoTR4a+)SfmY$Q@r}7xU z_MT?AYpBGPHTD_vKOe21F=TPB-%SgEIza>T8{ zd>7;Q=%BT3k`^GHqbC?d{gRGoXdmE?>?|!1<>+eq&g5XK18-$Z6Yf;tMw8h39Ejh$ zbs7bP`eEu8=y4s>31vC+s2&B-?snu#Q}&5%sR|qfkPW3ZTp>>er3FZFM9K!6&mfq0 z`4^~7AspaAWelT4fTLm9&_ZosAqj9c%9?Tn&Wbs4XJADPZ;Od@)R8gSz2%zJ);b$+WOUOdjOVC0bbLY9?aQ3%Im8`+HB^aqLHW zx14m^Ka;vOWaLdZNA^EMzhJ3d%iO1XM>`&M(#m`bRq}okqDShGm9`)?4+M3cz?0u-5);RLbIg5oxTUStJ8D~U9HFz7Md@jIzu)l zAX#W`n?YztT8iz@2%Pl~3_lG2c^|VyGU%B-*%T^S7YAn=m|dze;;EaR_0GU**`~*; zhqvYt(au{@o|R*tnaun~Doi7EGVlGN?ipr}x=KuNG)~UM^&b!vB3x+rzzE}gk@N+g zhWAuj8Nno@BO=8Plj_`)rIOY`#s!-`-HfUqY?bsmDnmC#+70Mu6rRnVZqo;_!}Xd& ztW@AmvHb2*65_YB`sFzbRU*HGW>f7$uRe;p<^G9xOhVdC|R#@uQ=UbNjSni8dJGm$`h0_;tlH zWCPGO`Me|^rHL=(B*$=CPXDPmb31>Swa-b+uC!pdOD*MhX_eXZ6@|pxP6sRm6(S&A z&V1z0A@(lf%`rcaNbg88O+)ZuWA^O^Owz5d@mvT3m6}(03oi^x4;V z`idSvynJzO0McWg^;&-KYrLqS9Gi*>?gPb&Jm|Zvd}8(4J^QU+iREhopH&5HvrK6v)z<3}yz_i=Yt{Z4@nZHe9uOnq=bImCW1L09)&%0j0048_^pTX~1ChJj z=X9iSDZD);2zTjx(%;_?Ww>(9xK~Lm?ZWfAm$+)^6%A4u$EjhQUd>P z-;C;t>CSU1V)Yh~=}e(Chgc0A1~Q#{qTald_da*|S)FOp{;lemfQc*ec7T@(3A0{b zyEZ9zamoZ|5Jl%A7jz|PCavj^)&w>6|1>h+WFIrqwkcxGg}E?G(?0*F&!CJw)$1qBgJklB$Vhxv6W{`;gP&-|=l zTmb&Gd0hz~F}=^!FC~XdrH2o{a@=VDk6HP;^I^gB0MS*~yeOqR&Cu=-OfBBz{>Wt8 zXGXwIo}t!8*~fGvr)~}Z)&)$-akDVHZ#>CtctyrxK_4V|<^wEC%p-H9}>1SyMMSc}xTK_v-5V}}*vDq%SOUxb2 zj)N1uOZIHrd_TiBmgcqw29^v2`j()iv&30cU>@!QynJr3F2i%egQfkW9n5mi5zp}2 z)0cCBWU|0hIFrevu|%scTNz?dNb{KM>-Y3oo*UlIy~*<*rP1G$>*JYYT{#egL%HH| z6&9?S!fh$w+(zrPbb3b`r%C5RwWG78thZn^34ZGCL;xX)u=x~EdWX7eM|X6++Vw4$ zZQ2t)eLY!_J=bi$nkPlv)qq?&c!oREP#P0JB0<3n92jqXQ+1t(B9Q>#)qd`scnK!0 ztFyU5P7(>wW3{Qt+08s#uE6xLnc9?U4_?~KsWsG^PK%TZF-D9|;YctQQ6=;-rO8C! zR3hzwniHsr=KYTd56BW$WXeLQ5WJbGFYiRCR-~=&pT;Gcl+5w;5Fi-VInb9DXR!T( zqO~*Zq0eMVLhYPGDMA}T@B_vO6pUyuvrGgxtP~t#00bk!OnEYY_B5SBnGM>DPDt*8 zLV6kXW|7t0AQk0@>E&)FlIO;uWvx({#iZr-q)&rYVc z!<4tsjr`cJ%>R6x`<1c%+V%jyNJcFx?j6+3cy6AZgzS4&WuXZ&l&a`*+3-GX4szxC z`Z&zFh4he;EGY;}E&E_vxt7Cl##^9EyX}Q`!`1SE!QOpSHYW1Sq z5n4#vouCbN_J_hAScoLOVLH7Sornmx+mC>_iwP=xCr`S~s}fT}E3o`R7l#jNwBvw< zsq_jpi5f2ObgK?CKIy4pmoAq~O?{60B7sbGzDUT_@N3crrjU7oeG9j0=N_VH?T>WQ zMKbgl7j{QDdOKIFLdws2BFh_b7qCgC8ZqwuF z4_}^0QR&R1*K4#`X~SE}5gn>j#ayNsa{?|riivGlGjoTuRoK71(3!483j@o}K|zF< z7cl}#R|h!>XzN>bN^ouUGI%k)mxxe~IRre!hv zDZQ72Ep$ak4?Q?pnocv8Whwa^62w~xJ0%k{OvK5I2Lz7nHr$_N+*1{{1svHn9qBRj z@`#Laeln4<8nbm}UyDl)A0j!lbP>xr^Q3QjCz2rlc~80@p@)TN-rk2 z=m3O=CuHx@1`eYb+-+|MRX^aqTZ^os8=1`E_k|z>Q&sVA{6{{4ZkfMJFjay zuQT3?Pt1?%=qbDT78SfF0c-psb{lD^G4gmu_jyF|jeKu^O9go^-H=$nFfl!Vq;m*O zTZ@aB=iOU%3}^UA#lM)&eXMe=9=>n}#_Mx29T(FjZHUHne1y`Kv6dS>Y3e;Je~(g^ z=E+b?)2vx)@3E3o8QJRW-|sweAD5wCN;0gs=#pMHRcG7PaEBno9NFC{xvhCuQFDv# ziClH&IMsPYth2K%!rh@On_MtvSKICC^rm3PiZBYk(TMj$j9azH^w6S~giTOz zs175NGqA@QB?xHh8xpX^0y;36uCWq_f0)RI`rDbg+HgTP0)?q+5sVAh2?AtjVU$c- znG~Tdh$$B=V9MLwYpW&= zw6-7-B#%2^AcH>9@Eec9wzP%%j7+lkMmUI?`x!N#kw_=x^H;Z+-u_^45;HBZ(Z0yZ ztQV@O>w(ZlxDwtl3_;bc=F5e1@_jj z;taQ_yWQ{t5rrTKJ55iP;rr586BWQ0_0bL4vbjyQOe5*7 z|6roKp)nmLAXgEzqz;!s-EV($BJDH~p;i1_Pge+foG2h!=khmN5W*9_7|hGf)z?DC zh(e2)GKUc)R3Dw5@3b;acGJ^jm9+VVbwpAY9hsPeBODTado6Zb zS6^-ymcM<Pk}MQcYX0A;fp# z8dwW^!%!4J!uBKzbz8wYY`Or-bu$Qrt+dj|bh_tDrO8a<+WAsac3~Zn&549)VaJ^H zM47s4BN_N4zbrV(^Pxg00R0#8yEU((4oqK9p_Qp>upphe^q@jw3(y_yYRKPBk2%WY;Yzw(79F4?5;S68;s zD;JyR%MC$I_MLEC^RFgKf@Wi)k6SF)o2$D}lxyHhc^Wa*AC$;!RS+IW-e+ z_NigZZLaqFZKhSkJhXHsUEE;(A>TGF3CR)&(l%Vz1v5tnPwM#x%?flq!`@k`*|sHo zdnUElr>11$NzIXPTYDG-+s<;><5Dc{VnP4&x(x3I%nfN0I1C=q zB}fM@Kddm{NBMhcS(}Ak!+G~wXY=MGT6PO8+a8`fcgk-#<+rj{L)4U{vA0;(-%dZw zq&kXWEf2)dLi!HX=A*%wKi~JbwO)nZwZiGiD*`uaqLRPa!oF`Fe5 zsbpq=-tMQj$;BW{$d01SJ=S027r!nA%kfRArlIUDsl?pECpFL>C%sT)cs*I~??kOS z6d2~1erHrkJ7(}kCAu`kDv5UTwq;b8Beiwyu|A}9u9!&b*evHR;BTcq|E)w(?``r@ zWg3ZXxDqW0$O*ccO?vRCL52wo7c-9yTrIN9w}I^H-;NG8Q;QNIZfo5T_<$RZzI=JK ze`v!!xTJfs3KZo4l+!2kx8FMnFS?tsb!!ih%w}{?Fp26Dy#4ff?S{Juh~xcT^rm@= zibyDO^K2Oisk;eV?cYt(YoSQ;b_Ng+ZT+Xi`kiCL2vOAb_x96hy=-NmduQ?l(gbwZzGB_udk8*P)7^u=qEQf{(76M;5tP${R;6sV|d5L+YeA z)brc&=~>y%GC5O7o2rS!QzdUS$xpmJT9s{*^OtD5C(F&FTKgKUd9{}6Wc<(k@XzKq zv^wE{Yo6gr89i3zpDm}k@$SWQ&L)#0>E)jF)yx)}3LR)QO--7!K|ZufZrqQ?Ud@>n z;x+6apig89=!yzrBx%%`p2l1YH|yj%+~cX-7M3z{Zm-Of95LV5qHZ0`EwGQBZLyzi zQQKMvZ3VPFVyH6y+(4S>!8TLs*cUBNebIs`Z4Yllc&w=wu|O-3ZwS$J8_o$FXhZ|i zhq7s2S{};FSbB*l8U$_gpPOlJ(V9W#xEgoL=pZ*>p_9^hbyzEMX@w^Ga_2Om8CiI- zAnNHt7{{dp{d9UX^6O^026Pnu%wptgEF>VUP>-j9)0I=T5NDyc!+%}O z9OHXE{GKA|yU2rN?xYc)Y|rm>q4MzW_Uq$DK3#Mlzu7`o zaDEG2F|``nW3h+LJSOQ5`#jhZ_i(26VglCa0tYVL^zy1{SWuP}Ov_~^p+?K9h?7;R z8z5wA2xpOcR*zPI-n7GgnBn8Hq_2~pNlp9N!3OTkRMISsh+meiNME}EJ{e&q)ZEOZ z)$qh1K7CratE7EeF8*A*0EV(0m)*5U$PBi}vZU+tFzF44(11h2R$D*Ky|nm8EbrPF zFM7Aquw-b64XLN2SyUjMN($LZw57a4cE5^oqYC!sd})Rr?GQ1NOQWxz&QK;ML*LA_ zkOaUTdb9cq_7_7^SxF?0j%L&KO(Xkdn7PrBGOj2?($nvGU;>GCt)sy#4V&kl z-HBN?btMcvt|ocLX4zN267$)-v7_pXk5`YXF^@jMrikI^`NbZ<&oZ2|GqS&8H4r;* zJ&LIX3a54_F>Yr`gOY_TEriAmL`u+Y{e5N-c;JwLW?3}APo6?Qn1Oo9jCmZ$E&n<%kk?exGAzb0ob1 zS~mp)^$QF0Vm*-BghbjWk`Fm+C7n=4C2np(8J6;>hM?nqA_j9nmvr*!j{T1#Wb@1u z^r3Xr5F_pMKH12B^lf75Zfk@KR8WVEUQ&}Hi0O(#nT_tBcbw470t(S%kS55^Hgeap z(neWpSg^PABp(OJt}WGu@WiGUN)K{RP)^rc2_#n6$!oJz%mHe+5`00_~RkE(Lp1%#Hq_y=L>bS0;dI$9XJK0DFJKB*y)h+*8P{D z=gKS}y~@;t!Gtcj0;Y%QiqilAoN zu_b*RrzY(&t&>3&M(?STM8a8S+-^{yR^V!i@jv9Fu0@Ad0xWZ|iZ;t-$Y!GMRuZh4 z9(OFg(IxTO=O-}CW=@dP78j1#{Ae8Rf~pOmx3z{NecS_b^fV(~+_KvFa|kXj&R@&C zYQVnQQndGlEZp%EDR**vy?b&4>; zH~Ba7kNgcCP)WMdpSoCqWP=b_7q4gvM#7^5-Y#`BPLpn~u~IE$`Wu=!&my{BYj~i| zJTpTtHWx)SRPzQBLu()xUFM!#+^RFI7l&vpxa{lV4lL|_PFTh*U@)bDAz7cUYeUtobTjjd-}+^ z6%NA+2bzW}9554l3Pft^W7n`x*Q6_WV+Mj^4PI_VPPQ$DRb%=&`XBR&WGO0F^r}pL zLo)rwLEq3~IJ`_t4{PZ>JI$W#mBTJtvzTI1R_skcQtG(fq0|Lh}7(m*a*y#3O$ zdBi$fM|Vx}^l0h%EV95WJoUQc1p~HJ_F4k-ZxXbApWru3EtJ~t=t{j$o0&o z`l_XxIwoXUI4Uvzea(PO7j{Y`)(@3z*X186c~u_M-}b_yz*XGeO!OO*bRUyvP?VYW z)a*o+gg?=Z`?dKS-nqnCGvKr??jeHmCC(@^Bp=G91>Dl5*N`gSSBE<0i7oVFFcRtQ z@yw;9*L&7pW~NO}e4DSRt&E$n2Yl50LU(RQG+;-r7OIfhW_&{`n?m~e^{w2=#O%Q- znPzyC^NNr}Ts9cU^IpTP6S88akC<9>r;<9h8Rq9hm~GZ7!EX}(;IvBh5=ai07{niN zOUX^$=$vh={&lR#$rU-RrHGu)KAFn@vD*4HZ~BZZU_N4Uy*Ek!fi}X-a5qFw+$~mn zFKRTbU|QjnC9@UIO;a>w=YGtVRRvbiCNFah`uF@p10^qKDARUsCraPB>xirlRLjq& z)5{lOUd9Z#Co{^)Oww8tL%fXl+!jKr=(^<0?vSY8hH;~B&~RuG*l|kB)@>xUa&2oy z{j3R|wv6;5EK?B(u;#XjHNJwekE-!l{86{NtA^|4|Q`Ym1E$Jwz3J(qzE&6f~88KCoIMs2 zGlJ&*z0Q2iRGO*f?_!mvk>3rQSlWNP*$7_DZQJHxiYlyQ@ zqNnFl%v-g@k-en`wS}q6hpwtASw_qkYMA@oROgTBoPFv1RW)iD?-^r@Yjv|b<={yOHdh8>aN%}gyRmSoc5ggE6)I!%uWf_(-p@58K`xs?8XkQNU(m&%y=o(|b#PuC!^w(f;&yOqJEDPtm{ z>xiZe$;39uoq1{>2;MOO0`2!ql$h(-$z?B^v6Q^Rs+>N0r>}A(2Ch)irN;9aP7afW z;_&--i1e*OppK9Qwr0nqnvsc5tV~23(%wZMFP0(ciEhXwIb)xAqMvx^J#PB428PbG zWLm0$Sf&YqvbZgVVsu9u-=F~lwTE*h=*Ow=9fS1YOZapjUEN_g~IdxR~^0T`d)O3P32M(0H<~)>=3p`G1=QaG-3MNFl=&oq} zS`U5g=9f|4G7os)er54m4~BB?;9WdXGo!D(ZrrADzd4Z)$G`qIeM8mOKx*Ed$d)?a zpU6J1**Qr|nVo1_K<|Dro!NkPF?VR!nI*>CRcHUk{OfFs z;T;vd$v<@GAZ?lIEFg5FZ{!H)W!ny?YeCKdHpk*%XE1MNqwr^|$2~h!JtBw-vwV(Q zMb(SxZ5nz^>A7N(=AVY!tu;TZhRX~YQ_sx9dgF1cNn8E9$!5OXIy7&p{Yw?#Uds9G zPQ!mxn&?O}?oRis_UmK`Ygh7-eC|b6$(IaD4$sQwzo0T1&v>mz zRm>en4swsH7><5lv#6Yz>&%g|DW|4V-69 zRv|l@k2nd)>TKh6h z@}Itu@8g&+&)!UgGqw>!E>Y|idU zBbRa$5^R@vi@ArCfvd+o$_PAa;YR}Ltr2AWA#)S6_BU>)4|p-%J3h=k90F%JfU0(D zB&ak|SDGI7UR>ig)+yDXo6|;DhBHMue-xc{)RdeF88ihQ!%k+Yv33o;%gC((k~9-q zHb|$1aSGPi8c}>%OEVf2aP7osTBBcR03Qd*egUx75)Enk(XN@1+O=Q*O&GcuT8m8_-_nDa`4P^TPK!@@Q zxy#Df>6!z?%V~qqEY=4w=54%-9@eDH68o>g$U?fiN0ASsB)#`eMv27=J=-D?2I-T@ zlI|YvYoEF;za1h2{YD~;UCYR9Ek5Rq-Yt~-8s|4~vK!e0#qRB&5*ZZs+`1aP)u(!> zcVpqDJ0cPHw`iRq$#*}uJB8WJb{1HhunNh zvZl!Vl41B&zu0(=kF*nRhQ%dfZJ9kOS%tW(#+dFoHCVMk$|9mIA>%ejO z@&mNMNJosKn>)&(#nSgzj(ue=GEhenq)YhYtohjlzLLHCBMrcbCYV{`t&MWiWe2Y@ z&e>1<PnUuVqnXS6+rhx_E78|0o=1GneV%%%1iU?6CA1HF9bJ^CVh zUZK69VEwFz?v$rPj)8>4cEsDe_cCxtzH)`AVz`)Y?`;_U672JQCqR2H=57)>uQILK zS`%IS7)ffJJ;FUH4>Tqvo8B6_7HPd?mZ=UIZ=*3^(0G;1aI4%0O(-qa)Aiy`ABuxKPi&!#~7n;jHMvs+Ia%vxwUr&*qx0E^)i`?pDT8RpIV#sdUhz(Ex|=Vjd@ zk|Ff-1Fcw&4bmGn@}^2iXE)9zog-(;wkIodn4}VhDOtOp)RonfXd2^wm_eS-5TfJP zv6{UY;b3TR7~fVTncO3cJ%iNmC3~+RnFu3}AQ{fv9q2eC;o6fmTS+C8fRnTx#myh{$)y9{KV4hC7u;Z=7i|9)*Xo?i@Rqs zogZ&_Nx6I}1dR79B#FaOA;>2VF!@4`M>Y^f`d24eR%p z%u_xwRj2m!cLxU8)zyOZqgM+g~b3gRLrT*oin?kvb>!6m+@N# z?bN`K(Mu0GAQld9dh~!pbMW|PjgoFmRshxIfbLU*ldEaj*|&y;)(qX+yiUWO2IiYr z6-s(GSJv#z0-GlK+EL^Z)W04 zl8IxXaX{`pTJ|ZcES2gHO_$q$WYR+e-MItHd)k9RW~X$fy;H8a(()g6eRsN?F(0qW z{**OvI&UL$n5;UzXKZ7`cN-h%gLliYW4ZfU=_1vxU(>=F#d+G?rQtl>NZb!y$I^7I zbmg`;)Z>h6I?Ybo)VO@GgCBjW4gEtikl);5O5et^3%v z6Y#{8{mgir?3hQ(d9+&OxWcncZyJ(R1oq5yKR;13nfU~8_I%t9FMmYp{Ad!h=*;HF zd}QS9n(~QEV&J}X@JD9%!$Vky`bjQ}9=>Dh2{T=~eRH`^BBmd*@I=z{4pLFGWG0!V zv5cu1`g9!`WP(EhU0Y_H`_oH7v9)*NnE%wNqsP=L+#j*7d^sP(;0@i)Ks_x}5Sri0M5r)hG|bDsOY zuj_mLoA}hTN!h{&;5zCJ(Udh9O!XQ&`FbxMTb&6t54@ih;r3E!c;sE2#ZP#hwIqB34S_%Plz(7-aF%(O2h$5<>;c?PXV>DZac{JxW5MSLy9JQu3O87JVwmRLMNg zd%86^59-4Mj~C$$X73ig#lm5G#XwDU;2k262I2iwGE8SC;A%0n_ayh+j zpKx9J&;m;3g4}%H8G7*OB>DW>ztHl7^kHf+OgG#|X)+(3x@zn78O23&(BrT)DuPAh z8E$;kX1KGqK{+l~LVtWBGx-DNZSmm2ti>BND)Eq>Dbc^H8`tf`M^v{{)*7O#k23!W z-7s`cSCKrKcp3jAe$~@Eg!RQFFT6xP*YZ@NX=59!T2w~(PmQ{=9n%KQwO8rRTgKZm z`L;~w{fU|LQLkW8gV&$8&Y+Enfh3Gs+7y^D8fS}D)?A}Mi+BrE;&fu_Btd_XO3_$V zW8{BL6AEUj>ah*zrFR#}wJbdC)OmvsZ~=@}ItH<0MLR4=E2|a3Wx-tP>S4n{al_Lu zw0^~S3lg&nKt4nSxp!>}_hmt%9-rw9_KD$v{w_=#TM80SzO6qi)MS8B2{{R+|@3QArG}Pci%+R~Q ztZ$^p1&iIFm|b1jNdNc}U-ulnkZpREKJJ2gNOoln|9q#s#QG{n=6IQ>Ik%p-F?Rzr z!dIKsN2#rh9k311DW$UEJlbD6D`xBKv?2FoSzcEUI3hhZI@_jy^Exm2X<*^CcZ^k{4+=LgP)sS#o<=A6D*)s>$e;) z=38yWraAt0vG}RUG*cAr+V8x7@;fU{=C)rsF(`tOu}2qh{9^t#wenN>mCV?w7lT3m zjCg)?&^?+0uhD(eWtB}?Pp(_6^=}~FRF1f*?sY&GJe8xVoD5k%?~THB7HZLP z*z$`p_!*rajGF40Pb`lqQ4|He^W(Fo>IywoRMRZjHo>M?ZE~??p+Wj<1eLCNIg68) zzi4u|M~&st6~Xpy7?&jrS{vPOOv^5-EYC6K8R>LZ8dzopZ zm6=Ar*i6ekR5=gWGGiVri7I1R-?Bmai6DHG!qg~cL%u<`C*i&}qYc3}0n?&y2@4-0 z-nVV|6GD+3YG?E1i`j8|*E}R#b=3|Nia{bgD0GC)M=(4iXW5Vf8au z@f;&E`rxQ(BW)`}k6#aOc}>pMENFA|(IjI_U(YfqOz@A}duh!GKF8_dbRdHcx1wEL3L}81+?qEw(KU5Jiz#!%`(XM73KtzSoxk1a1adSF?Ad~c{mjFW z^+};qBklBqUdCq0Z!11{2jb!#PX1-90@fpssNsa&36@c=;uuG7j@Xtn9rT+VZ1lFi z;4K~?hDpW5m=$mS3zwBve9}A7DwWmu@cMhu5IQ-JEB?q!SI#q{z3K6TSm3bi%xn8M zH3`D?xY z{7qShWp>fOe^T2#wq|=ph|rcSWU~wky=O{h68Ue=PNG=c@3QmDhmb7cs6eEgtTLwV zm35TxZ)KL0l7>D4cF$dS(Ah_Vr-tTg7&CG(#yOkF*%U2v8f`lc$O0X-<0eHpd<{qc zR$j|uAx}2(J@NV+n;@Urb}XMXjZgZ^kx75i;vB-yMi0p zY+b=UE+Up=?3SUUvrT)gYdHN43N6lO8#>4QqQih$9$k}MYjJa-KQkrcmL%SiRMo;# zH(PZz3yJK#(Z}L8T+3DEFwSip=w4_V0x4Xu_V)NM@8b1$@#E)BwdYNg=*17G$Q5Ao zUpG>4mo%KpVw55@@oK({7&<8isZ1~7kPbp_03 zg2GTCXZST#)1%(N?@%pG3dmx?`?jXzPvQ1Z#eL}(S@QVtd7a1SVS*7|C^H85o6Qw3 zb{>9yd~f;>d(##5-F*Z6cWL@Syy4|2BAaq(;3{j#M~~HyPOB(iOH(w4?Q2OF|J+tU z{r)CQP0)W4kk8b=l5|pqTgAUQ`kg26^@R=T;Z$COblu>=ob&#V?(JeByF~5cKX*{cgDtftg?LOx* z!mlmTxllp`N5@}`kGvSq+*X^!LAHBcu2`2|6m{MikD;CxsB5=t?IZk+D@OZFt>@y; z^QI``bK#~v=h2wv>aSuKZQ{*G;#IAKQEX@an1+hAae+1zjzBXvz|X%1>#;jt5s5(F znXpS0LU2vjnWx4(c%g$2Hf&Ah708%$)4dy5r_)cLjalwghrfw{9|*Kh5j4BHXKkUO?es7lfhkZ*wbd9I8G@X@pLU}`is+iB9WHmp94a%8Ru-6Ju zeid7n{5AMD>SNW8ktv89d2rJlHZVHWJcThIjT#oSJNFeqpm9B-j+npF*VL5>>V1ai zGRF58;Ul`g2yV1G3`e`^t7mwOi26@1AniPtp#STb>4@n$^PhAo4^UZoqsbZx0}CeW zd)yEl9V^`aY8foYnn z8pT?1;fAN=wQk1wgj@m_LgVbscdac>sFJX{djCFp-7~ZznQpC51E&OJ zz^Xp_VJr=v|Jj3R|I$Ye5SzK$PcHF2XDbfZGydA!>(lorucvi763#=|C z*>eBu_*4*HcXY%Cz&~^3wMWNyZ;R{u9 zB!u?QKsbiO8#{w&g+t#1hhoGCMoCWx7$^|A1ML+az`#gi963&vHZ9&JDNlkifs(6KH3HJp%iOIhD=RhUs)bp2 zZjeeCp?(Oj3P+Fva0KiE@PUkYGR!IxlkYce@ZmKMA@K!f=m|D0(|;V+xQ-x{b9e_D(3Aeg=&cF&F` zvtQzo=bC@V)#6o`aJ430(p` zwoUR^r~QtlZE$Q=7D6W(9jNOpZO#uQr&I0?(L}Vr)4r9D!8nMg!mJ~)@}*3CmS1Xw z!))wR8fv~wlR$g`H+E#bxGt9{CV;fNOkbzlyp5RL#A1WYyz&s57*jGUaaV@J0&@~( zmmDFJG1KmxEt#OabPvR;@!aJnxWr7nG{1hkICbGvF2zPpOg^mAb{s_nsV=g zUK}Vk0Yx9`L-uwa_{BiL+Vjf0(BnNbKT$qQib)G(DkONiJfF+-2 zBS8(Zk#Gv&+vmc)H*9Pyxt6HS>#ifov(9ZHZg>GO@xrkqSPNDZD;dtP?i!*jJU4>v z9o%=OcpzcmML^17cd&g`Xhm!pJXe?1LQsGf-$dHn1bzu!1G{$N+M^64DI?R*wS`?=k(&L`e$WXxd_I(zrob}dfeex$FT znp)itAbxuk0NkOaonm_g<#lINd4|~bVPet0W2%H(E>F?vPm%g%{yZO26Fj&WfQ-v8 z0Ak&47F}}XEH=XSJpHpdFXUIRXsX^dea~o9;(PmTQ0hs>u~Fzn2CgnCazo+I|{;l|1t*p$%||A1;S7 z!tiPW3)d9g$1&O(PBVe%Z?7;bkb+aI@PUx$lgNv1=X7jd>?60Q7m{XuoYGXlqzKn4 zRGd-9woM?DU--#__q?}y??$& z{x@{@#!r!|Q)GL)yUpck^LTNRO5s4hHm?mm&o-MUKj`$@gCT^;JI&0YTBkB+4P%Hr zM_Amr47V!k&Gk+88mFrf3m4WfN4`hWEe0-1tlxP+7zYDM8bGk2{buxfB>YvgOfLyn z?V^@dRJtd86#ZX}xUiBo{T{sm-tF-3Az8>i&kK>jxi{YD7LNU7_&ZpV+3h*T7Na~^ z)KEm+fPqL9fS`n*jfEl8HpweKQ~M8M;|KO9i&iV|QTNpL4Gu{jV_Rvk=Y^vkkhSA# zo7>m7;)SDO`G)lJWH%|5FPC)=Hk?*@O;mr0OX~*e$R=&w3piD1CGLlOJ*R^Hc_66F zI9SPzHF`(YOgViu(fo~?b>^D4OhW3$ak}aI#L7{+nL`Pamr$I|^F1nDk>Vj9LmlyT zmG=QP@N1z%A z6Y#{Zd4DG1q4Hb63-_v>@A9)?4|byuj1jN|w|6^RAI>Pm21|;pz#i^$WCfj^d3P4PM80gNY?1^<$djP}^>CXClkoCq{-Iz5STm%Z=^ZiteP z81bNkdo+{b#lugpI(-dsW6u!o3iad~%U~>zn&xe~p84NLWev~=>a*k+ADJpPZGfK; zx*cXZo;6pAXWh~G!Wdd^p;fkqVDAg}MkTtq9ZCive?U{SW&*2(gg(y*Q^EcrE6xF(@uKo2wB0*w^BXse}9r_TY%2x3j_*TLC0 zmdv-+mgdkI93qP?Dg~A6co!R0D`a_Y!wq`>k%Wos8JAMc5(zRf6h=7#cNRvhGgAvY z;4T3n1oRk~3&Xu(dY1wHMEz4N89uz3-k^{TM&!Sa?fI$#yDp>DY?q1_DXr94RvUm7 zRdU!!3`Ge}JZHEcIqbNDz%0@@2mDt65{=lJgOqKoY=Y7gP(A*>bUNRVi7%4|qS(sv zm#7bqM+QNrMNtIVa!hF@M$;uGgu5rscOYw#JAVZR4F>!F;g`tkNKARHq&W-qU>wgN zE0Q^@!EXjY6-}I4Ey?FFX@^D+oCH)wMi5?*gqoXOh^-@c7xsdZrgbG4QQ&v;4Ueo<1YO?P1_#t~S+bLhTY7jsK$o zrv6Bu@PMAX$f5wvIii)|@Pl35^5guaBy9+K?iHan+p27gk|3v%R7ZPJYBkcehRcxt-Z@OGlCw3}Az$9^N7H~b}0 z)8B)atkbq3EhA4#cyieFNMR|dtjqH+1BLz6t*ahQB=+$~6aV`mKAKpFyNwU|I888f zqxdc~IxGu^?+ttn0piTbc;i9 z)&Ih!Autf92fo%D7!(%1YRYy_QpVE2vYhxkmR`M=uH;Sx=f{5gFt59fiAM<9aHdI6G1 zJ^=_n`CH5HQdCI7tbS>|)J`$9ZC=!RSHP;HGY=a_YRl6=4T2fp@eqHDxH{okqb zHDDb0e9TvLo>qQKc?StGH;Jj_C=4p*Uf_Z%ub0F2kioyD&GtwEJ1%7G6tt=vp`ms%K6$EJ=XU?Rx1K-j1|B$cHYQ*Kd6$!@3iFI`0Ra}tCj zmXRWz8p66kcH~#uVL5qmXDxBu&SLS~65jDP=XuvT0MaS~@>@|>BJ#YssM%$4hy za^Re)()_?Zt60XGWR`$AKBy+WUaxEI_1u@1xU&|pq;PqQnA%XAP>~)-`Yoj?IRU>d zPFSAk3@7EREB=k#tL9%xFl6WABm^y3on&%5+9$@4so5-UN`}(hmy8t?OyvJNh?EE! z5-sC@CXv#d{`W~_Wu%k;(QZa1IZ~l!p-q+X^1I} zB)b6|83+aM9KSq`)HkIOX%<;H!60`1?>tiZktbUt%_1YQ;3Wl~Yc;(wheX0N&ORyn zCySS+FwXKrBxKqt#~iZkzr#oTfI-YnO8RI?&hY_TLwBenWJ9Ye)YArKh9BF!Xj^Px z!gwmpCJ~WRfbElLba}b;*0u!9Sc7ap|8fTFoi7UpGACw{E<0#N^2hnDBHlzPt11-Q zp>r<(ZnXCNzXiK(sfR_Xq|JWvJz5?=!AY<3uzHrQ$YZgWH#!DPl`J|~%xxAP#$2}e zu_>5L{`;}usJWHR3KU<-){dp&9Jf7`b8Ybo7F{8fevs<;U2wtK&Bb{Id3)k9{$RpTgoOVw{svd|27)k|JA}GtP-bzweb0#YNSabD9!D3%T>c zVSGcd&6^fKe}^=8^)g=4W4>M7acXH&JPU^-6F4od z9Dj_fs%P;vRk#FGOf&YKh~w{=9@)!PI2h;liK(Xx`Iv||A4&LfJEz~y9egBlY`yrx zl#?3$Le4XM?+7%>AH6O&nFAi&3(7lz1(vlBQ%jOcqE+VC>Q;g~@O&M|Z(dp&wN9hf;+k6}>(6B*Vf8OS=p3&;J$gKR%7- zc>`}B1l0H^H?%P$(<)|@z)T@eJ$_LwUQ}ZaiglLYi^Jyj{h{}%b%uv z^b;w!wXw#M7G?)Za&JQMJ5z{vY_9ctbv1%;mx@c#IglcTy&^jPo&w&?9XUV6be?{% z#(Oy-;Qy47B$fPWm*#i<{*IAX(~!W7wcU3mb7b$=XWMk@ky{(a)ol+GUE%Vb=F9}l z8n0M&-DUkg1fv{ltSqbiZTx1I$1pp>Cb!eKviPvwU&d=oaPR@+70%hbgm#Kb;#d)g zu(>>EHqD2;IaY)WRYXGVTt#H45bOQG1d@Lh;ifWeY*3KRpIs^aMQk|)t?K{(0slWo`0|ofz>_cQL$ztkQ$&Ee7w+Wg zB6aPn$+XSRFUd5=^ILzhiYup#|6={`$D6-cH2}l8n;i7*sVoF6t-n~2tCw{q@{85) zB=|b(FIL%slrJk%^%bePQgZEo(+J+lbUvF%KbWdh)5rL#W12Z$DQtNnk-*zl(BX9# zINkiBWL<%Q)sQmt5sM!~$JYeDAYN}wmMfUPw9C;1NErL0QQ<=I$iHDkrici&Qh0!d zo%E0yJo_>UZ$3Z|WYV|w^m9FFqA#Xo52EOiNcJVzxjm6=gu63{{RX~kzq1EZybAah zP-#s#!VtWKEqE!yJo`F^J98vma&Vacz#%DU=Vz8lu%M_EH^5h?*`o+3Xl@MPUk#xy z>T$5y&5hhvV~*c=S@HlOrxvlG&`{>8|4wYl@6&`V8Nvn*Ykv2n*zw^u_(ou5;3+Zz#WKMoAn0q&pK$0E~74(}>acbC_l=I3oky zJ;Tbw@RFHX5Rq(OIBz8Kb_L?MdWBK92HzkBG7Q^2yTqr7#BF%UE3EO5u)01IZHIYC z*Fqa6J<1#me24xj!=y(SURQHBG3G$r?qhODpE^XBd6~mmGm%_^TeVP2S9t&&>J2fQ zh4Y@AeHa&Cqe9yN5e(0HUdXy6Ej*K0A$ zMH{q-=Zn}F0A!!m(}yHv4YMyN!_@5_g!OX(s*nx{+4~0Mehs>P@VXxUI?Iw4T8Ui) zC_D17687Q#tX7=V4iz57m9S_&9cLLCi`)r*Y76m~7xF#Dq#|S?A4!|7>8n? z9PEtZ%*vl7ie*QLsV9y6t8U`=8@NulLV!-#NN88#NG%L4$11>5J`;Do)AT+s&Na#D z)vT344hWsIWYc2K zZj#vS2)a0ZuWA0Lrg#-<;*LSyVllO&=^Bp6WBPm8GcJ15OyInc_+($eOk?|l z!=|hG%?V7od1DT?z$-Fj?YSo($E7zIH1LXf=C3)%_V5T(MW;ylZ_g}7lOZ;8S|Sic z2QPj!Z%e^fB^HJr4eZd<)7E6S_Hk1C^{r#po#->dnk0l^bFKvC8T}JYy~>6LnFJI3mKDc4QU_ywMO~p9~TM=q^S2~ zVfUvi0aYTB^{fP6>k2YYCw~i`8jYVJKV@1eYi`^U#wCZ|b!-_;D@Capnj#B#3bRVm z(5nDexGNMo&vCZOm81~_47Laay5cTy4_bDSlbg-z zA}qk>!)}_VM2A*cM$EL{e)xOKs|6KTyNSbjwOfDJbnAMzN(%1?(Z}srbWXoE-3J{3 zaP>&Dw&z1i?KkfYEq0Hf--xg-ueNwguusxXayqmkFu)M+YnjJ)v`*Y4S z;%paRpF|fW1u}5Bf<)3`Hs&mPHtFTXR{qB%`X?^_FeQkN#pz}#++%DI@r>#lzwAuX zcP25nl8%!j^x=F!Q`haz2!kN<9lx!)^=d|dSM`?SI* zxzcnQSsz<8%x>H?Fq>dF5pf@?iMWq%`I+&T(>`d>|KGR|QTRV`AGbLW_t9f_4!HcU z6v|(4K3Zs+;!L!yt9@=CN^_V+M306^1O*o7fY8~2Q?!Nt zu`oN-0lz@h2<>${m&gWf)rh+6boJArT3J;eqbv)#`i!Nb@d{ubLg?T|ySiy^u4XXO zXRHz#ovI}a>LZQjXu}X(U^6dFVHF`;b%-f$>U5zhEFVNBM;M3b12ikx(YJ>Fsaw0I z3=A3Ert%eqk(ju~oxi34ljJp0ej+{Yb4%ERQeRmU+Fwpj#YpWlc`33$g#MT)iw+Al ziRUjGOM|_aGmJb-X+fD}e4uY62@fWyFtWoo_pL%`Uymm6yD8~o%A~A_STAj&FjPvG z)K!I~-S&w2WW0VVn4Cpqm|v%)-@~&J5t1Kf5JOrE8XNQm? zEdZ)IR~pWCA&To?c&7hiG2Q8>n-ZWlUYx{FO~h!= zRR%By_^E-#h(T7gqmfGgfgVwBU`VPuCeqz$k%ArU&-Fe>q{&zZ-?LcwEdh=z%<&c~ z(=KN@gkt)O+6}Xdto@78RxBBxl=Fr4pRCh1S%Du=s(S|hg5+BM39e)+c+c|pRI@Ra z>&MDMN=>Y+KImX8j3no4Jgn#IHtdPTO!#4m1oH zxaDB17zyRrvK_YCsYGd*6~dv_0IgZc-&BUiU<<1FpV{9P3)2eKu- zd{>1QY6Dm=^3CViv-g{O3o1^ad${TZrw>+?RC3_x6JK(SdG5`-J&-KF$7wJpK~G2r zR({1_tAx)A#vb!tDtc2>%wL(`(!(aFKCsH!!=Bt#k`n*UF`XNu?0&sZNt*bNW|#QM z*wYUrD%yP=1BVl_0jOiq&s4XVe@HLh7%x1|{^zL*|3ExxH{ZP&g9PiyUEG1nMa(Yi z1BGKfxGcBF`^9+6p?G0KgMX}0T~N{-|M*+pgOg#2=D){igD+UdKg#jf;|Ts!3Etxs zD#&H>^9($jCbqXbR<##&7T;IFzukdG<}}Xx5J&fUxr)qSuNZ6>2bbl=^#3X)d4IOa z7Dws6T1{+VCiKl`ziU}fG?p0tT1M|Jm4yf7XY?tFi~d%jZ)Z&nI;-j0z}%B5LLHFv zZk4&TvcgQyD8`?u&_7dQ?Pr;Nh79MT)mXs@3)!!kM2WH0LzpWJ9)Wtek%S|+xy8>} zrb<2L{E;n&g%Qhl8v%po_^JBrQ?lmGG6M0lBF%G9fC|IFzu z-VFrlQB2jxHxPYMJX4;0qkc|2tZPiVe-lw=F#3`;B%tMeiNU0j&s{B>Co`4KEec6k z?LpVep1;SCqJE3d$6CWZ1_%h7IA0V8mEPH@7MrzJh{e7iMh^ z=g50UFA9Sym08L>qZ`vuTJRU<A&SH{U%S^ZCnIDKxGfyNp4ut4A0o&JBefLmZuJOta?^*W zG1+HogcP1GRwJuA_Wjb3@lNo_#<$I3ip{*SXV=B^a2u6=*Nb%Ahd{x z3GU}V(MucahgCGJ)m$bi2LB0uWd;2roxW@^{AViNZJd&RYvWTB`tNv43xy`5gXQqut{*Fd-2!Ms7eDWg^LO1dF~ zf(yFcMU_(j!1|p`YpaLuu$N%*X5TNiKgno>|ZIj|H_x^mRch<9+6_VaUx)Zy6tAcMpl=VAzFwe3O+|b9$PfD~{sPtZ7=5l2bzWrw^Kizv-c&^TqMmIBy0}Gl3Hy;_gC5k97ebZ&DxA1ObWe?H3%W zC!aCB!t62yS%}F!W#tKEs5~JnTfcdMsey%yoQ|2w%d$S!-^a_Ag=lIji#Wod4=N*J z{mM@WJInh*{Hb zX~uO~hW$>Ta5$ZRIVt&BV+`68l;*L4%74{NGheuh{+fp6huMprm2vPSzZ`aHM$00q zXM_mRY|t1!WXpG%tVQoahtoDV*8Kpqn)C#cCVQwl4WR|-E`ni4k!6SUeLK(c< zcP8=6b@VlhswajVMB&B-5^d_294s;+L;}&n>D0vq*30cEg?CL$>jxEM0*vuu7a3qu znC-I85Y4cSo~DBq;Ry@rkf*|&-==>iY4p;)uX+}q9$r9?1uG!6OyQFf&CiOsHT-D_ zR$-yFSN|?@X!9t^$js|*_q$wPUqRma$E`o7Ud$}n%@h0AFPpcN%R_g^_h7vG?+(k# zH2xHCIG1LflVqN$=C)V^JZOr=0)#G%h!@Y4LfWkx*ZbQO@oD~MUSF?OQu7hs^7cBN zNpEK{MBz8+Xt}>IbXP-{zCBnp6aN1J2(LEWYYbx8EJhcMQQgPQ|?IP;k};*iJ5 zTu$@=?Q-e=nH(>3x)?2S&|kH(kV}rg$*o)om#H+!3yE9a${COu>S0P`Zl633zg8U5 zBH`_Db}-tim4s`;ufXx#9JvA=2;!9A!u^$lm?g-x%3N+Hjw|GD!NZxHopt(*OcuVv zA#-@~^Ih^f4#~XoKjPt|+{2(3atb_pKRz`F1ux(v@+Mh zx$g0MnSy)iK8PCnh}Xv%?lbRffg;<6?DC6VI@?U_$S#%;b|kkdck-4aN8dU8JDQ1CW(n82k**gtABO<@ShQA^SW(HxR;|;m;Y~N}bH(39h&NGxJIZVlndbmO*&hBSJre zc_naA&U}7}*{aXrTe&E#J60-IA0R7Jy7hp^WkAxyJ7qrKmEo}+V_2lUt`r9 zi%qljBtfrE$U11~6Gr|aSB!XNr4R}CXrVg6NIgP*OZkOLo-qQZ918Nwd8xZUBY{5X zR6woQPv2MRx38zORTv9*@UuOTLshnrnP*CZ{SdQ za_xaB^yTY7^s_m>f2P^2e=?f}=P?C=Pm;m@kj&~xOL5RwmHf)7$v*Rvl)&ba3}g9e zvYOMH)7McIyDqRu+&8JDg(N>zb8c*x;q}g6?-74!X8t0dlfKE^R>WI^a)Y=a#WZQ` zLZQ9Dl@}U}F758j>+T#tQTCYuy(RT7UTOL=0!unQ+oXv0!rFGWNoIHOr`Yir>5P#~ zyE89@Z;~e6Luun-vM3D7pTkX`+HCCa5MvJ5y9@YLSC03RMZGvoPNf!KE-4nMnq|GB zgSOem|3RYvAXL|)$iwS~a#+VeQZPP9qJu=?hMkZ9p&Pn%40vIw^SNC%Cla6R<6n?< zUy$-+vhdQeuVj%m7v^&O+Pug^RQm)UQ=Wru#$mz1XO`WW#z&l<&X{dVA97leB z9^O%x+l%+k2ZBm|KEFL{{OL)ZPfuF@2U)&{S9ajnXbTgBB0#m?yHxB1$MuwEd@1Q% zN=#)!8LM6CaJutd1;R>pd^zb{PSo$Q4!aB7EPN59F^Z7Omp^WZTV;qtj#^$9(2tRU zysk*p;Tn&SRS{DB;beuwDcWhKdVG7_s_k(a+-U}Sm!2mwhe)wAmDBO3na-yfx5EKR z0iFnWd3e&y5K!b8BD%cd%JE?m#Tm3{P7M7IyAA$SZ??mc<3(b60iCg{%Ay`$!Wx#a z8Yw<&>A**7ZGmVnu=^b2x5Pzni7V-cjnF=Rk9_Mr@*Rj~LXqkkUpr-J?G)k=Z-`6F zw>$CyQ}&IYm^5@^lE#kZqB|BiDszcj8JVyb&HNZ;&4x*8;b)U!-*mECsj_~(L~4@h zXYG-J^VC$OfWJ_rQf7{*LE%j*{LGtY zq?)_jjDw0vJYCKq#m+JV30nLO6ip9Ws1UACUe4%ysD8dPbZGp=CHN?QafzZaFuPnY zGxN6@AiE>|6{mSxiEk56^UlmELU$tZF&9p}z&AX`7#0nuw;5hel(+eh@nV^ZvP!zm zj~fJwWeuzctulQsSt;I2baNdKf`uK}T>BeOKTRunl0L6VqPA>xpLoL(M(ZrkrZ1^M zJ2b?0KoC~fP+D1L>M!3xLTeM5U(iZw4|g!*j2YxZF@s5-h8JLeJenPLyAv)NxW>FV z1H;J@{!msuOmk-{svytjAGC@tfd7z-zSRF^lKISbe$P}=Mt_1ks-LE3^A)q`0$#tr znkqfA4lmW^AdncadhYSSn56b)>r$%wI^kMVCSHlz%6chD=1t@4P@Tjx#q@ zHkR{g(`e~V^vhAe{QXuawb+at7UgmYFehb@_Xg@QmIb2J*jQQGXe`H0KG)cu*B6<~ zcsrSbtB#a2z<}RVPO6!E92LPh>6*1j$mO+Y2=HNu#n?*+a~&WqT1o^$zsYoh_oG3~K_NHzI37W8!A2*?b^_m~{do z5tN>iCb3m^UCZm6wTKYIQg0vgp|9|72-oQCSuD~i_};9vnqXH1wNjA&DWxaeF&H#v z&lBg1t=(u|!^?!>?p!n(qQ-1>wx&H441xHh$dUDRQMm|CHBPZ0{g;p5Qw3E3_-l}d z4WFFi`3u!tOIov*>vv@8&!mvmjv>s=#%rA?_tNlAuSrN-Ww}iw1*SEW51Fsk0AtbL9i}hs{^)UGX}Zi@Nf%8gb*+!=W-9;| z-qS~CXIU1ZC#erJr#G+)_a(ShXj4|+KO_b?B**ulKaehVn==M z??L~}-`^8l#XrYw_hWJLS5%Zcr6}eKx2K?@qQp~CQDCpYU(l{8b(g@k2J~N(u<20i z&P3K+-AKEaK7WK5+T*vap_aA6RvXD1-HcI4#w~Ap7(>L)^<$|i#&;A8F4hWp@wM^+uVPX4@jyH+_Yzd#mRUfe^oa$cTd zH>@Z%JmJLd^7XA6<3v!Dc&gsSp?C_|`DqEsf27~1(a-G6i-xOn^gLBa_BQ_XRtz3s zDnWG+UL8ImEK5i(Q^fi^R+cl#1u^FKUh4>njWu&A+$k^}JNx{hK^wiQL6H{=_0qT-*>)63clRmnrY`T1bo#T| zVhvydF>>N$W0`2YaP=;<4PvPCasGAS4ArjTbGe;Y#DUut$TgHZc;9pu>I~u2t?UeA z3D9G>8$=N_fes7<^n2HEL`iviol$as41{6)p`0xZH6ps-K$@1+!J-4Fb8H4OT3l{= z-#l3`mxk(8(5@iW_?oQ=*49SP&lAK%Jm30|^AAKd1Hisq5y^)$jE&0I6f=wi zxw;l@b}yuwkeJ+9c5SB~7BK2=KMt{>DV9i=7*W~Nd8)46P6 zxY&P~rT!Ld7jnma?0r5qJJ;CX59x7%&EugbSXBVMlo)-?<1=Qva*QsgG24+-m=}iZ zMWjbD8T2@E>5l6Gbt|#4uriFS#9%SnCof8|d4)@*LAiIIWi)of&9>GLXc6^(yYMYIK%CvF(2HWKi;RYKDOsKc4CM||VPSlwE%g5+lcZdHjw%wq;wowkOgc z30swft=PsIX-1=wWQ=2D6KoDoFJTKC+(@?ZBYp_k)?$I#;uhjU1Ezr{O(3mX9!XDt zQ_{06n{8;>ODpgQ#^iQ5B+1dJCJn|hg5m)Mnt3XyX|A1I-w;^#PjMR9x(2!neumvM1h!*dscDE@Il|t;v zjMS4Em9m1jTP2WZLaM`G!Hyc@M-8>`NBa|fu>mQ7$G;lV17FNpwaybT<=tgYuob8L}@ZAz!z2dH9M6RuPqmscac!#Wf z5ey<(mZJg3(S6vcT?tYECQ(?$>(K-e;C*QwI&*>#N$zH%u@hOT6In`z{DX;B1cjHqCJkj=*vib*%1q@Z zDZd0teSmSsEpT)1CF@0tiFWvSDZuvUrT6CnBJaD1G*oud@a>ZZ{g8$gVc_4`hoAu%rhxf1)>$0KZ^z5m8w#pUUNZ=ZivN)u%6`3WSqZX zL3h$rANx^d`bU*9xAgryTcjMg&_G0Ns$ti1;@5I)L3FWJj2>G1NbmI@SR#@P7^tZ3 zM$8-^B4EL5{*Jg$p<7J_L0{OHbbJO;F2L3V(rW_l*O7kTD}@76SPB9~Eo?J$|I3Q_ zbP=y2SnZ1?{YpquBsIh?msVOg@##$4ZQWV3^LY2IHhGs+31TPf~sVNRQ3Jwc2u1ml7~z2j`rGsXtj}8_e4` zn5Etv^r}_6kRe!-y`?0335qBFLU4@+#=x@*{45{>%N z;ZYtsLJcpYthG#Sds)bk3K=4Z{-GV=N>KpHK*?d7r2rOzP(+xlVJ{fgzhEees(wss z!_`<|EQ(5zsDu(S?6u4-uVsqo)gl#-R&db@nW*jI3S zVVAh%CC+?{9uJvL=^`tkLtcDg)TBgrvz*ND8x*Bb9QntK{re;!JI zF6BeV4do-J=ob1>-%V2U=VBuidN!V&9?u4tv3IOM2_vXxk7aCmEW`Px!-o5B06HDp zSv zSw+!kMUnG{$%d>$7{nuNXK89@>6y|3$Vx~xU+bX1E1<8=M!(O>PG@Bs(VLxraFd`z z{64GCiPz`Yd-!0Oy<;DJ2TxDg-w|uWW_EP21}@#eUB61YE7sYmubi&4v-|Q>_vItu z-PW7%4-5!*L8K3*^UK4@!&+yMKsxITtFeYoHCu~9i)o(uWmgGy<^JS2Agp%Oz_df$9^ z8Qp7E>{J||?Kvu)c(vZrwoyv-^ouV)vSkhNzAZUYK4f4%MQ;V^PZmh0>ieF9xa>UW z^YqO{X3UzyPm#i=(hhq6qipQpg|UOC6|tcn;hA3N9a-0g540H(Jy;#uZQ1N{|E1=( zEZnW+8m(`&u!E+>Lp@!Y^_=7mSwLjar{=f)2e!&utPK# z(%9IZqI<{Th|wM+ncj~^iF+-#pzu@Ms-d>@FvjlbD}}Q$>27s%+N~64SuzBB;e*PB zgl!2ttnlm5p3K}^R6P_D!w1;%Ejt(4lerv?iv`Po?(mOw|qlFJt+_zge^d_o$ ze(f7{zAXG*1klxA&*OeWg?cfna@3oVuq;mwr(-U9=q;W6zkH#kFV>D(IQz%+#mJ$b zZQbg&NEfRMT+JvxMQ=A${rAGE@w&B5b?dLZC2k7Q)wT4aE(k5%wN&dgA=RT_{N^IQ zuETS)yL6Mcr*5jEAU*#ej}D-mh3BS6qCc*F_ee!(zV4#rQ$|FK!bGF^58&1{3QItkGTbp8|a>p<9`;coTOVH7-$Q`IEqd~^-YsV}6c_3TjM@S#Neq!IiNS{`XXMv_Zk=gt7V zk`#V$6G=~?{{MEqW4PW~T_v3hmRr{T%m9Z@zwnm_gb6R69UpIpL(ET4*Obg)oyCzc zd~);w%!AK6heE7AaXEPB&hJFF?r{D?E%#juPVASLE};u+6-A}|03Kpt z6;IUi13lJfmi!HydWt!&)2T)CBhUZbxVm{YZM%}$BP}Q99rRe8C%K^TpN4iH()fmU z@%Goaa}(o>D`z0kl~WLC`pQ;_5FJDGPZx!!mY3kHOmm=nYx1Vy(Gxo}Cp^z+%*=&Q zWKi6^U>Y{f?A(0&6mD90<2JJ5T*B{h{}ozS{OXhB_Ww$*ekt=F$DCh;khgg#iQ(Gl?Dwk2YX2a<<`@Bc$U}vfNtg88gpLjl8XF|I5fN*-V zq>+LZFklbThZ|9%IEP;nqi2_rwyjQD?7>uSAxG~Tb37=8)6q4UGlv3NilL>MGisg@ z?pupj&#PNUhLr2?)l~EJakKEIS@H9vb^Fdf+HDk`yy}_A7p|FqxU-M3+UZ{}tJP8I zr(OZLMdGnUn?)@ran+z15<=$o$~CA+;hW;GJ4snlT0(8YE_tUO-l zd=6eCR)YUg>#hz3a#HQnP5ExXdC*oo$JvYQOT5M+tH~b9B1ZevTC4_qApX%Oe$(YO zR@>~r=}i^HnW;Q8T|B1WR)UWZYsh{$SUYvdias*L&?XQJBB;XALu{pr1V7%rt|BXc z?@MMjZ~Chan_&S&i)x$kySMscdV8BO|JLX#{nhm12P=xKo{Q1e7S%q#9`Qj`1>)D=J9{2~$+xC6&Rp>orW^wH47ubV! zBg7WHchG#jN!Jt|6|)cez&YS+ZR><$5R`(hGTl+u<|^4V^>^76BW0wV{Kh~Cs_e-B E53ICynE(I) literal 0 HcmV?d00001 From feea154e89bc24d0b408c6927230b4987a6c4357 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 25 Nov 2021 11:02:38 +0100 Subject: [PATCH 3/3] remove working dir after test --- .../eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala index 50fe73d9a..477a4326a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/datacite/DataciteToOAFTest.scala @@ -4,11 +4,12 @@ package eu.dnetlib.dhp.datacite import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest import eu.dnetlib.dhp.schema.oaf.Oaf +import org.apache.commons.io.FileUtils import org.apache.spark.SparkConf import org.apache.spark.sql.functions.{col, count} import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession} import org.junit.jupiter.api.extension.ExtendWith -import org.junit.jupiter.api.{BeforeEach, Test} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.junit.jupiter.MockitoExtension import org.slf4j.{Logger, LoggerFactory} @@ -20,7 +21,7 @@ import org.junit.jupiter.api.Assertions._ @ExtendWith(Array(classOf[MockitoExtension])) class DataciteToOAFTest extends AbstractVocabularyTest{ - var workingDir:Path= null + private var workingDir:Path = null val log: Logger = LoggerFactory.getLogger(getClass) @BeforeEach @@ -30,6 +31,11 @@ class DataciteToOAFTest extends AbstractVocabularyTest{ super.setUpVocabulary() } + @AfterEach + def tearDown() :Unit = { + FileUtils.deleteDirectory(workingDir.toFile) + } + @Test def testDateMapping:Unit = {