From 02ec058efe24348cdd3691b55942e6f0ef138732 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 20 Nov 2014 18:31:02 -0800 Subject: [PATCH 01/18] [SPARK-4413][SQL] Parquet support through datasource API Goals: - Support for accessing parquet using SQL but not requiring Hive (thus allowing support of parquet tables with decimal columns) - Support for folder based partitioning with automatic discovery of available partitions - Caching of file metadata See scaladoc of `ParquetRelation2` for more details. Author: Michael Armbrust Closes #3269 from marmbrus/newParquet and squashes the following commits: 1dd75f1 [Michael Armbrust] Pass all paths for FileInputFormat at once. 645768b [Michael Armbrust] Review comments. abd8e2f [Michael Armbrust] Alternative implementation of parquet based on the datasources API. 938019e [Michael Armbrust] Add an experimental interface to data sources that exposes catalyst expressions. e9d2641 [Michael Armbrust] logging / formatting improvements. --- .../sql/parquet/ParquetTableOperations.scala | 4 +- .../apache/spark/sql/parquet/newParquet.scala | 290 ++++++++++++++++++ .../sql/sources/DataSourceStrategy.scala | 43 ++- .../apache/spark/sql/sources/interfaces.scala | 22 +- ...tastoreSuite.scala => parquetSuites.scala} | 178 +++++++---- 5 files changed, 458 insertions(+), 79 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala rename sql/hive/src/test/scala/org/apache/spark/sql/parquet/{ParquetMetastoreSuite.scala => parquetSuites.scala} (63%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5d0643a64a044..0e36852ddd9b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -361,7 +361,7 @@ private[parquet] class FilteringParquetRowInputFormat private var footers: JList[Footer] = _ - private var fileStatuses= Map.empty[Path, FileStatus] + private var fileStatuses = Map.empty[Path, FileStatus] override def createRecordReader( inputSplit: InputSplit, @@ -405,7 +405,9 @@ private[parquet] class FilteringParquetRowInputFormat } val newFooters = new mutable.HashMap[FileStatus, Footer] if (toFetch.size > 0) { + val startFetch = System.currentTimeMillis val fetched = getFooters(conf, toFetch) + logInfo(s"Fetched $toFetch footers in ${System.currentTimeMillis - startFetch} ms") for ((status, i) <- toFetch.zipWithIndex) { newFooters(status) = fetched.get(i) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala new file mode 100644 index 0000000000000..bea12e6dd674e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.parquet + +import java.util.{List => JList} + +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job} + +import parquet.hadoop.ParquetInputFormat +import parquet.hadoop.util.ContextUtil + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.{Partition => SparkPartition, Logging} +import org.apache.spark.rdd.{NewHadoopPartition, RDD} + +import org.apache.spark.sql.{SQLConf, Row, SQLContext} +import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, And, Expression, Attribute} +import org.apache.spark.sql.catalyst.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.sources._ + +import scala.collection.JavaConversions._ + +/** + * Allows creation of parquet based tables using the syntax + * `CREATE TABLE ... USING org.apache.spark.sql.parquet`. Currently the only option required + * is `path`, which should be the location of a collection of, optionally partitioned, + * parquet files. + */ +class DefaultSource extends RelationProvider { + /** Returns a new base relation with the given parameters. */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + val path = + parameters.getOrElse("path", sys.error("'path' must be specifed for parquet tables.")) + + ParquetRelation2(path)(sqlContext) + } +} + +private[parquet] case class Partition(partitionValues: Map[String, Any], files: Seq[FileStatus]) + +/** + * An alternative to [[ParquetRelation]] that plugs in using the data sources API. This class is + * currently not intended as a full replacement of the parquet support in Spark SQL though it is + * likely that it will eventually subsume the existing physical plan implementation. + * + * Compared with the current implementation, this class has the following notable differences: + * + * Partitioning: Partitions are auto discovered and must be in the form of directories `key=value/` + * located at `path`. Currently only a single partitioning column is supported and it must + * be an integer. This class supports both fully self-describing data, which contains the partition + * key, and data where the partition key is only present in the folder structure. The presence + * of the partitioning key in the data is also auto-detected. The `null` partition is not yet + * supported. + * + * Metadata: The metadata is automatically discovered by reading the first parquet file present. + * There is currently no support for working with files that have different schema. Additionally, + * when parquet metadata caching is turned on, the FileStatus objects for all data will be cached + * to improve the speed of interactive querying. When data is added to a table it must be dropped + * and recreated to pick up any changes. + * + * Statistics: Statistics for the size of the table are automatically populated during metadata + * discovery. + */ +@DeveloperApi +case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) + extends CatalystScan with Logging { + + def sparkContext = sqlContext.sparkContext + + // Minor Hack: scala doesnt seem to respect @transient for vals declared via extraction + @transient + private var partitionKeys: Seq[String] = _ + @transient + private var partitions: Seq[Partition] = _ + discoverPartitions() + + // TODO: Only finds the first partition, assumes the key is of type Integer... + private def discoverPartitions() = { + val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) + val partValue = "([^=]+)=([^=]+)".r + + val childrenOfPath = fs.listStatus(new Path(path)).filterNot(_.getPath.getName.startsWith("_")) + val childDirs = childrenOfPath.filter(s => s.isDir) + + if (childDirs.size > 0) { + val partitionPairs = childDirs.map(_.getPath.getName).map { + case partValue(key, value) => (key, value) + } + + val foundKeys = partitionPairs.map(_._1).distinct + if (foundKeys.size > 1) { + sys.error(s"Too many distinct partition keys: $foundKeys") + } + + // Do a parallel lookup of partition metadata. + val partitionFiles = + childDirs.par.map { d => + fs.listStatus(d.getPath) + // TODO: Is there a standard hadoop function for this? + .filterNot(_.getPath.getName.startsWith("_")) + .filterNot(_.getPath.getName.startsWith(".")) + }.seq + + partitionKeys = foundKeys.toSeq + partitions = partitionFiles.zip(partitionPairs).map { case (files, (key, value)) => + Partition(Map(key -> value.toInt), files) + }.toSeq + } else { + partitionKeys = Nil + partitions = Partition(Map.empty, childrenOfPath) :: Nil + } + } + + override val sizeInBytes = partitions.flatMap(_.files).map(_.getLen).sum + + val dataSchema = StructType.fromAttributes( // TODO: Parquet code should not deal with attributes. + ParquetTypesConverter.readSchemaFromFile( + partitions.head.files.head.getPath, + Some(sparkContext.hadoopConfiguration), + sqlContext.isParquetBinaryAsString)) + + val dataIncludesKey = + partitionKeys.headOption.map(dataSchema.fieldNames.contains(_)).getOrElse(true) + + override val schema = + if (dataIncludesKey) { + dataSchema + } else { + StructType(dataSchema.fields :+ StructField(partitionKeys.head, IntegerType)) + } + + override def buildScan(output: Seq[Attribute], predicates: Seq[Expression]): RDD[Row] = { + // This is mostly a hack so that we can use the existing parquet filter code. + val requiredColumns = output.map(_.name) + // TODO: Parquet filters should be based on data sources API, not catalyst expressions. + val filters = DataSourceStrategy.selectFilters(predicates) + + val job = new Job(sparkContext.hadoopConfiguration) + ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) + val jobConf: Configuration = ContextUtil.getConfiguration(job) + + val requestedSchema = StructType(requiredColumns.map(schema(_))) + + // TODO: Make folder based partitioning a first class citizen of the Data Sources API. + val partitionFilters = filters.collect { + case e @ EqualTo(attr, value) if partitionKeys.contains(attr) => + logInfo(s"Parquet scan partition filter: $attr=$value") + (p: Partition) => p.partitionValues(attr) == value + + case e @ In(attr, values) if partitionKeys.contains(attr) => + logInfo(s"Parquet scan partition filter: $attr IN ${values.mkString("{", ",", "}")}") + val set = values.toSet + (p: Partition) => set.contains(p.partitionValues(attr)) + + case e @ GreaterThan(attr, value) if partitionKeys.contains(attr) => + logInfo(s"Parquet scan partition filter: $attr > $value") + (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] > value.asInstanceOf[Int] + + case e @ GreaterThanOrEqual(attr, value) if partitionKeys.contains(attr) => + logInfo(s"Parquet scan partition filter: $attr >= $value") + (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] >= value.asInstanceOf[Int] + + case e @ LessThan(attr, value) if partitionKeys.contains(attr) => + logInfo(s"Parquet scan partition filter: $attr < $value") + (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] < value.asInstanceOf[Int] + + case e @ LessThanOrEqual(attr, value) if partitionKeys.contains(attr) => + logInfo(s"Parquet scan partition filter: $attr <= $value") + (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] <= value.asInstanceOf[Int] + } + + val selectedPartitions = partitions.filter(p => partitionFilters.forall(_(p))) + val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) + val selectedFiles = selectedPartitions.flatMap(_.files).map(f => fs.makeQualified(f.getPath)) + org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles:_*) + + // Push down filters when possible + predicates + .reduceOption(And) + .flatMap(ParquetFilters.createFilter) + .filter(_ => sqlContext.parquetFilterPushDown) + .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) + + def percentRead = selectedPartitions.size.toDouble / partitions.size.toDouble * 100 + logInfo(s"Reading $percentRead% of $path partitions") + + // Store both requested and original schema in `Configuration` + jobConf.set( + RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, + ParquetTypesConverter.convertToString(requestedSchema.toAttributes)) + jobConf.set( + RowWriteSupport.SPARK_ROW_SCHEMA, + ParquetTypesConverter.convertToString(schema.toAttributes)) + + // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata + val useCache = sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "true").toBoolean + jobConf.set(SQLConf.PARQUET_CACHE_METADATA, useCache.toString) + + val baseRDD = + new org.apache.spark.rdd.NewHadoopRDD( + sparkContext, + classOf[FilteringParquetRowInputFormat], + classOf[Void], + classOf[Row], + jobConf) { + val cacheMetadata = useCache + + @transient + val cachedStatus = selectedPartitions.flatMap(_.files) + + // Overridden so we can inject our own cached files statuses. + override def getPartitions: Array[SparkPartition] = { + val inputFormat = + if (cacheMetadata) { + new FilteringParquetRowInputFormat { + override def listStatus(jobContext: JobContext): JList[FileStatus] = cachedStatus + } + } else { + new FilteringParquetRowInputFormat + } + + inputFormat match { + case configurable: Configurable => + configurable.setConf(getConf) + case _ => + } + val jobContext = newJobContext(getConf, jobId) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[SparkPartition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = + new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } + } + + // The ordinal for the partition key in the result row, if requested. + val partitionKeyLocation = + partitionKeys + .headOption + .map(requiredColumns.indexOf(_)) + .getOrElse(-1) + + // When the data does not include the key and the key is requested then we must fill it in + // based on information from the input split. + if (!dataIncludesKey && partitionKeyLocation != -1) { + baseRDD.mapPartitionsWithInputSplit { case (split, iter) => + val partValue = "([^=]+)=([^=]+)".r + val partValues = + split.asInstanceOf[parquet.hadoop.ParquetInputSplit] + .getPath + .toString + .split("/") + .flatMap { + case partValue(key, value) => Some(key -> value) + case _ => None + }.toMap + + val currentValue = partValues.values.head.toInt + iter.map { pair => + val res = pair._2.asInstanceOf[SpecificMutableRow] + res.setInt(partitionKeyLocation, currentValue) + res + } + } + } else { + baseRDD.map(_._2) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 954e86822de17..37853d4d03019 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -31,6 +31,13 @@ import org.apache.spark.sql.execution.SparkPlan */ private[sql] object DataSourceStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: CatalystScan)) => + pruneFilterProjectRaw( + l, + projectList, + filters, + (a, f) => t.buildScan(a, f)) :: Nil + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: PrunedFilteredScan)) => pruneFilterProject( l, @@ -51,19 +58,35 @@ private[sql] object DataSourceStrategy extends Strategy { case _ => Nil } + // Based on Public API. protected def pruneFilterProject( - relation: LogicalRelation, - projectList: Seq[NamedExpression], - filterPredicates: Seq[Expression], - scanBuilder: (Array[String], Array[Filter]) => RDD[Row]) = { + relation: LogicalRelation, + projectList: Seq[NamedExpression], + filterPredicates: Seq[Expression], + scanBuilder: (Array[String], Array[Filter]) => RDD[Row]) = { + pruneFilterProjectRaw( + relation, + projectList, + filterPredicates, + (requestedColumns, pushedFilters) => { + scanBuilder(requestedColumns.map(_.name).toArray, selectFilters(pushedFilters).toArray) + }) + } + + // Based on Catalyst expressions. + protected def pruneFilterProjectRaw( + relation: LogicalRelation, + projectList: Seq[NamedExpression], + filterPredicates: Seq[Expression], + scanBuilder: (Seq[Attribute], Seq[Expression]) => RDD[Row]) = { val projectSet = AttributeSet(projectList.flatMap(_.references)) val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) val filterCondition = filterPredicates.reduceLeftOption(And) - val pushedFilters = selectFilters(filterPredicates.map { _ transform { + val pushedFilters = filterPredicates.map { _ transform { case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes. - }}).toArray + }} if (projectList.map(_.toAttribute) == projectList && projectSet.size == projectList.size && @@ -74,8 +97,6 @@ private[sql] object DataSourceStrategy extends Strategy { val requestedColumns = projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above. .map(relation.attributeMap) // Match original case of attributes. - .map(_.name) - .toArray val scan = execution.PhysicalRDD( @@ -84,14 +105,14 @@ private[sql] object DataSourceStrategy extends Strategy { filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) } else { val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq - val columnNames = requestedColumns.map(_.name).toArray - val scan = execution.PhysicalRDD(requestedColumns, scanBuilder(columnNames, pushedFilters)) + val scan = + execution.PhysicalRDD(requestedColumns, scanBuilder(requestedColumns, pushedFilters)) execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) } } - protected def selectFilters(filters: Seq[Expression]): Seq[Filter] = filters.collect { + protected[sql] def selectFilters(filters: Seq[Expression]): Seq[Filter] = filters.collect { case expressions.EqualTo(a: Attribute, Literal(v, _)) => EqualTo(a.name, v) case expressions.EqualTo(Literal(v, _), a: Attribute) => EqualTo(a.name, v) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 861638b1e99b6..2b8fc05fc0102 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -16,12 +16,13 @@ */ package org.apache.spark.sql.sources -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{SQLConf, Row, SQLContext, StructType} import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} /** + * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source. When * Spark SQL is given a DDL operation with a USING clause specified, this interface is used to * pass in the parameters specified by a user. @@ -40,6 +41,7 @@ trait RelationProvider { } /** + * ::DeveloperApi:: * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must * be able to produce the schema of their data in the form of a [[StructType]] Concrete * implementation should inherit from one of the descendant `Scan` classes, which define various @@ -65,6 +67,7 @@ abstract class BaseRelation { } /** + * ::DeveloperApi:: * A BaseRelation that can produce all of its tuples as an RDD of Row objects. */ @DeveloperApi @@ -73,6 +76,7 @@ abstract class TableScan extends BaseRelation { } /** + * ::DeveloperApi:: * A BaseRelation that can eliminate unneeded columns before producing an RDD * containing all of its tuples as Row objects. */ @@ -82,6 +86,7 @@ abstract class PrunedScan extends BaseRelation { } /** + * ::DeveloperApi:: * A BaseRelation that can eliminate unneeded columns and filter using selected * predicates before producing an RDD containing all matching tuples as Row objects. * @@ -93,3 +98,18 @@ abstract class PrunedScan extends BaseRelation { abstract class PrunedFilteredScan extends BaseRelation { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } + +/** + * ::Experimental:: + * An interface for experimenting with a more direct connection to the query planner. Compared to + * [[PrunedFilteredScan]], this operator receives the raw expressions from the + * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. Unlike the other APIs this + * interface is not designed to be binary compatible across releases and thus should only be used + * for experimentation. + */ +@Experimental +abstract class CatalystScan extends BaseRelation { + def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] +} + + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala similarity index 63% rename from sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index cc65242c0da9b..7159ebd0353ad 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -34,71 +34,52 @@ case class ParquetDataWithKey(p: Int, intField: Int, stringField: String) /** - * Tests for our SerDe -> Native parquet scan conversion. + * A suite to test the automatic conversion of metastore tables with parquet data to use the + * built in parquet support. */ -class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { +class ParquetMetastoreSuite extends ParquetTest { override def beforeAll(): Unit = { - val partitionedTableDir = File.createTempFile("parquettests", "sparksql") - partitionedTableDir.delete() - partitionedTableDir.mkdir() - - (1 to 10).foreach { p => - val partDir = new File(partitionedTableDir, s"p=$p") - sparkContext.makeRDD(1 to 10) - .map(i => ParquetData(i, s"part-$p")) - .saveAsParquetFile(partDir.getCanonicalPath) - } - - val partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql") - partitionedTableDirWithKey.delete() - partitionedTableDirWithKey.mkdir() - - (1 to 10).foreach { p => - val partDir = new File(partitionedTableDirWithKey, s"p=$p") - sparkContext.makeRDD(1 to 10) - .map(i => ParquetDataWithKey(p, i, s"part-$p")) - .saveAsParquetFile(partDir.getCanonicalPath) - } + super.beforeAll() sql(s""" - create external table partitioned_parquet - ( - intField INT, - stringField STRING - ) - PARTITIONED BY (p int) - ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - STORED AS - INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' - OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${partitionedTableDir.getCanonicalPath}' + create external table partitioned_parquet + ( + intField INT, + stringField STRING + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${partitionedTableDir.getCanonicalPath}' """) sql(s""" - create external table partitioned_parquet_with_key - ( - intField INT, - stringField STRING - ) - PARTITIONED BY (p int) - ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - STORED AS - INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' - OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${partitionedTableDirWithKey.getCanonicalPath}' + create external table partitioned_parquet_with_key + ( + intField INT, + stringField STRING + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${partitionedTableDirWithKey.getCanonicalPath}' """) sql(s""" - create external table normal_parquet - ( - intField INT, - stringField STRING - ) - ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - STORED AS - INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' - OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' + create external table normal_parquet + ( + intField INT, + stringField STRING + ) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' """) (1 to 10).foreach { p => @@ -116,6 +97,82 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { setConf("spark.sql.hive.convertMetastoreParquet", "false") } + test("conversion is working") { + assert( + sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { + case _: HiveTableScan => true + }.isEmpty) + assert( + sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { + case _: ParquetTableScan => true + }.nonEmpty) + } +} + +/** + * A suite of tests for the Parquet support through the data sources API. + */ +class ParquetSourceSuite extends ParquetTest { + override def beforeAll(): Unit = { + super.beforeAll() + + sql( s""" + create temporary table partitioned_parquet + USING org.apache.spark.sql.parquet + OPTIONS ( + path '${partitionedTableDir.getCanonicalPath}' + ) + """) + + sql( s""" + create temporary table partitioned_parquet_with_key + USING org.apache.spark.sql.parquet + OPTIONS ( + path '${partitionedTableDirWithKey.getCanonicalPath}' + ) + """) + + sql( s""" + create temporary table normal_parquet + USING org.apache.spark.sql.parquet + OPTIONS ( + path '${new File(partitionedTableDir, "p=1").getCanonicalPath}' + ) + """) + } +} + +/** + * A collection of tests for parquet data with various forms of partitioning. + */ +abstract class ParquetTest extends QueryTest with BeforeAndAfterAll { + var partitionedTableDir: File = null + var partitionedTableDirWithKey: File = null + + override def beforeAll(): Unit = { + partitionedTableDir = File.createTempFile("parquettests", "sparksql") + partitionedTableDir.delete() + partitionedTableDir.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDir, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetData(i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + + partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql") + partitionedTableDirWithKey.delete() + partitionedTableDirWithKey.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDirWithKey, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetDataWithKey(p, i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + } + Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table => test(s"project the partitioning column $table") { checkAnswer( @@ -193,15 +250,4 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { sql("SELECT COUNT(*) FROM normal_parquet"), 10) } - - test("conversion is working") { - assert( - sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { - case _: HiveTableScan => true - }.isEmpty) - assert( - sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { - case _: ParquetTableScan => true - }.nonEmpty) - } } From 8cd6eea6298fc8e811dece38c2875e94ff863948 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 20 Nov 2014 19:12:45 -0800 Subject: [PATCH 02/18] add Sphinx as a dependency of building docs Author: Davies Liu Closes #3388 from davies/doc_readme and squashes the following commits: daa1482 [Davies Liu] add Sphinx dependency --- docs/README.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/README.md b/docs/README.md index d2d58e435d4c4..119484038083f 100644 --- a/docs/README.md +++ b/docs/README.md @@ -43,7 +43,7 @@ You can modify the default Jekyll build as follows: ## Pygments We also use pygments (http://pygments.org) for syntax highlighting in documentation markdown pages, -so you will also need to install that (it requires Python) by running `sudo easy_install Pygments`. +so you will also need to install that (it requires Python) by running `sudo pip install Pygments`. To mark a block of code in your markdown to be syntax highlighted by jekyll during the compile phase, use the following sytax: @@ -53,6 +53,11 @@ phase, use the following sytax: // supported languages too. {% endhighlight %} +## Sphinx + +We use Sphinx to generate Python API docs, so you will need to install it by running +`sudo pip install sphinx`. + ## API Docs (Scaladoc and Sphinx) You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. From 90a6a46bd11030672597f015dd443d954107123a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 20 Nov 2014 20:34:43 -0800 Subject: [PATCH 03/18] [SPARK-4522][SQL] Parse schema with missing metadata. This is just a quick fix for 1.2. SPARK-4523 describes a more complete solution. Author: Michael Armbrust Closes #3392 from marmbrus/parquetMetadata and squashes the following commits: bcc6626 [Michael Armbrust] Parse schema with missing metadata. --- .../org/apache/spark/sql/catalyst/types/dataTypes.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index ff1dc03069ef1..892b7e1a97c8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -84,6 +84,12 @@ object DataType { ("nullable", JBool(nullable)), ("type", dataType: JValue)) => StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) + // Support reading schema when 'metadata' is missing. + case JSortedObject( + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable) } @deprecated("Use DataType.fromJson instead", "1.2.0") From b97070ec78518fc12a851179aec8b98a44d7ee41 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Nov 2014 00:29:02 -0800 Subject: [PATCH 04/18] [Doc][GraphX] Remove Motivation section and did some minor update. --- docs/graphx-programming-guide.md | 77 +++----------------------------- 1 file changed, 7 insertions(+), 70 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 28bb98175188a..e298c51f8a5b7 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -57,77 +57,15 @@ title: GraphX Programming Guide # Overview -GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high level, -GraphX extends the Spark [RDD](api/scala/index.html#org.apache.spark.rdd.RDD) by introducing the -[Resilient Distributed Property Graph](#property_graph): a directed multigraph with properties +GraphX is a new component in Spark for graphs and graph-parallel computation. At a high level, +GraphX extends the Spark [RDD](api/scala/index.html#org.apache.spark.rdd.RDD) by introducing a +new [Graph](#property_graph) abstraction: a directed multigraph with properties attached to each vertex and edge. To support graph computation, GraphX exposes a set of fundamental operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and -[aggregateMessages](#aggregateMessages)) as well as an optimized variant of the [Pregel](#pregel) API. In -addition, GraphX includes a growing collection of graph [algorithms](#graph_algorithms) and +[aggregateMessages](#aggregateMessages)) as well as an optimized variant of the [Pregel](#pregel) API. In addition, GraphX includes a growing collection of graph [algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify graph analytics tasks. -## Motivation - -From social networks to language modeling, the growing scale and importance of -graph data has driven the development of numerous new *graph-parallel* systems -(e.g., [Giraph](http://giraph.apache.org) and -[GraphLab](http://graphlab.org)). By restricting the types of computation that can be -expressed and introducing new techniques to partition and distribute graphs, -these systems can efficiently execute sophisticated graph algorithms orders of -magnitude faster than more general *data-parallel* systems. - -

- Data-Parallel vs. Graph-Parallel - -

- -However, the same restrictions that enable these substantial performance gains also make it -difficult to express many of the important stages in a typical graph-analytics pipeline: -constructing the graph, modifying its structure, or expressing computation that spans multiple -graphs. Furthermore, how we look at data depends on our objectives and the same raw data may have -many different table and graph views. - -

- Tables and Graphs - -

- -As a consequence, it is often necessary to be able to move between table and graph views. -However, existing graph analytics pipelines must compose graph-parallel and data- -parallel systems, leading to extensive data movement and duplication and a complicated programming -model. - -

- Graph Analytics Pipeline - -

- -The goal of the GraphX project is to unify graph-parallel and data-parallel computation in one -system with a single composable API. The GraphX API enables users to view data both as a graph and -as collections (i.e., RDDs) without data movement or duplication. By incorporating recent advances -in graph-parallel systems, GraphX is able to optimize the execution of graph operations. - - - ## Migrating from Spark 1.1 GraphX in Spark {{site.SPARK_VERSION}} contains a few user facing API changes: @@ -174,7 +112,7 @@ identifiers. The property graph is parameterized over the vertex (`VD`) and edge (`ED`) types. These are the types of the objects associated with each vertex and edge respectively. -> GraphX optimizes the representation of vertex and edge types when they are plain old data types +> GraphX optimizes the representation of vertex and edge types when they are primitive data types > (e.g., int, double, etc...) reducing the in memory footprint by storing them in specialized > arrays. @@ -791,14 +729,13 @@ Graphs are inherently recursive data structures as properties of vertices depend their neighbors which in turn depend on properties of *their* neighbors. As a consequence many important graph algorithms iteratively recompute the properties of each vertex until a fixed-point condition is reached. A range of graph-parallel abstractions have been proposed -to express these iterative algorithms. GraphX exposes a Pregel-like operator which is a fusion of -the widely used Pregel and GraphLab abstractions. +to express these iterative algorithms. GraphX exposes a variant of the Pregel API. At a high level the Pregel operator in GraphX is a bulk-synchronous parallel messaging abstraction *constrained to the topology of the graph*. The Pregel operator executes in a series of super steps in which vertices receive the *sum* of their inbound messages from the previous super step, compute a new value for the vertex property, and then send messages to neighboring vertices in the next -super step. Unlike Pregel and instead more like GraphLab messages are computed in parallel as a +super step. Unlike Pregel, messages are computed in parallel as a function of the edge triplet and the message computation has access to both the source and destination vertex attributes. Vertices that do not receive a message are skipped within a super step. The Pregel operators terminates iteration and returns the final graph when there are no From 28fdc6f6828df32d413d6c76dbfd2d13b1991c45 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Nov 2014 00:30:58 -0800 Subject: [PATCH 05/18] [Doc][GraphX] Remove unused png files. --- docs/img/data_parallel_vs_graph_parallel.png | Bin 432725 -> 0 bytes docs/img/graph_analytics_pipeline.png | Bin 427220 -> 0 bytes docs/img/tables_and_graphs.png | Bin 166265 -> 0 bytes 3 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 docs/img/data_parallel_vs_graph_parallel.png delete mode 100644 docs/img/graph_analytics_pipeline.png delete mode 100644 docs/img/tables_and_graphs.png diff --git a/docs/img/data_parallel_vs_graph_parallel.png b/docs/img/data_parallel_vs_graph_parallel.png deleted file mode 100644 index d3918f01d8f3b8d39d0d2c1c3ac11c99a5e8253a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 432725 zcmeFZ1y^0mwlxX_cM0z9?gR}GLU4C?cXxLP!QI{6b>Z&r9^75t%0B1b@7(+C-QN2F zueGAJxhBk-qeczsebiWl$x4eMz~aDyfPf%~e;1Jl0Ri6u0RcOLh6LW3A6Lf#0Rd$+ z6Bd>g7ZxUxwYM=gvorz$p+Yy%)n&1Dw6ixt)zuvur=^CqcaaYbjg;5z>hA989w!>p z9n;N7*3{g>Lf!&^fV(}pl@Q^ed$tX9L!-8o!?D0AMH9u)wqzHB7at=b;}+y&P_i;H z;eaeJ?~_9a+6{0EHh!1qNbUgbV}OORhYTkKt$msyKBXmcMh$!e?_&US2P-~+`|ib) zJl+SYPbsv-LgXxN4l$$+dHCu-k~?-Ai3&mb>C-2O!T8sF#h|T`onLcn`Vf=td#}0v zuep7{=00buhrS55Vu2IG3K19TnVOm9z+e=RfQCbb+GTy}*&d~Gh9nNIrfR7#h95C7 zWHHty>Jv+57W4P(@uB|oX_PQ9S)B0`01O`z^1TF_nOztX64VPa*vkO~P$g?~N?u+M z)F6!QcQpqP5H#*TKcF$a%Ag=1f*|4|LW-`SryX#fO7kfXAER#s-@h`D34(#bCxJtt zC<+E7`8N)z7gzK}H_mp~d!d`CV?RHQzH~l%2lq&e*{VaC(Kq@lL|BC>=g^1)h6 z1c@ggex^Xc`~>;?AscEy__q`PbrTag8~G4yXw3f~C8X-BLHeh;fBM2f%Z56D8X61z zy?8*GKkqC-{i6oEA;78nT0mIp{#im;yVFm9yYxRdx4?k&Yl5Vx{iB2+L0tF$pjqHM zZ9zd{?HGK==>Abc->-Nt|DajmJDYuh^J5AcWBj9p!0q??C;RaQ|Npn&Q|l+x|K5Gu zPhe8>2~>=X!fFMc$|~CfQHc}sPaz~MY@bn>r?Jp5Lz0J}10k-Ra?p?;{=IX$Ti}m| z_*?*#%&+*`9-g3?Yl!&;+@C~0=TZcd5<8O=QWOz!8^OvKQW)sVgOP9{J3?;hr49(b zsC}l8mhA5Ptq{xNfMG42;a8&Yt3U=p0gpibv~{HNE}w>Lk>Ib%;9+-_Ti zmQUqx4k648yvluN6ar&mFfyi4@YGTXyXzy9$IA?)YLiXny}bwey82s5-eyBkjCF6Q zgEd}=X+2NZ`>5N_q zVMV?&%Mgo}3Ywk(P?T(BP9o2CRTR@o)~KUBz7=i_%-{J&hWh=0QM-%Y)!0Nl1ZVBT zEdquT`1-8hZ>+1Wjo50qS93BfHCzoISA)+q1l%!#b(!&~+jEVFM+F2yTu%eP2ic#= zg zLgYP{ov1Ou$xf@Ke~GsgaiC|!6nb{Q*%o>mNqylBoA&N7Rj5pm5S5^E|6CXyyFml& z7G;s1AoCQ#&v!$_{%iW{>6z+{A>@H%wBKfV$W<>clbsrgIUY3!KR)EJ0 zD$~hTl5Kj1*-)Dl*Xy%p@Vn;hp(NzC|F-(MkYEp)xG#9RWcFn@uIm~t^ehHFVKww& zvFX!tTDAET37IIQ-qpxq&EfYPF<8FnV}8wE%R$M^$K3`i{ETT1JWQT|+?$!BN!1d8 zMXsv8HyN|$#I_IigM`e-jxl9?eK4~Wiy9Jk=y8giE~El8k@j7VmnpAPjpiqB9h^*}7`G;pWU2H3L!x9iFhYv*n}SCDfUcV!)%Tl}rNvu8RzP zX~qpLE9U-7z}*>Nxe#&6C$t7;9-W*8nKwLU6Q##WK_zblkGI1SCv##gQh`w*=<}q# zFY`>BeAC_L^|f)i@0q(hpmHHcZ zjtYD*i5YUspy4U~gvh)9SiaE?a6V~@?vfH|$dSFifVdpHEiBJZC@5KxZV^e_%a}SL z#>M?5Z(nM7@*?v4bIsHr0j@8OT9_!oDz*w*_{XVS7maPo7S=}|0&o$bgQxv5+x=QN z;rBMetpV^0bR&I!=1`PaBL7ZS)?(eE!d2xW@|Yx=?hD3y7>Yux53tG%8lijqp|+)^ zoq8zWrL{r_Gu;u}qRonFMg}LwEwn)Y&E&kHDZh?{bI#@zpf2g+-$2E zY%=B3X&R%-+Pjd<>e1K6GSlNR;!`=fIyZwW6S%?%ZEp{`mW*a)gh(i@wWf!`Wp#J1 zD5%;L4eaYqcXb9)G%@hOiTztXC)bn12ZMJSHG)~|(Y}dx)Kiar>M;rD%mj52eVRnR zI~y+J;;$R$^h5b1CSsYOfH;uBAVA%^syQlFB`~ofEGFXi>`SLN!B<(J4{2X4@t&Kd zXg|JY?re(}+Y=IK-PUX0U-FN&;$xEy5@Y-expWiz z@wC^l;Ei%Fz066Ct^}h3PEX;_>can&^9H>q!R1ZJA=zzKxHV;uU%&vnD z93SvG*Rn}kvDxIe#2f(mHzu-$0)x_a#iPZ;>NT?7h-p{J6l@=D7la~F0&nr!`MQ^7 z%sqKMfsa)OPZ|%@0criVniZ-oJutaf?vJmfOA@&a*ACqCa2`UyfQ0GzciT7LX-}3o`g(1G+huWoG0zP=zJk8Ic zY^N7<%qvaTUP#kQCZ4jk+fB;fm=$GH82`d;zI8BIGP^p5X_~646d4WlgpB-&=Gc8+ zs6>9=Xj{$In@6t1Fh2#WfOzW!o#LCl3z*;VXthVv+ih<;yV{ZhPx{!-T1Ne0qRq%! zfpY?yRXpy$@nAE8pD^PG?Wu9T@mb#iB~@>W6lo0+fspU+*BH}p%KqSMi9it3WQBuK zUIo6B^a}r_&hh}0gwVqKPERLJ|26izZ|UlHOQ(gDKNv9>2~=&0`owINx&(XZEyif8 zd}@0(lI}MXG0=mV%&0Kpz=J7)djHD|L%_jmPB(oDs@9C$czggj>iue9+x-89Q@g84 zt1tRZ_6|i)=q(NznPa@^%X2P6|BXb~fM&zRX+wlj6$4I!u_Ms9^!WTs?*wl3<~ex^ zS%ec_D08efz;}9Tt;S|UFD_a9GVZS@b-%-evafxt>e2!#`X`gfk<4n6xnW#sK%?RK z8jRE!QEidH7&O+__)16RBf-Jy2sJ_?m6og1+hgkvVyIpZ-M$kD#@Lirinz_Q&n>WZ z>oN(3vGwE)5HS#gPEV-P&FBiAQlB-Py`>Yh{cD$=oj+Vh4BD`>xtrl?vO5 z!&GWj$l-t99V92_$^D2+Tr1S(sMx&~HbA4H9kDx8r^msWd$SUZFl1Rtd1#)68wUrb zv2U<`vxMzocxfY*6WU=SX-qL+ZmUgF<&G7!l#JXLifi&;pPnE_cX?oIm}5h22V84q zGP*-sM_}XQ6LP*~dYRwE)d(xb)5x%aK`~jMD=@Dy)ec6&p=(fXhygR*)%qmD3tz+@ z4;bm=$-L9gl(+~Optr3=VGn5=Q1@H)=IkJRmp+dm(i6G53RtQ;VRXHzkQ%CaBVNf> zH^VNw8dKODbipWc{#{gXd3Jck_=S;_V`=AW*}A=D6W4_I6L&c;ss)OE$LbUlqQKK+ zLo(+2tkvn7yJ}LeLiCTT^RmP@(c=U=q(a9p>x1Z;Cztm(0zrkH%e|b(1O>e|ucBj)#6?e1q4J zVI#sq5WI$ceO@NNZs*9obtK?-|L{89xWj04;Y|J6S`iF@4tjRQ>}+*8pKLkZ)2T(hS?8T;Iv9+6!TZ1amiP zH$ho0I9FR4!uGci{(bu;6Y4=#b*KEFdKIPwi?4r-%Yd6Zi_*3rh^A7ST9>5stzQwRyZD~0$GZSOz z?Q{vRbB#=jT@A}z9Y3=#>9WqO{Jet3NZ8X8_dKLeynnR$iSP57{{8*3XM7m};>M*-d=&OL7Ry-a27m%tEN1audl5~Vol$17tq74s3 zYOj>5^Vc8D{V3nED=QJDq@phlpM{A|Tw6)d%+n&{0L{p}g<&A+RBQ7_upy$TI zcEccA^@zTvWhW1*1%2WMJ8qaf#YZA86)k^MjLSv0a#{_&VPL!G3YZ}#Uazp1{hj{A z@S8x$L$mCaQ?HjH*4hK{!!ZONW~+)vWc3WZj4*%ViWf9|rP-t4v6%omoABsa{dkj% zdZD+fl#)c88AJ<&x()mloU=`1sy?L-DUR|*<78eNvT&!;JGMq^D$cSNiM@64$+Ym* zv&+izo8;J;i!*zMLxb{G@pMRFnN)|7dzhQC*+!yd8GllXcagH;Y8Qf%dS#xwvGqxM zAq&|Z%bowb=EVz#a;G&NOCI-+QL8v) zBr#q;B>qRi)q1nI`7GGogBI{>tImtqMmdL5aKJjYD=>N%TH{YfI9k0muB!_LoqbyA zte}ClZx~nmjhM+=Xd?6cyspkELlXgRHS()PZMA0FeJndu$D-+M{)ag}f~%zXeBz#j z+sx@`T%$H^&5)I;v(+)x9rn`P+%^k?V!hSz`DqRDU=Yq=kg$VFOuZ)a49(axPnz#I zK(`y|>tVv=Mwf7K;^I12apNr1=yQf!2i~b;;=^1cU0TnNMCa0zE2+bv1HLpU9h+C9 z6vl>UykmD&^Z_-f`!`Do?~CBP15_-Wvie*Fx0MO{lZM-9(AtXXUzJ9fPO;}dZ5v8l z+Om?u-xf%A65sgX!^*B3?(jP;X6n}yr$h3!3zdJzaLs2~C|Wy|;}k{1=T3@?$T0|*PD6qvO<4tBJ1 z0MhgrZ7WkwwH6-`P#rw zn@v@_Ls*zJ_8d+w_e(!_dpBHUqTxIy!6aK33cp-}oXavs?7)v$?-LmEn|&cPVZXM}5s^76A3;m>H*rY-i9oBSD7uk+5B9iR zK*FgR6D~S-T!{Aoa=uDR*$_s7I<WrcAJDM|E zGNS6Gm7!RcB42+uq)6P%3JIap%(@NoTsg*nJnA1ro<522ta)5stjS}-qorHpu~faG z$W*(g)8y+rt6UNaa*ujt*c76iPUNk)3zi&T_*p#64w{1uN^@sawtLLoRg) zn<3^~t__K%*!0fT>HYdrZ1r5}2ffd>7nZhLvCq2)&&&bhAGo}PR_#2-aM*vCWCFj% z_n4x*3Ln6xmUxLo>} z7#OI)&Y0|YWq;hdRd5}PO3#lTv&%6jHQ5A$!&DFN3w~~fQ+&#STJmls1*@>jWbkJx zB4o~S3`Pfl_f5R}?G$n?^xz8pGF36lRsw-&Gr`R9ycwnK_(mYYs?;a@luZAkgBF0z zv~;mb)#Dwj{JwYjvhEY=C&M#WG$eAwEp`rXm`3Y~YKEzW!^@mI{rZJNx4($^1c_~q zTKa`Y_kf8yQWO=~5^Y(8K9fy&cuD6tUCRIuIItH^N! zn}^t3YX+>~qTt9S;tgT<~4sni_kc?P1N0*<(@Q&p*0JAxpd}ZlNQx zQc^(G1gi(>>F6*F24VnxCBwz4^@;oo19AL?p$HJ%gG_&g0R|S@El>Fgn$Iaw>O)Z1X%hJ z2G3kZfcLI*W~;csBuPO{G* z|AABe@e~w;cxo9{t%48Ca}j3i41L z+5ro6URvC4Q?R{(K;rs6dlS5HOowJsU0Q0q2eXJ)G!&PPoB{gxU);b!E$)`aQetnMnfUaw0O3E|S9tw;BTL7WQYwwpk!jDh( znp{+>DW+Y2N*p}E!-YR@4kwP1E(!~ezk0^MnVdcXa4efr4vgc1Gzbxm z`NAJ~V?q}8ba(pR$9;-BWFs!DCI`@@FZFhgzhOyMIy>2|RINBuuR=7}Ydg1mq1J`= zZolX-aG;!z9R`M4S6;vF;W}c5Ufrjcoy-{N-Q}PF^1de$r(u@-`|ph~N?mdGPUgaV z+Ir}IlWeqj>x7T`J9H>JJiPTKOAOva5YjzhS5Y*81_I7#IIYKxtz2MW>LqCF06+tc zysuY~7JGXwZWagJ}-@u9r)R_-~&g+W? z{YbIRFSpaEPG!e^^6uYzh@o$~Xy#mlJcksYc`F$Z#Vm7Q$td?W*A6;)e*~wzsy3sg zHEO4)Vi>H`Ot~(Mhl$wremG2Ch#^Q7I|B@JzVX^W{+Q+qJ96w_su_Dri4@PJDMW>;r)_hTe>m<84kdG>QtG{vfEz z-Q(UL;>wvSRmCOddchMf%)AtkSJPLO{Ud9CYT~-x;9z!xoc+Q}$`VL=X_o_?S=%lB zuhI^Yi#M5O+hM!-D)R`!+zj=W@!QuMx@yJ^7j6&8gF%f}W4k$Ig2dKTzgaJYh-Dv( zFwY7L3T<8ye|gNWN?eTqU{vRy&Nt)+Zp-gno%bJXA5KLa$$vvYHAiqqMx0E)qnV|a z7tNdGHM6yl2q0YL`SYl@T|`H;w896PEYNZgPp4)_vHRVL6T|{f1btsnL%STXS^AZ_ z_9r7xGMA&31~dD0rDUCSaQG6&ShLI)!)4T!cz3s37#f1>M%s%$MZ+<-Z{S+u&aqw) zN5fgIrVI*;XRFNg$zuF;ixbR(X2b9KP}8#*P7F3-J-j7Jp=fWI@qlWO{wLoOaUE8eCMgAklufG?rRwRSRQ_pnLH3Pxbf{B{B!@q~-9> z-**^YpAV{fB0N~GY$Qz47h#qhvdjR(Bayqk+qWB2ZV$=#yblDxqD5nc8*e;#dkBle zffU#Th3yumK*Mu`YC%?`GihSqLpe8QX9LEl6Y_P7C2_g_lkX^pVSeKh^e^(zCf{Z>GX?o+eOpB(~(3trcLghVA4cFI4$Iz4PX z@$~~r2?FF)f#M{1-wJTB^4BtX*sBmkKm{<&){|}hrJng|XFN>j?5&9@G~j{YtyMQF z{7*rJD}G))hnW7A&;xLlT~2{i6aJ(7y3;)q<-yPd4x0HQN!wp$ zL8mgNWUWFV`eRaC<(vFXTYY8b$|7%rc1A|PZm%@P8m>9;I6k-)BKgA?I-bBCLP$JX zgiCj@F?w}t zF_Dq0-h4>jZm(;ok+<~g@M;S=G$H~pSE5kyu87qXD@x?H9I@2J4XB*KO_TuuCcF5+ zFbl87Oj2tckut}s@AV?*ajyhP##Bsj-yb3$ax!_q27~xecH9>KG-u#U zjP-|s)5Ki#jHqolADUTISs&1GoYVI@NIbLDuz@)cHWG|&P@TkdNHV`)=HG=YAF$k= z)HPyzX+V!lolIAaw9SfUj?62Z7;U=zN~#!8Pur@HndsSN#6Q~QGmZ=8VN>3zI3yg{ zE0Eb8&r8A+(M5*gO6ndfXtQ;xGf{U1d4&uepsG|!B&S2aC`{Dgxv$b-IrYTSSkQgEXj9X1z>$@yNa7=k zhC54P>bnEFQ%fed!zrd8#b?6hfJ4Hppyc|3C3Caxg;;GI=RKwE<$f=eZAr`*2vv>A zj>opHJG0}|nrr0Dlmu`KBm#U0i%nn>!c=xX0rZ>Gm{0JhH^5d)O};N_9qZ}TZsYHi zX!12^-*?cXSll!%L)iGZqr4LMz!bf>W~HFA?A&%PURLYCCswi28)5QqEi$x`u>94i z(Mrzr5~d=0{#wy&qE1HOeSnTf?OaK;sHhdnSH&08uMMMWP=UIf1PCIwzLOl? zh=_qC3dA6Eg>b-Y)>cdFK!79FgicW7Cfic>8rOq)!8|Itt1arW%i%){ZcYUrY^b0S zf<8e!LU0!(0$Uxpyi5tT-L|o@Pa%lSCgjUk&t?5S%`B2sks!s4pKl%+j%YooCUE;G z*c4T=Zx3lq0vL_?Ik>4(Q-W%)p)+;xq6;h<^(R$~l--;N9(&Xg#|ADoxHsHZtS3W9 zW?wF6;B2z%*BvMLj`dWg^a!{9(o8gurs^>om(dqdCDFBaw!)yv-WwbIuTD0SvT3Du zn&8a|b+1fknUob6VRpCxWHAt|Z@`9|8gNuhfru6V^&v-Dz^PQ3$W&P1;?CgJ46cW5 zYjdrP09)!Ww%pcwWT*{qgh}2`ETP!An%iL~jzo1t78kpp9(=Qn1|`rrbp#*W8~lzq zP91KY&{fAfq^@TB`egL#5UYAd0|#%9O(DdLo90+Fs*sL%_X^hPKl@3WWt| ziu)o$YKnUV?%f?!dHWq;=RIq$7E;VLW7K%>x|j=A>}p-yBcxvD@p-zze;`M9HX5{qw*Mjm`q0uP z0FhR!Owyro*X)Va%4}#zGxEzpp{~3B8S7kF7ynIr#cn9yuv+uP?U(O#TM>jh`v9Bpl(n%cdSk@d4H6`1qVeGbrS}nkJ zo*q&oPozhitsg3#mNtTd^bQg=+dJ*G2~C-fbx~~xI}akCfw{j`AMgh~Vx6Oc;c%j2 zA^QROJ0#oha)x(_S!J7z67)G7NJaOu2#XuG=^ngAQa)jg4(EzSXQ#x(*vtba>6K&p zmBu@4#7U=X@ma&?a;3&ZsNmcTKFy0_Pi%GV3Q-zsS)_;=F0;kXEUWY2yf~HyY2)wB zD|27GB8->k79dU!r+?0kDlV+N(ks9588~RiBpgk$H-;(iSZx+m-JqjQbJ9h)~=^Z?_ui{OpWXyyHVATlc-0tSkC z>pM(y1Q{OWYGi+UVkKbA9lD-+R2JH^2k(mP+^(uOV9m0wVwK|!dsUWRQt$fL`LSQLt|`D6Dlh&K{i|;sK5ru zh^;oxk~8qE((J2Z$|7QRr?z*g7!r)|l|T)h7HjNwnVo*H{Q9-)afmsIa6Ekii&PZ) z7ebgf*!%4d?d37!sEmukdgTz?a&5DshP21!mgn6T;>VPSit^b zL8hf;B5vZ?BQ*PU?eI#R$5V8>c(WKp!U#Sh-v)?DNN}#~PsT|kn zGMrfOCHA$T0O2N|Jg|f-CKAa+1T-ngz!=Q9|Jk7#X$f}!RkH%xt~MHj^WPmL)udZi zn7#kiPx5!|rbR$Nedw4k`Safm2!CMb{~V_6q-1E2m= z&Icefp5R*z0(Ncx_w6rWiSqHW_bqf^)IYd#whOlm4G4{`=8C z7ngt3&s)r_hWDBXX>td0wSGlFI zUpG)_r^(Ff^7=aQ#B=-NqNr#-!uVK^qHGCCJ;Ot)es6VH3$IOKPV8g~C zk4EtW{q{I4qKn?tBK6K3&aNfMg^GsC{IH<7#_WT5u)zzbx2<;+f9FnxC-c9|9-i9| zWVAm5@F}>Rm+oI40W4BvBQFZCvaxXhyJ+1tsX8N}QHC)ly3oy73r}VxG*(w^8!y2+ zdGFErN5A+R=Ulv?=7RtYhk7X^)A?H5zU5@7Lx23plpfS^*}( zOy`01P{{c6#NUGo|9`6tm=6Cn;4y)=>~2aOFzvgtg6sO|S4B*1m(I|O&*u&1^E~>+ z^T_DUxaX;ak>X*U&1U+@+CIA^?`*(GO?I@D%@!HYfr6oW zCABL>aq|lLYZvU)1(K3RINReo;Xnli1^V9AqEebJyVO2x!iyjuRf6y(=FX+o3b3m8 zcLYr|=JT+zi_n6MYuv+GVKcYXfAI5eEnp-*$uZI&5IKTl#S?JD>3Y3W_H<{$l(@)> zC{i}oRW|=s9(YX8Dx=}#Ce`FR8&9rT_DVf1#UXVT@wi|Ovq;^SBFg;~@6h_wWDc2R z!EUp8Q%Bh_s%K%b?DFu`5@ok?clg2!%4MQM30z2kPwgGp<^7-G@xQN`+W&I-jyZjV zq#u=A`zk&$F-UkK&$()@$61}^a4clK8n+F~HO4poO^6ZSZz;(@Fh zy$xBj{Vg2m8XQiQe3~;i4;FxRm&foUi~vq&rR`lA2V14Wp+xTol(j5x%mI;aQ#R=n zQ6K+pxBug!`x01Nu8e3bwK450VKuSKc|_$Gu!nDZuZ04A?`*zXFe|n(?U~*^L_Xc5 zd0%fw<{lvw6577Jjz-3}j?Z_M<4>qA>}$E7M69V`l~n6V0q43Y3NQ0#dwg!teU2CO zywH%lyu0r|?-09;&{5tROSyR*`1G)IkixokIC#Jcy4&^lI@)>%FW%!a&D$AGx;j7i za{UOv)1+Yx$l2#X?eo!?>H7};sQFFpw^GfIo8U=PA~snwD{q!hN+MkJ;&k)5BH9Y7 z30YhXDM8KmGAIX&I8hsfSfSw^Oo<^xTgo!~t^zp}=i3))eAR&PLICv-&mj%@QMBzy38 z46JX9nWQZm^!kPWeigP5KtzHOACtiqrt@*|uyi*;Aej;Go3NP`(RRT`#83;_2}+Xs z`+QS`&cZ4vrv!bR-im1m3=Y(rB%M0MP942n{Y3QW8k4$0E6^eE(!txQzjXme0ZI6I%qgNKQq1NrI-)Ya_-qWxBE*G;)r#TALZN1XqQex=bJ!d8c@2QmCU8eFc~IfNvrN0ve_3 zu%cYInA+fPW!n*!O0A!>qTEK9lzJ?D^LUjF5l#l*Jhpq<-t0OkP(SWQ+}dAPt-j+C z#+WG-h3Zn$HveenBRmIKRK1BSYghC&v4Y8IK%Kr{(rJ4?!Rj06*U%Z1>@h-Y0gEdC z9ZUZ6Vv7ZgR65kcQr=&korb0=w>N7`1Nc5ij*2aO;U8eVrQkF!^^!?Es)ix)bUOVK$TM4>S)I#oq>?_ zyC~#pu&X`FBdE8P5ioAmK6adFzXygMcfx;HW&S#7xi-N4|}CE`5V18+g#p( zJH0#wrQI2IS6@1=+n%x z+s(?Wk*UY(DI3@!MLy|%G!nTiip;e za21PUm@`Qc&qJ`+F*ZAcPSEv*P2=7la%?xM`PHiWqi#pGLKkH%igJ?Ir-<+x37Mq& z)9B$R+8&(ei7)%H26dZcaqjfBP=G^15OFYI>7-Kue5~K2=r&|O2~@#)elV*U>U$IM zbAbeg)azQ-mtP*w25Xo=ZOu;7)YyC!eNW+0!w7&jlUb9<9iGemMSoOoEKiiBznLM$z{&+WNxodmsAA5+Vf)>Z|!xjdXPTf3Xs* zL~yYFhQR0D|8yfJUKrPO_ZbSXg%tmrA--1#{uLi&v6#&6hE|JEXhcEz9o027Asa*f%omj*sBI==-W3*41?n7f|>{Z;(^<=Z^XuuM3wVw2Gi}( zk%+7$WPKnyHZ|r7Uqe&%OX)UwL{Mhy73-RfEf)+MfY9ewTHtE5P=qV)s8FM`#KMpm zvVloBjqZ8M+;SGv>vBev%OT|Wv*Piw=jFrO#!}lRk_n??1mbL9@{!=7{SOfP8C+oM(!Xaxv!TG=`ngtp&xDCAX-_>qdW&P z3KKul>ky|K((m<$)x2cFcw`Lu!^3t5mj^jj{IcFCE;pcFWPz+G&*kn5F^bn)eh1-; za&V`rSl8@!(BFokH;OH}ubTVwG@h?&3>$72UU(pkePJY;txKVEzdIiX$8`Im zuY{yBFeh}_h)6U6>rEa8U(;i(jfw($`|Px_1Q?*jq&532k{BA`Ofp(Z_$s6dm&1vQ zuTOXl7-T#`JzZa(zq79(<8wo%#CVV5`DICyP0g+OZ(4nj@_)QSvw0aIonXkF<*$sI zE6DLQXmmt%yc(h){kA|u9n@RqOjH;Ba~}U+EaU%N_?|$!Mj|KqTz_^BtE%rZz7TSA zKg05p35;MPDDBiz15NBqC;yG~JqG1UGqc zasHk#_C`>+k%oGBFa}DuwRl1+)|;p-n|^M7*AXJ&+`MbpB##v4Z6VoHX!TLh0%Def zph$8s%HI3bsn70m<`qS8rQV;>SP_ZyNXP;A(z&&bpLhClr%g=< z5`Nv7R+@ENHH{AxSKA(<|D^s?EC0X7ZJ4hHjGYw;16$Kj{4j_QtzOZ!I=Xcz7PIxK z>f@6drrOgKuxm=M{W%1mq}(e!IaKy_7wuv9{pDci02|Fvy|7s23@ka7!M&lBpv%LH zbK}i}A#sfrS51aDw1)ZJmsm%7@i<&XAx^bK%dDTCl?0i<{IiaEs_LQ;=4|_liilEt zhha`|?oF!dQbYI@<#Pr@3iWci!xbbWN|S$p$WH~53~19Ifn_$909+p{42vR)&xtgn zT-lHEe1%eoqMGqdlGs*gIyS#LO&N{3C71c0zRwRvwuaV;CNcXgAguAQLVtN*^ACnh zVK`X~z7p4e;E)67u+gaxn_4hu?pGRTLHNlfzAK^{|2v)`S1%|08Dr_V`(M}&1TleI{I3n`B96?FX=X(-YD22K)f7Q;M-mXpJD&%nE1c81fBxa zLdk;3;h6ZW@W8m{yfyH*MI%ls~Luif$?r|T9xEdhEkeWOq zjF&ujC8-EL*(okqF0XxS-J05@mW8!njwCUw3bkWT6(h_v_IIGo$GUHYkLg$Ous{ho z*U?-g2+=6l8!H`=ADwZie&R5Kwrz~Y3Bjf&%~U&Em`@1BC02l-QB(>idHwZyb4E7y zA7O@P5Ct7(B)ut|LPTlT4~2bc94M?_+lP#UBZ8CmaiMIg&5xnVsZg-Ah--vJf)O6Eav`;z z`7Dw5t!&RrbgGhuR@9=d{2X^HBF$t18BVG(0iBHC|MnZ;cl=@CTaaBEO4W6~Z(Sxm zT|V5QU(tSJgJQ(vzDIw5ogmo5TZ&;lguVKBJJk`1JpN1y4@vrd|E~RNKkJhxi(vx% z;#Q06j)xB5by;ESg3^0hgqL+#&iqG~x^j+a`Bvx8e(ViE{2n;(oeYw}fRwm?zcV^h z=1(rEUsalIFUzEc^ zXK>8LYqxPUmGQ!1y5`m;>aX&pK~)ovp+u;W=rJ2K3L+Y@9WP8^!t=+4wWMn+5h3?e0yrFP0F8lS%X-H|kzAtJGk5Eup9GxnRB#X_F> z0LWV_W^Aszc!+yT&A}}?Q|srGlfeVOiT0SrOB=~+oYg*DUkeSVcw+VUveFo{$rnn- zdK+aVu8YF|Eo#PM0~4m{TQ9ch#vF{!D=$YKAzY}9ghIwajf@c(?)Uk)N203Y4Qj>Z z6oIx#_??LkOvc~mg^eJ&;g|W~9I;b|BZXOAjQYauMn6hbNr#V%&3~9? zhuXF(rBzC&?9dprjTS&EYTw10WoxqpOntVNuQt?S4EeRcr@m~1^59SbX_LQ6;G6vu z%kwZ6>S{R0fr^?h45IGMb`s1+GGnRm#sB8lEl<7(%6@c)Ia0W9!HFIf8&VX_FtHJ5Nas8i*Zc=Eh{q>HP z8czU^J2<82`@s}T*D2m(3Dv}xa8h{o11r3 zJJxhPPFVn))UfYfHt+i9$Oz{aTt^+X5kOhi6=C9my+Ol`e42$>knY4HbUXR#zfyP% zRunVs&?o^q){?U@=snIiUw5N$H0h~lAX?H2r5A#vR#kZkxcvNgBx-Z5TL6sz4_oiR zrCG3bYbGjf+qUhjv~AnADl2W!g+;~O0vEZ9`u5;(jam|&nGvKW9WbP5<|Fvi-B3gX5ESp z3)=4Pe_O0tBbcO|0;8CD8XPbnfV%Om%B3L$z#( z=)IyKRj4{tH;+?W$4)FiKlROw=Gzo(HZ-H+5CyV$eZgIuITYv~*#CTda5(&Hu5LCP zbi8_gaFbsUXC;s%A-c*=PDhA9c2b6}>>jN|Wyx36eFQSow*NuGZ?lIFC>pS|(3wQm_?tZL@Qkg zcmv&omps zyj&@-%rK~9;oD)6cYuC|bw+D!M1qULCP7S7sqdz2#Q7c5 zW|a+n>LSI!!3t-ads*`OmPnr@ME55pA}cGa!HQ{ZbgMMW_oKqd*MEuPzx!aA_`kF= zGDR#Z2hotW=(Bht6JSml-9ANRdQIv1J%4m{sC)e1JFw@dv<8!tFy>A3d-HY?aokj6?k_-C#rcB?3(mT{6W4SkFY^1r+;{-aQ@=Z=yaC8i>YVPF z6ymX*HKStO2{l(BvK}fNK=?2e;bco&u)$AJE>kZ?;}cjKm76Wd|-d`iFdGHSa(if%6*>XXkoWx`zo*`@2l((ztA$SJu_8>D$PQ2-^pwKC!F zY1y2XK&dQe>MFftxLER2S#>7jKQ}PtlXaZjlH>}e+pvJKk$1t*@gp=GGrzAa(CfM8{X$W6< zKsb(DH@wQGlK$`Aum7$QU_n9HR974hXMs_(c>)s(9?B4n)7hCEtP(A~SnS$W(Vc8W zrkbU;cG|o$Vc{bC=pL>G$A*kllJvw}pV$`lYUhKyf?ot62^f%`#^v*SN@S{bsYSXf zF@9c!_SGt}R%mAOn!A9A44EL|BSkF+IoD!E6)c)gsj7*Ajw3YGNnl{X$C_3kfBlL@ zhUsv4gGoMcczp-kcTxoA|9SnLPV7BEI+{Mz2F^5TWeHfcET`mj)e-5yNpNcm3p6zH zL54OZJE4G)dLBkgL4K-9N#463df;>?CPaY zQzn;3wr0}DeZ52^F6eUn(KQxLcoH*Jg@Lu*Zu9=eQ}{ll*P-seY3Zf*Q((n%3T6zXx4|9kbfhPKBLM4D0_y;5y zPWyJvLux9eDL8u>Ib`Whx9FjG_QHt5c`NHg32LH=hB6W)_kw~EVp{IyB0C-_kzzuu zlxzXez%ZVJk)d$1eG%QoL{hQOWomt|Xj$3!BwbMtLhVKk&76d@l?=Q1^1-EPS>N^; ztkj@B#X>R#*NACnJa1w^^LQ=1`gCr}j2h`E2<2FKIB|wn=@L@z|D}ITPn1_;vb?l> zif%ow-s!|u6jX|cgyCaKT~=7#UXFw@le-X1&GvD&n5w^fIp%}Z4@*oGmO}*2#@)Vb=pZT14@bv5A+L8(I@5y|Z)_CzHL~Jj&6;fPX$b}2^#iYTGKs-egsaX#AY}YI z`xLPteAI@|ehv;KKsgF!^Sr}i2?TGA95vM$xMm^$8BCseg> zf?cU!M-OHJ8S3Ef-2@PBlpt*;^aq6C-Veq~UcJpaH2TY&wLd`)t7! zVgD7{Lx!!~L{jt{o4rUl%PQ#C9SjB<8WDAA?6zV+fbZZ=`X$bWL(49A04T^^;0<30 zujb+&f_+!`%uwiN7Z;VsA&z@kZ6nZJ$RI(Mn-c6sPf`i$1EX#_qN^}#=mi`2rBA2u zU<0HHFQ@)Kgp^MB4z;q9w&`DR%`l;GYpEioN)@Z>eE~X|ElBFkDu=XZS)+8xtOX2x z1*2q_*)~i#iWy9Y=5d_x16DSeI%nm%r>QKpkNhBrN>z>G?}|{DEY%c;hVY~qy6r1t02bFru# zM#-v88ME?ib?r+@ML4Gi^)NtHl06%l>&{5-Ze-yIEwSfzja8vw`Zz8ZxsuvEUaTY3 z-Fs?ig=HbFhT|SU6jhH>*fMysx3+_Rno$(>+M@aU&LYM0-h2jwYC*K~SJKn|>LF^n z9<8q?M#xoxg#Z1Kdi1z7Mo}y^L1>T1O#VH*;hyE(0xwbGx_!FWF{-g-D8g6(3gtUhm~9= zBWi469ubd%;R{`RX$e}_X8h+uBqy zB)xUiMbR@uXDw-rfXzP;p-Y|n&BAw+tdX~qM_F^^Fq+c;WItFs96FGG$(5LJ@t|i)~t%E`uG*`kJAR! z4dHI)s%9C4HpEN6&gm)z{$tUzLR5qS83&o*pa;6=YDMXFyO=5@ncJW zfo?}@NMdh3SoqNQ8L8*I&K72{_kvo?~EEF-*O+H$8W$$~h&6?=$K7&d?MhL<{~R)+2A z0U^a*#S@tMtG2T^0z31cURLT~vU|hQmcx#Ae)9Qco12?8VJo#nCt12@-o_St1=)hx zNU@Jndj#hPJ!I0K8OLjr3MF(Xb!mAcBN8U|OYM4S2^}Tw)r@lAbZ1or$&g#HznDxz z%JJJj_>)Ab=m}X}nl$Ey4oa{ZnaumQO1LS7<_hen!DO>;)su9Q%>FW?y-<_OW>X0t zQ!4NO{=A8UafGU$D{#q(xVr^?nJ9oJFm{*2Lf^rnP-B*5=DPv+`_g90x9K8|u%a>B zVg_^H;!@U=^A%B`L1dvr+GEvp5GqN{V1H9lmKD|IbF0u(!?7KzZj}*Jh~tD)xzmar z^-O=LdSPCITFUJZj77Y8jL#;`A*i~ipzP2|o`u3_k*?XYY9&c^hyklI}<)%AMZ%e;lWQFptz61dQ z5m{xMvgBw@7B_MzWn)w(In#3dBqbC|9qcY zrHp8>kwUztEWG>}0}yRf+r)7wV;$ua`D#JwpU-%q0IR(BzKf1*4phw`Y{$P}TgEeB zvXQM({9y8bA1YtYe1Qmje;;W0tYqu>8{f{dm?Ar9gQ%*icEqAhfP{pEj9Xnc;PU)u zn<*wSi297rUu&=t1${@P5V7?AvhB#jt$|sU*x?Fh)|K(jLK~@1tZYs=riBxdy%sXa zXeIx|FC`Yt&Ffq+1+n}ec%10BRd zFt!QVf7=c!^RS1|6pxZ=I_jgm@>Rwt1stThCVqrzM@%v-gJi_WN$4fz4g4-@g5ocX;`FaNTjL=9h9&UCGs?fzVmj^w3}O|7GvNrVOzzn2T_5Q zrq>%xN-t-&b*=dqX@-liPVHApT#|mi%^V3-J&%~5yFb)r&~hEE{zi;Zu{^o&PjBk< zavTcl_-mQw`ATC8z)N4ZOMmu#p(D`ZM|z`24-F+i*K@;&L~$Byd@sXkKDpFfic3ON zC2;tp8&!-|#C1^SbJt5MuvF?*6jp4j=Gi&X7333It>W{AnGxQvPYLcJSst=2*p ziHku)t{P-S+AQjq5e_|vJ+IaESrAwHy>xGm5;p}50sDc>{v~8xu_SL?Bzy$(vu-6DBzUnhj#bTK+_hws z7%uc_uiO!}hPv1_?NTxq2rg0KKe7{tg3Tn*g_b+11Ip1niq{BL>e7hFC`G zA8Kz-ZHo&$CNPpeY5vR`AHvY7^e`TnK4~^rSei_7{tBwlM_Tl3eoCvS(!%(uV1Gb> zOkk*6jd^y)RbB9Gku3Bcd$iOgOzHv^sgG76?_!4Yd`zqN(!yJE$%;b}W^)|XjIEfD zr0jNpR^4yi4U9!Nb$(8MJtym3e{F)f8}GM>OH+=;;(=Fe*pKy6DAQ*E?~NL#?SxlN zHI1nM$NLnmO9c+-fpQjTpEINT#1;!j5Ov*YcS+QDJU1cBhkk3WsyfB3FgYrGc{$2c z^O1^*!3@?mFOJZ#yw<0u!(#cjGvJ>L#ba>$!%--zoN}3r!T-&Q^eP$uGu`l~fr@HB z`v>o=X+g&_R>#QhTUy91e3XEn?)uRFVlW%NN_+1^K;)uou9W7Z-jr_7K~iV&D+r_{ zcsj?Ew>{6Mqc`LwmrEyi`b}eH!W4XvyBL4)?9^nkuy7^$g$j6bW6+D(+=A_ZfS zY*`InAErF^5yqsPriI65)c{$>Au?DYDl+Nv8amiFp%EKa&vLgm5~}EN--fA8+54{d zdVYGM;c&wTiA6@d6|J;IOfR^t`2|vFg1*R^#eYvRM`WU==NsMDKAt$ERW#}M3krqY zNJptbuLLSFp*Y900IW%#F8Nil-m%T|8-!}cLi2PFbn{6tL`GazFrW6^Aj7L7UySxw zFRu?`x%0`vZclIIMHx5}?AOx}R#N;7&3vQI2H%C1Pz+A; zY56rx^>Nuav;L8l{*-NBee)p@M?%qYRFhp!u0qE|FY|nPV}Q61X{`+`9TA_dxU?W= zRt1)Y)npwLQBp-3-%K+=M#m=d+~uQl-Qin^YQoOgPmyKSdtM2i zVw4-;&}9E@;cb_0>y_naQTj#!`EI79Q%5KjUN^{E#pkqqM%3e1?;3`081bS}D{jC1 zZ#h>lb`Rx!-D8I9@|CpUU`-W&SGrtfz`%_9Xm#Ojut+hiOWN{zv*(}N#uAyGNE4u} zt_^Y@JZB1X?KEU9xkM9^(s^1nE8q1hUKH#Rd{UgfOA8$*m%=c>hr76#U+{UXTx& zv=+P>Sl73<;#e6>b75U(;aq2D`U27!D^FMI-v}zGfJ*>yiZN~X<4d)@ux>y5OcigS zzs$P#x`u#lu}q!1W#e$;E*uhJUIz#GoY&!cJyTf*%2f%j;QAFkgPzKY>v;{8WGAS% z|Mg56*+}K|pZ3i2yYrk1f5iRtd^szv@t#OtmgSqSwc-mrL@49)6NJBcE0-j#z*DL4 zU>)bvCU}LDERy#s-hQCtk8L^XoNBsyh;3!q?(w=fFRf3O*VSG0^7^LgG5L4}w0WGO zUB~N#0N}=eolN~61agw*89*x%qL%1ez-(Yv(d+08wEJ58ZGksHTEMm|z*u`%EaYjb zTF}!mFNvXk;$&j2p(&yL#KICqK1pW~Vx|9%rO?DvX?|Aqa!U|JQI@g_a(roj@0u5X zd=i!W?m?Y#d-r`F*>s4j@jg-@Vj1 z64S=fLcH`C@Haj@9POublGGWzwR=E{zz!v{TSCu|KNhLeBHjbRHW?J;sKB{G6NkSe zapzpg!ZEY+5?gaz#f?AVkYORqq4cy?l)^tgONp4YMLl(dr8!CIIMtIDt1~XbpV{$+ z#QtJoAkWUS0b_hr>XRovItBi8736gFLFK?Az)INri!?^F9W=9PE&_@+8yPl{fCW@H zhcr*NM`P{h{q*y`jzmUEpsOvN3Dv3F=-FAd;}#i(47sv3 z2c4Hf$t60pYUXDogOuR?*;mBEN5ZaZpO$x|}l$6|%={_;T}v8UfLYZCDcFD*V0g2(hLq&Xoo89uWF;-)7B z6w=aI0j^t&&iWg*zE3kJC%Gay-|2=A>i&yBKjG(d4CcPGcZtpsDIy6e2SS30m6@g% zH!ZFJWIr|g9J(^GyAH2X){P3eptH%N;^if(V8Sa5Bj{o4q`GKHcA5Le@pyqdRw%C&PneYqfIMhNgEF>m zI+?1c5YaH^1LA9+*^|4j1Bnog2!#l7q5URH2Z6^+V~ zr$x#QWEr()O=^-~l5WlHbak8GQ=s-ft0bd!CsH8Q;`i^c55@2^GK#<++jO!bCJ4t+ zx=#Frc{m=~AJ@)+$YI9qmQ;%ex|@io<1VU%4EZolpk3#kZ|^e#{|q}>E8|MQQKiWP zAA;mvhAlE00;P;%=Q*zjMMTSaJDAJ^5@hlqSus+9E;JmElHTk03e*)IJ5W=DXrc#? z={3?ogD-^_YTJ~=dkRoo#Y=h5()!B;g(>pxz2z@XzmEfgV6;}^+x}<#|0TTtQX()d zBbapg2>|J=$3r7z;{w2djaDF=4LkfRx%R()GV#P zMdhagk;!XfsCY-L`?{JTS`<-(et7{?{lj@?|jx~^E#<7ts{4)+H|{J zRp3|9(C@bCnR}R;Bn>gyyq6E`U*#l8405u1@exHSTzoXGa-v zS}AfrYI=f}UN;qnxu|I9C;PibQJ`PwyG^fPN(Yf>aity-w_i!GJ;OKOUyEVCa9Ro% zNEKXBLda%;vIt8ZN~XbK6x)(Ksjc&>2=<9>9_YLVG@J4Movij!O^^)`1#BJ&{AFg& z)}53|NkhT4Z)8ECF9!S&@}6-@(+3)q)K(-MEB&c9&#~d&5t7GZ?1Q=tr#23rt81lC>$~$yl(c50sLaktSD;+u?P}W*; zX0r$X6A7xCu80RCA+Bd#NnktF>M*?Wj!MfnQz#{|*c=W321G*}effn?EtRedzzuNE zwVY@w(*=>pvYu>IoEgf6Vbge#3Crqa4W@%yXttAfo>u^%q4_AdCexr@i1jE{b9Ck& z#z8~h{n0WEw&}T1w5;HVs9T$n%>)5TFn@nD(_VV_y3*s(pSbH8p**k+kxb0`_dM5u z^A6vff>Ukb{WC}To*yK+@qi`Dl~XwQi5oAp#EIHsIq-aJYd?P({P|YCdn&3m4-{v% zs89sogT04qbXlPDV|NFjytM*M2E=6 zLO$^0*B3(bGIJ>e3~O<@-R8mfU7S{j{$q9jQQU3?EhCqVU$rrV)I~fWW5qD3?U35o zTGPa#Z6|W4>`N%3+i8?+5+)GOPC>!HNN5gxQ}HT$Bl)neP1M&A(zyM2vXlBC|5doy zdrz|Vvm_5Z{1si4REyv+=92x{X{f9m z>!{c0aCzp7^S3~e5!&&Vs%;R@Cd~inG&U}u!W+9NnUGcmR;g2{b4(J z5e=<2`390(8Na)f9=Uu*37irmgQhaNsR<(lIazNTa$U9bJCC)3VRfj`%I}}WJ4FzL z@*ePv;_f|2t{t~=+m|bLTK*};lK{^Xpw_YRDTq4j_?Gn*Qx`~fG6OACBWIQ(yK&l8 zT0T1X<6qEaj%z@U?QMS395Odv;U~~oE9(z5==|pZXdeSTlqREK^<)e$4G}-k2 z6#6@4p5me}Qn}zoLgP0r>5>Ya|B{)aTzl)^iEEf@1#0pFs%b5^=WgAu@_m96fwUJI z=hz|*efb{e%|kI9-+J7Ol@?0ohh6gf3-@X^Q=0m27Bgq9gvbbjmUI8abNHC=J6nsF zpxHFJDo^m-bEIXn>)q9RRJ@+G7cFO`m4jNlNeBnz*f`rMpTl)hi{+j9|0Tlz$mD-T z#J^zZr|m8>*w0^sb*l|4J5;K%e)x=V0=Kr*(6eNzHSHzK!lP@=_auL)>$%9B?!$l7 z;lCboM~`Q|3rW#c^_0Ci&V{5_%eo!l1uv-vQN?7_EbI)Spi47N(sX9oFs6x3O5fI# zMt4CRyj17ICgGbHyE>3Kn&FtyNUnvRtQ9vJ;Zk3F-V*6e0I5bK2Q5TW|y@ zQ5A-JPMpPDiJ$i95L!5?w#@kv;l{^IT z8V99ADnWAdpxrRlS{$d&6J zSVi1F@CXO8!&}5;GD`wo+@O!O&AoA`n_8u@VFkg2U<{|x)M4ry#$3;Dw5NV||kxZ`w*`H&2XBiJfHMZ$tds$7vCn*e=nT&)$ zV*RLud@czzjjI26D&TM6k^Q_*A-!#l3=}sZQeWx#5{PF?=A+0^WWJzwSg@x>FGozE ziP&+*>D{kVtuhUD;8zHyQf+s1#1~|`j(L|D0O*sW*dzP_tDJ9TkgmasR_nLHNW7ue zC~CQr3Co?hK?~2_zL>c#dFK^e&*Y{z--<$APCfRUo8@a@$Pn0e?-y5G972$(79N^v{WPSwV>ux%zP`l@^OAmg|20M7~bt*Ecl z9oLrJmkd-!nQ1B!5n>r)L%>l|BIk1KTi4T$Z0EUe2FQeBdaRI8%KGJor(Y-Ce`l}1 zFR5VMa^2KRGv7RurH)iRUrd@0uRl6*p>JXx>s{3^DA1Xt;@?+4DFgaH_6-j(J7fge z$Sm6g#TekLLo5_U#r4xbwp&-#kW{Q`3NohUWW)eQ=2?5LOU}ncAjt!N={-nB3q2N9 zg{yOm0Rgpuz(j>*RI1;bbyi>OiORBI&;S4lDIZVnr^Ov3KnYf&^<_;T5wK=D2o`Pp zd*l3~>K9OzMl(F3f+$u57X#3`1BGH>_iOTjSUMXkEoL2^9P|D1n5KW-Z-y%*j#X*O z{Bfc~Tmg6*7g&xjuC^#Y+wss5?4g%h}3G*?8Ibj>AyWx zKC1(@JtpgIHzWiQ5}QQg`WrNX(74ofLUW9ZIWcaxc%@Z4UMK}HQ%cQ_I8tTFVO(9L ziO_(zIPN64G))B)R3(-M=OZ`4o{-2B)2Wv1Rp+xNott4B(EIlK3D%g8)7Q43Ksn1c zt63Wg6!|4q)T}w7y;vxg8&vOBmis9F=hV?r$2Dk7_k|Qr&$C^Bm6fT{^$LTvwwNe$ z`11?eyQd~B8y<2B1H z+J5g3SXl1C;~mLDN;%?i&bI|_Rrr%nN*nQ9h;r(@=(7W^(MyuTrW87cO1c6hKTnlN zXE=LWd`pjz*`?+F#D?)mIR~4DLSC;6E>Bjmet?KbRNe zvv0r}W;r9+l6!&k!bQH)`6JpJ3jo%hf~VWu+HVuJ@KS5tqAtIy{_wSDK*ua z#Zqe5{C&+20b|C82Q-Pus}7Efok-(w2!opN6fzumhW@aLIPe`s=|aF74(Jl%zIaKy zmMXG=5~CQzop!OwzM&z-2zQ~O{JW&IiO@Yj5fCzr1d+bR6;|GAceHunN?V)*K^e(e zF8-WmOSI!f&<&edfSr9z3kkSH3+udoxin1T5N^1EpxO?dFIY9}x=7LoJ!C?;0 z2AUdA1$T_M_Zu=unEBJ=dTnwfs%U{-&dOOs*A(niA0HoYmkDK+3@NAr$49EPt{V1% z5Hp1x%X%Bv6wDb|hE2s^?G$ELfGxDBiT31Jo_t@tQrXOZ+dI>poPvBY1T7`Uj(vej zctx#zn|-#2NWPb%TF`O(J&3ATG2P;6Z`A@Np-?w4H_3dr)4(P-h<$Ktn{@qAxL>x7 z`xS&(HC9s%MsFp+QNfc~2uXR%sJq(xcF?6f)VcGR%8+pjnIyVq0klB#$Q1U`(c3L* zrjRQ;LnBs3l7iw8mwOOpy2@V+fajG)&swR{fbhI3;d`RLC=KM0G@R{sJy0@icUG6^ z-fl7bO@(kYB%-Ofw@L5a|KF8JUrXPhL2cI8*pB2%d!UlvO)07U2gHuEpxW{~k4aIZ zB0c{XT=~{RP}1g8{mW;c6{MN7Z}Vf^#iR%1EA%+!L`ZY7R5^A#;6UdQ1Zh5E2<+e@zlaVJ zTe_+Hxt%u>!4?qbP1wnX`-*0#2i+G*Y31%q@8gpodv+A$3<{<>wAWoJpMbi*_&? z{P0OX0WLX(&M(2Nb)gSS=oJ>YJ%gVdb09GL)T`DjW?IfExKhGwl0jf3@_8WFKwn=a z?`zibpl4~Os6JSLqkZoWks!difOg<>-f3)ddSshT$@jU4Nd&n4{@VN15vu>EhY|j>NxNSdzHkRz z&`8vqD-`R3UtW<|=Ek6Gvt5>ysh>{Y8$hPL0x<@@s8?9<@2*phsk;a?F9hYh8CDCq z{cgV*KeesP+%(vJz5Mw_ZknC%xwuO{MrvdFcP9Ppo^~?uWHdMwSa!&yR&^KwRo`NG z#>-iT91&v8^f3Nt3sv1GH@TH&kIFheVrnt!mQ5MGx{W$49-nisN==CJv4G-UoF_d0 z8mQTqlbla=P-HJAS$n?TTyT+x&XM)9k7im->E*MaV^8h+FE`EJ3&Y=c*}164VPHR$ zyNhtEY{CVfU2vJC0V2gwj{z524nQ=}aZA}=x^kCq=}qiQ=^Q{0V)u*B&pRJy3t`9H zGBUEsq5LmDcY>hhPizJ9-3H_Z$Dx_~L^(0S<%x*v#qTB0>i(6LcL7;`N1A6=6=)a; zy+BhRS!0b7pk4^%@eB4>x(Jfc2%VZ>Bfi5PcS6u6Eh~DAiyl+mB~o3|Ak{KjXvmi3 z{PZV!TfYlt#PDx7Z+cE{~(T)fWstuXDiw>&}1%=0*{ zjYfQdz3n%As59-0dX;m2mVcO=?%Exf_$~i*N(FrK?xb9q{*TnCZw$kV_gb9wwzng-K3?rKk?JRrW)ARP#xCyU zry4)+e7tN$N^BDc94K!$t`y)Q2<5Ye`Z$0UPZWVU$PS@7MM{}owPZqp;iVW0_#)Sh z)72k3%#2Q@0-7_ANH71b-)H!BPxYd;?_SGX!?8p6FJBhJ=M|+>PPIr;zBIT3q3W!Z zFZ2QSUZhJ4n-rWd+yw9rxdQK0(G{Dvn}oP_V%Pg7LveRC7;7J#wjHMGqTU&a`X;Gm z3~Q^qh6ZU+2qzX~WXfw&GBO@cEapHQFi>}3{`K^(AVXExFUjeO!6m4a+cJ-PyVBY1 zw#$Dd5_n0%$_Xer)4y90d?NjQ3O{@z0~}E@Zn9KNxq^qlJ(NeJuzcffCML%cLA_r_ z_r4g;%q;sjDm2<-9F}dJM!pExK@h5Cqn++Yt%3YY`Jy^B{l|rbwd}=k1Dk$x2)6J2 zW2#)+u7O_5{z$_(t#P&1H&+2Es1E?bqfw8@N%mu%g1buJqe zqY=p;v1F?@|JH|GJhhWY6MaN{>7%09bap|9F4eZm7(O~nEg922Nvcu_{iV#Y8i-AV z2m%Bo)6ZllD5Bg&kRy+=t6?Y?mvjgn{%I$*yA)43wHJ6!X4$BlWUSPv3irE8UFV43+-m zZzF45k>JYEZ-#M}oj38b9fBzfTT_2>$%;z{$55n1o#v&-N1P zqSD+HXG;xi{cYUk+5VuMOQL9Dz^L3`L*=wWwTgz1M{fz_$S)reoNY{(cB1U=+?=&% z8)a=3z_e?|`f>`WfEEO&GP)_%;q@Bz5wh9%k|Ya$C(viVNL?Q_Xnkzq$Y7}2yrh+s zmwjF+=pOrTgWX*j1EDWqiTQKF09+F@aX!*Zilxf4^N?EGdsfN+4z~7@8au8jY_syc zoVv5Hf+~rsxb!~t>PqEwKWhyM@tA>Y8{ZZ6=czpxhX@1>TqR`fTvnTmo?l3gz;_^# zVv24=?~mxegOV!EhBV(%%5r#~fj&;%N(xFumg#GZ=fjZpbS#r?bB2Q4J|w(|py-Rb z9h9B?pP1i&kgZDYltKkQ-eG0{(-*A#CJqvHGV-#@I9b5GdcN$kODw^=fKvwuTP_7F zx`D55)JssFi{Vqj8AvT`7N%S6&ij4DlK_Rq3N~LZK-_L?*&e zJzqsHo{dA$SbjTU&p-ED_8LYA;3xH2<{U-{S8?1ngwwEF$v=v{Dkzrtrr?^;Tj#ArkCW2RhO{1OGk zf$2W$vVC0ZuKfTU9E60NZ_RdvlN8X<(8JkyL(40&J0ky-^kS{8(o+-pzd03#?%d%g zGN;PoLQr5k+&B#Jluk+s2D>&}ZGUxqT|5-ig*!ZMtUrngo{)D?s8`q47D<;vUnKXa zlSF-OyUO6Y!+ES`04X36!F;6`H)3;lH#e6_$1oH{9$PJazC{8Nv#l%o2?hX=g~;(X z-0XbVfZPI`^1Z+Co=@UCV&iWGgZC{)BUc>2AS1MlfqLe~(v7$R-qQpjzqd4#cxXtI zNVEND9PGd>hsD9`UFq!6iDCUwUxiT(Tt^M;GrK;!|NLF{2GkATJ2!ZgS$DmIGNpTdtfRkUEs^Fo;vXZ&@e{YyTsp@jV50KXSSAc-~IgQuI;9po?I`)2c{VR z(}%TTV4Igm^J~Ab3c$zy9n$?HOuVnW!_g8ZHBH>RRSLZSEi``f(RBLm-A)NAvt}Jm zW%-Ig5`tE0pnNsN@*uRtRzX8vCr`T_8ykd z1QY2c{thB@H9`SjhjhvjIM zcX&wZIBd2NnWs!cMH0o+EeKz?<-fT%92=&GbV z1Uh<#RC3DBCj%vw_lsh>5Gqj2l0ga+BEx;A2 ztjQ|V0xpjfnOL6xD&;atvfa|StcdKdWTU)sDI2Y7xA$>vxR3RX@FQ?Zc8>ah#0bz* z2RKD>K9|PyO30XeZYJAaMLCpE0_IH|r-qZEVOA0rJM{eK02`ZoTT!R0R7lCGzFv9N zngZoaZ@CytZH#F+xs3m^&i)oWqzPVM6)1E9K~t+V9nDEW|)ougpCKJeu#ki;C!NmpT47 zp6Lbksld`pX%sEl^F%w65TLlE&VL3B+d8~38k9IY57y@KGw?Ec^XJ@{x6Z(X z&V%{an82S4Z>P^eEY68_A_CBck^r=E{007lNT2_(0Nk-3oO48Tk$*|B^<{?SCh50<@Bh0mi1?xU!{T>h!_$jUhy2n=(89kY#1w1?5 z9smAYS6sl)=F(DIxOONyI!O#6q|^ai`xOH-#clDjY3*GEiZ~6|_8cDjQ*zn!gYN_% zM`3S>Fu9KUcE$g((?d!zlf~kDUyWR3?`g1L$>WFctfI#A%$gw4)TvIrY~Lk(THizG z)PLJ)&{I}Gl68N?@EwkNgrPCXZl9S|*IlUmeJTBi3a{JOMt9ZQbX@iyaFTUW^H1Vp zMhEu$_V7k!c)jFydfGp?jWLt-+jzqf-)UYV7St`2JCg z@Qw7th$Z_d$Yd!SHR=KDMm_mDOWN;r-FWWBaz!OYP zIr6`8HFvGN>BXi>PdXIGy+Ohzre)OEQx}&K%VuPAI1WawUa5{%f@J6Do{rgImuSzD z<^{Iwj=~q_R@=!$o_m>S{=C$VUqdILO$ZM$@r8aVr2>e^Q?VRNFs#iE%guG}EBWZL z@L*CJ<{jE^t{teR+$E4Fn;hZ{;jmQQGKBs13>Wg6Hfd!L@jd~UmT7i-_keFr&o|Z7 zDlH}&?G^5#Dj+Yi5rrvK(O~(U-;enOdEsyokN0~d~Z9ob&t(lCv zDx#%ZVHBrF&O?T*==kWtypAz8YkHJ8%hf4#ILi*z;-)hX4A`F&O_td)Pns%Hact8J z`EWbhQ_B4h`F)ZC_oKcFdP`E3m1=2ODok@3eRhUij_+Bpuz`LbIiuRanH*f5{=Qig z-gDc&Ir1=v=P5nUV)A8G%>!lWP7C6J5hDqM%+%!DL3Linla?-u(%Y)dhf5e!11Y*4 z5uhLhrTL9tw$Y>H?j)eu_V>|LFw6>bIc)vx%I9_#9$}k~6Yx|F^v73y!i3TU^tQI6 z3YpFfxCp+hgTsRf2T}C}dU=gpKqK5Z^dP(NlKtG<9)sitSXLiAf+uhY9VIN{a31I| zLofLd{De3wRc3nhyuB~$D&3rIGZJ1}5t}W}5U?8C!td+Z(>hh<=9$67tgL!HN7<)IecZV9#LI?;c1+nqGq;RdHPNA=q9(bKTc6VmhX)A27b#8LMi zn8H8rjAwRU`kqeneC7VPXW#O-3Vz(BPN7*ro^+zgKH6of^bbXR2#}Qe-ExHBoiJLO zZC~kiv7Ag0t~}o_T|@&3GZ5uM&zl}$NXrSun;a!f*lx&D`r#)Veo{=OtJPa&qoSXT zof>qvWyQbhd%hAck^8v#n1o&NA!lZhh?4ktW9Lx6?*L#m{uzoQ{mWH&3S5!Qo5O`T2 zAQV=FAS{PZI3kbEKgY(yF~3iV?Z<3-p4|9%6Tn!wWB9M4!EP1-uH@IkpUO(MV1csT z{BKk?zw#4>r{0AICj~`a`5m^;m-hjKm0pMkH@LRN80ex6KDtr{R>N23a7gu6$xyK8 z^mW~S&gJUAO8#<@1v<;0q(w`>gn~X!WyAv&kjy0c@ceLaloG@w5>}&bVUEX( z^-OkDQ_5tElw5!M$-MROk_q^(60}>|4ijU8q@8f1usg~VPj#Ac6b8?0pG6}xU2JHi zvlCKe!bmRLyl=YX@!L-9(gj%!QCZzH0-wyHfmNg>q7q2~`$onENPb})xdN|aGYaym z-k30H>`w$>a>&T@q(meT1C+N7w_V#2(1Wf-EWAvt!pNhLnwpA+fa|UgUxC%R`_Qnl1bSn}4O28UCtcIf{fs?LEsvv5no zUu@g9ZQHhO+crA3ZFX!Y9iw9>9ox=iX05ws=11&v_PeXXs# zH#I5u32F!=H+6O{haS*hz0>Z>C{uTQc%BzB*0QN!tTZkEz*gR?cnE1&Qv zuq-HEUtH-;OUfm!oDz3>y=Irz{*u_?9YPrP($TMvnI0*#%65;WaPEAaiL|06Ois&H z6GO`UYgpume=e5a*fw2qzVpmh5gZiFy$+J5agbXt)HGm+S}P({hc{$i zE4BWD6h*D=AY8dK!M|#^@qeAA#I+Im>8mxyAl}{imxZwcKk_H{_vkzE_ZF&X+@b_R z5`4K@8qhErLw!;Vz~1QP(X87R!3{Pk;*CwKtx}oYb_0l!&8BE?R07-V_jSetNY!*g zUX|lN2wBkVn2&#dG_&*Y(hOYIB!(nUv(13^fU#5W?C&yGy!gNe9P4AQf&ya3CFW%lc_Q!EP(dC zyJE=Laa@-J4WMNKqgXztglGA6bhwIJ(;3pn4>i2-fFscH_hpZ=e=Y$`BpihW+8xA_ z3(1Jw8Cj|K0AmIMLjQI=q%h%f-y3@XrLY3ZJ|q|$0Jm{c-ZFW{YEa2(k%$8eO=D)S z;8!m!mGVE8Su(7N!;5uX_;WjDeLp{4>uQUVE(`Uq6O_HAWTYuMQB3RlZO#W>C9V-{ z&G=skWE47}fJ4%~D)^53R-J!km)Wo^R#_&l&eRJvM7~6%eF(QeRHGB8?qB<4qPPGm z46bkN@XU-v8l^v3%DVjolmq4*9O4iK!>AQ5>gLpY?XQ6eU*%J`i-+z9WeuFo_l7@4 zLB)2dsc{%cSSRWWOxj$46iOsk#aUSR2O2|lk{tqne3dB~Ksp#SBFr$WUof`yUOhU< z8Dv#><1bE!UJz+)8q_LUEONvV?SqCYkb6Nr%HK3|mNKu7*QCTO<53%@&-FKQtMe<< z`+qBC_d;j_JiX)&VGrOH_1UvIU6g*GAzz)Aou%$hg4OV>=IAud1P#^O6=;=3jtScq z=~$a*p-qT90Bwm^;Dsh@FzSYw)p&KeL=%eHoAB@tZ*pzJUZIwt?XJwQ$&>W>?|&v` zc9!E8h8i@p0;8}(k;Z{4W>D)Rt^HkgKyt1dJ~t8zKD+PCBRXmvJ8at#%9b9vu^vlO z3uRkLO|M9I)$3faJD0bOvljpHj`#7SV_xpH89g#lb^lAm`6?<$62ZB&N^ZYMvN@}U z&PUt6L$VU+eda?LrW$tzC+g^#{YqDQ>2|!!sM&5(Vc&7bI4o4cI_r^yZ}k?eqQWAu zPhHxCOUhG=|K%L~z-2K}zdY89HKd)58-%#KjJthUJ=S!lQ}He!<$pFoCCCgC;~6EL zCcwkTN2Di5QR000$=v0RQm_&U+iz)8A0UN82u!C%Y`j3IjXG#)ld|0TDazUceDruA zfgFUn40fU>1xd*Xd%^q&CK2&|I%X6VEtC2?=aE_M6IKtpd>i}Dg#l#soKSBOWVQ+Zx0v2DZ=h@!u!J z;X8X2g+D7DL+@Fd zg?=7cel}VFE^tO#vK;5hl_c)Dq7-?(PUxcC$#Du;KhY*4b`>q>BcJ7*FLUMTq;7!b z+#)rLJvf8@EtQ!7d_3@OqC{0miLU?chgMSEugQFWZrwDsP$_HaNZJT5j44p&Gh!zq z=VQ6~q(o%e@2|szLy=iMU<>!=*adOugcTD3Ud20MHQzVBMHeN~n*NvW*E!Dy`>tbbe8R zfzOr~K6)$&$3s)g#^KD)hlrHe0vD*80*E6uC9{K>Ky7zOHt*93X@I7xDrqbySeNV4 zXF8!=&5kz0*~|unaI6NcO2MB#74N%EE-XAipe|;kHTH6ZoccU7$nZ_d;;Fk9(?w%1 z{B{K^(9CdRmpl;Zi;F+?IXO*i3-+AL>Xgbf8@ln`f!^}NW;n{B9uC3Cp7~d{lt5bq zTgBw*$U|beIbo`LE{7i=ITw#kF?+%&JD%6uEFqW-x(ypcKs3mu8Lnp)%K zve;iKk6!_w>*{h6Vhx(F^$w06?#+!|odrA=>sjbuH1rkB_YY2yz3F)k-Lo1Uv)5IS z+u+k9^U9vp!1ajgq(kDNR8H}j?9}sBC$yqW+V|fjFUEz}o4TDnfx~}hxE1U}tH5bT zVfyHBf$htEM{z?q648d|ZIBJb$(xBQ+g<7sX6Lw#kw3AB?Rua{P-{Ajo;cbO$dpiDCA~C28v4sI~5T% zy56&cj}==^h@<`_7}!v~st_?semt}kUBv%2Sv=H8`>#?|?>sk3@g?&54*6|V$HKNF zsIoi*|2XAw2rL+kLk{j;)*=5AmW+O<(m~%(@j4q9t({xLrTR9-DVPq>LzCeOS%%Al zd(IyByDUz!%2MSzUFtc<#Kh2h$Ff(xc9d8pR^l5se$u}=E9h~!m@|Yu+4I@cXV8i6oXBUbQ)`YJoKPWEzf)F zfc+IaX%V1vIM9_7=&Cy}n=w2ZE^SshYTos)O8`4%xs~vB|CpfAp;CiSYeugs@vGP0 zAsyY<`tI8m4vp-HM{$1jY@nC^^varLFDrGD20;8d6ktcG#?IaX(8dfp1d|KK8^G5Q z|EhMx$K)5kt27f7SXIV@3-k|E_cB9sJ|5$@3F3imZaSOlf5pAGXTT1}5r)jY!2nBIw}~2waK|0)*4ERS2$>Z~aNR$&ijf=Vy|q)iM1xn;{Hx{mE8zUp z1@TFfrtnvTgiB2HI84ITPbTF_B|<3qtIH5CTs&O*Z!}GaZaJW1GrYFthN#{IMNWCz zRntc%sHEy;ZBcVFq)Ee>-9GImqts=UilnLbh}S`DlktSO)SAqeT|u(xncG(Y?Sebx ztVSoZDF#vQ(!^IddP2prH*zV%KN^aU93uOuL1qIUhRK*JoFIBInIse+WgEsbCntm2 z414?hSVD+_w&$(WJCp68>t>jHCwZ}njGXK*@F)6}e#5dnu|{cRM1q6kU+M3HonVj) zv<+Z~W-#*ijnebK*GTc}#+WlP*&M$-zcTU860S79MTC z>*k1dmzc`$z~&q5WH;q78qVQr_gr-YE1UEYk{58l$dzM z+JX!Rd1hJX8vFw%smXcTO+(VJoU6VJ1STsNw3w47u1BQ=X+2(Uq2y~*YSF_^--#`b z*;x+mK%`7c`-;1cMDvV3C*4RC-}RcbGA9d%7I7&Gu1=%FD5yPMpO5f+R^wYgK95tv z)3Suu*C7QKR`w501Wri@mol}q&`&!sdTeu}5~*u> z#&5+{VZ1;G#$DED9m)%WX)udI!h2iNR?B8|K)Kvmbh+nIx>g8f$Y^K?yXe^j&q`2J zWek>glCO2Z?T7%$2c7uWXB5T1djqUQAA7eye~=+KLdS;>YjQXRNErIqNo(ogUtbRT z$ey?Ta5`>(l!+WJ51`-L4%nBQymti!qO?qZskp#aXE%HD@)$lK!5>bns*Xpo5)3UW zYCvn8zztn-|AVx+S3Gg%m%H z>u9gp@wyuc_2CCe>VO$r`D#W!vlH7j+nrNAy=b-0My{9D70-cDIRT}tzfT}|htDEu z-;BMz+Rk=cI;*?zxjq(J!Wtzr%`_SGR^d?FO$?dkLq`gmR?P48iE8#D7Ypw#iny5s zA5zCDkk7#yl>2)11ypHg2^!MX4M9GO@1R~}XcpbH@R6VPomd zLdr?}fh3_JoFI8U`{z|mhmOhJBSD@ujsU#Kr8rMj^St2tu64?D0|NEke@qt2QRm+H z|ALnMFiX7Odj9VSGC$_77d^j6{-P(tnl?RpzI!`=9iEbhJOxQlTw$4OVXD>Fg6^;9 z078{Rn#Fs47#u2>OS=UPpEEwF;mW~NEn#Jb9DeY?u<@X=qiBLy0BHc{5w>3ABo_9qfD7{($ z#$~v!Sd8y@$d5SDv(f&dp?XU`cpF=;K}ah|M%M8Z^73cYN4sFWNIF16Mr-Z&?o*6|$W)bFKf5bk;v35>Fl{Ty z84&2gHta>)MN&2?Q;!-r)LL#1sBfWD7y|_W2(LNxDgIFATd)BZF7ayoz^96*j1>Oy zxXz-3D5#yUzMDMY@ozvv@!0I2Pml2N{4f9CC1c66b}ls+Ya8O6rzxmt;?kVS*7o4< z+Ve5tObSz=1{a>^?#&=XV3I~N?OIVoPIG_qreh0JF?W{kRiA8`EV67Cd;mJNowS}g zmsA*9orWV};h+=q$;rqB=LsnT4?mM2JMveaEQZm#Ujbdae=aplsR|pVqY4!~QwPt96NCWmE+APqT#@ah9i_7oXa`u( z5JPqw?b#_XxhRO(r)Pb#vAHck(XC}I-P@fe`5T#{11lEMvU(ga)C`6mPe&a!RM&?K z%bhRDIC=Gk)taWfO4T3;Mdglvnn^EXW-)`4BY#er4cx2cr%~-r-y6ndTNQQZ<62s z+juPVYA*kM{{L=S{q0{I_VQ;tzt8n3X2%pW8cG|2*FE3H@RI zJ7hUI@O`fZQdawRfibSan1-dM2^+TL3kVxRCQ%SKPdJtN2B4EZKRice5(uei7D zYB7?nT)ePRA*e3)0}i0@`2zH_#OwoBq9tEbXiy^AK!NhaM$HD(L>icgCPO>q72y8d zB!H=R=nf&H}ZeWBE+{_>?K|L^-*SLe^Qnf-OREz>H3=GsBouDd*E|vb8Q(Ul; zRA>zC8YM%QHnlC2RREA4mraP|zvzL)hH;LQta?Nra1%+-OT@!KiA~Mm1q0g_!ovL? zKh!Kos#Cm=d*YA`(5N*oi7~sOJRxaw^F949PQw{v6i?w;#0zZPq_mRhyTo{v z;<2-xiH+j#R7p&ZFIQ4t8aCl~s*ZdvS({Emus=+BvJr8qS)vWiNGNQu-5XuBIE|02 z2-)N~+_7jpC!yi~VsJ3gpG}sKstq?pKZv1n`+l+bMpdof-$2lAs1pv=U64Ep4f?Cf zX_p@>E0~}D!-(EMNlW1W$VMo;UNA&BtARl(%HHq{@cFURs-P=^1c`?5aj;Tm%D|z1 zk;kI-PsBXEpXY`*KD8o_a<4)JveHbZr$eX?KH>+QeTd=v0_3;hdh=~n_n;x9)dJ#iszslA90SEWTUZM)979??Eu~i1`m2TJGAYt7mcWepTgjN{!b5 zjbQY;%`?S8T1-Krthj(tbDR5Ndr#>wPjhH0g*&?f+^q}PFE0OwS)dzQ9N((6j}DN&^4I*vd}6@3qZ^t$E$WgBjmX$y!de*!!(-HIMk(j($h)K%V#aN%MHHeS*H zWV}|kUuP9D)w9152dCTmNE!TIjWV42$ig`IKqFn;YWNSn!8~6-XP&=mbBR*?kB>-p z({n}Qn`FR~BgJcfo32UU<=Q0$?=uII3aD|APd{bV7Fb%Um@J=maK22t)Ygh?VOcmQtLbd)lY&E2mn@XE%@TrNEfMxHCr!#XZNk$7&TGNz@a;$&-AX(JgMC;W`lIJb(XHz&=m6I!w8qaO|=yT z8I_gaybUz!f1-u|9nXeN(7Zp%jE&(ZFBJK%bsNQ{q@zuaN1|Qhfy*)Qr*Yn7SbhWI zxLkGvs?Kpri(^W{*Cl4zylrU6SaVArp>sf~p|4`$_5_jY`9hr1G*XC=2UG?WPRU3c z>gx38lP905aGkV;5;;js&^=6g>wr4Qfja0DOPF<;C#jjL+}BNZU$gFIaEo16wvV*! z_uT6>a-HpO9s=4PFi zSW{2Sgh7Ia5#y1{F0e!7EW=HVLg2RnrfAyG6>p4>8n7o4o~8^PAIM3nx#~bAli_|> ziYTMdCzZsI(1f-6PXBaGDCBeZ_9kDA99~;%5QNZ#=3Y&M3S7#y8VVc~-?$ZduP#z* zc(?cGW@(}bf1_Ae6P8Uxv;rqg&VJnSO;82;_$e~vMR4~zVMqIo;JP<3XDST!@FjIQ z!AJ}J$WzUj{#BGUez7reLXKf`CNNvimuazrLVb!Va;j&-8&+hOI>OFPP<;B(4~y;_ zQyFpz4v2fvRW!#BI&)AyKPTu7!=Q;?_8$cOAJD$QG*4~@&P7>W?v>u8Lr(3wASUYU3X71{nf~r6Nbv>hx%=GA9nd>m;Zi;fQPBfXZ%M z94UlFBH#df;F#z3AO$$qDN3kPCakb%Y=8HRWnh% zp)HAO0{1I@+#`Lh4j3<-Bkf6=A~%nkh2Xnw!K!lr?gvR(Et`?-l;B1DOuLYNfLIdL z*3*6dk5%#oYoOi69pibq8yqr8ZMG6tAF`SjY{Ayl(Db?>U1X6I1wNz3a#Q2>p@!`x zF=_G#)wQJcqX*GhTu6&i&3X{`JpE)M_LmPxwS1n~23<6m3%fqTIx9&mLJ4Sk2oUA) zK#x%F1Jb?{)$P^y?$=9H7!>=Vv~07-vda}e=q*RV9A)+`Nk4K5P$~EDU`OsmR(Y-D z!C#Y4k4SgT79xie<+{%I%Zd||#50-}S?K&z3&y(Q9NtwYq*j(tX&<(2f$vjxD?=@w z*~~Okup^#l8$)EouF}TLwBfOB@y|bhW3av7;hme*+ngYt$A-kcPB~EW<5pfkdu3Sg zsH`N1VLf!tg`H^$Bedk+86~7-SpqKGZ7;K#Fc_wujw!vKJ|=7x?n|Mem5lqUNMwpi zyCvl!Hm4PtwTKLCedRDA65*$f75*$+e-8c_ci)z;=NELAy#*Bl2s^ljByNlnwX-(?^bZR(3$asR+%xv_l! zB@9LgJ1(n}KR;@C7)#-8CI4N}Q=BpmrKw3-;oh!if1RQjc4<>}uIDVTjP5A)i>eqp zMQ|GR?~DJP>Hrwo;l!Wy0e3tJ>FB`qIT~9_rRxx{#{wr-KlGq^(=Yb__yndQv zhiFoxK4J}B%)Xt7tLYn!%VR&mwY6oJLQGP^rrEadk$%-!gq-%TaY*HkbQYn%1X-CH zqV+muNN!1D7H$I4I6)$l)eA$TAwsC-{;cPgi5Yfyf63&yPE3Fs?g}$!!Wb z-B5DcOGKD(>UW+HrBNzsO}jx(BzG*F8*+Si{w%t%f>IEe?&2*>;q2`MG_kvk{~Fzb zBT^t@!%SFyiQl{v_#O+aBH0eDU?wxVuT;(d~mIjB2wXcVA>(rDdTR70d6+T9GN&F#(l-(DOi2Mf_bOqphX~$5Nbp? z7?c?*33(XmvS@sp?nX9Ub7vp%nVfWzS_L~nFbrX%S+I%mhK<9BYFY%66>Y{qb;P~0 z!n=#+Pi37I!n#}?Sj1ziGrLz#7j-0~bE3Yl3!Q1%^>}SeJ<`)( zLY9VJR#Y5?-)$&A9>#yz3KzTt|vmUtWD`rMS z&El`GGnP205D@A2OcQs%%G{dIdRE3ss2Fgayj({Kg!;k$lUv^Tkiva-E(GHfL{o>- ziWJ0BoD|`}&wE+CA;rE*%XeeZef|7|WP`;MriAIeWaYs}2t0!<)~%px8a6UH;v3ZK z)%)!Yy~R3Lq_hbEpP!gbZaQ!@{kS2UC_;~XuV*hX>%R>mvLe4WjF__KOwN4hu6*#e z;c>zZd|0Z6x5WRoRW$VL-a|%@_nL(>BpCGEAVW)jiTq-_7E&LPz87m#)x73Rxud^A z>Iqm}P%Z%&K?Yp#?6_?F_pHmqv^xQl^H|fFUu4^AUcYDV(VwnA(@ps821JrDKgqQk zYP9|DL99<7U9rFL2OwtU{dv6LtzqO$?3U!?Lvjb_lVkpek9PNR4wk|B+zSzA>V1nZb*v&(ka{(5V zA82xN2Ckmn0~|)jP9_bfAfFONZ2pEqP&>l$0)sf`qDM7o5nabII4&aJ-Sm{Ce=+$` z;CX-}kS9sYV(1RE6vdzw95h}lGbMU78X6KIjpua{YPg}#{m3LOKFZSjt>TUNdS*#Z zfj^jut+z)`>t+t_sH7FNT_lffR6)%?ulWhxr!;2x`oo$vMi5Gf7j`nV2q7V*r&qu7 zd*Lv=3o90sp}bUUpzkU7q7ZQ4B(zuiCrw2)ot`W)yFU#4+iw$i-L=70k%WZYY^+QY z6es5{UjQ#XS4X$0DOT6Wn#@2}d~5(4W(Q1U%K92vD$+^D9eoT)lG1))6R4XQ19Km` z4htEvGcwzK$XAH$@MMB1kWo#5miJCD z7HIExOw=tjN5UjrsgfCPPp+A%D{9yzcjgf|vq!YUK_J@%7T2(O@<%}3buXHxc?9|G zN<)`bhWP86S5kU9Uc$WFzG}gTM(7sM8=2`5=JK2{%yj-@pe|!#*@*|a)2&Pt;fc~f zpcY*DHbB-onNU#9&r{cB0KZbJ135S2&zx~RK%~$xziC{RjkQ@6<(x)g5k;i$J0(Fw^`@!Lxo+*@p`r(vEjn&KSM-4OAr(;gYW5IjMMO&Uc%*meQr!cKa z;ai?pq#M{PBtPxzv-S>c$?cf&)d-YK#)FTKpTzjSqY2!?=Jw^%tU{P5gI}`EERWHm zmHxFXT9nFhDU6iZVAdnVH}b6f!1=SE^NS-}=cd)a&H-a;|2ob>9sZjzOE}sIQMQxs zcgsLZO^tjp7eH{H2_qsB3gpAH`?|z%>WSY?eFgTW<9*$moZ(Cs7*c<2zYaGoJ3cuqg!i`LgHr`AfoDTQSyz!voiPFFY6HMn{gE>C3-Cmau&plOjy&5rZTyco{ z@I6fuXVnadaA#mzx299y&4HRj3k!-2BW9vAqN6xS;fP?P#>!B)x^m;zbp2SbbG-cF zZ?*>jy@^9Z9AH~zM0vmbrJBFYVxFgl?^2HNaZAzZ8N4N|1lJ(HU<$B56~&gZ)>qz4iVvYZ6tH*HFY9Wip2CL$gGT~`PXQwbqdO~p@PYNK zCAK%=xO(F6#P?f4o?e;wy@ zlTs~@Nv+(rjIMeJjpYQ1rn(N&?R+-s#OMAnhNv`)h$BPDjS7PSiv@KkD9y~Aym3pH zHLPt4lc&uncM}mJaH@NmJow>rA6z{fe}M?{P4|J<<6@`(Fys*=cjHE=ri2F@>#Xd$ z0{bXB*c$uVGiS~8zU_EUjZ5NH5CQ%?>I1%z0=JCZlNr*qvgbXLH{-4-FyKKT5&|U}IuyBIyS7kj zOgVWmg<1wpu1dA7GcgitsrBGRztUjVVc{K^A9wS`lUeF+e=sg z8O#MI?!;YO>iJjPs;)K-Y8w}q-y8U2Tw#n?&kd;kwbx0x>r6x-i|hC6c0{S|B3)YJ zXI#BL7Q!$Qnh=0iMp8aC^!JbcJUt_Szbl`z`Ol`rZ+7v)NfjQKzq0@#uWs6}9nj1) zqT3kAo05POzWTa1Miwx_-HQv51bLW;=Cdb$XwPmFS4>R?n77;&y>(WlDJT4XT(HU% zA0UI|Ev)O6!ambkJ@5fE<45nJ*2oy#d%vfpWfHadJv;+Gqh)&pS4$>1Rskm*()^Iy z=&DJ)ETIhQU90Gx9CVC4Ky!Q*0?rTR`hkNz_M(~}7t0uWL>0A91Yc3ZY4+$A%Jdv6 zyT|eQa00(!dL=RR=1j3*lX?7s5XQ6?Kuhu_TyR0y#{k8NJ>n zLJCcDZ_7;v3S=BoXQ7g;nc-C3LR*MLAzPZ8gIqPI|FDTkJR;tw%*b&T@r#3Gq)l&) zl24AILl;}&R8?X#bxYcbubRQmSEjU^5oJuHPfnk4O$lK2Wn<>f#l4+S_BqnH$OkgS zE1B^^iO&|Nv>RyB3rzA$2oY~z#$5lCI*?}#wZ{)hQSOi1M^QfEgn_-!97-)A2`XI& zsfT=qEmmO<`RPtbuH<(`%Z{1$=P}!%;(?zdN20uOmY_Evz5DG=XzhMV|Mky~{5XL> zw=8?z632->fVsqBqOa9mEMv2YL^#jSUeq&Nsbyw8>wsmrxjXQdj*bnQa5=5iT_kVI@nEqG3%???skuEy%pWg z;1^CNfHRb>#bPHTr#?4n$A?!Q83?+?Qcn#6>d04`>~Hq%CG9c2E^v;JUZS|)d&TkG zVLDcYHMI$iq7wEoAuwdmR&IDjDc3M-L~h>#aQnhkzCeUD9{V%`hOlJWY#^lNaRnB#in?u&+||DMnN!7E#(F?2%;jVrGJrv2Se^^3v0JG6kY`q($zn{7$ihk|NvYUGsu`X!8)$IJ*4s(%98sFG^t$7AjH{hogG-995CN(jy(wS? z)5x_f!kr0NsOlzaJt;d5e2^5fMef$2ue|VCZ{pgzo+q5m^0ns}wTo`aCfntd{ zYEXbxhgDy6df+c$5k}5-J#cOtQo63_r_OfN2q?a3s@_aq2d{-SZ)9hwk`fZ|!m7H{ zR~^XuI1&Qf)3V^)(6=WU9O9U$iPK1f(D7{xfAGDcR9xTbrhm&_4v`;@);s-?u~$9v zWOQU={NHOkBoK=QxPAu}zF`d`ThC&XbUL}?ugD&<-HxQuziyqd^^eKRuy01EAh~&u z5ohT#994TzOKIcf!gm9^%y7$hw>PfCkkSSD+n~Tbv^%+W5{rke@1eFsG({Uejv(-q z0mXE%=o>Y}qN97y-pgqSb#~P(!YB&@XQA!Lq;R#1cLxS=L!mBz!qwz)9r?D;l0mA< z(bMs7*wYs+Cd|qvE@O#&-=SJi=SZPO6x49xR8xW$w^V70heCKEbDEsBzyx!?8H?^Q zY)l(G#yelc;Lh00Q5lBl4=P77^v9c+%-T4ON zr0uzX$zkMvQEN;jqocL4>ik(P78ieiE3L2vLtXJAoY*KIp-_2Jqm*VTJ#YQdWRHS| zI3do;*_j2@ex!s~R4kNPdQ3I{CmXTiK-!-6wEr4CY*QpY;skP=cYhlJI*gwvlAeU9I*`k%46g|qe zkJ0O-5I`znejCd+w~XHz*HE#s(J~pP7?sJc82bYrPx26a|7<8e29oy_PJy7yax4ji z!^S6+;PXU>SBGkV?=#gqeelhSq^cI)xW7+6RIl4nS|%vHrsx*6?f_rPU~?64_*p8d z>2Z7;#{Za60?D@ZIG%^BoRYm6#4u`-hvg6^G|#!9u4)hRdUCkRJ)2`J_WBD6q*b@A zqh|C95h6#X_5Qx08a2`1Wyn6ZO=$iA(<-?r?U%juYy*AJ*ynZpgWv5A4`<0@37oaM z`e!c|8$8{4 zFN7s#pWSBIQ#-AEHf!VyB)!KnJ@DkkF|mQ ze#iZJJ4;D2Bv+OR9T8Pk(QCT&qhhmXWprfCEjn9;#bq=YXCI~kk0!?0ErVf7kVv=A z>ADex4$dIXFRcMyK}U0`YvsSbFZ>_`byqSUfy~1TpSQsvL6IvwCalxQHkwSv#GIsZ zIOUtxe){Msbei73vxlWx&27%?-52qA?42CWdN@dlM6YuTr(jirzI{C~p%5h{!GjJx z>{wYGw!5x-9rQb%H}7*z+LBU_6;0Cqz+4mA9f6aY*ANaEGcX|`7p#8X|BX;&ropIv zJT_!)=o3#+jE=_)CV;~=Rykf|dP3EDM89Dx(U_6DkA*%XGZKj^k6a?)d~^@y=;fmZOLA_r6Hz0b}!LPlja8uD(;Uym2% zmFYmSFmKGqL;Hzq(#Rpi&NK|2#A{J^89`2nmc6$(-`fS8Ezx#}8wmw)WfXbQL%vI7 z#@TUpl^w)sy35TOmj$_Dg=i@R#Ghr5}YY}W<4rSM2ick&=%~Rua#tKLg<{ASJK|M^& zk_2oM`t0}^NZ3-W1eP#VFUifU>$WuL(Fd14V$3G;T}G_BmKo{0g$yusB(X{;Cxj4V z_WAqYwO(6q7+BxGY*4cNz)D3}=TccCM@er@WA*Q!|1FXv^*0`uF|HC??aYPT_de4T z*!FGG9O4sORWqNvwGamuR5qf^#r+g}VhYN;qaz=fv~LXjscUU~%0&-?iViICUO$`o zlL8f{6rD5j)p!c68Kd(EioN~e!<@~>O-Q1Vu2Wv!1C93{0CW&Ct7PEp+x#&&+Psu! z#xpZ8uMMx`f+o}s&hO0`C7!S&#R3Ji(>>y2rnHg%OxWaK4PRupGMnJ|+s;m??IB{MoF!j~vXytLm_NM) z93X!)>G|Q#z|2p}7w|*eK4Q=b8yulYNX(6aFRNVst6J1pmURU8=!$q*2qna+BK(=k zpEG1n(JPqCioCI|lP#6OWb!mP0^PoG`@~bxJ{}7|X0c_Bhk=7r+c`!vMTT5VmK}}? z$nya1{~^g?_CC6NpdjMP6x87`LAglEMis#>BAqpRDQlWkh@_`9!piyVh;HNSRz*vJMsQb=kblE6vee|Z!$4tK zO@SI$wVaHAfsZI<)5qAhDt_-nY5s==aFuPSFG%_@W8RwH>F~8DneT@1IEYt4absaY zUAqEFoIzN?M2YOA1XIbd65O*9Kv@7kCeBJ)0-hUp+Q4p%iCitn3^VfDvcC`JKW0n* zgc7Ev>3Tuv-N}7(n4V8<@jv&|&2>dj7~g#gG$n?RU;_`w_!(`-fnA5Ml${@qu%T zt!VADj7D6f2PVqN-&GmSLhVrp0j3OtA<>D(iPAOe6#4W{NUU}$>sjTV2C>m}2W#TO z>;3bR)2jy3NQj>IJr>DE<17L9Tm~^Z#QaqO&@O~HlMy)QH=~;B-5kr}A|e{n0F1S$*{1ZWFUFrg zP!^NBV?#Ih>f{9ms=CAh<(|$6V5nGKkq6}fX<$dFCVGqu&uC+0uMqA$ID1pPC)|669EUjB{FI*`GB3yHJ>#>YclJcHX|hw)Ak zj)&goUU;o(7tW7qiFf-GZ{PZZ>As^uDOHUW#~T^JZ2PFZYOCRB6JOQ9jCF)>tR?|3 z9&=yt6&1px1iR5yFf4EG6nwt$BkhHBuNiUjk}AD8t?#+$@Vo$%%gL_I4f@AF0o{`k z%N+YX@d(99CS6iI98QlwH3maV`l;qT#$uPn=s{Akfe5oM6j)BoJUnMslVRd#0WS?M zXv*zSUt>Fg$yV_DAH2?fuNV7=a>uV*khv`wdoI^^vV^ggeHjDovT+RyOx5WDt)tBC zl@a?V)?$GKY@(}cXC^+IXZYu9o`=1rl^&Tj4pcnA9uB|2hvOn!o|OJG(Dw46q7YY0 z^?4#1k6tZ9b0xHvAnXWC|q-Puo5H#r(LZHw4ZyxVT1;^^We!WQhxi>s2FGrY;MK+muu~T z8w2)^94PwoAG$$rl8Lb$g{dVa4Wx@hIbB)P5ede{EKRuwy3jRiWx4)oQpDDD2Nt}~ zZ>!24uV9l-w~GWdsbTOd66R$5ofuaDi1Dc?6rhqs7V*Tv?L&R@2_789|9V`1Q7C6xbEHu)&FI8(P*C;xhh$sagFlezhN z(v!vG3sEo1*fLdG^U)iZd?vF~;p85Qm=O`;oVPnaSsvP3KCsmyh{tOqqGE&U$Yk+k z_HG#O3RgF&zpou;op9{7u~_}dO;)z(4>QW&#!0{7`VkrkjMZtl|-V{F#i642*Q|^ zGZ}}zOCwBI5$vN5)-~PCwBnwVpgqE%^(l|`QIeUbR95m)-{CMS?< zjpAmNF_-RjZf#I|Qs7V1;x$>fGcC^b9tBvW;G}y`E0w{Z<`gCx0*RsXzd4`STl2Df zd6dMPo!=Tu_BxOvnz(N-UV-sCJ+*&-Si5jG|Daa2X_$5yi}Uf6clc~&(FTwYlyh{@i{*ELi-<&+G!vGkclZ# z6MDn8HTbHR{#Zs<&4{U5p3MP8Qakw#P4NVq)2tJVvRl6kn;wAAs!quNsP>d~VqZ|BOOI#byj&D%y+otmu|F|qC>Cf(q=3|v>B_qL@Lce9MryR0TVN0Lbx*oqH5k5d`q{ak-; zdh+zn|EN7=7sAlY&%CU-^ZoN>4PxVY?R3(BOOmHLF6EWj><$>7fcmsc{?JgIcfvQeYG*GI{f^F}h#@?~!eajpLU1q1eQZ zKUbNT+mNJ24xn_hM$8z|@hfgRqlF$5KXD;bs~^PR-1+ zy1Jxp)3#~WvpAta&I#$isCoy+%%W{swDaYQ)3I&aw(WG(vE4z(wr$(CZQHifv0wH% zcfb4IUzlsmxoT9^sKHzP_-fI1+|-}Uo{$D33LOCxEp2jl08DKIm(sMQ&eX0fAROOm zX?tct+ma?w_QAUTUeuIE9!%D~ABh68d*Jo=q7q%f(TLq!zKrQlQC`zC$)*wewnQRw zS7bwNcK7b5x>rLH(2>I7Yn_B#@ilw(--LqO%EVB3%);Mo3>}$8* zdsHCJ5NC*Tz7cAVc&N_^SBB^9TgmDd1cu-D%BEa?L+CR!Fh0)&H``(Bp9_DpYHg^~ zvExpvC2 zrsN*5X3hH^M1aS(nt|i4VPGTs<-6Rzc}*bT$LPs5HWaCO4_r$Sv3I^A(XI# zx0n4HAxYM=EU;6hT9|uwuT#=igNs>z6|kxjo#h;z8^lNf9-2y>;31)NDz{>Wuq#|I zQ#@X_W_f5D+Z;z&zPUDfzi5K>I>_wl1XYX4{-ikVl~epg?h?wsX@G3Luwxq11L6J5 ziDgkXvu-K)=M44$ji%>iKX@nMUK}q!Lht8<%UD)objist8a-$`h*1_)^eM_1g@%nl zOw%IabCqN*KuA16sJ?)KN5w0*j<%nPpMX9himYydzkSRkH_Bb?6Ing}M5m1S<(*+# z2K*yV-EI#EGaCIbKZvW=@FTF~Mguh(oq&J?jG{`mJ)nj)w)#uCV08w8(m~_8l=vdZ z98KxpjPot-xq7TwZi?ZJR=ZNfjzF& zB7Z;wM-)OtB|f6Bj#*sMeviO}R(Cj^!B*tY@H?dOGbYPc*h#4r!ssOSOK9`&-Ya$Q z$#fD^Y?rOKxJ^wDGkEBwE&qm&moOz>(pp0;662*U%b{F$FMr|d?y>U~xqKLJW3t?y zfM~R~Tk!L+2a#=o;qAKNg#g~}H`@%pn#{dQIIA|xpIIIUAXtkLQ*xboMAp9Fon(ZM zznc|koDIX6te}Z8=w5xuAI3>R-fhT<+Ip1cN6cTNRt~q?c$Z>TZECxL}IC?|6-@J&+JjjW)NzMP=4ti5T;_SV4h{RuebO zu|LtqYi$wQHQu8?P$fkt)7ufi7NTL`;Pl;0=-IsX5GQo%{E#2m;w&Sy@q$_gP^ylP zTsmOTQ_C=gKCV~q(1>Uq(4KY~;aFl63=30Ls{=KlS?FV6d3u?^N22beKe*jEtir5NE##lljt3-XiK69j-Hr*?T}XGoIQb0Ml$Lx zcAQC7>jD1U{2~SxSa@z{(j4yZkWts$zEMkRB-1IO0Rsz^{N*;71U6;`k5}9xvc+l> z8H>>uG~do7WCTZ%TBZ6)!5*0*2)6Yfn3gi0r^QhH!cWAEgkA_3cUB4UsZovR>igmw{Q2mi%F<4)Qi0N$iGd{0{o!VNBSh=jodPrRJ#~npzFxCXA&D%% z2B)xU`*y@TYj(__2_3Zw7k|d=IvvfxIo;B%{=;xsJcZ)~$F{x4BKImS@McpU(6yo9 zn{Y_hz9Z(|QpuUin}jFM=qcL9h^j;0P8(+(yFMU`Y$+_tDD7ox7wlQJ7@9ivo@ja@ zvfP5^O2b;cs`WY@camM;POUuI*XwMp!yxDLh$NU}8?3m`is+-e(8IcH%KqZMdY||b zl-dHIwx=(Og{JasWB@nMO0@o7e5l)9&Dcx@rMU9K5@Zx1%t~TrrPy!QhS3Z1(C;uT zaV@uNuaU7+7EgwSDMQ#_Y;NZtEHM7!YOi&?<$2$(_D+o03~iOywq0*{XoTd@8i%}y z{^w(CcCd;TBSBW`)ne!%AOx3}I`+{wj zlkdB_hFSSv`X(OVm+z4s{_A8~FPAPOONO4;7uj>?88(cp+iBh#9wgR26IUVz5Bn;|J7Sq4A;KZkbaES-kX| z+*1~B?*;kU6#qXUlfyp+*V>e->_!$tj$$PyQc)+hT_Kw9Y-ubDF^h`S68OKcjW~v; z6BGSdZT{duX50nyT?`3lsz{ZMc70D~kQ49`@9>af6P-=-C}wxP2x!rNMe@W!SV5QM z&B%7=0s@Xq8;OluySTngX^b-xdG&`pjXkMcX^mp1T^em#kbR7ai!%8^eDd(}L);%W z+>Li&x~x(-;W(eCoex20yKB%ReOdH?_OV3NWyn zdHlXp(v2HfS!3*vjZ6#(goyL(69_g>iQk_2knWPJ`Z@}!G0R_gy09#+4++@lcK;9; zoov!V4TSdk&Jds8hgVH`kqkgdMiubiOp^ACd8 zv$^LfDYX^xB+is;iH;Thb1m9(J2CZl@izMXHy;c%Jz(``p-8G^eikcp_(ap}x`|0w z8{mTkg9y4~fXjA_$_(grr{7w*X>44kyd85aEP@|1du=2V zJ-I2_ffx}D)%QwNlO-nAhskWuY{vsoiA+{}!db?!(PFif0q<8c_RaZOzo9w@3ezwv zOVrD{OtnEroYwWQa8ZV@`|H&qREa}YWC^XSbV3E{SN}{DZ0ntCwcmfzYzmwl<(tZ% zf4023RFM)=#93k1=^P&MX(Ilpie$VZO_$rWZJ9fy){Ag(To@EaAA?Q6)sbzK@#K?r z`-{j5^+KTUe!WqHwe9jK&p{a@U*q&FY}xSmAM`0B26WS+a?Oi|8yGb&jg~nQ8MM0; z(&rJUYeDZ!9Nwwo7=gnzl@K@IFdG-sxC|5hC!-#nP2GjrB~XRsN+yP4BdBLEbwl4M z|KGo6j;LN7k=@?0Hw^kbNM@gi0j1}$H_AA5m);@Cb5t&DE%k% z5znjB<9hI8u5G2^qbZxJl@hdX9)X`ETcReFXKK%mp3(8NAlIg{GE@Dm=w9E|)0W83 zUKZxV77q@#Qi31oeJVYM5YCFWmwA1H+Bn`|e=@9cw;nd(`J<35rg68s*ZCxEZp3QnE;DF*TxadqcBcxD_Bh`5; z?*?NJkxMv)Ja_7CaHsri+1`mTrNC%gFegq)m1a(Ds$dTzy0W-D{-e8>>t&C~gfP2o zj4r3?Jr4{1nEuy51koPzTT}1cJgBl3cANQxf~4uHu_e3V=f6cDFR03<^2f9ti$$3h z^VG*Qbw;EQ194e7?ekIh%I(f37ft5$!lIe3MOy8?>{41`@`7|x0kj;UzX`c4Dvt9J zEV2*GPQ|ks`X8?;8J^pf{^xe@S9zYVG~CrxkQ7#7@t+*>nM@Z+@e(B&wwE=#gUjs- z8s~jMtmn6j^!&nlvMh`5wK+X>)OilqT@#tY z#@4_=1Re1!>;j1Svb(u~+6m8Nh{J7c|B) zb2^7Hu9dnHEUgmi3ql@F$f>KdH|7TjvSIKK$pM$s%cy2XHNutM`2utaszp(^%QF+_ z{%|G-RWbj&Ta32x5%8I5#pOB9C=j6b^yzgTm6654%m@FfwH^BdYXcZu&;CNhDjyL! z@R!zPKs5G=hh~EgT6?Lfx$Vt}!wC+?Yr)UndX+pdn%n`krmFT0^J1r% z&b7e9L=*TLLe1Nu?@!eFpX{edDxiW2W(SU6`HWuzm%=MRM6TJ_cK zG856kqShjlDcyU}rpHGN*1 z=fDFZxr-9XL9b&v5~MZuOwjF{paA2K)1ca-v4__3BRhm(I5Tx z(AZzxT%_Si5+^<3_Hs320U z)NtL?I#K@YSHj)N##__*p5CtKEv;<{A6C{EGwBhc#}kQ|MRdx+{e zq=@)hap&J4dq{CJa{EDZ4x${cF8r+5`85{(n^NDql`5*1j>MeXfvs&g@kvMEbxR{= z`M(<$^4pBxf0KBTXpN8B*QX;1;%&0N2-Sf~dJ&`?)+pyuu?5DlH=bCO9~?V3D3Q}q z@PsNUz#xeJkefKuJU5qG`7MSpHTWv}8d4|2QVNvBMY^d9=(ROfs8&RL(cPrWW{O&nB}$CkL&J7Zm{tI@*@4!*LJh5 zn>d5c2jq-|&|NAdV}QR4g zKQ_GX>gVv-Ln$jOx2swJdUmHSweXndy(1zI#?vSi2Yk^jam>zcgg;n9BckH59YrI^ z<2F`%9>SDVmJW9L7vd1S?bHVQ#b!zL3cx{zg>61xQXBOpKJEEJT=gbrx#6VlnT*6c zQH}o-g+WBck|G==tlV*B&te)`ysH?4;R-=nU3D!;f&+vTpN~k$_SSp9l3ENv89Sc9 ziC4L%d*jh|Aw`{Q@L>{SmH!leI{a7-YH~Vp&iCYZeCzE;aW*5MT)M4|m5`Ma)0ROo zpyZ+gJqf&nR#eiAa$rF$Ew|2z(xJBvT3G+~i#NsdgGJfv{`qj>W0uwEinrz9Ci1tx ze*Lq5|J#&n%ZxXhHz%#-X3W__t2Eo&D&T$k;l0NFS`ctTe$Zc&9|NJIK(L<87Y~#J zH#Fl3s-R?lOlpI+U~<98$5yWt;fY>!f7R`yd=^t>Y!UFps&M$V_|I3CQE8em>L~*= z^>k!>O2k7OXV0MYkb)UfbC!r<5V59FfmtP`nMe2AIcb%QUrp*sQSJ2mE150;@$!Kg zoY3asSV6}dgzut4M~WPyHtQ*0JM(PLc+-Z@4pbQiF(xkB2Y5CUNJ_v!2l_}?)cf^$ zzJRGOKHI(E-Ol5a!D0a1?sMmO?qT*73 zFPC$g$h(xA8KCc!1C(-t2~%tIS$N1GR*_LAa8cR996h5mVY1Gd+9cCzzviz!H~XB! zQatSvn58(QhEXSue+=tzZM;#o7}-?NEQViHkpYqdM-;Je)wU;=oFznWQy}o)Q7znA z*C_yMDU*0~FAvZ~w10k(5QyI&>@JU2i({j18k2%{DCzntieVitP7iRnUj=y)? z>ka>Qw+U;3l`A$LS0KRUM=||cN|Ko%oo^XE?}vHauQ+;NQ015WDy>c&>g0N4FDS&8 z*`<-d^YEIQ+TO9gRaYwGDR=LdSzhz?O+3VzTMxz?LW6_;-hJ2$&Z-N{Q{XXDt;a3k zm%DWbBm9}j?LaI9)fYaE$Nr0M*Ubi9XZ8Dm#`N3Ssv27YS4Z{P<_>>WGU0<=XWrdl z2e-qpM+q!#+#WU}s`!4S5g3w-1x{Pgapy&O=^l^r&kPo4$Ufco(d3x8xPl+T(^%@i ze_D*z4EFQcF$2Oq^uQTxd>ygiNRXW>K6F=!tV7X!N5g%z(Jvx<6}{oQWZZ8`9AUS! z9cSL5B3{4(%Fmsjo0Iv$iVg@ZZa>UkcE~H@Y4&>iK6AAJXBp_kIQo^Zb@k|6hKK;> zlqSS%7|#@jE25ooxpXs0E-tLu_s?!B_N7?$_~zf{$?%hyd+gqc0$VsX?0b1nRW2~y zj|`EWkw`${a2X%7|5vyxVgr22crpWnb?1VxSFxm}e-=t`>A@ZvRjE}`1OD;o-n1Uq zJmK?&&pmmErlx1LRb3YMkL7{rh=-B4g> z3p-sbJijRZ4gVn3cZE_w&rXq=L)0)Gr3@)36?QRV*^5`MwVEU|P~_uqzo=odM(`8l zn#9L(f@n624f=gg0{dr$lki$yBAttd*q93&V|+HX1H_8r ze~NE%TFf*q;E|8hB_F}%Rw%=EzOHVOnc%3#ogVpPzWgOltSWL^7(q5w13@jEY8!*T zc)P*1>iE+4%8t|qB=dhocaw*hDHCK68B27>4V&gUzhr=g*Xq6CayXgMGm|`w9KCYZ z!xZSV&oQN|{qg5t@~Gg~dmrb?_X1hARue}B`pU}5fT=7SQTK?bu%3wt zSV(u-rglw4wz1bm>Yb;`?xAS!M`;{hPiSZ*O}I;8XL zNI?^%4my0#s$CFsF>qS!HwTYdZ|{d8Q;DeSLntl&l${v;YS`7Dr`=-qk+Xq@yo?-r zjuN6{pQH3~h54Egkn9j40;VXRNzk0pM`1!8KIW*&Irp~zE1>d>#MDS!-T`%@a$7vO zMOB`V#2r(kQQdoKo31<48Rx%*0Rt})Lh2|BzlG81d#V5732vieTwWQ7eSXkU8qA=p z^DX`d^5f_JJf!rP&_+h$@`edlB>^T|m_(1)Od%u0_Bte4D0pjuoXIh|fr%H$!xBf3 z!BIr)5P+*IIwVYtg7q-qv@k=e)O|@ot26>yDm*+!EI&B$oQ>LaMmhK~|J?&IZq77k zSVfj#{1jxHqn@gQladh8oQif)&O|##7k1T}h4bW+#Lkq+p(=iYTYku23-)NW@y4SJAjZOP1A{&MIk4PYmz7<3OXvgttApakE7M189XOx zf~UDN_Ti-$14lZpA20FGx_Bl#cO>Lba9j;P)?c;y%)bc`$ZOj~{^?<#rL?){p0HfC zeG#%K+jdmdMVw0@CE)vx8W~Mf5$vc5eCvBhZk;RtM&cz1Bv!XJ0v?|1u>o0&iG>m} zQo=_S9)L{cbYI8ydnM9Hr5*|}3DN;-psL8vOdJlFdrQ5XkHJy>3{jT!r8wiH)L1`c z(=bIHtznc7fdi9yPa`q#R6a>Vi;_qO^finBe65WDNR{O~Fu z@5$c&*r%oSc{pWdT95#*gF_=?pCX!X*kH#k(qmp0+`jWyw>*I<22;=^^Iz`*87rV~EzR}m*zTwq z50_d0Uj$88nnwRfjP~+zOaSJH`wlgWtqYvx=k*;!(6ILMqjGB<{w*}fqKH>;{abPx z(N$sQLJl358$zPd_(MGAk+?xj3g8tv9+H57-LzM!$h~N1Bb1neo^#q!7MW&2v0nLA zd`?MP4qy0ng6FKry6;GyFRH2$8&R6KjZd9h_MfAjV!3Rw17gUX`VVcF3mL3;Mdm5Z zpXv)Ve#Z5Ry7nPGn#Bg=rxMysjcPN!j;L!`at_=@-AJ6!kz;}?a~>>9I0w%qQLrrR zt9GOteAsZzk;DSZJSK!MH_sOd?2JqfeWD6&ZI0XzP1Mi04;QcDF?^Q1$`a`qpzzZ2 z2hy(|5Gg~N6NZ~1yrr1Hc^a1b5Rw`+$;@_OfMPUH*02$Y9wr8g$bKMc*kw{GJ4ETJ zMnpOOSV?EI_c}M#r8DCEFZ60nai1CAusI~0l~201_BBK~|A_a%@ml?Ha?};q)8Lj- zTbu8AN23jr&beqxU3}vfoF2#aj0+kPa?)75v3JwoEOC5Ar|gjtO^+JkJ@=74FL6qm z?(k{qu;xV@HxFXGdvm6)N@~V+rU`rF>2Bh}KV>@X#~Y*t1^KSQRQ#61(0sL60FOyh zzsBJE#U`V-hLnojmi)trrM`qk+zKgbEK$fZJjj{zJYNZ(g9qLDM*7N8KzfDil7+u0 zxh%i}zEe}ViZn;`4m2xRCF8BP9+mHLyd#GYpzRpv3c_PPSD(rgpswvSe-eB6mNNC4HU3uhQSvbQHY_ zhKXfF?WEyjz6oK(5q)5Rl2c~FL0FViKdYB+2$Ke?U^wr>(cuxbHD6;s z<@0e9*d7^Ek|x!Wh?sJ@ba>BXq`+*e0iOuRxyZ2L)(8Eiwj)wsf1c!^+xB9mjh2i9!+{7h(=9g1|P30hTz+GG0E3BPC z;ywdtqc85lHeSZ(L&v|AL67O|Hit!HQnRx1ym$*3E~q}vOW@@R1h>N!&)G%haIVfZ zV#FIsoRaCz$kt^EQThyBNdYkC>#tq=SRBkR&tqPy2lYpK5m*rDO}S>Nhr|_G)mfjG z(y)#u-T!fJ^8#$P%);rD5M<-Y7jIVUK+a@D$>&Ogxd5PX^K4?V=}v~N98j$>DJeb| z6Zhu&GfZL*jf0v%72xZ|!tsf@e`1dSb|ju~mR^7e&~%A+f>uzwjkT zbcal=2LUI@SoZ_5w}rlUP3e4<86h?vUpr%M_P{#-j0MhmMz`K#4Spr z1370wp7?-F16pZdNk9g>?hZOp(7&+WS5G5Z^QwTlthF6c|Jb&C(0oIqeK43R-e0HfUYEB8 z$5%g~2{=Btw|n;Jz3~xGUy!@|qxB;-{(6EhzWpZ0ctS?udsLt2129!d~|IA4AwN-gr#R9TueZL$ITy z1Ou#RkhF+^)E^uT%6`FZeTqey`Awds*#JT+#TLU%NZgXrx7*AS`N!RDJ``y8w zoi_4=hEa7y5W!fyt@Sr`svYPD9amNn0?z!rgx~iOICU*NEMh}~D2c5si_6akQiu!` z)Y00@_(zI}U1b{WX9|iqqFIbT#nPH8R>h#8O`%g25k34rRDZF{}2KOYo#`)s&|+kZX_9-Tz{YFrLQIU`Ih=EphKIC3Z$~{vK5n-}< ziut-<=rNuCfmfT!{UqlK0=snTR4`)9-7C8I($7+|Sk&$UL~{ya>Q{hI!78xRgouJP z8_Ra+R>fiSd?jsM@olV`f<~Ls3f#d=)C#Sb(?9m#4m%X|ROeN~a;}s4^?zlTZ%dz% zNDn^;j&#>MeaXfL5?#k9a@cD6NZcDS7*-)or#qALOKq9*UV5t+wbb?-DHA(S)XqwR zx%YeD6g1S&;wcS`J*0R&RxLZdVRsIDGkT$gI-`l4)4oi{7e@p$s{FUo9~R7})53o= z!LPiYp_o=GfbjrShQ-yTFmQmI&Vq3KrFImZT-PJQj)M-MB_NV4g5h*IP^^fFcWq=r zhG{Rwk7|w({OprW6<=|cpAgX=5>_Ak-#oC;u&gYzXU!Gq$O?PL8Eo{5be^>t8%u)Osy=Up;Y%-q~+>6tU5b@N`I#)Zop{I4!& z(6f^!Dos}dM??gi3YHx|VWC=^@eiY!OahnT{g9B{_6t8sesvZCn=x-dCCGo?2^lqD zSQvdAl{Il1!C!ks&fsfI$d9^Jcua8Je#oo%7@Vl z?cH!b$CQi8T`D6p6Q>`EeVzhq`-LxqbvPA`(T%Un0$=c7K>HVt{>wReC2hXAq}Kmu z0;!4~z`Vrc_iZ1u`#khkCj%|^da!m>%M|?-_AW9X($S@;{>A+f6|C_1S*_4`{*-ce zvRwV_)kcS~Gn##b%a%st$c)-pn2Tl&{B~}XDLCgsj-ICeYBBh!X59fHXTp|2I$vJd z3c~r~fNEH^q#`%(>&&vW+_NF75avlXYpvCOO7|H49-8~Ya5z5?gVJe9=7$t8zQfkY z@He5Z-jsx!E>2Gs){X0zxI5z{r8PtPd+L;hfLbpaK-ZUo_S2`L5zo)m#?eWIkkFIU zKsnhiZz+0RH7|50J-=B8llLA`fcbHuPfjv+r2vr9&#d2=S&YU$FZEzF%NVUDPBh>n z56K+YcLIqW6mui8c$=W2&A#fuXLKSti|80H6T#=_oTpyA&p5#Npw=_4bKN(1DGVpFAACX%vJY z=u+iBxDf{tL45Wxzq+U4+IyEpx!Mpx0q-I?A|G6mdZ&=^gZXq?L0}~qx85g#Ct$i2 zxtAGD7$0Q{Ma805mgxD&x+Dat8Sj}E=g?hMv)62xu)fu8Dn+d7g@*{8zr@#WYsb9$ zzxaWnk{AI2FFbJi9aobcr*{?o6LZmB*S~a+294(fH-N-Q=DbP%ct{f3?tTKyjH4^E z=SV&EpNnNLHlQvPu8~A`3|J7E%EgwRsA-7{X>exlp#+V~-@JQ2`|*zHfiR{rUN(o6 zfR0Ydmf3Hv>;V9I3@a&Bt1+Wg7BSSXX;RdxRLA5#NnJz0$&%Wk#&Jpvo>|u@6=2lT zLm;=qm2v%fP#&+RNw4ePiVo^a4;qPS7Slrm6O2Jn6HWQBn0P7lxB;Qh)|}h9i;-%} z@^`0cLjqNQb=6IN4#xmasqR{UM8nahIrFrjb7FyDJhHlX_moEs;C^{{V$547pFBQ9 zQ-}wc_mW6=CFi?+J~kV=0T~;0W`1&1yjq8Fy+fm7%c%i^h1?lKs6D`Na;9f;2g~#` zXwz_kp|2wdWe)A!*cD9<<(#bOXKZw1)1}E{Kn@I&7wgZH;|tR`y%g1LXquj>h)JzC z|3|fJRsm{zl$`z)o6usAhe{udd<|nVD4^#U8hJ2Eg!m7HYIQH*Z-e!x*g zdY0_)CfB`uyxOd4z4tOPI!vU29(RVt5%KXg%G><{I`CgBumMMvrmv@fXK^wm(1C0M z^ma&M`Vh+{uYGDq9M7ND>)uj2+|4IQx{5rRw9Z24ya6k!^0oCx-&rebfdA_^WymB( z;I@QXdhg!O4LdC>y#4^Ve||E{RKF)kRk8pBy$*_=m`P`ouQKq9k#t+ItVzXZ)jTll z9n`jzg6VoR|GdH`WQCJCl$1MNPQt!eAUa#+X03RguAN>&v82RD7OV1leo)(Kj~!J*^O8dQCcV>DSn z4>rQ6T~exi(}c)9%2U?j&u}4)(!*QLwT9|DU#er|`<9r()F3}s>c0)vIqhBx8&N4Y za3Z`lsrXQL*9%kmw!D*G(HU1wj)eg+=!_;I=;yb z2o5wHHCs~?!3pD_2%!$LNW>!=g$ZO zm(cmr)?$cY=MH(vOuSDxPzhrT3rI{I@o()*om@u+s-yQ(I%tJ+LRW4S!4P5DB3V^K zWLllWiO&(y+Lus|RH=@Q0+-`&ws7rqUmlbBUX5){w&;n>4=#Wj9n%dg%kICzR9|>O zmKPZ4a8j3|@9sm=7;F^peFX}6Q_s+j=sa3acW!W6h8cIx9QlL2&q-q9mxxbgaGOIg zoFSp~)ekl1OZBtVvb^tZ0@c;J`+rb^1rV4X5;X^Y9*<>I`HwQ($1&%E7WG0w z!6Ug6V-CK}B3;O2J2V6go`}E;yte{lx`LBiYwk9!Y|cTS|9eE4ZsStc5HGmq+0njL zV)OMcgIKYgs`CG-EMLTvv$MXZ?2kJQDRgS;>cMz$ zX_wC24^b2ya?VoP9yoFFaj4ivT#f3Ml;B@KfvUN&%5;TAB;xjxnz)Fllah%jgK9z@cNd?RQFN9Lx;i#kVz9HUXgb{Z z_+0(qVhAiekjag5i}4Avyop!w2&I;dT{kj{MiXp+@HqZU(H+ z>)bC^h)=7Hf4u>vsjDq6ZSj|v`;4s^PjGdYZQuA2OP5@8{=c&F_k-UM;S;hYg-H)d ze*UY^(tZb%DI^A6Z%18YRb0Jvor8Uk(s#wrdqrt*6MB z>)oX2blN_l6S=74>KhCGjSWkA-`u41n~aV#eZ($fGGFv@WLA&ZQPA*$uK0_2;a}%h z_U%JMdjZCdJs35@KVl$-Bz;vSa&u5xQ@8@^lVAY1?AE=vQEh@YtEDElMi~9w!2`Oq ziA|lxs{&DMRX2-|-M^|h)ouS!yeJ8@WnC%Zu3XR$fuR8)CYz60fPy1h4c}FmBnGDi zF~dL#6=fK?p3>Y8Vs_R|t4_26)8jZOw5XH-BNE0&Rw}A6SXT07bf3r=TxHJ5OXCcN zn!Lej2?jdcT}Doc%vQ8U#tJ@6q?93+B{!+ehTYu@ zX76R5atq$QnJpDreMOSfq{v#uq*)m!5P8*_uZu1VY5 z1jqDr!AJRj+2e06A{CONxsqaSOBxFV!Y^MKK^-ruyc{#Lc+2PTJ(%GS8*_slvJf+R z4?zA?DAhJVw6b41T*Jo#-O3_Vq3iY0C~ zK3K>8sOH$j#K1D)h}5qAYLn;E>(o#-GtTP}A$-}lhF zzauc(pEf+<`Q8>xW1F77vj3X}5F5*QB-WS2ZT}A3^1Rl0zc1z=Omc7Dsn7nP!Nevs zw8{5mv`6Ildxax4E@jJyu&fdp$L{TRDgcaV`%Q2)H{^kw$g2Jk3TpOuQ`9jbQ)F}Z zuflsUK4S8@u!18J2&cXWPFXmY8C=!Aya2V|RjynW zbp)4g%Vj2T<}5a$7N>O8Qm?Sx+zxncdf~m8aCP*tocm}C8$t3$tMZ+xFl0o`MfhzqDGsghZ~Uzu&sa<`>Cp1)97*X;w)sgeKU%!3kq`H9$yOo9B4il9=wf zlQzZ#h}qx=;VdsqXS}%&`&J0QnIa=9>mSkX;CNqq*|~KMDL=>s&sPICmKV0tSqW@L zMsqtC|F3M(ECm!YtrmZ(RbSoRIa!=PZZVlptfw-CzHx@Z%o8c#a29r`*%#=^p)cfHjaws?P%A6=$So2Cg263eORfZhIlY);K9 z9B%G@u%`J{qj$1w*cDCWZ@fPgc~)Ii1pal`{pCN5=MB64Jj7n(vHFj(dA{o#uW15} z`e=E+tID{_Tc(oef}I=$IyGf(_EagwQS(+r;Kyi!@l#Hin|mJu3fitCOEKfP=Otr}}{*wH94^u7DNJO+6xXv@RP(-0>)j5&N63SIMvURZmGFlPoeav4n zsz6}7t6VFkdLFTlcusXYYAI(qVs@>c^_hs_qTtGDjzy#Mo$Hx-OTRv;*iuyL9iE`o z8B$9783*G9fnjC(MY~LjW&r6i0m1MCM7S@Vek_#H(rILfY{4zyd*jv`@`7JJ3)Gu8 zpa^W9p(5FeBeN>|2X3b3sHNfS-2Vml(!w#)A4l4XC|Z{D|5p$4m;>$nqH2OmKS4%hDS!P{?Yk>%{T9;rha*q;^R(gM@^6&<_SrCc`|)-j&h8H}!Sumtl{Z@}VvE})W;wo+ zfcD`OdND16DFI}1T$+IdL#C_SmW#B4n?sh- z9s7?2cwz;dpusgZ8f~QR2d*v!Ppgn?xWN^5Y3>DXCPl2x=|BfK+_r<9wU%~~ech6B zD?gI@*ojL+aivess4hyX2ip})Z%rhu3N@ORil`LLy;ai_%ICFx|z{Vi(hqGx|$P#p0(xlOB^cB#80O|ihLRL%| zN!hj=%Sh2~ht?W+SL%V^EB&XzF-1eWU z_yVkb{x74+FiAvEuata`sF^ouxA{rJ4s|Jt`u;^N-iUPAvVYQg7UL#3Mg^qmT$R`o||& zZ>4n5yaHsi7dXv(eY+aLz(33P&!=wniDFXeB<(dR_q*GO9xdDMK>W{}VCbAoEsnoJ zes;sSe<_v@@x2eDQ6CO(hglhHgaGBLd7^p=kig+$!YG**v@|wHl?*`@lU)i+*-IjQ zF0x`%LhfoG^8}N#+!y^RKVufbkp5fHb~B=xA@I2vpYy;#@wot>Il>Ea-1TN`@@G$o zRC_sqQ)(+%NTjTQPb;2@X)qUsJuT{oUU6moMf2~vF@Q!l{r!-H$czACN{=Q!w+5Y; zDJpPVPm=~IdJA3373&MKZpNO^0v%7T&0r|gidb>Uad*p}Ri1+XG$PKFk zuS)td;@x0_=vbC<49Ctqn9eOqK{iU((2zJmFE_7 z_65Me$?L7>>qVtUf++Kz8*$MIvS$*;RP-uXx!yyuEU6(%G~^~zwxLgF3L1Kn_72O5 z43p;QeKKvll}^X)E)PG7?{Oac%#cTvzk4vH!Gn~zpa(X6?gzx&>l=riKCd`68Tjn zmPYyzl0ZwszyD0*hZKHVecByv0JYCSnrysHk}m%jIWEczXYq&QyQ5qyoAEr84VcpO zFL3XY^ACQO1l@l($3YY=(mFri4)9N7&qr;Z7+}i~l^+19_W|AK{)tQ9c^@|y5grlojv;=t z?|A|Nk2?VGbp!go!|&~rk&yxQQ%p<@-0M9bfB$rvKXPMZ!@jsp&WyNWtO3$PhA5`} zFdRCh{rPda=3U;KgEpEgp`vmT`;|)9tcjc$D7ECV#yDyV&!>kJdu_dRQ%prqXX<_F)HJqrhqsQG{>`BD5L z8oW2t^y{nDiIp){Sz|1g)klhk9e-Xt^l4RFbOmcVpe*Era%T!5L&QWQZ7%u$;pv>h zD}k169jvfpvt!$K$9BiIZ6_^*Zjce?U-}R0 zIg8G(OCAXcss3zS4ugL>nMkO(6_DM^jv%XYs(^QrWr;8(Uyx==ACO$rV zJ+DWr=V`Z|pd-IG-oU`XPQCx?$1?hW{)C43)@wP|zjIB6=V%4>N3!}CsLIJQSlHk} zWpl29GA%miZXHOeS%H&TE5E&-z*()9%Rt&2C(}8b{V;`Eb>_pY402<((UsYT?t2Bh zw7Zx9;U_e+!*DPJ0^kpON-&;vdUE*OH0kCEi`eOLp}<>_T4&X{`TRt3Xm;B(ahvA7 z?-_EbASf`id8f0cuxgws;aw->+~zd67K9AatX_}+0g^Mg*fGOQqgjQzR3ix7U8Ph* z|6!!Z9I+YF%k5Gj@*q)x>X>w#bSHrT>=!Nb(cXL2_Ns3sNS`Tlp0r0MN%EuwvbJcW zoy$T^y*pK3i>F>@4by)Dw*y)JjU9K%*1O|0(~ z3;S|nlyw=T^Ytt@UJ)yxpKLi;Nlw!Q%V z_iG>PuWJ!BWNlO$dkV9NYBk8`JSXl0gCk)ZRdnB%9)W?KAgJqW60N$xS1H|{m5POY z9}Y=m{TWF!IN|&HDw&)0sLhDP<9sZ;k&~yq5_kiG6Nf@$ub36ApqTo;{F|`$J&Ru$ z*1`0YuxoCY{Syp5Veg$Q&+nMsoHv4aT6+XIp{2#K1!XCfG3qv(U>MyxqM zj#zYDe!hBd4aJj)UysTADiy=PR|vk@uZeHx_I%B>K5cr{IhUpR%ILPeU3Q>vzpZV3 zV(5K0ixhU^np2tp{zK$6mo0+)gGt@2kB=vZ5xuVrpO7;21A>bqbKBCPwnTA7A$-ns zM$5{If*cJtj4JIQ7pv`!i9Dgd5GKjLcJ23)H)SOtR5IM)DKRb<6n>loT3BMW+7EP*B zl_FrW1+~QzK7O@xTh7Y)wB%f}C(N(oQKY4ru0}PkKLAcrP)I99;Z|JP$iRp_v-CoL z8WCW-Azb$4TQf}I{VY(!lO^JgFvl|3{q;o%#?czUIKP|lV^*wL$4;J47TdhKhy|6G z*Z;l~o;O8p0qA@3rE%}=+EEJegv7r2#<=NuvgGXbaBp8sQ1rxMIQ55)`rrvp7XN;7 z-NK@Ja0&~F^OC=vODm59Q7uoJ2Z;)X9^lAs~^Tv#yzYb3`i%%PBg?ddo z8j{>a8a#A*8yNh~$!t?9wOFnu|2_eiYH`q#>$V$oZVv=-J64Z6-}3#Dep_qHa5cgv6>OJNm;{Rh*Y}Z+)xI>O2swt9kI$KE--jVBs34!}#ZiEcj!x9e zs{`zMcXyZ3Xc(@Ti=#d!KG#HHVSc{ob214u?loGB#L&<%>CduwTdvebbaZsNj#~S* zWM)V}a+*Y_abN0)Np3b68TdxMD0ta?}dQ>B~Cx!Sl`DDF>QSNTy^$qlB!j|sX8KHi-ZqluzyXP!E$HjHOXtYa-_|@BDw#LD z;|6B3d$dVFHJ;28nihm!1`Bk9>`bGao*t9HHI|{IpL_{oo|YxzDsJQohjXnRW?)~1|Dg8h{+5Wo)RH%~wg zjf0a8a1uwuw#r_g4xNERbU!R9ObYg+45ajN+*y_eJY-RerZ|lnPcwxqijIZ*OAG@A zNuTx{+hB&9m}HybBGqa}hVDuf(wmsd{c_^GNZ`F@N1 z+3}f=IkU7ACDKtV5&6EGC)YIQHvb1Bk_{77SI6}I>i^+>7QHkpD$6;Pi1YEnRlPX6 zT@kJFlKc4#H6CqoaJsPCYp5Ncxt{F3yuutjj8YUKY=4UPi_QxVUtCx0sm;b-OXj`G z{Ir)p4l^ln8pTWex=>kl{gr{xa2oPBEu#IaSLpp!y+=^vMAS8AU5bF7j!x%eG0_FU z?FzD|?uQd?=N+o#;_8Z1vz~bW0?A!I=W{DNSm}E@j^0S738+Q^oeP+loc!hC!6i?M zAm#l&ca&PxY;I+R3pqG;MDTSiZhmzQZGe04$$NWTh-4J%^Y*w?#j#bS-{`Zv2?j0U z6JT9ZSy^c?(M4cw%q4+yo-YzgNVsS&>c7b4bhw@4btxn)+;3hH)n1usxWMCfIXG7+ z&K1_4P)vv`xj_uZUmw={RBs|(laWI+d4kX5y1g9$?hSN~trYf2X}`O@B|K*UJ^17G z@-pKWGsFL&0YSPM)r^6LCWxnc@V-)OFfg@T+>A&EMlbWcbXa? z7X78`$pUt02S+G)=x3q+cgsN;r6@*UTh;`-z-TW$K&ekMNuO573A3Lw90dnCq*oA} zTT8C6q9O2{kw;nj0&q`_)Z3_-d0aJ@)=QVoi=npx@~|w!$|Vt4+>oKe2WG^o5%=aJ zgA$Da-K^0;{fyT@+>15ZxIkYm&8IJS{)#l1NQ*@k5iJfPTQ5j9HE+^Xhj-DeGDV?P z6Fp+Ed*iiAS2z1oe*r)iAy!JR3(9Q6|BJxUVpGl=p%K|odpqrgRl->`nHJtSiTAK6 zsFoGNbltord9g1gm-h199r8x-M%$k{TT12ZiREh{nDv_8==b09(B4a2@Hg_;!ybVF z#ltr3yh>W>-`z!%0z`YH+vh#~rmGft6YrZOWPR$!0I%gg&g0~JBfw;7DaWm!W? zL>o#%K%VOwAuO}M-zqAF+;(!R2mGxsp8omBx<;2ocnb+>DS1t40eOu6m%HCXW7v{U+0}5<<)_ptYO2(|PWv}fyt!&z>Iqw$;*H9lUGIn|E%H+ag1DSJdAwi8jQOW5I)f z{mIF`2y_~i>_RpBi$vWZ@LoqcqwcX_#Bg(GdQi3g!mDs7C%|#_8Msxk5l(e*+ zhcEoAmnL!iY8D0p2iH3T-;~Asuaypj-qP~QkK(6IT&g&v^7|MtR0q7*nbY&wDAn*yf4V@du38oJo?`oLc{nH?W$wAo12LrTDvz3v>$ApQ6SwiPM zg6z)HDUvUc1W3%mgZmA2og_u+PUE!Ue${bq8c)#dC)EqySqA<<w0qFS|E& zaJ(0SI&|%5&A#S&2;TW01~eRlV%vYkYJhUstV%$NC3{vs8asi_j|RPb6kqMe?(EQA#=+!)^`H1}`pZk*EI1{kR#d^PTJoV^B@>zrZq5N;`?ztBJN(Tiue8|J zY%HY0!YTtb-t*2xHZ_N!EI&En;ZFSY+37r^Cm(xku&*Pd<+egc|H>4-ic`-D_wxr) z-!dI-!?tF_zNI)igp4h(nYBu?XE2U+f4*9A@~{~_CMiV`sO9@^VYkUqRy3M%B2bWa zM`JP7!Zwhn5>pLeZ&lx^!;k|=+jfAslVF+x@z||OiJ7;@y8O3la zO>4ZlV|(ONZTe)}4A#^PRr@L(aF_ub#6g>_Zv0%|SE_(cF&UY)p!lrW6-glXDR&Bw zS3%!2sBY6i*bGmABo$Bm0ArzxgL0k6ohdZx+oG1;#XMiHG1N@|bn;fs;3PK7%**cF z@mwL1jh2z$%YS!aDI^s2+l!GG%&uKxz2>FeOJ~m`Td`>i;F8(^P%+NZcf z=30eVQ0fbZS&|2$T7hn5TF&;qg7=3iN74tcx`^64yNX*7iD*;t7&2FQ-JRTV)Up&TSmStm=sgMSSY#FK+!88p6^p%yKVVMZiG4V)y}!jm5EntiuWEs zuaHdiJQLWqo|qPk%=;>>k-%FdJ8yvhYO+q(n3&@UCMtC+aCjF+h&aUt;)I>AgLz+n zCK|t3E2b;Lg9JMm2?>1~bG4TDW#9NZtufcW9S0R3V=n2*`a91*m{ZS$S`8uHw(Xt zF1Hs>-hnc|zwc;w#OJL_p}?(@?^rZmiRalOPdxWEmmyuM611csyXK{}xUI!y2L}Td zd*G^#B`4Ru*xs?Jok@*BnxF7vErFESncx)811yO8+Nbo|El+;GnVucV0bpu1x<1ny zX2T!MEp(NQTYTP^6o)~O2>o7HU1rr{lKZ7l=&T6`w zm;J^FzjD9c#hG++J-;JCCUi|6{IMpNzr(=^KDV%YQzPY(vO3=lMr1O$nL{n5?}!A` z`T(74pSyfmb^9>E(VdgE9TQio9w*NyJx}?~q%})|uyE{Co&zXE6DzcGcH^2dX~+E3udJ;nLg_LcnuqUe@%&DG$FSxqT;Ppt2}mvj`q zzlfJBQk)~PR;0|lG=umrXFSI-cXV=V^Z?F+-_+Jk5&litAUdwL-bY{vS8ZUUY)t!& z0Z-->VnVG8?FH4FVj<<8uOR~z_O)O~+Pv>&$tkgp^wj8>S)|gogn`7NBplJ!jqTK* z*|02Yz@0qN`GOzYi8B6NmVX_YI4Z9IDr!7?Z(m)JjYWI&mR2B6)Px7Wz(%4L!tkFF z@mBq^{0)iN&Vj_XNGwf%K7wB}P=qr>We(P&>(6D)X_YmyH z#07Jd(-nHRZA;Kp9kOqm8BJRk#L&L5;O=$n)C=+4*M=Uiq7YmnQ4plGi6ab zDtbEOgMG_$ADdCxg*KTF{=p{{WMm=~?e_Dbs0~ZSf8yb-YexS)d_Py_DI~ucby_N;Eo++O=IXcJd_$ zCDL@UDckqmft5((yit07dn=6Lb_FYybT4xoillsa#RqNm6$oH;I$j6Y3{$XUF?8pBUE+k#|}N1O0St6k8r=apLCHTwL}_ zI&kxS9g0@Fp1Qqm#wTjh;HKDIstt03W4`vjJ_F2he8u?u;NGUXBCGRj;STUXY4p5z zL?1W|s=e*ja^1`$!7|VuLbIKm?}U5OIY_)GX*p4Yt?W>kGun*M6MUb;&K%g#`j>Wk z|Mm>f5XV$DRx?;HBx4$^-SrUdRsUi;!RJD%Yp@mC?sWhttOCCk6%!JhyHp8{7MTDi zS?|RwkE2*DxxUY^y402{#}}f7rH|qZ-0~q`~!7mhHNSrT2orl>;E#A<18fNaRom zSvk~Lv{{oXt<=c{?Ts*PR3GP20&VGSyN&0`@|t;*_jt)2_1S}|>yL$Bqr`q~Q5nZv z2=QTRkhmx`!K#6U=WS@N-NTVX-0-`c`}e8mkAv%daLyo zH_*{Ug4Z?8e~Dm6n6ai#%@+!EQI} zY%+w-t^Obkudsj>9y|POXY}mmK3Z6R(7ZU@OdjbU10!0WK|h|10#Wmg?@0+7Bx?E0@Jt&FVC1b-qj(*nd_)U<}_&?WsERb+k24QmJpc;my}N zVjl%vi;EK)7u zL4{A@OQ5PSzYD!S=0K=60QU@6ZnC{MMf85!aMOOV`<1=s?!z&L<7CZS4MGXC&|s{N zr}N|OkC@M~$yXU*f!BUdmV=0%UM|ELwnA*#_1_;^t12-R7KyZ{PHfTZo4*Dd@|&>1 zXInizuq)OxkMX2Xc0C{LF({mC4!z*0Ky_x7CV*ZgjDK*LuJ6kUOgV^kC`|RqIq~c< zr$23rVW7Pg`t#%U0AdTM7iqA=cVy>xw>OtIHc<1T8;OnnpXy6RO?SEMabf-)il$~? zFpBc1%mW2@Vx zGp% zPR66^>WqN;pW>fSV8a_9S#AHg*Zr>dC>{v$Kx4ljs`&bWHmOlqK9$oQ;y_;86S48K z%DKm#!+bJkxfPOHuDZK*?AdTgOd4cMul>2#69eT8Z$N(JPc|5(jMKA~Rf-wnxu83E zXrP`T(ydd2gZqqYa%Ep?ZfMNU8JXNj2m2#e->#{!Z~{1j9uQzEiam+Hs&3MtI%Gg8 z4t+EDr7qF@nK5(duF8CpNnu^LFxj%>o&IsVIEd6OLP2##FU?nj%_xPH4b1?XIC1$G zToSspjwGDXgQ0r~6!9Jd(eaYZqXsF0J$-y?{3c{DYHlN=!9bkiUYxx>fmFl~=z6B} zHg;sM7Tf;ga3hsKXsw}D9zI~A!=LK9@K?JPsiUcI4P`!`xBjIuod;EDke8FUbpJqu zVgL>qXbAX7unPd`Lj;nD#)NNTVPJy-L!GV@VnV=M+9Wd>Nd*)d7iRwtfdG(Ci(P+s zJp?YgO64i<;%teM(COG5IHW8-ol$qb^12ECU@4f@hx-3(dq%{4V-o`w4Y>tluCal! zB$zPMBYkPLUiio3Z-Xcp--aF*(7tC8G(Uc4LAJS*Dw| zJB+v+q+WEke?j!h7glf6Jy5|A>Iw=)kRBKhu8 zTC0Of2=Gd!3o7z!`TV4rL7EIaUm^-UN)L?rVk=sv18(fE1X6Y?2DS&|U}r%8DfnJ{ zooYLv{}xNXkhw65c-!sT>J14e2M8|8-1$=>LYtEjW-*#LKO&d9;gm$Q&D#?5RL1n* zr>1jPO?5-dS;KC;Uz1Q!+QOxJZTCSDg%b>Vr_IR?UPM>rB%;JaUX%ZjYuqZgw61eV z*0ODbkmtPtuwUH_w;Cj4;iyNW<9?uNO=E>679me*+gsRU zS7x|a6M_w&q#0|m4Yg)4n?!)^MMxnOv>#~BbIB_SW`Tqa;}|QY)9zcFn97^UO*4%1 zM~R)DNpP0eKKtHfZc!@evRU58f;Wsur07UFGCw|G1LtH~QIwPZpy)vVtKk53uqd;j z_NrEV^1Ap_JHF|CCi}j3^>vH_{k;G?{HD8WKH@_!wOX-%&s2{I*UInr8Wo`&hB06g zojj||7!}_|EQiB!3JgrDAjDzs-md{a`)Tv-%3zw!2`#9xu>K^HWV?w z`@zcEZ^w3M{*2|uGK^rlSXEt>xvf_bZ@CTLLxxy?#_K59BDLbpR?U+-pplq9Aw_6> zs%?k;{j@!E0jwpv3ISboL}BQyE>xaVxR9ccp%Pd*EY+&ipj;oDa)`J6k`8DiZLnZD zYZkISH#3TzRQW`}Dp&;r zqM`QUG8w?4#Qr2+i}JLVIVU8)gHmVq3KAi?UKqJ5Euvv|GvB|N5~%FW+bP_}ohKOL zf>NBhnqS6ibGN(NAGiaLcXUo3k2pO(6(?j0?A3$2ZFoffWjsoCmgta^ty2iLm%-&q zOCz)Y=?UhMq<(!C$_1eAdETZYHykxavt&tkx0L}wju3pDNQ5DaIbW02wXMTCfB?2)SJ;{rJz!(XTZ9Yk==z)pUMBDWy7_@ zu)b-*#^JE;r;Y7#(lytdWBOPH=$ zfh|W!1a2DbZv~=mw)V+CP3sdzyvxveon>fHz95M)HaVKtYlj`!$;ZicE#^eDr!6l^ zJz8D~0`t)Hir<75U^gxxC8?L9XL{yOKd|XUk z&#MNX5kSt&i@~7r7q&m0M-il&JJ10$mY600>m~r;CWmIt=<^E4j4%z@fqQ_yQ>o{K zHBZyp4y(U?Av}0tG3HrSZZ%mg8ZJ(1VDxZE zUraE;?D!mQxN%c29zIKd`M|{T-AB}9LFG1;^?X3R_((5qgF_)~ zC&mg`_${B#hydXfd;a;nJsI>r!>HxDb;FuoY5rSvChe=44m8i758-J%$BsDQDD{+5 zjwNrnDft6Ck=dT0;#Cj{%?59qMBj+~!mLExDB zhP#kVtP9aIaGID>U><-EOWIkv(%GHg`gfMMR))pT+5dgAg0WtLgO8{&_&WnBh)f=a znV`(;t)AR=pI~wlfwJ}40QZv#$qhPbTTxwK(7#HF&x}@v$<1j)NaYYq^uj7CMMIJ% zv3gl*2C<%m931lSi<7F2rVxbuvov<)oXm~o8gV;lg;{o;Pf%*QyJT#t`Dk|n;*rHgXs@{L;UhW zz+E8MC^u>tdfsoNe%RJOh-9C^4S}H&nS<`|(roQxN4MSpwa`&7q7qjxIPtiVL@#}* zl})_8y*moce9?w8esA$>_vVqAwSO;YRauD_6&0l-q_2@c91MY2Kn-&u8({rj6FL6U z--?=8S?ZZ68Tk|LpCA9-*1vjC&j^bjU){1Auo|<5UN2L(5l#?}umdOGCxSdXrb`-_Wzu7ss6Aw{V2KCUS#D zu!1HtO--+Ehc;|L^_J%y-Owp6mtoOOKKmM2r!?fw%iWP159(!W-CpZ+YsnJ010CL3 zvI(BugUEKTGSO4j_XhD=H^+cTdBXA^4JtdU3F|TP%*w)89yjV67Kfz=8~JdF4;NnX zB_|5|*U)ciR`7)Xs58C=~{>xXbokH@{RAE*-L@Pyb!_(FEb25|9(X1VDj+FO-U zqchqZwJ?yTQhtc(8f#%55s;u>LN^p1(jbikCDDHgs52K}pyR93a^bVsi8Gc3RCsZ& zSd*15sMeX%yVVopR-2}vlai2{!ok+R@2qBxIG~Rhc*Bi8f3q!IWg-ZBf2(@n1avqS zcPthasyRzTGRwV(g|$j!lXn1afjnI5J zEqG}B{$|Adl)Q`zC=q+`y;6f?#I4@niyZ-be3G1fz9=eIwPpIy=enlu+5YwKMQR|i z$hFjs%^5Q&<2!XTjym&`74+&+9nbL`gh?5m8Wv-tB<>m#{w&1gaymvH4nz#fI;1?} zNO6&(x|56RxH3kA4?sJiv!MTK(w@uU{k)V^bbCRv0xQhwKK2p{p*uy)4v|Mv?~@uV zwf7&^P0TI}Syqf+t5(U^wI22^PnqKYGFwPq(OU0rF&FuHWF_;}A=`QRb4 zNm1d}=@Lmjc!n`;Rc#fSKjF})t3+9i?wG)D26pyy0*s^`E2#CvIB7-l!B)8O6AB#- z=z)xeg$327!SQ@PI2=XXfjUXzHIxA){7+^y`gxo(Yh!$+XoFc(mBm`=O|7t6m}9GD zGnL7G)0KPzB*&MaTC)PEDc^vfgsp1Rq=4qBVlhkx7&Glz>$8DhkgwKx>{D!$)a|+< z#^bcV`Zpor>tiG}BscJwkP};mzh@$0cv}cr6%f;slU6Evh>%0BMlCHZRpQ${!IoGN z)%o1%qBw)F(=33MHB-&6_N<^X!7rxc^^_X3{iNvq!A5XcAEE(vv9l}@rg3A`7)tpw z-oM2*2(H#d^)$L{7#Bxi{-0(F`u zCuM$#in%;EyW~8NUw3IL&z-VWc(IaP9H?u@Ty=jSr1*1s@T;n*#5r(pWYBBUqRvqW zhYV=a&i_DIOh#v3BNamH9{lB2MPr&JAv6+|qQeGrm-Gu+r>Q$tXA5=^J$pGQV^_QN zHN?58UDrd`)svcSN=XH}+F#O;c@ufKIW(06td&t$?bmFkz7~j!w2wDmS*mMr6sKv< zk-OBi!*<{wWA@}a4Kgy+a_Fp;)#g0PGbHcyh4y;dXbwYu8fbe-UodP()g&{*Nn4Zw zjgi{!U#hVU4HT%%&%No`svE8LO2Z+T24XnGq=3Q1ec0rK6A7LdlHJg$0e^$SIOlUD zmPSX~>AEWOMTp>FLJmIJ=#_oXBFC}oc8>>G?T%-1;D8$OAO$Rf%%Fl+y7;xVK>Bef zd!Pe^hS!QXf5k}<5J;};Mci*R%u+~$`c-5jL^5E7ojnt$On_#Lx9fs4oDUa-?#Gza z9->)uGDcCidxwafpjC(w`*t5WvYu%hgy~WsK}Z^(6|-sBnaPy$#BmIdCQLw z=w!kRG-&XhTqYfv4uRoJ6A_a#HH93|Lm-Yn`PblB-xl#ZUl_)Cme>@8Vu0k;9*oW3 zKy(1TfsvZJ{o%L#lY0NWl$2D`pRi{+NVW@ENJb%{K4>BZ%B_}QiTwlg+H(t_=9|-g z0|iB(u5J#w9|<#;*H_n5t?QH&nbg*&H!_yoQIRfd$vJj8B*LaOHLhmQ(&GVq?#y}D z&S?4-7zz85YCicBfVXgpg?ySy2<5=(Bhbu6E4x0wR^gIxk3xs+vD}6r_-I| zqIy(906jH-IEuVp3N{1p%Iw*;GL+DiXed*kOsyv4pM=NisSlmY5%#*Zltb0Tu0Uau z&toJdO4wGU=$eQk>9UzXxl3d-7t@t#Q?j{Ds4xAteIrfw^WJsbFrAY;J7K~khW9pY z>AQu}KKg;SF3LGyY-FCjI;zHo=lV9_`6TpB&81_G!1MA1Ayt`v9R^ZDx#2LECbkI0 zVNi6;=7+q5&IO0~p~#&i@X%;Q9EWDSB?MfSaF8TEOKB@E;9sh18GkmeB?`)v9}C5R zqw;6sim11V^rF7ZC>FOFeRA$N0A3TE%Ryjd7mM9XjB(D%_Q?6Bt!|(BK3`V`jsx3# zUuawmMNfrZ6BZ1^Vl<=R^-;En0=nSRe1 zE-oi~O@FN-YVAX)9hXEA0uy%V_nHG$))&tp3qftWXN*Sv|Gcj}kGxPm1zol2P|Yr| zP}(Vc8tRFg8z_S9>5j`w`#gI64RA#Mc_Tjb$K?pS9ur{K2dC#fBNPSmHSzUuTQwNc z)TGDLSQ@Wg@<*Dr9yJseX8G=m4vYG+f0Z#b^)R62-j4nT*Gw`&??H+biED^+)}{E7^f z49b#RW?EiWQYDDuX#4jvw^w1vCn_Hbi(Xy0CigNKIvYnpxB+g)hcKU@YpVRomj0S! z29xPH>L=i`L@qnxMq7q+ty0sV$)gRMoV*uaU6WvSsrxFrpR^eJi7Ra~9La!q|K~4R zR7^cQ$h^63hR2(nSqVnn1Quuz68$#gPHmBmW(;O1D{Zr~_N=q!Qn$G;OS`{rW~Xr2 z*24~_d3zN!i~1$G_gGI|G!OUtTm>c+`igEBPd#6J^=(-H&=ZSrmgqXG2b7J<5Et;z z!tM3vXwIWxf@&zRLlqSjse?Ld`N(7zvmP$io}!MD+UiptC$2;=Hhp1Ix*u?pN%K3F zEU(TXuSP*P%Ei(=_~(L%tT~w^L6Dg)BWxE|GUZZk6wZ5z(%km&DJN%yB!&1RU>#q^ z_c&!Jo^gg{a>Op0z6k)Q+p_x86C;T*to3DN6xhEixLg}UBDDy4J4xbF+hU2TJgXGY zH!aqBWs1{{14JFXML}ZxGb+5W9u~aQ;TM7!-1u=V<48urhz} z`Vb@a8f(0q2%H=t;o-twFAdf%&8Edm@HwEoHQN;pY?P)M*-U(TX9< z9Mk8SuHRD$v-IqDGv16h8M|JkCNHc#@QaDjmN+b2SlJ-(?VIl^9F!8H9!Xc~D+@5; z8z~8Wx)>~#OeH)ZZIU}_c{VUY9vHgWH2gLnwYp)d2>xp{919@=LI6%k=fmivT70VO zaJxjTPXtR0#XmK$rHOoWu!LF@ z_Ss!tPrI;>%Fa*iEQ;}x{Uy2H-{F=Q;3{96C&-H@@#)_;djp2F{;cIqFKb?KayJ=t*2z% za;7i(4PvdoJ4rFgFM+G^KNi5*>7rqSQRzg|qaw>Bp+(BcDm6hk=KyyP4}50yNW*1a zA19fsW;EdT%o|P87fTC@s9|val$#aNvX^96op%W9?hVt>~Ak(&*RZ+nIGR zTD5=oOo1nm{dK+!D*>TsB(vKLd-dF#B^PqxS`(ix4w!qS>1aAklDC8y`1l*i|7OOy zP%f#Sx-VC&S~ge71h2H{&%=U)*%ssZOW0hTo1$oID}UF4<@!EalsGR{!ap@P>)p1{ zo-dFxAJKt2=xQ_movepCr5Oz@K+tvpMFquMi8<%9qRZ7kP@I{~0TtpKFw_cYjW{Qz z(fWEpyL(&^&*Nla60vtDVE%(Y-?b4FS;#CW#aWA67EZxsN~<{8c+zl0R6AoHlZ~g4 z?W12ZLLpNo8bAFUwMwptDbOfgk1sY=jLghHOB zF&my^z1%CH8<7Jl<7a1J)n`QXUK5?Jr03R!PBy_VQmrTk!Muy^oXP8ns)~Bh(8wC9 zMf`Az&$7o-*Y;u=)|NQ|hi#Wc3&|`&Bqh=&Yv^fb>L?F&gLc`1owK)m-yDouKN;aL z+D;3xf4T@=y+tDrCXpBs{{vt7!aD!|SO5>Zw|`xUJZEKq)t6>KRSqzBm-FzL@Sol= ztYQfdm|}u|V~OhdOJ%1Nm>Fe_`^H`jqbq3+1{Gz}Jp#jW0hqf<6&E7 z*C-3g4#(5(e0y926uLUFRf|2_;Xlp^ySy$iU`QAS`lsr7BJ>M(T*T;ANV<8=jw>nI9B9zPI3OT3>q znBdn#8jeM|U+0Uf2qWUw!x0~TK^#D-6`ntp;oG=vInWz?z4{HrE(yw5M`GDs7CDe*wo$Xdo zd_Dp`uX4Z4To?e)mlk&>0O2{L{DnEKVhwKL0at=FPScgdXqaXc0bH!v)F zxLf(-(RwTRz9=zbj5d3UUn8X4#pj9cRFj^*#%{Mgx zyEz7~UD&jsreb=JvkDl@jIT|F_*W1+mYlvZ47+jLx{PZ?P(D!e)3}X`kkA`T#PFkR ztU1)v=>(d%R$WCQxF{Gr;)dKv;>D|h==g9nWO2KeU;Fq$45pzNmSj35BP5~ezQIjb z2s@EX+jGGiuH77^oW{6Qqj%^BKBj%eKe08Ci4&)O{dw@X^8Jr1x z_|Mp|WvbJblT&Q}G$M7UnBCyvc34vvI#Wv9cEc~xUylP>XLbY41Zqr1<%q&0h1 zll*{MzZgrekB9@#cE41^BiURLo8Oij*>?Z@Tkk&L%`dF}w~X7Sb?*&DB-Rrk;D@_N zYON?H2SiomVT|L}bcx|J62M_G`Xj#O`0$Q`d+h?LRzn*En zGtffy^M>=VMQkodOt|v`U!QK4^A}gnTYxNh7#JIyij-IHLt!a+pLir~JzxuVFV9Mk z#~rzM>pdUB{iHmp4b}`2?iPXH>s~H8>hQIW`(PBXQ@AZcE87FE(@RS$Hg|`5>jS=9nhB=l)(?56m7>^~EIgR)k8i+5Gw8v=^32?~#XVht z)njg|Y1NE^y9iEh@I82HtFj^vVT{@(=+Q0t7g61kuZ=lNra43v0}U^Du25_*X6?%V zS!&#Mcp8#LHf;qc%H-sOyiuxG&bFvR>@@N;y3+EKj4HHJbKw^5EVPvBj&P^!Kx{?5 zV^;(U$oCk`)tUvFgKI^4cNAh$aHN)%8yS^EdK}z0BorkP^85@%%*zbSu$k{TkPgwf z{rgW2Z{UyWR-A(?DkziXwp7uNtC8=lXQ`8;S(qurId(HuyY1uqhu0erxwAsW)7nJz zgth`o7a#9peygJ^8#Ub$?%McujvS$J|FK^gxwjo3PkYo>YiwJ-el<;-h}4wWyL>l2 z%7BoBA%ZdO)unCagql?~&34LIR=&#+0o!!$`LY4Aj3}e1z@B&Pwd3;9tL!1AvfsAn zvpmmk7)_C^iKLvu<<5-Y55bl9_`+Pgw7&!u<(_hPIW0rc&_xx=a!2&%NDmPvK#s z%LP0F8jX7Pr9yse^-kzlV-8;p4lm-@{khB;ueSeU(SPMRR>ZW&-s>F_hwEO@Q=x02 ztpJ#~WGUcc<~MLo{HF(8f^jcMmb1NOXlRH_lj#(7=gVt_HzzZk(&ANnujgqk5yrL? zZinl6uh-A3@9Ci{Q9*wXI&7y0o3(cIn(kXYW78rRoMUG7lClqG(A5qzL|xAvkN2G! zKSfkSquPmiu7b7u(~6qixW7B*4G5Oy&)9Fj&SZNvf{OqAadwZsFm^~DwB(j4Id z2a}S^V_6dcL6UE!4%fI`Bla5Kj(ULZ4(OcO#KZ*IU3don^wjm!98pWHE}1Zr;>+c# zyue4%C#02j4RNblr68vPKvn&yx7)LA>=ggSjl*G=ERD@7MU)Ecq{E-hWIE@X{H?d7 z87>`KeJ2SM?08L}S^!g1uKZ0;p6^VCdwo)FZb_xmg=NIcvBezF{XRbNyUUY}Ts|j? z3RAu>nq$3mIr^3_7d|nrF^Ap2L<@`&ex@|X_UTALZ#9L%;d~-BKT+n}OJ7#J6oNbW z5%afa!c~Ew$>ypOF&UGvy2I3nwfR_1XBel2PHg99qpX1=du`C_FLGF_sD{T4L7>tD zV21QQ0S0fBTv^2UlLW_FXeS9{TUA(nHgvUxF*w3JfAhd;`ZzW`#4=Oo2xsMo@#oa=^mWx;vpPcD*CotrKb6Q zw-l0`S&D&@Fm%5@+h}(ks|~`xprbtb4V=-2CTbxC5)V5@nb&{}05g~$JEG{~H)MNf zDe?$lonrDSohme0o84}W7MEb4qp$)i6LDMRSucMfz9DHivOz~=1S1M~AQp#s+)J{p ziIGG1%~h8A`C<*PCAJQx23$iu|7aV9Yd!OJ@oU)~4s3%a97{<3byw zhN$+_08Xou$rp!=FcFc?;_5g$%{8o0t0Rr`zsjnAN?f3MkLN2P>S@>8Pk!%-L*aEv zIt`xa%`o7@GKK(Yn=ya+M(}8gY%b$@iLOH#a`plv3jaS@ZowJK@*j7(#^S~;2$!q5 zG)K=+knvo1UW15O%YV8aP2h8jZJ*KpY-_qeW7{@5w%M_rj&0lSq+{E* zZKq?~wr$;g&UwG*-ur9)+-pCpYR;-rV~h$ceEZG1kYb!iZGuT1^ARc*OgQ4N@;AUl zh6*SX|Jy?U<@a=vTc2P5W3AggyGluy6~PHQL1<*8%7rMj%g547M`i~9qb6;uck0cAtpc5Wust&Jb z+XD}9z_h59gfOHZqQa8_jEMfn4+)bMUWdIP!vME~m~4J}>mmL)AJ6tRma_zzrsoZ> z7#NGM3l>v(lh&0sMrfx?iv(_-d){lgcL7R;>yhtl@v%4gb?xet%0E$Q0sdbFyVz4? zl3;FA&8L=LM?7|{<--Bc5cI58WIz-^zhF_}!lD~M_Mvpy94?HG^${_p0-*HCdVbEv zW_0a{S!QsfXs?Sms|9GLI^X6_QLfrQHVuBrY_d{P7~0<*ywWcyJB9E(uHfK~K?jpJ zWY0$wHyd+xc!A}~e^@uGg-B7j#vUN_E#%TnNejAYtWmWgPeGNBK{>J|P;p@JEY%z} z6vj`ZP=Z%`C?E)vn7_dj1Y29O5pHx{K4=GCbZY)KjEdq84nJ@DV*=bA``l@l+*Q%r zn);JPK_eczr#267nw<#n^Oox-DCAZ|M3i||%%l#YA{%H6Du?%~BDqS#%`~y%`dpU1 zGR(W-P;-~6mE9B+81d|PZ8u5w`ZaU22^ukce$T2&=nrKyfC+DY;sqyeI2x=(>R4!s zVSNxA(Us6U?0mI2KpO-Q4|Mf{0x!6v@jGF3=Z-kCiAudDp3NN0y6Ip0emoyyjzoH8 zPP9T*_@gPKkMHdH<@oeZlnTZk_iz-t@qSkBn9MIUP!1^2Z(cqqSfms@?lxtvH>B`QqAxo_PZ_+i#nUz4srOdD97qH9$=&& z!jL3^V3LGD#lQ#-0EIOC8;T4Q#KJ$kc8L`{9zp&(OvEpM{0Wm#SN*LL%x+6mV4J`C zp4jyG9PAj+uO&%b(T*5<7?G|g$%%`=;RZb4&E zEG8{8H}y2D^zB(sFMa9!tiDveXd}outJZN6EI>r$>+Du>_3-L)IGDvhya4S2rVe+s z6>aoTt2US}hBZ z@m6&de3hldg7Oon0hS8RXd`Q?G#k1}gwWcY@_yk@A&3DEbD)496qY0MB%y;y&cE&3 z$~s0yDh%=if!r0WrWjI9#_kmqo#T6TWzVaxhG`e}Hlpy5)$;X>2BsWCq^ne{-i|3-W~(2&cy-h_3p>y%PKL z0z!^m;QNE+lUf7v5MD5&wubk01`ZlwHpCKTq>QwHl!p>`5G-nJOlpCx@g`tvwrOE; zaerQ3U??_Ec=7MT92*{`@L`wRmJk);RI(sN8(f*VE(-sSEFHf*vn*NSa%>eF8=Gj@ zX$l%<8E~Wv=osA67KLK z^?v`4F2^$v5-b#T_RR;HE|skfOqW0wi=z@m0Ke9tJ|V{b^Kn@VY>W&}{s9PFJK=r2 z)8l?E5tD9)Bp7*>Rqf|7QiSI)G92Y*2&S(4?!)2z^;}#eC$ywXzIl^*k0geiWG18W z2vSf;NDjSKBl&xg49fRWIm+hFmO|9*saM^AE8LdXH6N{Rr#iVl6L-;M@djlyp@7`b zHGzvjhJc@MOT9=-S!W3UL?#oepVyw96y)Sf_lodhMim7e-Q-zhnJG=(p5U)v7$(F^ z`mdPd)9%IU4xSk#S>Q`Sx6AVsVUqa;1(UzL=zD}4vN(q!kAJJhtibD3!j8ofY;vOvHEZ01+c#Wd(6-&6@lY7dc%e{$X}I2 z$S7p8Bx~tm2Dv(*)tTy&%uA5%_J1TAneBdcWv8(NrlO0)5Y}ZZLaxPn+JFU zz9l0K1Yt~8x-gViP5PTpvDoTLO-Gt&3J7RSQO5_~F-?TTc7+YDEXwr9r6h=tjL-pg zc;kM>3d*N&!|?|(Co4(s{wk z^?$9zQ~)FhHi`TA9=ZFQ-glwCC9zu6f#kBoz#Ze)*ZTF&-a+_a%N9i1jmuCFA35rz z)TlkhvaNesa*$x##Tyq$(&it!f2kbuL9^lU{%KCnppNx%5hkilN=?Ia~Ti zM%RzVtjf+sPS=Z2Q5{?=hUJuu0ZCM0`2wT2Y^J^>=vR5LJ=)WK2|i}z+q*l_?-zBs z&a8ER+qq{&v~BjM3gexNP-Hdn&nUJ93~mpSK5L1}Mw+xT-{%f-M$d^Ab{mNv&;A9n zaWT*x&;Arm{No|eyNoPu7wYbpJ=n^XlaN=I2&j=HlAaYsU==E=Q7jJIP8iIjSRN{d zyh4)5oPMtpMT(=+{HdruuoYB_tUhBNCMM)7)f{H!v&quEifRYm0hJhKmg*pl9Yf`S5v%ckXYu!F*DLf;qSG<#+H$K52XjAlsi ziNayq-D`Q?0BGYOz}Q5sF7bfihfhinP+3{onXirm^?KujTK%fj(ZZIoaGL~agn(A6 z&zts>4f^#xVB})9iLTXRGgv5@!2S8bzF4=HKUJ5tMw!T@YG(EXT}$alc;vr7wcwJM zSVcc3#>SG5V}0`h*$&n}`Z`3*D(sWxQ$tClqMQk;)-yAey{4%cjC?YaU^dr^)i^GMR9)~Z^Aj=hQn|G|TsLh<0 zTs%J{mDBV<4EEb1&%fF*+~LA z_H<;=MU=YDb`0x^mociYdmvnhprPPafM7|~PzG3(QF?$H|Jf8Exkx4{f}GIo^ENfE zUi)_M^u~?K zQb<9mX}mjHo|F}cU##nheWc9?8OTM2LPx4h0cvm1JZeQDIwU|m(qggisdV?ank$n< zN@IC9v)yKdYbukIhefUwQwM$cwI9PzX5K7Yw;UOhRX3>|!UQ}N#jzQQOe&ssS&>PZ zCRvllLpV>L6}?#T>ns7+9?5XD&~6=iYch=m(RM!_iwX2V`3SpO=}n`<^B=5 zG!mget}tU|d%G0VGwsh95F`wg2ki#WqC6V2PU#+Kim1}ym=uI9w1iJVM^Ls(nF7wC$5-cGUR9SnO_C2)Dzm{2|825$bf!wK zN4i-J_yO(+mt~>W<7el1Dm4QSDrGc)t@QT?IyPiN+u_NM#LyBk8982`2p}x>4C7L| zVT)czZH9cpFGYLDx2DqCYAeuVj}MG%FybWRNdu$S>>rJr2gVt~$2aLpsmA*+tf(^fE<~zY z*#l0zklbWDiLK9#YJl zyCYv{!Og$==Q$9SO{e?#gR@P|igv@>IEd2Q?P>VmfkuZB3!+EKMWTyRVQ1y zx8S_aCbQ)K0hLdEs%-u@tN}$F5%zsbQlm~Dykp>3uKY8_`aZC>sBKYSuygN2jn+`$ zB>H`F10ZCB>B*)fLw2(W6WFzTFjbR4QRXRK{$!pv1F5TXaNp?*;dHMXesdAX)n%QZ z$dzPXk&u=Z|AirHfK9SQK*sHRfz_>dTQ%4%C1CCGd0(Rw3T_+?HDYR`XM$3H(=G~g z;016ep#up)8A}>Q(uN*@HAJjmvPO=J8QTX zKfzjXd?5Z};6iTUyzcRoXnVT}C1ere;J+30Ibh-7r#aN@g7KDb9STjpzw8r!&jj<4 zKW;nE%f286wC4!&5vbW|XEx#+Vdry`+ym8*bpN=45~Ks~&(K(k%TwBFLWioP6%<4l z{p+!fHX}EI)Lz5?knd+Yk@^$*&osJ~Vrqcd`j*}sKYf;KSvR&=dX{X0D62>FQxJ4V zH4D4rfdn`+gJxqXOGz4C$5TjC>*-K6D6c8a7d9bM7GvP5J@0~QTX5xO1^lA(JD<-w zM`)Wf^3V9{pKzyIJ+_yDUiPe=Zx42#-o)1Kp6`s~Cq#vroWD0FoWXP1;x2zl>z74C zBZx4_O-vPsVegtuEX(MJ=c2Vp`0lUB39)xUg}2=gPL#-4DvXOjz$YTa{?Wv25lqCS zMDCokf|gDU@wbRxcx!U9H`S4=oQv*FCJUPhu}j7@i*I8`$>#q=S{RPWtIOljP(wIe z+8adywds0CmGWaxnMR)F? z|4tFzbtPDH$gLcSKnPZ1jR1xdT>%u7U6)L?4Eqj0<<ePxST|GqrD)v&gv1#IG~Bv3~~QyQoSwVzhF0G*m zC1V6rChYfMhgjlT7)53}L0O*qbAn61GCdtLYDX}LYu6c|6T(vW-3l2zsFrV+!wud# z7_^MykIH1J;1<%qVe%v~X5Y7V$uA-0rmCNrl0r$AIJdG?trMZs;hHy_&CNyN7Q5Sw zf3-NiXgs=|er65L&ntYpeucHWG;1u*F0a=AX$jrqdNW5Vlc0 zq9}16BLW}|p1S8C=?`U|Qfn%tAhIT(+3lI_^RO;AH$J`(`^9uzwZjgG!--o9j9Uvi zJ*qe8^|K8Q69Py~@wT-`nOBK%qA1dp4p!&M7i~#m2184k94=!j{CYk8F3UP*k1V6w zU664?$`B8mMRyy^!!1Zxhz0t6>7grjOw9ptul!wad&1s+W#l$=gj!g^l z^Kov4#hKqNu*#uj7Js2Jp9&EV2f*JLfk04VGEnH*6T2T)4fRZd?Yg&vG)di2$qG`T z;CeX7Q0-PcJzaTok+6E=rQ?ig3hNYQl6lrr)?yNHq!H=jygSF{<-B&fB{C@rRh5q; zyzgl1gh!Z72pMbp{SHEaii#|*j7&LeNpNEPc%frs8n+?^W{0$l9|LJl)lTWe^AL!~ z+?dJ~h6=sdyBfZzB88fPS2(=HFJ%xHvM!3XaQuKK1@OFt$^sCkKK@2Qu_dE+Y=)afRTipkIf` z8&*3zRTUK7ZBJ{WI8e)~ke0_TVCh?Vm7nIfAxD!#8 zjsE^X1L76ifAHS{f?cEIsVuY^y{OF)o0mIQZav$fde7OV-+_UN&HD$V3G1|A5-Z!( zmJ00ap2$HRvKIYqYJO$jcRPWI&@-4O*a4AJg>G~l*DH?c^n5Ocfc_fS0dK2d1qB19vU zPFf#Ge=`C^lzhkB`XJq#06G6c>j(ap_r5F`xC_FBxt~rkf5i6pONqeVtT-Xt`v?XpJ+`D4~5_rO-*+`P!qt-W%;c9;2q=UF~KBfC-~g1_O0^?{E&ktP4D{SG0O&{ zWGy9OTIngp*cN!#hMvRd-R(r+{fYgZ)@Oz($YFZjPY`w61{#SCAl^&|64qgA>4kzq za=WNVAzriXW)iNfE{-EkeBb(P@D4k}GrJZPk$kI{+{$1)uKgj~}m zzV>IyK1rNXZi7KiCzJ7uI`~~0%z2{`hN_3lBHoI;2ey{ri^AFDpF8Y%5&fyK34Tnx zyl#uJQ6qDt(G+C?wXpaw0>;qDfq=wl2bJf*2&^X@Nj=j9pVS|DJ;Mm)g=TUxb3`)T z_cl7u3$IsgOWprB3w2%ty^2mvUA2Fm(0}rKD7!EyZH1MTrd_(+26cc@Qy_)*c;V9Up0GrZ<*Ym`Q&||6-Mn#_I@} z5xEEhBM>gS5Ye#z#0nW0sljw$)Tnk2he@k&6Bzu;B>_1lIXk$#-16nE`ID~EO6tAT zBX^jO$TQwgKX?l;y!q{bZW&L2fl(X}`JiHSdwc7CQ8$d1N}RU~A_0%bSqo3Jl-5ip zqpHrAe+hV<;DZBWQ$9MSv?8mrJW|2-Mq)Vbw&atO6ZX9PA)BzPOmn!!R$1|gDjx0f zp1oZRbSY|}RH}q`NqhTRO)yD$d2ulaE?Y;vyxOKI`q<|I8Re2PpE-`%NK}2FKX3feeD#0bUhy z7RVa?J>>ImugBTq#>Rzh`B_^GtI~w-H^AVgvRHms>N@*F(aIVI*KNlH5$VOr^e3;! zIcnT?%JkX4r+^%kj;o|?2jjI;^jQ_ z@JZ9vdtP`x8brQ5D&NGgOb4;C&MBxyZ`$!ik8P+7(8FoGFluPp0H-YlV)h1igixkA z6d+^aL`Noyuqr`v#!gh_*4l&jLdTMs4*Y?pRO^b?fEDI=Tr)?gG7?s zh?Xj;CknM<)4n+ze{Ue50lgq z@C_!wnCKVVY|>r^2cIdGjampoSv)*p$E2iF$39tGlcpr?VJPGC)eNS{-shX(B+>mI zgTlm=XemkGpRVfEvrhgr>a9+XNT_O(B+EUJ3fHnwU@LbR&4~j$LBbOR{ei`F@yK7! z;x|A3s^=rhO>)4PZPDQYVW`l@5dN{i0z-e5th-3_e0Qt4zdy z=^uXv_k&sSn~R>DqHe3>e%}g;i>U8}S zT(Rv}2DGe|bxpsc=<8h^B#7x)tX4+Qax-(8^SUS8p$jfb20ys&Cj-2X7fDu?6r*=o z0P}v$1<8MEbiU=Y?9AzFw~7;>OGW=>^#h5|1Xw6omd(3Ue_M2(mW&t07dVlB=XWTn zhGxCG(G*xvy^*_xIK^(JZ8ov}J=B8uZ+pJw=)A(9N80SO-t9CydtP4ew_XNgtl>;< ze$*98FU&{V+k*z5*lGM%d712-uf9CE^~awAXO~H6OeDdIoblwUCJr{sm>#BKH9oH> zvGlAYQcQ1fc@3B0IrA|9rls6w6tu5))nmv@}-(7S8)$`%0E;I+a*+^pkzKG;y}a6=8^<9sQF(&3DH=c{q3?OxbkMhGg<~5jSGF%T!aAu2FbXe;CMPC zg}k?)`?mlJ0sgkK<)-^<(X_~HnJeE^>?Gv_4eWqf-{ib}uFb3KRU zO*blshkuq6{j%^N;Is!~=zhqjeqXbyDGNee^oAI%Xh>5a=Y7207Knbo>OqvoVCVyE z+T!spmP)n<26l_%`#Phkp$;ypv-5g)JwmwG80wgFgH)4m5Y(R8fetMu$N;i3L^1U= zU!CWy+oO9Kj`>X!(1>w1zND6vPn%Vb$u(M~Q|n~-YrNwNG-l^VGS}o>3esXwjjrNh zFzrW{C{cA&y)tfGLbaHW33RmIdpk};p@N^>+Yu@sR<^NQW2-%IO^f<*ch=UN)&J4kXLx695JiR`1`A?%hZf%`cS)Am~P{Rg(< ze)2b;3X`l4r|=3_cGAr=e|YV`g7lqy11*>`9wA4p64 zWuxf~H`pWL^Huas>Z|@V%1o=*jfcqhfy81Gq7M+eQ?qhVQ;$`8i9pK5n@I%AOHMHQ zeR+R8GWgl}$ffh$(h{5um`!Qfb^kHT_JCNP=z?&nNsFj68ZJk&6{gSvwkP-g6;C3u zO>X@5v~D|$N3qL9D+2xQyw+&(@|e{#1Q;kJPHjGn7Z1b>z@?Dqt=RQEvS<6gSi=V= zxz@FM1G{>R3>^p^lyCfP4HEy@bpb*4{ZhB;B!kabn;01b?KMrq+njW|B-E?~FsgiH zrZ?-nyg(NOy80Ou-MQf2((FUPcjJUT9`=;wL9pxLoQp1NC76jWHy8e&a%e~BQBWXQ zYRl!h%l%ehnkXv$C-tZhBv70`vxmp4guJ|}r(ph~il`6gEH5+WSs9`ZgYk-y6%dS&~>Q z>m;yAupQQi?Uz(*X~Ka45^ivN48`@b^7bK==92WV@aO=bfISu`icjujdo<~7uawFV z$Pu{?@uE(DX6)#JUw;#PC zsWl+|NpavbONRSF)X%tSOfEzh&e#C+)4_{Hf>ch2z3ELlz9qle3av(SWDHt$|0=B( z!~5jPmy`V%G*TI%Byx>VAdPxP^0(4eja4}C#zB8zQ2W6{wVkw*9L3>D7YU(qI*UIz zQyphx747&Cw1WBOU18aK00WE#S4+)u&uXX{if4O`85~~sT97lcwqV}n^>u--FGTQz zY)+5gLO1YPp2-nMyV49@K?&LqsA8Y@47}4!BAyk9g@}q^T;Tqwj9lE*?LPPTz63&( z>J}xGGTr-Dsh1l#PLu_-U*+5C%8k+{7p)^paf3PzH2q@Jf^W@sv2OOT<#+RFq?0ea zkZOo8gVBO7E90ssX79I?xgO8AE3_b{A-VfPS&4{vtW&MwEX|}tlfia929`Zu3OP&r zKZPlFmChDTkl`j)8!!ed8_K$iA6u4jfX(_)ddCs`3c;(2p{9eN7w zv^swM2Q;VN_y8!)lY&^=jf%bdLn|hB$;Q=0Ci^c#~upvCbb4Mc(n}_4i{Wuu{PeSQvuXaTVI(i z<%VTV~~w`$kH8Tbs^le(%Z`t`UA%!#wv9@!cG2tvx<~RpED`g)K}#zaRs6rW3eXJ3Bzi5o1UY9oCZj_mlVp0%x_~d3{5P>IMdA2_vDtddoc%!^ViiTR(fPE4 zdgDhru{a;RA6&%zOJ4o3`i)zgq zZHt7ATn2$T7hKsr?mvLSGr9G3A%lSC6l0UY@=DOz*|nqpHMW$o@e04 z4?4O-|DcOO1Q~x$M2eKw(<3eWml7gBcX6{z(!HaHfY0SKEJV`)X_`W;Pc&c1@L40u%Lm1dQjthH2DuX
Xz>vO(INYoz3W`+;&c+znVOzLI z7s0_W($F&wpnwVLwbhXzLkhDJ$pgAX``S(oyln8t|L>tUwR8ag9rOG#(3 zva9J4VlieaQpZ9uF&C==U&0|yGL_E}k%97fe4#fZSV_5kYu%ELv5wINh!<f&}>eS#p&=Hdv_vG3IQQM>0rGKUC6*qawV?BY#D;I_>KBqM7%UzhEm*-)Fy}zL%c1>ltG$zZC7~PqaEBrudcoI$ZL>=bEr@1~LlEh1As4zgRt`vt&cT zcKL=2B9;E&q|_w&l}d$BQr9*C;N7oTPDfi#*YhcK)BSiWW>b77Qxn%4%x+gymmYkC zewcKXv&mP|RCnC27k@Jhj0K`xH`CcZsu8A5&J+~vQ~TgQ-fy5{>-ObJ{O*L-JKT&8 zjP8I2?jBET?ra;1oY`Y)Crrka9i!9Bp?-jzBK3v>TTj`KQI({65nqK={Hn@#N%ifv zs?ZsaxDFT`6!jK3|9kH2S`4r_y{`uWov$`Xu9lrt215fD6-4G_1Z%cl8)?9AzFEt{ zH7E5uuwLsF)zo6pu&Ql)y77$CX@ANs=Iyt-{oreKeM|{fGtzx#NvZWdB~ku>{c7E; zu=BgGqeyxC@jiQzW*vc{gqRJY_cxk@6Q&Nslvp6p5k3Pu=W`2%xi+K4Zp%73fAr5X zKLNDv6zS(q>cvJE=KC&8xrwVBJ+tKMhGJa06#74}r2uw^6R`Nfyi~7Rk(o)Xe)3YG z_m0E&7xvn2vzh!Ailu}N4hT+gK-l#){1BTkTC2sHt>C0~#D#lUi9UDg(;k6Nr$0hY zjZ81&iqVy#>AE*V#Q79O#*fLd;i3%WPOVy7)M`azvE3yj6VJWOUFxmf#R@wxt1K(o znn-`vIt{~r_aQYICM_O(M?D7<-EQo>)x@z<(&u4$sY=T|S@SK;F_1;_xX0I@H+yl;lVAJJn@Z(g*M9y&6kaB9?bj-Tya`@z4 zo0Oh_!|~;%OaDf8Io%>P6UQr#s{S$rkK*;fW80;w2f*IfW{_*ileo1AflPKZ?wV9< zBYd^{Q9V4@+dU}`YXi3frM?J+)lPN{?>J|n;{-HxWq>FO7t9g_QpP5HTi*s~bp-Nr zw(Lo3Ru^f`LzoFNecm5g5KqA{|^nvt)RNL8>Bb}@krMBP*(|YxHH&obKyH@xSUZ+)D?Ds zNLGMdF#Wn@&}Up506QdkX`#CR^Z@k}NuuL$*!I7zMEy`ox}9g@bLGKhF{jZvS!gpd zhE%b3cQ`#%yD80(pESiRJNtbdjlX_UCr+>%=lod;SX%>W9b061t{`*0SfSRh($C&0 zmJ9K3RQ((CBYt4b_ro(yyy$3p9U1Iaedic8M`amH-*o-`V9PLsNZiY;IE_3pILW8g8@+c&B%>M*{BY2jUY9 zi{_lX?$YjsDJ6|1^X)tlxs+5e9 zC3?D)?lE;V?6~;5|6F*~iLBY8+eyl}zV_%5wcri4%J3rrajqQh-&Fkpf(*9fuLZDj z?RwcmdmD}ULgahd(eqeXJ?3?Z6<4U^uo==~Iguv#17W7X4t!|Z#WF6FBcn=yKwoJP z94BZ0*T7+E@*{~l%V_w<&_g%tjoIf-GR*<6crS+vAMurH`a9$6@gc#~SdIFa@C6Rb zo)h0`g0bN~{$}t9v}$$chfG5t4W4984}X^FRA!pPSk;ZJ*sOrbJwJO#)Ne}mRtd}< z&Y*dzOiP|s=$0saGizZocx!nDr>}IFW1Z$c&S!QzX-M1#YR1oqETd1Lwr<}g4X_aH zVx5W|B45wDgoh`;Z!f{m%a$64|2I<ac$)3lc*2`)*8|?*utqXy9`d?7H9$@E zDi700+rU;dYL{`?=$SHb&^Rm;6HpOs=&}W3xLvO$yWa4p3U=NUz6*kDCpY}f+AGIe z6;l~#*p2+@olLt&S)`UypAjPC-yL6-)&{NUk8$)_FjH9B=e09fZEVf?wj|(B+rU7P zzxBIL47Wt!u+V=66fk09DHyAEv;bZ)AcSl{Lpg_A(xz=icFY9|o^)=Q+Rltq~!EZJN`S4~nZ%!$|`$xW$B$Fu>X1UEt zjb+YiQuDnxDkXwYI2(ODc|Jk@W1bT+PnaGU0M1_A%T%kYF8DPy_vx&-`EEaWlH*XH!OK?1b^5H4I_8vh&XgVjQzyTwdH+x% zQ{j%mer?$DBF%P^Ee_Q)xcuWFLeXBQ^q|(9+trEzM@epvo&+facF!?_s%y z0`vDNor$Z$DCzV&^Tz0f-{RJNH&agoTtOk5%i%Un5b#dxaVlsj&FzmTMtC~0MbLIi zOAXsV4YoS;uDHghI^40Eb+YaLDMPRR_>NB7UKvZ?ui!k8^H}F3GLg*|%W>k>e@M@3 z1JoNZs>+aKI>*WNogif1HwST}d_KN9RG#|YA71ysu~W=Oi=xJo7t6|$MCHz@{<;4Y zBA_PR_u;djIMVntc~YoXb{k^8$(=GUD^|qkOL~O5JfE3(&eKFcjT{FIKqzN=A^H(?BYyKmQ;CHeP@kwxgG@sE={~D_>6~Q-_!kD?5 zD~dQ(Yx(?5-*w^0on6wJPW(M3nOuo5M?|ya>UFhh=AJSuxzHspzdy6qPnw8<9`k=* z0PD?#t7O1paBn3+JwPPU{(szVZxQRs;I*rDI+`W%O(n;j_KvF7Q;)Y6copMbn+*#X z7v(C&34K_zJm0V%4g?4|loOPjz}qnENbk1Eq!U^pX7xh_`6An*;@$7!~Xdi*HZR<*gZ^WQsF0gO7Tw|ueUBtc$)se`P@ zQ7nKSxMq5 z6-1ylMDq^xG6+?URgS8#N~?$8N*r%YxEOKEFUPj+kmBu#gCW-_2G_v2|Aa^oNTspsSfZZfz?0#^b?KBd+*pR=4gLwmFLHI zr-zK@07gS@fX^q;po@M;JOWH~*xbw->GKlEL!TD%k5bg|sq=+4`r=s2Dp_IV^k z))fDn9J&A0Wd0=fwIu$J^YK**Oa5k72RQmQhwa6g+Yo1zfL7)lbialfxM2FmEjQ1> z8gdnM7X2ENZoQF5A+Ef$n|rSuPsyb8-_q`plrgfz_v5|I%wt$V_nc{{!w{9C>zS5@v(NL?c@#x@_cviEwXF5rWBth)_{Da!J?-mj8S>;X zMK^a}98dleQ6O{eF0_{<8=*F^B+C#?8=I#{_qeQo9D{)!cHlQ9Tu3sVI}D{?-1-pd zEW6#>$Y)Y{6Nqd1ZSpleE<%44E(Z#Wv}S+&W&B-WM0^@M)Ex$s)jOtzcN{bCM(5->``;7PkFs-;3)-<~RX81n!l#N%8Y zgQV1UC?MfJ&%8IknaSf-5%^s0+!u??#pa!}#jzu*3vu%tLnAuJ>|`1e_ zzX%#Qg3(&hXwDEHAjaY~%4WB=3?P2w(a3?>!r@m13a(mjgb`aaFk4Zj>4D4i40}|E z_&Ph0H<6`9l?9$o;!VPGEQST#t;|*_f>+K4KlZ(GY6e26%1>%kfG?}e%2(u4K-XurIP6%PofQjE2)Y(2XmDj12bd{BZc7 zo`S5c_Y?kMiU4iKE^aZ@X*JBwP9Hsz9>(;TDbR>WPI_C>oXSYE+3IDn-k77BDL}T< zU8?jLmzhlWKB-S|%dgr=W0mzlZBXf>B^+>1s-0raB~o2IlT#Ddb|J0&__vuXr2wn- zI`nGTDq5@<4^&i1OJ}xiK_ji4Tl|wyp}OAeeAq&HLjkYzJA{qEyJD#_FT5)aU!&~h zo!P*w$mOtNzWO^kTpm>Sez)_UA^*PHBU|H|M3qZ2q|b5nP;KVp71sQYwS!_cx z59?3&i*PR}VMp$mLuDyfP(zvUaP-h=W3mIt!|2#L@pDvZpwn939l{oq3pvl8K5HS2 zXcVp@3p+)n96pa* zq20Yl(53;~|64Z$PK#1v|1@gYasGoScz#x^%cWzNErk81m^x#)i&>1SwHg0{5G9VS zQPzY&>!b6UyJGmu((T>?7CBWN-fvV-ueXSWL=Jv`QcL+kg0;zJYuGPA?!CYBoLweM z>_yvVxE?;IeqWd0b1!x}KQBGHI%JsNU6Az{uZ2DD50A@?TC$I_APDyiAUUjNGsQAl zqd6MPQEO1RhK8-JhiVNrad6(F?nylyf!@8whu|OP8opUg{2gq9A9C8Ql$mX`{we2x zMTzJ@_&lHT<49wU%QAWY)TTpXve8;fU}pLe!-q$)YK_a`N9sCtmo?s*W~5z0aqfmp zrsAZq8bWS>J70MmE3yzMrr4aURpIKmEUF$wXjOwY<~lK#q;Is9wrg?=MOuNjMrX%j>jQ+nxGlD4?djreKeyw$&jInd z8&tXCILfG^yn>;b;Khp5#cy8O^%k3?e6CqU({9m<9`&EslkRjhY5Nm-G`gnICTT>= zH>mPO$dTi6u0wp!e@RWbvfS~%v91w%T|oUF)hfmAoJF*wdMl&; z4Iza`D1FZpjw$XC(=Lt(fQc6G;Zzyq9J5$I`mOV{(-dD$_ghm9xwUU?SV)W7#b&}R zu?rD%VMPSnPI|-r*?Nj!M#)69Y%GJ0&M2fVjA|GKga}s}@u`ccaEl&#(FxhCwK?8;0F7uqfP+j!M&bXg$s4n;#vG z1>dQ5%DNTRwTzz+5It4I!of2hn3?j3pO-Z%Q#OULsTd9d*Hi0RS-G3-&!r0~%g!Hc z^8dxsKR6OvfVW6)rcG|!^=C@fublZfVR41&?xKa*OAkLVjdwYnQq*o#><&enQ5@|X5RaJ{%dRQ*Ih^`IlhJn%itnO?>I)br-Zia`4uPnLN`6H zKT9<_HZKEmlkC0RLllUYJZ|vi`h5Fu*1`gJB|IKFrzQ*gG(LfyiE*^)mDD@}MQ|FX zC!wvh0dt1`ww4E^Abehr`A*Z{6b33vz_)aK5s(Vao@jUN$g&x11x_A?tAi=p@y^t{ zI9sI#$5RVdfq8V5i@}eF<;?zsVyBO%n<*4uFO<&QHOk4BAqz5>{V%U)*m?zaI?iFM z$n2PqDim9Qlb$`AInq|EDSX*Wn&q>pwKWg0)pYTQPOBbJ3azITXr-D_fP(4it=Dd} zWy&xn%JE$~pAB{_z6}{Odo#x0+)t;>1^Ri#vfmCNgRvVH8|^C197P=BM9<>byz zPqa!?)mA6Om;?ZJ`%6hf&cxL8WYc$CgeO{6SLrq^$uY^?&J2aN*F=4DKOX7)iYlmt z`EU^ZAg?hwZ+7kj$DF0>`9Q~a+X#)sWg(lNl^+=R+LNX)((-e_1vvXQ&dKrRP!c?3 zy^QNJ)rf}7+E}^c*cl(I)i92+AC={%{-S~CuP2wDi`EWQuJM6nS8#l>Z1d)d$&}fL z6Iq9M=eEJswIajpRwx3Hu$@QAQxqeT%P>bDRf;)ub4*=t2hD%N{~G}RJHqiMp!9vV z82I&sR#Gd?uD?Ehww3-`pp&T28|e3ND&vol7#Muz8<>Vxwbm4aDbLCN#M_}+`7Ic zBt%Liqy}k0x*JIWDM3LxlpE zo_+1T?!ABOw_>k#SVr$q4Moq+(78%uk^h5OR!>oEg#8KiD|PBhI^f$VI-`X#M|Q89 zA_COSLauPi)g4XAMNTSQex{qhovT`zJn^edn9G>|A#$ZDW2V5AaI*m~sSvc+f(LuArUSwb zkp;%1E~k@JCuiAOe@xpU%h!nV#L!Fsh$p5SiW2Y>1t!6pSvN zB%ycA>Jr==P!V}1gk@@b+>?z!T;Yo~Nh-1l#tXDI&-V446SK;{uX(ht999us~SF8m&@Sl%l zUWgmVY*vYRQ;1Y4T^mf-ezIn{Thk{OhicZZES#mt`YX4%9x_60;G!Dj{D;Ch3g;7d z;K?;YCbi6ZDsHC$?DhnAJGQ%1*i(LyYPtqvG!D|BYZB9M=35`KA5If$W$&@r|L6N* zp}av*x^`p#EI+&4D&0f&>*j1RJ=wr~i63Ah{{+#aV|vnr)*p9^SxQjabvs+*%1Z9` zi&HqHlnhx%gr`E?Y&ezeM`3HHz>W(FqF&SlC$?<#2$^z`qBWVuWx97IU{Hz}P3t<$ zt>mNYgBWS4sq#x=c&c}#WlQz7y6K2e=kr3usZd*OQk_fO>sVqm z6>@Uque`(g_)$stn)MDchtoc~nL#PlNu=D}=ONBl;5E_=Nw3?;;J3{PtYBpQkNf>U zdar@7%qSDf`qG51+NkeZ zuG4x^v;}t>Z!>#tjXv|b>ua8G>(XN}Ki;bi*UM8!5mTdW&XB8%F%m}C-Yz##m$el5 z69pN+qit9(PhLI4B;>EK2uqA4rkjsGOacLj0Z8P4a)gq0Qd7 zbc1jGme)?RJGSuWbVxQkOi9TSX~kfT#@hf?T0GSKM1PsW8^Vv~A0jz$ClUU%xbIy? z2M$MTy!h>wh7Kn}L+9sc>qzVSv<{x9B#Fn%by3d}1dKY@r@x!ZZ;nTDR{umJ6Ps7R zs$p$cPSta3O1KEZ7w4oKdS9($nH9$^g;%(Y^e3rQE}*#c#+4m*srHG$%Xm6jU8Qk5gKZp5C2x)LD(qsUAXh=A%q|1 z>xk;B07e?X?;p7;q36yzV*r2MT03Rab}te>ySE_#mTPb^lIBhlN{6Y_x#bwYS89pX zx9sY7F1sA)a(j{SGFx+)#7D~TrS2OX5A@~8PfCH;VUMWMPy1@dEV^2Sct7?=v`4~0%L)}9rN2fpxfp5#F>{X#KeOp%PzkSlM2W=I zqtchNULF{@DxH6;%*!QhfsuHI1Biv5l$!;-0XxU#`v zrGnh&G+`*@kYj4iG_KUH4VsQiTr_DcI#L^MzSxR@JQlW^ANVR>)Dsb!5O^%?_Jltr zZ%zG8Y8EnoNLdh3fJqMdVxf68ix!@SQiJ_c z`cB0YRb|_`&)-7zmkG7DyP2hB*9czaWp$+4FV(XZuuc9QH$xs=NjR-Pev7{MRX(wj zL}AD$z}Y!l_Pp^7GXdF@@cPr9n}(7R-SN*X%P)8>g6SN7-sY!C#G3jxG69-%cYv2D z$mW3O`?lP7ytSIeqN6DId>hV+|+>d58U zhQ&@_hZlN6_no}1M=Z_68b8t=Z(jv)`PP+C%4i`}0sph<^%I9pwc zEH-$C_L&r}3PBmFlw}eLcwfjor1P(1^*h8mV2%FmaV^NdFfRXd zE9eYZ+V)yxzFn2L{odzA+=1>Jc{~yJ)2cLWb-tbOp4if>NBD(|t)nuZ z+*UNrjq@|z;`DQdSy}3GuKVN4Gb!g zNMW~qmA;tn@``er{qb@NKI<8lin@87(?tjU=Ihr7l0kZfDk4EF9K!ZLPr1K+E6NSB&HFbtUy=OLg6vV0SS3>^Q(7^k11L(Unk zyxqY3BIO$u{FH`dd|@$E2cMeC9~_WU{sbVqzn8d1qdcu`NR^MtkGW~K8wj(uxUO=d zj+pbmH9n_rua`$#lr9@<=`?#Z|BATql>D3cK5EJO$x@5JA=TTYv&A0U^kV!U8DQe0I&|AEw5HD$g^wt>vTVGXEBN3HqHgo71Ud?P zsA(zxUxyGyHm`~RPZp{`NUK1Ml;jC=UG_Y#=ju* zcgDeAOVLK3CM-1%{Lcf)QLN4`4EfA|1<3!IZNL%X_y~DTzoHV$zev-62l}sQQsj_J zN(ohA{oh|(_$~gU?~{rDD6Ic?tVqe?-wP|wpNjn7UxQ$k(Xo4ftu6XzW&fSfKmSy- zM6OoLa<8`Fe|;@p4E0~>5dRX^{~S`2JfK@~ZubE$RM-58vZ4nJS#AJtmb1|xh`Z2yg%)I~qA5&M44!osqbT{FWAy#1{6 zSuyjEazSQ1X%aH8jY$^Y`-NuDaCr{`p1YTsB&vE<&{c(cNk8Tld`W--v642A-s>}s zaELe(nEu!q5|i)r;`0hBKu;!bP%10;^fUN7_R;rtQjbAl!zb?dvQI6Q7x6+H%vb6koC_Rh!IlxTd!%C)bL8|B=`+@VcsKw?oKxdzQEraBv0i z%uTg!t3LsHvqC29se!4dl!@kV6Z@kXXSk93xnc(|J_%qbf7al8=Qc%4`5}aiFPOp$ zrMh~1b$*>f>A4n)_m3DyMr)^Z=xBPVuw%bYwCeIVo>1d}U98@>h8D|gw<&5lw@NzE zbwbIaplfyXd;A{5qG!CugZ*=mPfNiu<6!MnXG?4#M51ZLI1Ge-nKdj8@^XAM-vVG|vdZ?-3lOe8^=Pn;|s;aMtM2 z!d#u$?o(g=)`=g69_RBb!OT99T0#q1wdkjM|NSdm3DQ)^rlCj7W{4*`9@zln=quLv zZTjOfcva8KcGpV8wrJO_zq9Trj&sf4e~u0EFgS!i_ur+Vbn3bwfZ)NWLD-HDC^3lS zld|YKRc*#;DENzUtiZy8RPlnWTW*Vo-4J&xgtZuwBhpzfLE%fhRBV*9NJY%cbOit%jW)1(=%xi0M55~k= zPmLf!qWJk-M|7ri5!rN_$JxSW;JnRV3<0XBlRC4?h_bc{NV}N)uYDtf;7eboM#-&q zCQUGq&k4~J&2U3a&TB#Nb<=iC6*hwTdtp6o573kY5p{Cyf@ydBk7VpLeX_c_&{jns z=~}TWlE)sAYMXuj(r(SP(U?@vh6a~KU3?Ya$ojM+VNq(m`s%S0pDt)_!C})yfhUDh zck1zz$3S^U%1x%`FK}Z77NdWK92Bt94Lt8-5KNtD|L0f{ym~MualgJ<@s$vhLjH&{ z9|Nz>{piDU!Lqj9H#X*DRhJDvT|;O4Q7lO>+UXYwgUI{9X5p z&eZB{@CpbNd4C?Y@$fEPWbQn^jIJs}L^hgR58dhc>m-N)L*ZPaga=w!vVGanCYrmc zsnXcbSA_BGmV%!>zW5}cp#^gbNcZD=8t{fQDe-(Vy)r$Qag}?B#0C^{M|o3J5SJ(q z#2$IKHA#2WwJ;3+Up*5>fa-xk8$2c@R3GP7=ROUpxPTb+LbSXk&fA~<7%Q$`u?+w} z_JZRup2$7(T}^gYvUoh!>aj|=a1SP=kQf``yEV2Y$3sz4{dKx*sKzEEbB2tqeaZX|0|q z{FJ(?-h}0T)!+~u$nz7Bn z=fC=kB3{O$I3~z$up2HyZZT_KooL^7mhavue8?Z@k72SHo1#*G!K8ULb^+f_$oZEm znuZwFW&$}SVnPx5vPBV3J8<~%!I5d5`|U5{H#e=HFSN@R@1X64VDGx?b_%aYtzofp z7kIzpw{EEe(!a#!W0Imjp~{AfEN-DN@!B=^;i3)q-Gy*5+tPm6@VleLUz8_=CqbUH z-*8GOpotKP25B6WY^2uunmtL5J0leeE@&6T396T6VpYD>y28Js9Kd>?C%DxyKuWG4 z&4{^w*javbRmYa!@ed7%_m`~MWg~s--A`K#v762z5z#7JY_S3#-r*VIp7_eHx>J$7 zSnUZEk?$^iO>~a=XR?L^t{Y`5B^!`Er=7Bj2fM`xL zm6x)Yif@3&vZM%hHXM~9^$r-MoxKnA3^;ApD*S5-GvS@*+rySH=v4xVPMwvkezQwN z(f+HP=QKa#3a!A?Y~GmWNiNSftiX%cM7NyQUPR!R5BeWUP=9Qc5kugbxc&smr$S-n zyTJBb1T>WKI^U-RsdYv`XUHtIgwGD9$kQ8C_5@S*Tf^DjNCHlJ>qiOfE7N;r{ecY+ z|D;+JKZj{-{GQdhuI<7q@X>|+vLpPAgS=a0oMYBNO1$9(H@W9&Arw=smk#@;?=X9Jmx!8IY%f zD}5{Pe5vrd4460OZHQ^du^*;u7_04Bcip&)PW1DI27-R85YHjb^a$X677SfPpzL5U z?&GN81y+hpUT33wON6ExARu1+f@wU=gLM(!Z6Y03_i*!33#PQhR$1iMuAArEG2VG4&B%uS=nx|z>y2&n~IL&E$Y`pJj^4rqxnRAE`+P*WZ zsO*BhYxtvL#|Q$ZCdLizbW@?Z5GNCrUGwPj9E2AlD$D-+oz{`dM33m7Wi4h}WKC|5z9(_$Yqok+{UlxLI1SM?XqL*wRf; zpWT$VX}}<{m+Jg5VlaM31;N-)YNgq^!O6$=S*9~K;BxqSG@uJrQ;JL)Q~pkR2x|u3 z0JiQ#c}|oQSgEGbKkXvH)%v4rt~wUZ?Tamr<;2zmPjO#rzG!D1&~d#Kkoaw3utB1l z6iy$qlM2WrP(+UhQ%GZY64{CBfll48wit;-1-Vx|vfeleNwZs`y_8kSM>gmzYEEbp zMv<6Vg~YW?;KWMd<_r*t2H0lM_!6xRf3hw>OobPH`1^c!K972k)4#Lu6*BtD7%X$d zlQm#q>6?j4P`{bK+4B%>&5i<1S2(h;>(}d8j#zjrxPjB@-Mst=Xex^3phr7KQeT6$W zkMGe);}cvUX4&&JpTx$R-6~OS>(;Tn0;;8sI3#z3@0b2J*^?&l0VJOk ziJUL;AhEs?THa$>cvfH??-iX=2u#`rGVpL4ueY8w@}mU1>kO!x)?1?IE_I*8oh`H7 z!O?fs+5D4o2RLI)qD1SGDekv)ZQby)j2qRlLtZ|?+@_JsTpoBr{jx03iTT&7jr#gZ z4%c{&?O>>7cz;m&aKt|KM_RgrpG$EU(k*X)Zk59ykWzihPxPQVKzINSzyEF8B$F9w{ zw*fu<&jOt$Fi5>3afH^o?zhb>=kJ6)YHqfbGgTydW#u$h2U(%%6gZ{j@Tv=R0r`t6 zgsV)}Av43@TR@T1d%R9_i-COlozv<)o3h1iZ`11#lb8yBr*$cGnOJDib zYX_ChEXnvrk+BLG9FUb??+F`G{mr8)V6;e7ac>I^i zo^RWWcn3qy6?QVw_>X9BusX$`^vIR*cy~*81jbafCAmXl7h60GJ8yrXxQT#dd)wnQ zw#wT8?bXkGq*_hFNHh>yzx6!qkO?AA-P+i4?RXd6g-t(K3gosFHT>h9xQ=cy+Un+_ zW>+CnO;vCQy`Zb`J{=^ssJX#>igNTOLrBp_ZAhAE8lBx{BQ{{#HLOc|HYIh!A+$ie zR4K!l$tMJ(8%uhci(w^>Qt%5@*Z)8fai?GY-)$>N_&UIUQG|r7?)ouuzj8nJVV@@U z*7{>jfHXWiVy1k2R+Z1_C%5ij{YL3&(#Fq~Xi*RS;}nmc#2OhJre{TPbNfz*N8+ru z5T;3Mx?d^li$AE_K8vD|_h`)BJc%b_Fx&U@qbaaeLob4EGvHYTCOiyjg%G6p3_pKE zu%MAFU|Ru?&N#&X`6BmjHF}hnDexaM7b!`JGVf>w=3@Do=Q2%D=RO0PyUuPl0WZ0$ zwmWFFb};CzF*AX`>LD4E(XIn$krx=a92_d_|C_RvCpp+Z!HiO)Q(RVaq~=e4HL#V2 z!|rqzKY)y zFS8R{y7qYu!xZXVPn@x7E1ZM-0K5@j-Nr|E42v_FaugV{(wRA)+>W)V%U=eys_dRK zAWA6gfjN;BgmogvVV4r6*Gko4(7AG4zEouZ(XIp2|JhP&0&UU!PGUJ9I3 z{(|J7P)YFj-WgtTc)}CmUYU|B^^!{zv3i3GPtWn`he|oUX}H885Z^95w;&X-19d6oa#4drcyMJkpoL z2M8*)A(cwCL(b5WnBLe(W?C_TH&pBBCXw2614Y9^rUN`a6Ye;Xb#?u?ye$8F&<dw(APDM2&;iti2FoPx9;LWH;JY+E?r)Ff?T{F0r5ljcYP27 zE)Zlq+ELy&Br@=$7~)~%o(KrNIg-k;k%@Zz?j)cU{CZ9>BW}0DsFWA@hYVcdR_T5q zrJrCi$NO)Xuq}?BBXq$2s9c(yoqmI*O$cYnh!3S_SYpi z8o^DQ+Jq%E?oiJ;9-a!xsnfCuT0x~ht|gWLgjAtCpd4efk99T`?}1@28aPZ&ma3(5 z3#rQF|EpNSAAHSht-BVzL;YFs-8Tz2U{}<@Y1=b*5LCX(3ID|1OB0P-G>Ibf$??ts z^uy(__+DTQ^Kx0kef%37;>J88QBol_oo?O8wi1Y~IeoDW0VPGP%vy{lO9i6E0H- zknHL+@8D+)85|Hl@yiaQ8O@Wj6N8Y;1HaWlOqeW2S6Ay=OAtD2Ta2IiYl4r=A3$bx zxFFL~y&sAgOj@&ukZ5dbPk!|-B8U;^C+FJdKIY^oa?NgdsQi=|#r1?(7Qt?+8uf_7 zj4E|=n#%n;eNVyV=Z5_{>#$4@6VECd;lkkF^w1}1Uv`*4;y89eh2lWNuy|=+4fy-*xkI;($mb^c$5kcqXYT5(Lt_y>soILAINbE4#-QkDJQ+G@zay1i8&E%=( z6dcQ)WZ*a$=;XbZmgY{Hy?~I;Xx;lH++OD`>uB41oU-`Mo%;Y7zvTTz!f^1!NAK+C z3h&-8>ma39v~=t(A??oVO}t0ppy^g)F|r+g)<*dCc)c=(<#tWn9xPpBKSbWXlyq0Y zc6{IE(6)L)wiLf;NAU3%`D3v3**zvD)_D%?HF_pFOziZ!jTy8T*6aqCC!8qsky-oZ z^YlP19{c*8v;8&zS!$kw|H)(A4wa3Pq@&y>LT9`(Kx&O&MobXjjc`x=5;sPH<kYCUr#{B>`XYquwD3Y5x<;6@>8+0Z4Dqv~pj zjh+0pvOTt^(1@x$zh%la8927Z%03W4^q+vme|+GwouZS3u5s7R@Yb-(q#@AR9i>;) zsNeFQVMn-*JJoA*pX*d!-X9k4hifWe$i*g9$_tSwU!A>9R1nnR>t zt-j`1f$%?m>7916;hhRJX`uLeF_0hR9P@<;7ieJQ8ZazRD@84^`b)v7D~I|h|YJNg0Id98&=;XaPeOI3@#6TUmk^HpVBiW?_> zf+KDr(FU&<9r}Cib?=de&xy|tGl$(e zHE-=UO!0iurRBF+nE-N(+;0lGK=F^gruUu)N8pbYb1<&R^mx^8(Rt(D_@4ALNK7|T z*By!_vN!E>>3lExC0s1RTO7){h+hLIrTl6DS-}tt)-D3BqR?XPz|pKgrf(8Gy}x@x zR{Bp}KP8@5|GHD5YJszHLo>&Ie~fysekhj%>32W!Z3x+lzlN+%|o0o`+PM&-GtiMN<0bXaxr|dHKJ>mDsot{Eo%qx zg19Y&af(T%XSU%}0aOVmuRx#-a6No;b}*OAz#pT1TOkdMikMM!fw)fpi*{?He;EUn zXYcyDE{OD0!Nq+0MJ%a-cCkW%F|2M?}aaM7qSa0R1yn#9bE&6w!a-WmmNt9L^r#r=oRH zv(^{nQ>EP@D@D4$EkdJE z2^PH_y*cfIUYTn!>Uvijs5akl5beP$jw}jm+)2l0VTI)~M^(rTxd{b4g5>N)U+8Y+ zG1C8#5c6&*Fv(;y_amx~&^v-9AHUnZ8*P5(GV-v&TUhV428p;_kj{^-PMRQiz_0GE zF0%ti9nj&Jz3Swy+N+HPNQb(v3;*Lck$t%LH-^pEZFyAvSYWrEkkPZR5!X}`3*m9j zOU4^Aipl*Vh#+ppZi&Eri*yKwHc#gsKm7l?~{Fz)Hoq5Xxu)uu~} zbxB@R@k$nsJdmu3B_~f8dU0@r8vRUO8>?H+c+d*y;(3E3jt@JuFb7wVqR7xz(JvQ4 zd^wq2xW`7AyAN1xioUu6xqtG^6~{G}#;YpVe$`Q)q_CDay+bdU z?&}+Y&fv|nLssUZgjT6PzBVeS9>PcJ^f?EpPIiLeGX{L_{C(m`iM?p|Tzj+gVw7vX zOtV{U&u$#yd~qdOOXFcZ9K$d{O%IYFg2)y!*qSz*QYYR-Q}X1_cQfWz)ITHKClMRn_Y z7iGxL2~n(Lg6pMD?_-wmxz>&4I^SZm;YVP{ybGsd6s>lyaZeNjOH;kk;6iD;HcpK> zpCy+@`sx%et2z0$&9QCQgVl5+uv44{Ev39?DkWIG*N`z|rN;)N1Z^RqpOX5{)e&kc z-u2q4o>(J_*i&Y~kmw>hkj4%aS zC%XXJdFvmKR^3gREW{zeC)&I(#t43Uzudx`6zIt?AdG}wb;mKPCzp^=TFV72K#PV0 zvA2?w>$W7b(>80c*83U_(11xn@Hvl2c@}ZU&>@F_F6*P%aff#Y_)MB)k4Y@LF7uTH zhMZogT5^yZam3SaeDH)x58i9PJK$*6c{UNbx86|pm&G1%Af2aCsuC=L>x%YUVqm+u;%=h6WV3Ta=Q0ijbqu z;Do$48#+Ix;{71`qKWu90Ahe5GMly1kC|okXbp3jt@;#$)R!QmV9iu9D3ui)Tw6oO zMlRDX2-E;2tsZnFz2~@(HyiN^+iBO5GyM8NSzDo?s0cF!51`m1pCyHilXKzr9^O?d zX7b}caePFy*Rg-GaJe&GetGh%>GGDpMWt`HQl=&=w#yIs zVNf_%qFgFuQ&{=Y4_UiE-*M?JG7I{nu;n!K!E~#E5!%vn85{)>sr>S=zc$ECqj0o; zoKRn}%x8+5B~v8IZ0tqhLnV^&48B=BwilN2VwjH$3gbiqVuii{QpiMVYao6?Y?+zM zvqwGkJA+KQazTnBYQ8~ay#C6p_P8fA56bZlonlG3X5}H@3T=xK4OJOR91?f<8Nw~m z^=OC2(O2_7EIe}SA)@$~4ZfyoUkr2IUZx?txXd@7`g$RHvNgZ=IJdqN z?o_8Dnht)9d&nKcb`uZUbpIB2LuK=@BWPXhA=n5OhM#XLpJfH=Dne{}pcE$Pi;^8E2}5CLk2hxuA28xzK?ZtVl=3dqh!`z=9U%9VT} zZdYZiR=z6Mharo)3N-jK(daLjo~9e?*`Z4%m~ZNZ0+@M9??X-+FZ&Z*NokLAzacx` z!$VJQNASymQBcb({KJB5R2D-R-pbttwo5-zJ_^C-=U#^+=KBeEb{j<#E=wBMlPRXq zCYy=pTChUc_wpm5X4`HLFPEHED>9(t5IuBr?DpmcFs=b?4UvNoT#P~D_G!X5@Fu{w z3y6Fb@_(gRy?TIuHV(l)e(w2gu5Q=Y0`S3mYzT1huq}b)z((Vq_r>N!40s&8QkC(V z%KIG4>LtYPR;u|^mVRLc(j2v15WcROD4aIZq+-fHM zv3VV*t(rH*fu+iHCW*NLyk|KQ6TvugPwsHZ+K-OgMn24ExFfPW$#pN@n_EXxOJ%p| zw$lV&U0R2Vd~lx-0d{;IeG_^$7I5VoedS)}9a7p+@r=oS#rtjZhz0}Ii%^okQie3M95bKrTb7(ZdL_3dF|#~Pw8RqWrF3Z z&`kOlL>AhEvcufJ9hOZ;Yh#@~ax~(-X-&1Ro!o8feAG%b3ZxJ1YTabZRqSYne)L)! z;arJpyXn<;L1TL3<3XbDyko?(_IwVGe;WakDx_kBu|J)kZ)D>*<20mq4etpU4hKG zzYy~_k8$)THt!N&4)T9y#eq=>EJ)*&f?loM;l~NCE5>z%Gjbv*?3+^9 zsRhQZjfE*pVf#h7Dg`6Rjb0HBlCq86Yfjw}_oKR^N?61ub=sKEbe|9k%9yCmfrjG; zV-~REAvu3Y79*Bx7$L1^-VK^WzTfh;Rsn);WIx~^n5Tc$DK<9ZJRjFm;JW6Sy9^3o zFHn6_7exPJGc~og?s7miwwN@i=IB%~TOvBLO+)+jvA)4vsHo;paLnZ;ei)M=K(FNa zCw$-3Q1`}QGkYl!lRHi}Lm6PUTCQBFYxiwdIOD3{bKDb$C$LhR!eN31Q|QxK{Ex++ z_#gL!(T1B``ezFFSo;yt;tNMx#4wFTZ(T{&|HDNets&j@d zMx!DKABRt0uLVoDXJJyR7I(CWA2%6D=8j#oilU8{&@ByuT?>fS{~vDN`7zQGsr;e#`Md zoU@LIogj(MT~9bz1-7pty7Y)!I_&6fH3#wD-dhE0Hx17CVX`P%l>vV`u7!Ky68IQ1 zw`B(Bg-SpTSLud9|6CD_LBn7p_KdrrFi;%wd^Q z_`tmQi`_@5lDSZYBuE#!qZ-^D6wq1tkJ{NSq7ck)}01sfa zUsjmPZH-cFD#4AL*fVUCAr1^WjaJijI-lv;zj5|^NxXk_2q+XFwJWh46H{#9MpRTc zCHRT#OY88uGb&S(J}s=dV=ri5^-PP#O^$-cD_nv)!az-Tl)_gBEmA*iu$FI9%XzK4 z%yhoH;p4+V%26a7`Q`w4OetVuMT<&73jy=3BJE>E+^uT7`#ZY;=`k}R44=m|p@`6@ z=dF!9hkNF}2kc*dgnGEWH6lIPwfuxA0mRtI6RkYPav*cO$la@j969i~rThpB=2JE9 zwWv?^bENe}JQrCqK61sMyi}c?noK#g++iJ?4Biatru2+%U)COd>N79@!wy4#C0}6W zv`ENBf^?Efb)-m0X`uIW9{ZF+4mwItBht!lo7 zmH((Pu5|f{1J&JO1MVp6V>}z8mUF~C#As?)cBSJLvJ6(_s_eBWPcFWUTE5e<{|0{7 zWUyFu9(Qk+@sZxP~G-iQ3PwVi&~?d;lAhGR8MvC#QnC&&Em># z)yiJe2REcCC;@ce6+PWT>|jp!iba0);B0m>DZbOV>=p`?;AKhP4DCSroRXy%ZgQuI z{q%aX5#g3)6u%8#{=30zztY4W4$}>vE^J(OLA_x1*SwAFj_P!(>_(I{Cy&yivNyY8 z9)2i6m9N2~&0XIg%B+2Q9C1_y;sbhSl8ksnj{yCIA} z+^r5h@zbOAsBw=JE0$+RtSwe{Tx+eUrmhZut$ulDxOmfCS8LKGf1d~xfbIS2*($M9B(p!`iCz@2fAV!sS9?0R8Q}Ru z-yK1jaWNE>@PXVtYblfZ#b#d_dYB~eC;c94EM8+s3CFTR62gHfo6Cw|$^WIPIwA4d zKuLEu<8=ENXYck-QJ8~he2#JQc>+&s^SQ^`IdT1+>+ z2bt$tB+2rG$wK2a-2IC@Hn#!G`jdh zW4enMThtC>ZB6%t!j<+)D#<<_c>OU|eZ1$3{d?uQa+oPhhQ631b=LbBmJ6~Z6DynF zcd~GHl?!&^4lg*wquhQIr@kn!iIi)OVrva;JY`pEFZ;+ZvXI)!xd#UAd%M z+T&eYTINger04|Fnj*A;NlPgP;w{w}F1% zw+f?pvWn;9egmI)J=fnf&kyGaZn)_f$NItu!&>>HVGw~bptT?-0lofZLd{)y*)Z4T z)b8w5+vc|}z3|_9PCpL{eO!U3-_9{219M{B}L`kUQt9A7aYefA{rE2ELM6hy} z+;;7u<1y_@aP;qr(_EzyODHkDHkfA}RYrODd* zP_ZB)RpnM=yuWyEMWIkv#Pf}3Y*2>a{BsW9Q;1h_rS7#y$aBoV0bGOdpht%JSx*%% zh81CgSHm}sryJ#y*RakOym0~jZ1HYP*DYbm)!?~T>bp}&qf-oasppBw^D+ru3%EF* z7fE|6ErR9e&CehTejbD#5wwB73p;(hE9Mcl9imb0jdX)nj<{@N8)9Ga!}fWzj=q+etYqS z*7$JL$Zp4)C_%*3lp0_OLs3bmCL<9uvV-LW@ln|-<8Kb`FhD~yHpi@N2i`(`bn_g` z@UL0FgSC>v6*5#Zc{AAC;jg?rPt2fG!BgDAeHI`W7E?bWYJ=CQkJt6pr}kE1^GUx@ zuS#xRHTF|MHX+#f945Q6n4?3JAX~9$zq>&kkY?M`egNs`XR|x~Zp=}e>)&u%0u>Q5 zmL`aXJp21|;^q;GXV-i0+$W^0AB7$}yzH92Z~MWgtea2(LRkcp{NUp#O-()8xGZY} zXexFJ4=9AY0K^wfaQmX2sBt;tt!76t`CmDX)}*~omm>Fl*XX0Pf$6jt^x$9vSrw&A z>C*=?YFzgjW{qxm@9?f4yrb^XT|@~Mtxq07gZZRPJ9Ay%)P8OOl|t-gOuF~5P(k6- znrQ>f#D?4>k+jEe2VCIhw|I=uLIol4k4cHU3xisgP2AHePb=)~cagX!=vrHmv2UN$ z(q#*rVZQ3#k~eThqjCZx{1s}#_wbR15bnpPAASfSlK3`CBdWTIJQH_Pr#!^z;df=Nv?}b2orrt?W9*W~ z;jPAP4qIKTsNkCQc}|?6r6-9$m9bVJa8TYJC5@l@{koL_BvZ^S3DnH z{ba7yyW}`hB}IHfY(g4goiErGEG)pCIcr8o zr1Fl@fhJRHg4Fc*Pf^@33)*3X^Dmi=xTzj`V?C{Fl3I8&?2H?SdETSuwV>~^`OMey z+D@?GiFD9Uwz3b0{L|lUnXFcHN`mCJBlb9iv0156AX7Y^^w9wBJ;Y!K;8Dqs3IEko z_DHd&`n~vK_;LTFeh0N`Y4eIc;AUH5KVypN983J0(|J%aI?*)|J=*a?qgs@&Zu3zN z(|%~cZgU|1coKM7MuKW);ytJEiDE6EQrZU$SUSqZvMP4{&eGBN-UNpwg5o*qS5p{~ z5mRJ1JP#|2J#FB7KEJBXT^annoW^5ltwfAnhaYcDhp#Y`a)Aio?zz~dzJWaZhqZH7 zgC?dMG6Sy_dkYtl)*Y{N^j%Rh8ZjG=+gn3P_%qRZ_o(T687+iY5-;y=w=ifOl8d-a zevP517^*rvTt4&7e&UwtvIZ6@`rzT4dxA?`CbI>^dvne~IbbrHoFBn6bdgjrDUTyu zp`gPd(XWBG&l8B6dmS)pj8?qr{h2X#pZ)7guN;|`x9({`R+_;zpe$i~;?8Dk;U>yx3S`){r50n(}H zw4(9mx(!eJ&*HVI`Ak$<2k=TLX`y!>CmZXVzkZ!MKi%fpN2E-#mwvw|0}_U;`ow)c zduuCyS|&V4`L@R&@(Z{&lJE1P4X&2v>jHjXYr84OQawK$p~orB%<`hh?fw_TZg}QB zsQ>Z~2YS|d{!Kzz+9>4>FiGzREH^~wS?#RN-&x9QG;*Euk0SW}MTZ!e>^ioWE~w4I z>YgQCr|bnRSE0#I?Ip!>GD^L@8$7Kkd3+=oB|GgGu?O(HrBofehoZ7Ns^p}5l` zMTTX6^kcXxNU;1D3m<^0b1pEKU~`@Lh0kuMofM)unK zS$poa=9)V)+HwmsvzYqo!!3O59gD4!Vmc9fhx%3yhGth@oaYB!7iw6P-KYr*9dfEL z@o}(nh$4)WJayLZ#v zo;^z68Mh`KwX$sa#B=U4A7&~k{`4=kFSG=$Ium%)HscVla*2EdYMl)WoaGsk_UFwg zD$71SU<(6@7>-(0*4(?MHr@Pqr^OMW9;s{YY&0yHC$=>Z>M)2=<%eO~`Q1<=>Ef_+%AGKev|?38b!-rfDZ8QQIP9V)g&{EuPTYloF!uJkwi_)5@(xwA!; z$Vl{;^|miay1S{c9Lj<=w%6|T$g<`1H&aRxg~}$-11KffyAbh_V8=K1$R~RRz=vbXlH>z& z*dpTl52y|xgGO&LYVf_kPWJf(b$NVra>;u8zLVp1aw2@f0z$0As*$?pXwB5pDKaM1 zVks6~)6KxuLr9}TTjA{(b`%YIF4Kj0%h#S-OwjnCi|QkCY9bzV`~KIA#*FOr@?7*L ztD?lPz+@ca?Y-8$=5p)5@G4K`lbtvA=VspB?c1=0_a;0tCsgN7#yz9+)8~7+c7zEk z77$xqMTb#*+-<9cO}hv?2H}(p+Y_pRZz~~DyEqNhe666d!p~*y6Je*L*XX7W@~lWo z{#w?s)t z`I}4g+YEi!!M6Bag0{;eFvCy!o;02H6%1z;TI;!^^SJw|+y}30SRePn114z-{#s7) zYZ+pzMDlmjfzWNkm9=DH279=?BNZ)s6w|egPUSNJa7C`{S&2yDp6Vog*YH`LC;gzp zeCj>9N!F7@D%^lY7k3_O$R#v6dbl;)aV?ur4DvlBE@${zgU?ngs5cmuU}XD={(|c5decBD)Dyosv8yliHcO=ji-*K@2q}QI~O*QI!3h>-BjGjPI zQI6IMrwSjDBmOM;?(EID2Er3Q>WO8?VKF))?Bc6ShmZu$89)BT>pJb|rbox$cB4F! zL(ure>1h1770QypFVhd)vF3D3aKac*yue|yywlv)-?Bj4kg?z;6-%)fmg&0ju&kNR z+jGFf8t^qfer#=fRhnY?=o5oVUM0viEK#9&m}vG_H`mAO1u&TIhQgO_k}zJe1QiSX zR`_qWqeU7vdcjhIB*D}Tu$Ef(8;~QQg4}YOd}*MdW*YFguVHKB3^b|BoaBA_IQ+8w zUEl*2T+HITIxFB@D*Z5~jE(?$HRzjwG zaEc4%u%B?nQqZ1TP~TqyK`8Wkcbn~7WIhJ-`o%#mWv!1A&!6eMiun`*zy^b;;^sp1 z&Uf-#;IkUP3hUS-gG(jJBseC-*T)qhe0=-+~D+Ak|Rz{g14fdPY*Y z&lI-w;>Vm00g3iNs0qsFP<{7KF>MYtukM)i!jdkxck|Sb;M-h`_9O|j8Y@*Mjc{?2 zN{6!a^!fvQcH`(<2i;Eyz?q#bL25aMD9FeAiTl9>i_tC6pO{TMiqJ3H78-&I6@YF5K*LiE3OIOw0(q%z!peCSt*Di9NC5+}h*|0GmG!u5rmh0R_b z{+EV1Asd#h?ydAs@AJP!0Ydy1eoDoj^!I9FoECZ4-<7u!CJU&^hF40tOiNg=2Q|Fs zs-t@@=)j?}yx(7D)TF)_q`s^M(P{m>t*o;ot6Z&SsJ_BxoKlYX{ml3{?e`HqZag*S z58{ey0@PZKHXt88t87wTB3V4wi1fYRYE)NPyG$=8$64GY=08;PeZTsO@M{ZSxO9TI zgP1Wyw5GNwPimsmBa5ciFR#WBahuoMBAj_7RbCObf2PL~;+cQo2L&Bz27G*!IP(ZE zaBlpPF)yXr;l!EAIi-`q$|H=t6x_J0A3~uWRZiu@J zC-4ly3E)IzY#%yAO`uGX6l35_IPe#UAoOD~KDagd$c#&PpzEg|QQ5u)2k^anzwOTD z!r@6-P=)d=yk+*y5IFeU;})P|oPHL2<;xC6kkx0%5Pajk! zvQcAsnN$NqE2Ux&`wKaW2g`ol`3%bgugLz@RAUVpVE@D%7~9nLI(}VSDmQ@QYYNdy zLf2cCp!7huFR6#YdM}>d;p`R`d>t@#Akw7Pl`{K;6Qj4O;fGe9b&2CuK1i?eC2haCMOyiBjY%xn4+m%q}zU$f5w&Z_s$mLAR z74IUL$qf?Rq;0HXxa<^|=^uBPsv~r>CQ)|&Y(?>ODR%+b5KJ=^c%X6q|E27o>pgeD zn8e>(+2W9z4lY@CbvI@gBz;Y|*ltixp>NQ4Tl5OM47QWCqc{lj^PVK@f87WTd-B5^ zqI*d|m`tKAF-vqc1brZ>732Sf1qzbd7QTw3>FV6i2ybH*Ca^Q7t2m*r;(W8Es<{=H zErWo;iILu^XOr~A>9qG$^NBR7{IGE@BRlbtr0aB!K91Ilos8D?nYz<50Tg@+I3J5L z1YZF^S=2J%;?DkE2FguyW_3@ibY3WUKCL+NuBfz&1g}$ZA9H%V3ab2t(xj8|y&=xu zl^>FMu@%pMb7m<1B{{k+cFCg2K>DQI7YUrh6o2fXTXQ-LFpuss)J2;Lo>E9FeCB+b zt#xvIxx$@W85yEm8iW`{cDZx8i!oehY4L6oeF2 z7QaQw3-nuzNqWc{{DBeDUYvWCf_fEQhRG%YZUi2R3qn-Mu_QQjMea0N?POcXR z=>Ey8hsOE!FS!E+R;z~d&Q%1apXGnFI-Xn7@fjS8#UZ^7CBm@Xw>6YL6ywq1V#mu( zj}-j$+qFpHELc_*r0!X(2g^;qdOA0UWBQ=@WUPLi6Z*-5I_?JtLs3@BP67M3H@sd+ z-#;C{xx{}*V&LsdyquPs$tn0EAQVne^vyXe?#MPhVb3{Fb=HA>h*s9C(UKOn;g1>B ztJiXhXz#f|)`9f}TmQ|GqfAQt{!9?re*4kNU*MoA3@+~W-6MlONf9Vt ztD$-b;kmZCcS`82IrVx@0Hr>FO`u=^Z6GA@nbvmcDwvzNoN5-ahy+kXC3gSN zUQ8UHJx?xc!3N7ad_2G~YUE0_m|9PnF^ySFr*kK2v>-FGS)&v9jfB0>P}Qs(I}CPo zv8g^C3(mdE%96_*T_jDEy-t1z{(Z0~xrCf3$UPA!cB>|>AB5>F4Qu4|<<#4^>u{Q> zuh}-fj&_doL?g5>D?Z-r89rVL!KgY!;$hd058TGcRqP7C;B{t9mc1TdhtKy)+KnGo z{gXxF3+Sk`nl3IcXPeTtvSP_}osy5`6&t{NSMXRhLx0?*VHPOGhjjHqK z5Ass~Rm^L$?EgCwmD(Jn{pfpS{{tzQuV?H{+R)mnimv5a`lhgfT^boD87UJ^Zh_E( z*PB);gUDTmszg3(o5~-$@$P=Eh0=C08^_V6JI~$v1YP8LjbGE*`P)D1o#@9~UK(^moVmPaRpUC)$7WQ(>sEZ9m~Y*sFxBhMz-GMsd+pFkYW@C6mtgh{ zk`>`NtG5Ov$x5u&A5vw+sB@%2K(DOQDftO(lUxyi?b5byr>V=*s6F8Ubh4g`_=sr# z0xo*hm*PDz8$H}y^0FK|lwGO(v-W{l2E6(@_ZBhvgg0GFgO7Lib(5v6`!e>pPgJ7x zDB~#mLZ(KtoxxsDEEe~*pigxnK-hAT?0~B`6Sr=6l;-0^uGF|$zmsafkMgiO^LE>i z0F0dNUL#%m&g1!^>9;60zO+DRo|1JAi3f7xO_{mu8MFW;W;HUjp@1o1>(e7m8xOzR z(YyVjlixn3xmd4njSr6X7{|M{05NBXn5SixhL_7l@Ek+S^_hK^v)z($HBdPZ;0-8S z3DMdG$YziE$IflFjT{}nqM@OEp}2ImR>yLTQr^EO$`-s5+ZJ!+-H~yI@*!VH55EEs z1BDHuobY6Te>tzH)j-ow%I1ScZG^7#Y`yy-(r%%*`9aIW7Savci2+Ou`ob`g-XWnj zX#=<3`KdRF7X$nB#UJ6KrfH%NYaGyM6@-DA(;;WoR8T{p%06bH_SL$Bd3lK{WrIid z=={-mTq*3R_i@BU=U{|44sN}>v($f#2GdnZbl1OH>)4?Nbf>b0ABVa7D)_o#o&{6b zAgdJgG+-Ls0z*I#4C^&`m>!unyM@5>cZ;brMU{Rm2Inen%1#d1!Xm(vaGQ=_=;kst z^Q=qKnyLm|?~P$)V8aDP51c*F)K68I0`?Zt^7|d;-&3NO%Fj=6(vVO2BFG*zdmem7 zZ>V*dms30mf5tK!lKpuQF=$(3pTBF+uO#BUFM1wiqXGiukHJ^h8ObQ7*(@ zlcCe?M34ODAmxz2YalQSn2r`_rwPRhq=$Gwn;9MS;T#pU5f3WyB%q|m(QxmRU9F+# zP*5&$y5ZgVi@SMkz<;a+?isiusc;}%aM@fr-hiNUQeH&; zZg1x4%h(J{U4D~X3L&;gTj)6tz#Hb5&G(k&KMe0Qg?8`2dErh5oU=l`SiUh4+yJnw zGRvB*jH2x_G#Sbq%f~$jH@v>dJkAbf*SjWxV9XoM@=rCSzn?cM$0dJTbu8=^RrWoD zFlen@tqchBRiW{q@NcaLCQijd0|q~yaEi!h9QJ^nJnylE=>S$G?%->B18lD zotej9s2RxQyOI({p!Z{%y||;oR|*nOSfaD$A5_Czvr=DZJ4a<@kA6BnUN2o|XMcU# z@e@O-QcebGc*5#6^cVaw`@RA6nAm%I>U=@6`(`}#tcCI6)2(E&K<*DVpTpME3-zMM zCeUgAJl0xe%#lRg(+nat+eg3sIupHu`Vr}vK-Hm3U4W#rT;vdAbT6bEQE9tofo1kz zo6+vr-*SyY8-r5ET#3@{1Ej{516eqCW}i8s8zBN~Wsws4n(w2DKW@i36Vl&F-Y{86 z&MlE;se1A3P)%6-Y@Q=Eb3`VdOis zm2lU&9wRI=K^L3lmYI(CHftXxe>zgKDj;dnRI(9C(F#Q|oe8*uq%z&b!x8%I{o9ny z0{6Byv1X#DdZ433I3IgH zG4x3msK$sVO(U1LRsvUg;9isf{b^!7>tqFL7~J+7CRUWI`&Y8~MCG4eZrk{3DFZic zc9w1KQbBdn?^}-z4>XM_@cQ#4PG8uMC09qa9o8i!&iYcPb+uRi!HW zh4CZreF&B#^bvvjIKtMT=@u#YhF3Q8(Pq>i_op`7le>X_Q@Us`?5Z9SJYN`akh9(2 za9^GOfqADYbB1h-*K!-qTQFv?4s%~&(lT)VrT0dH^i~SG_sB67Brfygvm$F|(R@!D zCv4k>ea`hr_(C^k+_dfVGT&y6ZhThzD&6L;uZh%+rKl|DL+z5k69?Up9D2%@LDkRa z-IQNy@a9gG-h+bFCh76lzAEuifM-YS9^$erj;4+82bdBo@3UgYoSS{fv1gIA!Nb15 zaB`(b?x1mNy2M*f1K*^l?VGz&<)A23f76o4S8ZY>^=Lmu=^EFZ98@dL>24SmL+OXO zC7=AC=fNJb9cNQbvvzBlQz=dJfXI3Tr+I(SIN3wyT`E}GA|G)hGjMoykT$jNYv$&@ zB4P`c6}q9z-lRQn0kk%nXWD4*`TG6>OwyF&!Pi;!$uZ0r`%G%jvrJ}*ea0XaOBjmZfl!!>}Y9R zlIk?sC@wvAL(Wb}e*KH{wejy-pP?;h^5!$-_~diFLJ_Gw+ah&V58ZZKK4`OtxWH6ja#=MyG9D37P4>UQWrn5B)&XsO zHGH8=@EB7`uVkJ__S=k=TH z_(rR8=n}q-l);dq_RJ;sPLc2*U#6_70>R<*qH&G|gDW3^tL)^yRrnb@-Z0AXiY5!* z2Jt?6(;?IP`}WEb_wKfxz^IN^Ur@9ZgvVcnTqK`uhLQK-Eob8ADw7?pHDG}Eol9^2 z82#NqHfU2($at=7hsO|FDcWw|%Ap;(hPZ<)vf;sZ5CiK^@Uq?eU8W)Q%4Vjz_qUw_ zOQN$9ul<06cpFqIfv9gj97(v1iT*g`TX@`kENcy!1U9Y|&Kt2F>jmJP0u&gOP8T3FOQ9-ZGYCTc0k4f<@U$;*uNu}~3M6_m8bfF|uOyE&#hB*~$&m|y*RKURHX zs5)Azxm1NNhp!X+8aIFQwV5?*ek!FINaqw+*>i1>ACD^9$z6u4!QmAU_I4xkCBkOS zO6Y`M^L8!pINYM>G){a>r0w2kln^5i$TY|tRqna1(hZ6w?B1B#Le0V*Z`~R4oQDlIAibnHe=|JvXu`{{Fe2h^*p&+8|F_8ft%{DEcAMEZyDWEv^Zu;4i9vlKwI`xy8{ zV%O#r%uv-PWy-BUp54aVV*!SK5puauU!0%ksPIu49&dNgeA-QZf3X;(HGl`c1e=@h znwHGen}tk&fONUJCW> z7Qr$_Z}aa4%2W3 zlv>d23krlTweWWG&N%Qf$KD&d8%~0fm~NI=XY1@&)GJJT|*n3{9+^cHcc# zRV2BCJyL5(m3=v{Z&lOPVQQ7*lXKR?6FJn~oxE9j*L<{7mT>n{6GQu|2C(`t3&G~V zB|22SnAxzSM9L2KgjCce5*fZ5)Z}!DqfBY*yCoRN%*SLc*c}ty^N|dH@Ze4Kt}s%p zEE^s`lWvC2z-udsN!$iash>w;gCnY8HzPRsXMCaW!s@P+3_EZ~DS6(&t?cOl0hfCE zd|WzgNUx<&{XHROz4=JQd5#c~4SF1+U`GtMVC)|zUP8+ZnRtG|T(EpyNNB8Y4x0 z5%vPahM4x1MmUsKQ$qL$4b=Yzh9A&t2Is*JKjvJK>30kDZVJ@bn$ODoju zAxM<6RorFAO>2#XDf6xfoxsNT6a0vQrn-sbsXTRKs%uWuq;+#!{{id?`$(24tF~O? z46WB1NM0XExtD>)H?C>(;@8l@{Pj;sT`T%pb>WXRJDgVYje1NZoY^~hOg_KL7m6J9 z?j>q0JbgZCdDf6qCc3)Z%)mF0B?LcpUMG?aGTAS&fT5?HLq*p1)J?LL^QcqTX@Ptp zK_u<^;up&fap8uZxoat3*AYPurMDbon3q*m2v@*rk$qx%GEJ0C_583aUuf$4jF35g zSOAJxmAKH5TlB%a?jnT6B@*G4znN&2d#pVU6lXw<1u1u;04C?rHxPMoS9LDDQL-aE zXSoOfwX>XpPg%7L@Y(^!uv!dTOdowjYn1fmb3oSsl%Xx!jB)(C4+)v87v;k- zZhtfiv61I%l55eyHl+Q10E@#=pHrjHlJ}3nD;0hGi@K=okK<(f!x}0TNv!L4Lvp&G z*REv^V<^tnnP>pr!#f}z&7|M!<((7H)L#}(;MhtRE>9Mdp@((Z%6ayB99}!*R239f zsb25u*lAN4!@1fd!|x3Zpd%&I2U>8=zPLeGbSkn+tTjaJU-ooMB&&cRk%d2xsq`M& z!HEtlE%NqF4*?ArTG4-YuFx)l`rloCG)Vr(IzBic=}B_;rQcr|z&vCNT=p=7icSl! zg2qBBf$55vD@BTSJmviD*WtRaKInj2?@89)&x_R?RKku~Wg0L zE&6cwnfBBfrGc@5<7{5ebcG0S>+L0xuf>WAg0uZ#d^gF#`=_>7b}803E#H&fhKcVI zYpb+~>rT8HdFZu>lK|y1tM`jvlTX=48Cx-_;%{Aj*KF+&*^K*om(|>GK*eqRqx(>>gFd@M`;Jr_kCIK1c(jKnb(2GI@;vKo+uCs# z-$I&nE86lk4xrb{6NgyUx=i}$&WzM;4!c)mk28KhggXMs*1`6nad_n2(?H+M*T#dZ zkUbztsV4Pp{sk4nY7CoV8d+({Y3kixUxG8T0-vBD^TO@fDyx{7*z?tc6uV`OCrLa> zwCA~9lkB*gDn-VCM_&gD);_l;{_T_~Wv(FC+M zRwc^P099#Fza}pI88}(Q+G*M*86wIweFeB3Hg#Xi<;=@u3)jq`3%B4BB$Y?_E(XYc zbGMDR({*PC+DST0LZ%@I*HIy5Vvfmvls`}-c=z1YJoQza~nEp3q=eDMT0YF(VFY;kkF z&kU-wNmLOZBOdt21O>64Hqa$v9w90YLK8*J$|6dJ-RE%urYSa5#8}sNcf%bWe!X5I z?q>;A2#@^ob$(_D5XSk%CLDgIpH*qIWP1&5e{aY~UCg2tbfa2F1mUv>Ewk(1ICa(^ zHx9HL;C#Me>GH)PD;gJ9PO8*q)F;oGJt*?$zuHjTl756F43lKqsCesLUx@64I~eY^ zQ%~?``F`PdSf+ohj}`AJu;VdVzT6vL^M5gkxi+8^C(sAhAH7(cNv(sozRb+$C;t@@4rV z-{4$2Tu~W#69U@dn;#<70)J#|77l{psb*Wdgp>luv#9=Vr$+wundevp)67%Ll^*X5 za@)W+t74LDm&}4M=ouuOO{la;|)p%^W9fC zmR-YsZvLJ=B`@z2cYA2dM8L%Ux^17svXaGZv8sLF?#n_}IVk^7CSsUF!q47%8 zzf4*BUj1`iBSwB*9;QWeT-&(Hj^1M_OrB==F=>QaP|~uFai_YS5@9Yl!_`ykn?f6>@8h1&$=;OP&+55?xFQ`f<}n2KUll?w4LNw+%H-#S zaMeIGCZp!8!&td7C?B|7jXWE+VOio(nse(ucPEGIAKZdRrX$C(K8i=*VHK(wMpOa;bQqb#Za;m|%Ygk6g_kQD3O2+^ zBLQys4JsKoBUK2f^p3d3qn1TqjA;DgSzU%?>f`YrD`^T9(512l9^QqO*r_t9pS$wY_#U4TRctV1> z{t}s5-V`BF$py6BP^?Ut!xy9F4 zE3_85?++!Jf7Qt-NWF*{9{_uA09)kU5CWb8l+;Ic;zTaPq55DyMl&s_jXC>dJMojL zYzymwKhr!<3<(HX%n4FqRLGvAm>aKgEkgX{kY=i-x4a97i2e3~Em9)#MYyM`|Cz%B za_rfRPg|0SJE@jm%(7qOF^FB<1Dvq$2Row4diMOd;sp`(_%Bo%KEwHsU&2X6#e{YW zOz=^MG5~_SaAW6-EtB)!nycW5pG1r`d2f=dVlXSX$E_{V-1WiY}Nwdtp!QfB?U*VG&_$EOjcC zo)RW=fDpN{Z<0_s3-+3ok3VUlH-vPAu63>?ZF{%v=S^gu<;Tbx?!$o_uyveLrU4`a z2-DesWq{5<6KcxE!xXM#s(--k)Lr$V;~QQP@39DVzd0T};g`Q(#Z;oQ8KSpO*?!mb z21PyRrmIV5<15uc8SB%AX&LNeA32~aj|xbzkw?9&V)yH|qRlK950LQ2a_%|1fhR#} zz10KSj^Z3F*afp?VvS$>7DOy+;nqj$N&GiNYM>A%pH6#6j?C3^KK)2q_O?x;`4() z;Km3n-L}^KCYwnFWd<%<6r28t=)46s9{L+V_H39Mb@L%NJjz0mMW}+dNaWOvBHtR^I-sD$A0188=4QGmSsJ(KE<;OodAGD7#Fh z>96w%?%qbuY7h@&U8cVMjq*UV)NN=fFhIUrL_L6xyY!+0?LP14hQp0tEp@?X{UUDF zjS*Vv;csttjlyxlCZc6l;(0Z-zO!)-Si~{o9sn)zfVr!*HVN6Dv^0X2D0So;;YjMae=-&|P2rzce;kjf4Pp{Lu#OtGvlrsZrmK68Mcu(K@9p_LK zX58L0ppId0)#xpLH&BY&nW9Fp2oO7GjGd?h2$K5!)6G>FHMxG<3@~#jO7%c(~SxB|+VSAz#ghF2+8E zyQ{C^qmbyA`htd2=4qd*So&M7BT>!}%eyTStY^WG;jA2qES*`^?@x3+a=axCp<6H0 zI3CB3(nNLeG*Ggt+@*kG1IJk3tLp(<+h)L%8kUiwpz8di)8>o5@e*V9m@3=; zS-y7WU^*-mpfILUN*6zGDB*1G_7@v~ppOq1L*V&x&9k&j7+pUOmiq9v`wO#TVl5(^zc!*gd+md1k3*7-mU*Y*+#+f7Q7@GeF^q% z+ioD8uBWcL!kGP|;YGc2W3Wl(sMxrIzyl$s(Mud&VRg5k39xG22G-!k2zCe8|Eq#e zslOc%$6IkZfh(=y98!A* z)Q=IPv@$a4Al?ji{f1lmv%)j}1a!&#e+h~@#|5_6Zu4U611`ZMxH*(tr%2>jUyMiJwVj75!F%u_c`3McW7(8pIgixCy zAO=?fJN8XwyYtyRqqbqJXsy;?+XD}@pq{c71ScKGqEe8kpL~CqbL`KtayLEYiY=8) z$a3ua*d@Dc+)C{>Zx1T=ab;DTGvx%e%{lCNd=gT}TlqaxyP+aA(sv z*#9siThU^vqm&R(y;W6;aDw%L2HrnPq~)QNEKENDalVaN@?!1$D^ZB?=e6zz@{Cf5 zf(mJl2e%b2N#~1L5NSX>6}@a@Aj&bCXa1LqCdaL(jMy!IX{^4Yv9YviZ~kv> z8Y{ghz^apnCWCehtCUwy3Gt9T{^@5o zqnt8NA+%PLlDJvxISnJ@5hc~ko3r9EZ~4*s+aPOu6$OHz(&s6d=0vXxQD+;l3Bngx zvW(e+diE=BX@)?5!LW~vtFt>L$&{#~)Qoo0J`&EiuGwUAOSd2sN@Rb`ev~-TK5RRS z-atv9X}-W)ae3y@?8vJ15Aq1o*%eCob9_meutUZKqcP*^>fMS6j zzsWA4H5auV4}tzisHa7q9;D|YV_Ue=36QB!TvT^RxX~1ea4H{_%l`*QHd;7N(c`$# zo6>d77rZ!);K<25m(NOi49n(NAp#f}E^~`p@?aAo&CxK^uE@zRH@+;U@F1LvC#jne z>aB&_lcbD4l};#PrX!=zu-^cFhg@DG!1aNKGHXQFh9Cdhgd=+L)r}q3Cv(wtR(7i95N{ z?D>BgK!S!BCgH;AO73ZRmLACcjNf!vd%uhHi`a)zLV5)S`dfROQ?_9N5qI5CGbWOL z((20(%!ZIz18@Z7ejG~MEmQZ;SgHbhadn?Cn^q?a1dZB6W~|YK3u{4N*BNy;Qf%F> zj3;w9=?=mJkS#xW{1tbST#eB18zXl1uRp8SI8XrS;HvI?Pkb4DGsx6Y4qkSXg3|bk zArC%A{iHd^rNW?b+%PQ_bJt^ErSPrPpYVbb;M3KP{3pLOKrG0MrM^YkCn2UV+b z9hSln=|cw2OJ=6-N(`M|*GPe7!yI6!K4?AzEz!~&5q3y`Oj%1GOtI0UT;=w)D^`E+ zHY|}8_RD$#<@{h3s433&4!y)TSG<>|#hHBdT}F<59Tez?l3lrJ=KO;B5g@so?{Kd& zEy}K($d6*!4;^;2Dk=*vW##-IsC9BmhhHzFEC;UXl3A&IMtPYt;#O)7qgX~<|Dc+u zG9eZ`?nA@iYovli=-x<}^=Bg9Ht|#u1H+Vk9;kL;sA=*8b<9ikps# zFTFXwS4#;Yy7KIOep_r#JjPcNw_#tZs19TNZTwz9pT_qkT0a*aQ^sCYP05Jciel}E zZ1oK?k%l)neYVWYc`=A-yNM-8+3k2Ivc3mTeISP!0R&xweD>=wL{@5zt-uwg2)Q3_Wu^TaaL#ILQUv`$^B23gn~w zZsW0st^VSyG8tz1FZjv%#ul@kNeSshmJ8hM@Ou*^i09F!O#=w6z!V#k!3cacBqOlp z7Zv?FI=_-}DgfuaV{FKehfGC`)+g#tw#H%0N^g{_26>72shrf~~7e#sy=)z1Zy?WZw8 zZzYYLRc@<`#W~pE8LA zs2NU6FQP6uuQ1|>^q6(0G;P(OWMk0g*O1%@;Q+elGdxAPb@qV`n&={;8L@vw6YwnZ zMd^+ao4O^#J9VZ_gOK^F#6cnO&ko4bCZJb)9@N3ehURZ#X*5Cnw~4FwbA^djCHG(T zn-ayH*{!rN@RzR@m+Fpal>heWbu}`gmAXbnThV9`RetUP_slgd$m|yRG>fpS zO*+bSa)`@xhu0c?uy;JrzfG}LP*AY-;w58N2>#zbAUlK{aY{N)w`NSe)PtwQe{5V7 z<{>zf5xeiTpZFAmNuAPdDizSjx2Me@N^aiX*b;$M7X_Orb-S0gaq~r6uJbucyTKD5 zgpb1{n%9Q+fgko37yLiSs3e-a9Nspp#vf6Ox9&r9Ir7yBX!=zkaU0q}0&w9m7aW1I>o*fX$Zi%|E6zKRXrQYG$_YPr1#7_` z-4%|bLVReg$hWupJdP8VCH}Bg7cjiU@gmspvITNESckW@4nlrh2?Y|$V?5ogRQEPH z8h04(oPN;Ik66wPck*(8763bp51U(<=S_5d6>n0G;=V8b8&dYwN@jYD2VuKdn&+0C zeM!|13<1=LioIvuH6<*%x(dpy?g$93iwGB+;{XInUgu5m*Nx4b%!t#Y19%&OeRkhN+E=Z`z11%g!t-E zFQrzRTF?Vt{=k>Yi89^j$oGV`&?)IWU~%89)n> zAgQoRY(0<|OeAAaA>kRs9YX$KL@}hsVzKOZqdEuNdRS*17lvWF_do$C(Y4vQ(0TQ4 z6TNXNEd_;RbMEi!WXU9(N10?ooc{@*8;dnB_Db-FKZTZm@m70-HcPsbgB^~a6q5PJ z8Zg4!voGIZe68dEfha0c;4))vPp=n82VlM#1$trn#c_!IKGiQ6oxCpd#+|0a*X@)H z$L7xv{Eov^a}XwR9p)&Gbe<8u$|y!tALE0)+XH~>=og~bne<;lX>5zP6Qjc>C4JAP zeC_#`m&Z;@v}pw_$PCV}K(Y64POSUx0uB+Ftr7fx{&Cd9XO%u-lx-sk;*E#G#wyuc z$SV6ZZGN2RQ(({=^(#V=jJ2QbE$;_99&3R`dgN^kz67;Jh$$xR$swK)GfcCwvTeSuU47ge8ny}>{2Smmz+Z! zTmqLz-c7mPQkd_r&JVTgO1wkaZ<}5t3xWSaZ3Z!Gogtvi|L03N-4`FiPsEx&Hz z=oCL^dfsiZ2_RPy4ohcIFGrV?U!A|f_K1j#&KHuv|7`2pv-N2J|NTd0yccbid`e`o zpTB3UMY-=hwKPZCZ)VbeC`x{gz=LSH>RbWNwQhmW|LjSnO43ORn#%j9EmM}#v`)skUB^en z@SC!IQZ9Oyv24p#J{Pec&Y{_mj2DPIJwT_1O%vH9X#lnBm#c3w=dxJ$?%+}c! z6C@tEM|1P@H1B#!!dznTQG?M#w@olGq>;jqj^DC%`Gn8zm49#BOVfo9Ihq;}v1 zyjXI@s8ftC%@HWJET`zX@_#ui+CA7g9{+I$um3x+v6t%%11-4bm;Y!^7*YTgW|Rz~ zaope!Bx1U^skJgOYgvMoeH4~>)&y4(YP-0K9(ofZ`?%ue$(cm2P&!q)ln`(-+e)4ozP(#=UdGQWF;u%-XzHUF#_1?Hq zN-KE^&8@tbURe6`yG|)N_x5mVMM!#BXDd4N8a1b_B3K+xpW%@Pt!AwX#=hZ~RxV*f zXNR%=VI0F5!8!JANx_a=(~By6|GArhKMn4r*Ep8x9%=0l!WW3j+I%UA@r_Me%`&?1 z_C{XCAzC36@QGy4nppX3Kg%m&tO6-UN8oy!x%NiDV80BW!}uke=PS3PO}cSyn*h3~ zzYXzRxY(JPbWy5EELgO^f0aAY;R7^Z{}jaF9T58JSvJfcnW&I{1^Wd1D&3R|42)&_^tS(be`%ju5M8pR*PTSNGcpZG{!dcPF_66jP~^Gka8;?? zTL=}1?KP3k^g@a)7vDR^fA{_5PdIjj1=|`@ZajtIrR2}iu0R6fo3f`0<-evPff}B{ zK7qEaZ^1`WP^RDT5+JQel!ETMZ>-pHA?7k%iT*J=_RksnR0A+vZk?lrp=M{{n}C~7lKKB&5at=~`?+mqhD@`pTll|cl;qk`$l}u#d9x}gCimwcPRE*R4XVPO4C{NFI&<9z z6Sl`|q6KpBJ_!ExO>qA=LLuxyj+THo*)A23Jjtg?0Ncy8Yv>0DE%U$oIZ@MXiER+y zl~iLVl8$+RLb!sFh@?u8t2B%ILLw3c=QW1IJDzVmm5HwYP4~w3*jtRT`-2&ePbqia zSHc@c+fT5IT;3%tiBJ^TtVhZ1%Qe_k-PMQXH1~?6J+b405{t6tu6k^*w%X_3cSWm1 z9hT8kwSs2TQeLD6m83sOKikqPW^)ecm+-f|Bkzzx&>pTCn|+qX49KSJjMn)N*RLY5 z?O~~P2f;5LaFoBZ1ze5AccPMnDvEFnDJ~+_Ozvx{WcG_viWDLL#h!3xy&e!>f=R@+ z{e>?g2rCfFVHxf?cK@vYvzjofE2@)RrvAV8@HTp{lX!bNyE0VqJ%1KQOm_0T z$LFzdRv>i>prxvy+qRvF_099V=e+0p2j<#y?RD>at)Es3=Y&Oed?HiNa3AW1C~?vs(b5X>Y14Nr z3@wUPy4S7oo%b0;lc9?SAnQUc?$Z05@>!N+>R6$q#fmW@9Ihk2)27+K5`G0Rgi8HR zoy+dekA%XKEO#CKaC+@RCuG#KX#A_P)*nzTI^hP$VL+8VlJ`gi?Vh^je1Cz15!q{L%Fg2^tNj*JvS$L z5m=^OPbIG@wpnR{5OYLCpxoA_;%p0`3TW>&-kh?q6_>ZLl@z#QG1~J>Q~nN7FMNq> zMREMHe55G8)&kB1FYOMili%JAz%Kzsk=J5`JM9z!MFGz}xSjI!A+-JQbe1WPFHyEa7?irqn zbMDmH3>0XV^}YsI^7}Er2SEm!J(Ajbj@Uf^z~bWm80j93eMXk9QT{_{{=UGDk7)7f z^I5j^xn{A$y5I;B<5bt9@F?iIXEP2SpztJ3b^HFWX4BD=F~I9m&`wP+KDj$2O{ zj$y}e&+>=fM$8y(y$WN95<5~zoPiXJ0EK_kT zaVOkA^@0N+b6-FAcSRBwHi|UNk68LtjO?&+{P%<9h6;%+rZ+qL*t0TH?QB#XB~E!s z*T6qE3bE!{CPB!pI^u_ffWdf5vnSIdih+(G4pUC11-FnqSyHx((<29+80RI5oJe-9 znOKU zH(5?r{-d7%MWB0jn!wD-qZOreEgL+)Fa{CVrAaA=m-K##8BiXU9|H*{15+a9N)PX& zvtT_+Gj1bQiLD@@YQg9OHE^JrNO}<+Aia9VZj1REI))2u^@nhwy7mnYJSc(EiQD|O z{uPY^fXK#Eh#zB@3rsQixo(5TC)50!t4EbdTzZ~1U#?6LpfH;PbPOL*npyZF{b%9` z=533+zeAEEv0iE(!$bE-=Q>y=#z0#vFSghwS#9121GArDXF1GLguq4o z9M&SsC&m0=^4fQ(IiSyc6y&>(U^N(L>U}iI?9_y;K;@3$A>U zgXC!rr^f5uqx$Q39$j7RO5Mf3?ijc5hS+KQ{xiJT2KHHj_j~sZ`lRLJi6!?{k>?y> z<)Jc@q2XU5Y5Vyy4zmacFIH8sX(9d++l?FhMg;z6xbQ$rrJnDH3Y+uP-HSS<2sAZ+ z!&z;MH-iVYlX54#DymsaP!G#-Z2J}}1@iHd+(oB@C(YVndy!k8;%wK&kxo9+Nr}wb zQ8ok7P!U5?3>2Yb=@qSo>(g_*o%?=|UiS6hH32aAFXmh*vhml7n=a6u^og(vT5 z9Ne0D7HNaQwND%FBhrcAEtggME5@wPB#C@vS7LRO;+ABIf4Gw&O>8_u(BiU^wlo3y zQjAFk4V$Xp zH~%m27{l%!i+GSY(YI7Zyykh_1$`D9gPS(32UdDNjA97kcT!hR=lYr)&SC%#s3$xb zX}63`;3+&0kO4KPxq_=;K0UNUwVyY%vP;@sxJ!Un3l5&naqKD_I0gOdpH6*;o>5cyONgSzi3@xI1zY22Al6NU~)%wXauHuMoqlp-fFRbUEx`}Z1k(YH`Z?ggr}GWBpszF?LKHNtz9{+IqRGEE(BxV$BjF2MAO zc;qK$3xi0{D+XS%!)`@zw1b2A|TnQEb}On79_fY?@#&7mgi*hgq+nrw#A7nr8noY zU*%5L?vsX7_=|k{s%9IXo_7GPa%CSq`LySj@Bu;);sKowu%-+-#3b#XzPA9<&mkf)NuW$=#pK&jkVl6>s$) z*16}Ozdsye%d)E?ojlHr*Oi5f0$9YZCrswdMU!GxU!+h_DTsC5L5_qLIOZ zsF=XOQfz!*6s3Od37#{4Z~7!tH+RWv%nK?sx9JZ9e14RP-tvv0jwZ9(H}=a{Ja^V6*J zoL1LfW51s$Qv||{beea?a=OI65Kol-$DKMukL_wm+AooYXp6hg zQsZRrg2wtprUpJFmdka+3q>@jJSIOSJa0s<6#j-;L+G4vFRCS5fx&!+E=JsVvDbKp zYAT@y%fJl6fweyk*9@;9n`z!9;7STT53VgA{B<>3*)NX_5--<;3Ib0L{-OC^@6+Ci z4oChE=iWAC48V;A4FjIv_sUd{bT>OO{j?>u_TRpY;L`Dak{|Xuaq5fMl{}Jyv06OS zpU2Idc_Em2U$Ya}sY*Vbl_Y!GBH+chTBZ%Q&##OoliB?=*$qfR7V)3|>$45xQI)v( z`l+w;SQ%fuh@9?xJLU?0DW(@-!@bdivs(aNE}FH%VEGeD#D;Eunx3LUKD5##S$b>X zxfuVe_Enb0kNuLFngN_tU`a>MC_rEprFz0p07|+INSUb8f(6lu?3Fx^Hz4je#9*{CLOaP)ELkol*VsL#5d5u_NV~Umhr4x&bLngINclks1=YvtpBMPMOCb zXi=k`pGmk_XNvy6aGYT!c8_atRoaE`ZoQ+7i+HQ)`RR^iwzDiYU(d{u)RE>vtW-SC zkvM753l@@=IjWhN_mM$=*uwM5kyBL=S7-CiuxR@jWSN&hddwg zXE_Jb!%3{I#n`->0oeqRT7^;Eh{p0VvOIs{oVs)8bt%0GxeX4r29+JF_|2sIJB!%x z9WM4&?!x1VHgT1xmh~u#7A9$cMzlSpg9s&4HpENaa8fe!VhGQ>V_zuAXtj2CE{UC` z8sB}|=*;ZAu`6dg{JH<}w+jOn+}r#ey7y_cv5+!4-1VRwc1rImb>A zboVRwBv*A3b0gz*16CCo&~|$uaHg7jklyY3LsV`h^lp)+gQn}2P$r~Q4#K*GQa-K{ z6VK46;2n!9lhhhkfm9QE7F|;AA8&jo?RQs2?VW;tikx2({y+0?P(7G2@1R$(p~Sk0 zp#JMm;R*1$f-X# zv#U4uYiG!sBi>_OOIvEOS4+d#gR#NQH^k*Iu6Lr8H52;tAyt`;@*|eE@bCW}z*&O!S+Scj1Qd z?}dgkH4Le*xv3&QjCu@48+QEo-?^5h9J>5Q|A*;bmcl4_ok1p21gdq!*8M(SnqNGe z>e|sGi$ARXww0^S)^h{-Ef&fJ4$rd>?^UDo$F_uD46MDkkv$<|em^?G{kF`BvR7^R zbh|@K1f{;cf%9X9#- z`l)sGIlo21xxmlNc{uQJov3@z49h*7lJz>11yq`F*X}Tk8PD9L(wi%m3nPwAnvV6j zSfN-0IX0^J-A6G9+?TnuUth`-v4_)XCQ~1{{Jo*xJv$~ryB>u68Djh?PJ|QuU9?U9 z4I3#SVIb|@#)3srXeN{?|9Ju&k%oYembkM^cu1W^wqEH5MJ0Gt6C_N=xbwP=pr^C@ zqzK+Qu=7qX^R%a~)dq4{kzh0{A1O%_*6Qb6e-(`#^l z7~SB0ryFR?uS_?DDJ`oO&{6U#f9shx1vi-=it!Q=|A)?*)n8KpwZT#HW=Ddc4I++D zjl*}Fj8!k-u)vBARLv>R|GPFg5J(42GNr@$`oRT_ARz5A(o^a6lMwio}UMc#HFzvS}8abQd&pRo~^*6-ae5nzmwLZ znj*Kyy;-3v-dVTFv!b%1iQa(LW2>K@mWI8eOz$om8>rhQcn{%ceFSX>p|6j_tBfBq z@YuvL8T@MI?uw~+bH!Gf8)*07XV#}D7s(qwkE2|=F zHwT!Y)5R5aC+>ou^#&XE(lsBRlmvCRp^WYOVII}-6Z{X_VQIoX%?z2spjkR6$ez5Q zWCcP3`oNf4Vx8aQ#Z|90gNcUFzvFzZ+aP=MEy{MeL)*`F%KOjVj^eHS%ko5-{E<#n z)CU4uD3Nkwz)?rbo(cG^Ksu!OSzH0zrbL^FvZ(yR{Wk7>_lX^HE@u$gPd9c;g?~|+ zd-*g}7M`7b_^$)$7Sra618|ILVRTh|rOay*og~7GC6SKgb0td2?8>|=_Pj9Z8zH7L zGcn(l-EFzW824)7*meb){ziB)4D}F?1t~#y-{$_3ojvJGfpve3}*Hyuyq$at<HUttM7X^e~-!J8JdrH znKXUc&x;&Ik5N1k`FrRY?O+^Xz7jgTY_hm(Sw8^>uEUPTaV6mi^PZ8K zWjDWSTF8q4k&;TFAA?n@;<{in7?mfvq+Hl@FUF%^56~$%>%L`Si&pKjqs^wc^$rR* z3?XckO={=>>v;eJpef<7H$RSWUrkHj^+K#?rn8&vba=7(NTt>X{V-Pwnuf(TA^uId zS$t)l!D$k@m_bHkt8(z5=vt^a2~)9N-;GHaCk4`ongEsiJj zAxwN=J{WlU$go=&i9QJapcE&#O8m&GDC&MUZIEp66J1C|k54sAy4X1WsCika6f>w0 zvhReD(la7WA%T7oSAHtj%nALYjWg``a7;bt@R7+|Z#yH_Zj$(-WAL3|rRmoS-H(=F zfeJW333NgaKPCg`4uH)UbQrB5I{;M^^;1Ne5p29PUZ9L`%~(_p%}fglm@ zA!%+Aa9|Kb5iD6J(WB#Gb}Xt1B2Jm=`1xyI(m;6h?dk}e!8|nfcS*Cfl~S}-py|AW z0(h)_nd7X8xLEFQBd=vV2nc1|+BiB39PFKIh0p9u@IO28WO}|fiAULuvt1wIe;*7n zU}l5xDp&HM=~w*-<5Y9$u*k9cm`qJOceekYI@R9Cz`R==*EG{G>C(kk>Qij_*D5m! z${WnDtyX%-7>hDNnQ-vLn*AXXGzllr>_XH@`w5jmNl=02^K znE0TylKv_3-@|A9oOsYEi;*mtMc>g0z4x-5=I$q3^}H=I0X;iE-UIsT?=F0;XC5v0 z+xNcQ{nxBz$bn#(Ff(x|C^kP6wU5@P!&Dk37XTjIo^_tqdV?$OwLbvgIx0Lo_>7Lz z)D!k+I@(!P2*Wx!i<4m05gJ*}e-OhH!c@d$tE)@j%4?q<1bkU|<19z8DH^Z`dgO`n z%NGY3t1{;@g$)Ska2R_0J&`6(+Iqa)px_pqwcB^Gn3x}kl4LWxHMKCa6{r?Wd&2eA z0F|2U4Y}91934;VJ*nrC43E(fgoja^>?JBSnh@!C^&ngi(fC3Ft<0j^hILmJl60jw z=4l9HvTA9ZU7eOK=Jy7r`KKZS90J{{>UK_}A0{-64Fc1of6d_mjgF0tiAXKnT=pI` zGg0fG3z&@#3yxSrYk8Mo@$>G-pFAkdSF1Ezf9~PEq)5cG*sKyB>dd^kGlUZqaUi^B zhc_IjNGF|Jaa$i_AuPNa^o1D~Tp$ntu~mBXXIxitHr*aI9p@cp&b|37i!Hb`4Y4}y z#niZJ$ddJ+3%E0+90~!~D5LAjFordbmd)%5Mxzu)A$CHxwEw}bVU=0L;QH7RRk*Va z_#j?PE3|(PfzJ*M)p$bQ_6&lboH)$D`vo%LH=fFKKWph5_L_q+erFp_E3h-;s8)Jd zFeLpl_+o3$qqSq8)~%y|!2YYF=`9Id!71uQ*kRFDy99g-G9z>!>t!-b_C+6^pvMkE z;o`4f2w3bObvBR;GsGEIPTZNZcBx8P`1crjMgXC^_ZPWvcl6Bq7Mj?yhmAZE$7y~< z8WMk{?;XCG|BZ@Z`321~VrT+}ekcv&8N9G-jo2TlF}bllA|TWACkoZq?i)dIXM#d0 z2~_wSM)tOWT`r#}fnt5dsDIjh3%62f^+bk4>PUut(mzA;U^(|nY!nN)|7nxf_t;ii zLt)Z#TYYk2uuhfpcIocb2R&!@?7L1>f-hl?D#A@$G_o&MXO_=+{O!%q8Xx-o zdK9Vf0dQ8NS%U!sdsA(hs?CUV-Q)T5c?(f2JfZ4h94c@DEH} zhJrt>8ogioXn8r?{vnw>z2J^w6zd6-dWQoA-lEJ7LL6y1K>~6ux?m+(Hii6KX98kB-R!}@muKD zcvOp449YrtsojNi3+F@2V!NJ|J7zow?+1^MbJ%we4iX`NlPa*#6QSJ0A&sR_p1_Uv zD;zL;{cY`OL3kN;M&g;OpjR)~jeXFqb%!I|nDKwkCJ{XZL-Kmr3O4Rh#%Uj3nGKu2_Z0{m^Nw%4 zzkD=x31*nC7JL@wIkW@P^2)0E`fBLjI3)n zoG%c!LVn0Fn{N}jlw`RkY~Zz+zk5Hw`Cr6s@i>%cHn}GF{SeL3SHT;PrE}Pkrd6dJ zRyNROQb&1dQAoGAim+Mh7JYblIDt^CIEiKg$A1hb_{Ec3Vkm;2u!8mz-cl>bvsAhp zBXlve2-bkh$}IOHE4WYx}{v8a6@nqd~npg2g9_OIMG=Cf0@_5GC&*1+;oYosFb~;e2sfBtf+A zUyL=;#O1v#}F)@hgQ0j&52x`_l5_TLU}EWSO!!OD!qI>Fx3usEa4{WO3}^WjH1ZfnF3Le zr*$D>gIXj4+6GRIzwl*15u`SXP|q<`@BFKrJPVBtx*pcdpKa*fx#fP&iRl9qQhyJ> z6Yy&&6*6M7fQMao;;}z`OTV>Q~k*K}9ftb3j zB9N;5lxCAT@oO9IkPD?1UBUk;AUQ!6)B~3VnHyrua#oBp=oI3f<3e0>k$$#uHT22I z;L-~o9oRtPI%H(X$uuNF9t#3M+-ec6Dg1g=iZ`#|0|OC)V4(}L{#_4n$hg07sGDIT zXoKwnC|_iEDo_iLpimBvUBHp9aUzC|gt`5~*m;Y>eX+V7BI-UVTX1f5<_{GXL zu5ANN%xNzHWYpP~AF%-5VFRa%Q*YpdKQNe)|8d0sc(3%`yVGOz7nVClsJU)VAR}%G zzPj+;a&`M`2KDhLm}u%Ejj{;!Z7=I&q7`WFIHAd5coe=3e|zdGF>|&E%dB^eVWezW zGPVW3G(oi1p$G5Pg?0ZR8T$ykAI$HbInWc=4RAQOwIqv`+CfEG*)JPKdoetK#0sU{ z`r)&0ye-fDDk%Yp=9^sB-Y!jZN%oq=XImVq&Ma^Xe`Ps^z%deCr)DxeqI zv*oIh=|8=8i*Mi$WGw`20Nfu$-F=M!{9{W3V6&h0Nh)0D;psq5~M{oOx}zrw_m zGvx)$GjkCfmFJU%M49d&LV|i`R=;Jw4XOFJW6wZypS7%hdR(s4Z%q%W`rJp*`a+qS zYif3+h~g;J=#4WKN3B7d4MPGaPLex`+Xdl8Wit;B_^fk!U_Cr`RTnfmBky2e78Yr^ z?HgR8xSBPpa=ZzaDW+m?pN>pUcZfeHdc2l;{$sxQ_?Tn!g?33lK<};FJx7#3jy5CC$aNoAM zrtDJBBNZAKadPI-dGx;?ic(M=!EQ*!KJvsq5&n9PaQXI+K_LFj_b&lEF_|N!*me{n zxEi5Zb_bS9N}1r$8bndcRY%>Uu!>jqQrM)%9m|P>6C`#oFsy&`GK=kg#NpUxa@BOz z#m&CCnKYvr*UxhwB6q~ji-|isA6JFA{qb=m)vrVP;U0O!+1E_pU`Mq-hYV(sZc@ih z{N9jY8k|N@%xMUGLo7I#^3wHdmDJbK$eT0MceaBvn(R^!bj~rimaF6;?u%IF{jik@ z2RUWjd-oku!{`r?Fh|QzehS$@ppYeZp_?4dLE$rlL{UqNW0_RsIIv6l_#G8wFqhB+ zSIa!8h53pw##59UAe``@|1*Ch(30~i<@7{-vvPf*rg1dY48d&{*i}P9Gs6t!e>J)A z(&ydHpxvsnWm5XC0T@LhAqGd*auD}t#}8Sw5Jg!(_0j{`tR^6pOQ1P^3{agPA15WI z)UeP4K4Y5deUw`s4Bkb;N9|G&-Af%lZtT2Y0_Jv@Dm$Rqe1HRod_%!;xPHw!4 zrvJP^=d*akSiN?Ssr=Mi%K!WmOmXHAGu$59{7)iL=K5XK1K2;?s$ zzOJ;5iT;Lg9^CgQq!$9Nw!p$6aX29{ztY2npiDuB+eC91<6-o{mGyG5G@CM(n=H52=VfW+j-*5Ys7FexF0?ZGp3~?D`iU2FqUU<#|HH!U&H~Vy4K92N&pd$=@(ckv;brzuCu~O zX2ug7k(n;WT1oofWtIp^)|m-12*dmDv$0Yoj}Q46*4{|3!Qz+cx;(YRd4vI*Lf8+b z$~wPR^`)LGZb1RBjU7Y=Y6}CcdXIi;-uACiAEzFF6m)!>J26uO{r@!mpC+kN7pwv6 zh^opNC{Zes zDxoSBk;eFI6zL?I1(F*SY?SLgmmH@2CgErQq9XrP1>O_QLbF-Kxxl(47lp>@wue)* zeiutf1o-zL=M80#vB`Jwde6U3Y!-(Mx*aSK(@^bk@+%2)hIrG$^DMf##(B!+x`AqZ z>Qo*B;*RA83zIM6q@$fD{SK?e_mR_Fy1A59iRS5X z6N0)$%2vAuJsHU~=(o`=z2sP)bQGhY-0T9u)S6oSI+MIlHGhWMjrV2h&P10QKddHs z?C}hTNJ}8n@wUjkuqqQK{MSLY(M-bIi)AQp$E?*lWZ)7k)D$U@B*B560#IPn2&vALGilv3ccgI{gohM z?1JqyX^AX*)sRUN;>C4cX18UcJn68{Lg*jctuCZ6`a3FTH_AY7GynZ(tVz=a4t{_! z<@*fkSO;#YcS{|W?Vw1>LJaEzw>+f#h!?QpT5 z4OlS2ZM>su-1tIt?5>E{TdP=s5wEFGM-A41fj}hbui|-)<-lLd? zS+~FBi~O?W$C#v2oB#ToA7E>FvPLGj!>Wmy{-&{HV`OmM4-&e&6Dfa+;}f`z|Hkk0&FqAZkue|-aGaahdt9PHX_u3zss5jnWunE9d4enqhGZ(gimANbE{Wp+f4^&cWp zfw=3qv|ul|!=fW_QMlkCMPK^k%>IU^4hC`(8LA1yVUb#WNyo=NNh@zTTsu`IrHCfh zz&XtngNK2LSEBoLn)6#hBaRF8Wz3pze#_+Yx@3EYNO_o?w6b%ac_ugLHtRlLXf~v2 zjF8B{LbbbvgUZVSBare>+!Xww6Q<@w#tO-QDi?jKNNzabTr)<2iTTo?>=`o`{LKc# zH*UH_CHl^0^#~#aVjfWlPGLDoHgR*&!j)piL=4vAC=9S;G>i^WG%d3FscOUV+%w_T zp1U0?5VpOI8S`h;(wg#p?K;g`E2;s7r%|6NF;+N?feqx6{I!Xl@V{9Ap!=RDA3LqY zpsX)tuD>44!a_{uiXG$r`uXc*m^aU?uY^$WWX*VdCTMZx$=F4^>o`TpPhnOuhalIc zxC`{f*#^d*Ti@|NBnZxGT)xtKg?=Dag8W5ft`wm$+>rF<)#?&6x6r!9RlSb;8+s&# zN&(?#=s;-ClI#42V;gzStnOKu5vxgH_8!0cRRS#cWpWtzd8!(=yLDVtXk$^MiB^z3 zyEuLOVN974YwZ-ay`ldLSRAWc9=O+nWs&a}1XKzMim&$3)a{Lm=BX!Eq7L2V=A4@q zQ=n(Q0Ec{k=Mx8FfL@$4)H`c72*Dc)toUVD_kArl?#m|XI+@H z1H8$NbNgsRAN!2NdrxHX$7I}T|H*bSonjZt+;2)0v=(!Ao(BJmz2lo?7(rr`ha6Ph zSBOQhb)j{zRbzM3E*06RNg_MY+^jpv<5FJvyP(1~%kDP+p!2qBsBg?{#gE8K(IZ9& zTCiesb4cdPYh5O=kr38GPTEM1y)>ne3A7|cr>%p#b~1)h$@ zZ%Kr=nB~n+FsNGZ^dXZE53%p0sxF7Wo^7FH@BW5ulfRjbzc~tgNL7bibQ|_lhU*mA zB&haJy?^-V?`Q;pV81?|YNc{E;oH2{n7rOj^j;)qWnY{^@ow5`sN+JtLuRb8NEl1b zVOS72;hfTTll!LKeRMO%vjlay99GCtDmNJ{vI--^KuJ-1CJc1R7+!pZUxJ6qKT-5Q z>)q~DP5y&&7_f}9}(Tqs2lK9$t)O+`1+h&a#sdA}>2{S%!Qdui1_!~3q zCubj}FxXBX;-@`zXa?q9`r+x{NFjh6xSoYimRz_^gaRw>ze>FFHsEIKe|4Pq1=xWw z|8=X>;p*3JyQ?T0t1gA&V`D;e$mn5qivPtqh64gUQnfx{}wwrB|eqGg?em8~43k0e5hUc+~>Skr9%#1H= zxPa~-cZS?rHXR3IGgrYH=DyLD=F)t{*6HIF_waJG@k`8CWw2y& zY8?uCNkyRZ(sf~sb)aSkJ0ufq=G(Zc!MReWCw7!_s}zn!MP$SX z(1s|X2NqNpxqQH#8l(u3aF+kug!zQhaFF%O-lz`jrHPheJxzjD%0vU}zzHy2w_}3D zdiKDSvxA+kBk`yi%^D?z^`Q1^2;m&lr15KMtt*OGsT~~Lb%+Z4f}Gbo z%&jsu>C{5KO+8dsw2H%$GHO-tnQhSuQ4%exKJG7Y&gj`R-YJNVWP4mG)KHz7M2YFq z>k(1Q5~4Yv>G`WUt8-Mt_(aAm0{G&5P3&f z(s8i52L3tgz+u1Iq-U4Ifn^L}lY&;xbSbh^>g{gq>Q0y~f~$O%w7KkFMA z>r&FPfpvCiPmaY`_Wwb!!}rCb|6*;+&ZEz*u?1n1#1er2#kzcn0=wWd>F%cNrPSuNKh(R==Y>3OGbEet>_#Ilg!5-g9 zjpCq*VRWfPsNp)bZh$xA!u%Hmm5v;(pW9=3^zU+sIwMpNg5Lx&J{?cXz-=*~)a~hel1yXq(d;Pe}gV^rMY&`;HmG;5=|Oz_>~LOi{ZAy^HTs$9p;$+ zya!(mTmqY`8a-V3r{I{hsj8o3uKo-6&>yzTc3-n~AdcTw3iFUn`3WR)QbAnEm&Lqh z5xc-uh$iS5b_7S#5m*c)2`|F1cHH<;G;Jp%rU3*tuA!iE0nH)t=xA04(@nUa9jE&x zNdbDo!MuyR%5MU~L+j+Or#Mt}F%rG--<$mweUheRPr|kOQ@^b}0uXs?@h>}$d?s_l zGpTCWG;ohJ#-3#U0vR;=F0|?c7XzZBqyMZYG^-^vQDPk-TI-94t2DVed=xbmaT;+p zZnvi4=G1n+KCWD#jIVgD5iG{Wa$SoWj5DW6Q>nY$;ah284FT@Ss+3KC=DjHsFf!_c4! zlK)zX7D%F)$rj|QJpEo5*|ET)_h8ff6n4_HmfqK@b4~`nJ-_I7N8@l-ZCkBqN4{tZxi#h zQMwO17RR9krR+37`PtFPL4ak!{%sBC$z1q@HT|5Db*7W0!O3z?F!L9HSzGyxtA})y6dufyt$%NlT z6Z5#DxRu#RfJ_}4A*ER#LdvM$4Ds#zz4C7j;AN42QXOA+i~wfNm0;}zWhMp#3hmf^ zVwHftuPj+dkjehR3SL5sZBpjnO=Jqbb8PMWAI%%;LfR7Frd;e@&}(sj{r+_qiApwd z1l0;F2XWYocA|G6tndz)Y6bp2J;(V3ON`2SOm@;iE8!APKU2}o{ZCyE#X|G!TF#>( zUd-_%4;DOZL3_bO#~91@0_=&+{B?s@Vag+I&fM-fut7{_bwIpK6umOND`^y=rS zt>U_2;Q(#Ulxgru=$NL?8;s78mT)(XFA|vpYZDTDA!#aMPC>oIc_`6c7tKxcL`LSV zrL(zhI-Kc#UOEr{Y3qC%!M7Jn2SqVTPt ztZVS1$O~Vd%70!$N|=KK;>cs{(%W$~4BwImSa(B91@o7)s zYQ65&)b_FAdzuz)8n}wqqSW-)zT`NnLsQ|WCj^EMTRO{M2{^z`BQm3JOXF^+a$$@j=9_7bq7cVsTWxqq!oF`^R~U*_Y3Yr!NqG8$5(hp zg21uMdVY%wg9m4IlhWhZqPMBuC0$vM5xWL=&aKFM|Iskg4nfN)J|EN=;RaYzhdPoB z46~9iK8MN#fjp`uAB_|V@iggJ9NA=|=4EWAgjSfrbJc;bcr^Onm`cH!e(cwkB_BXR%_aOf?8kR;0wrw}&`VI`IKXxV&DM-$tOtd~5Gazl^hy*S<%>vC5&k9yp4I zqb4P>LEO<*%fQ;jXJ7Zt) zcPkE5|JJ~0q}mCcX5w9NndeWWY&6>ZHMcmbB}dNK?9=a}L#E%u^*$lFv1iZIMvC1g0p;|#Py}@3(bVWqUSXYzO=lq!*Ay1b*a<5-yvfd9RWe6Mf7PahKc|;OYQfhf+(aF;waoGxE!%Evz2wFm|6!Nj(@C z%Nv=*AfTr))baf_&iKue%l1r#M2IXF^6mM4J}=MDm#W+jCr4;IhB1Y|=`I&wixY~4 zgIq)f7(UXTS}|7M$|S!oJUHW4T#M$%vBS=KwW2{;nPawG<2j_I$r3ad=4G?GdPnD~ z$jWH2A;GOXGQ9dj^Ml1O=UHsEr#b(#*=5zSmle1odDMK4V32iH_W68-5>lzg{=j=~ zl{#4e{oaFmG8s}oFNUA8Om-(t!E?4c>?2b>PI?~fB;0LQT0(_Iv9al{B$ z8)j6Ek4YR}8ol~2M=XX?+CEYep9uPZ>@<{#2nv2l*f?R&!eV=rR6c_`4OE_sBOKdK zUOW(8;M=?IUP%&2i8)KMdv{w?w=6#t-k1>aOLTNF zqTZe=KEI|StT3~hZ&1XYGlx-vMb&Mlz~1*FW?Td^J1!DO?YQDw2@R1O7$7hB?QK)X z!2s`Ca?eo#qHTbY{nKkVGk zXT7FgoIjx3s{2*`lsc*E-c9UGu(R5-2B@DktZuGfms6x@TKQ*Z=g7RWoaNY-zg65; zrZW+-ryDhji5dO0xw(td{-6W=)q(?I?;o}WKad`$1GV3Tbm}7FjdTP}H)$j9 zHxE1^qg$Mc`Stl`DYHra1z}LM$zSk`8W@y~(!Unsb{kjvkw{6<2+oCtg>Im=xJ)%> zH3L>&Fs2eH!uy5)mIGp2Ui=GRa2|kiWy&Dvtck*&$VQ4Nj#P3dnnngkv`qrDY;**= z?5*c(*m%Vxx3|*N;4$gAk4?Slp+W$nBtH5I88z85Ji3RB9~>{B2n9*UJGoFIUien0 zz-;UJC*w&zj#k~Py|AIbkGRfAaWA^SNykkuJokN1()+=}+<;B-e)CmIko!C1+{4?E zT{DftX~RRas?W(+mxmpqv4><`^YdSJT1=g8!%Wr_T1{|5oSo;c3~UWCNNaYrDwpBhr6ulVq-Nj`f#vMQPP{xd z6ikNNl9ndvF@dR&ttkmZ^iy( zVc$Ai!6uVqo9Kn;m(*&Ryix{%KsLk z9>P;=#yq_=NsK@1qNQ({PA(rYtKi^8E+*4*GO_7^dajOsa`zLQvzFWMsMw*C-*-e% zGbP7Lah8}~lrPM?J9NUe)2*L$X9mCP^BVL6Vv(=qp&2m41_3*~E(y7|d`!b8L)p+7 zx~D?3Cj6%ccL` zSNo6VanZX`>N0!X@eHX^b0XX%On(2^Wo*@dY49ad-^l~5ZyDJJF$1F0g2}l+48(+6 zxfeLThV$TANBTVGFPG4hJV(iv>@nd^XN8ytdeNJep_`;Aq1 z`;ndCbJa20@e=?gEfJT^)?Uv^uEJ7Adggt`R{ENgCC2U^LVQ?ir;BBK7l6#3-@e}_ zjP|mB`?rv}-za5E{BmXFeW5aO@!z6+A!d@YQ|GiC{ zsh%qzBRisI8e^)0il&+0Q$V&0k(Z#rJAawR;8PZprfK#tOng34phEhKvI=&9#s(&9%LH^?7~I-5~zxl>mB5v z#87EuaeN3_di~%|F*gBq-k%F^-(QT+72g0$sF(++(AMXShXPn+i@VIcPEupiJN4{O)ynozQ4KOSTT4io|sxk;3S9pl&)8wys^S$TSpRJPO7{~uFt!PN$| zZQ)X!;!uhctPP&x!Hc^~3dOY)3-0djR@|XTkYdH%y|}x(`{SJZ?tSkUWF(nm?Y-vu z*4*9csZkh$-=gbPG>J=5vQU^i(FjjnLX@s)B>N?Pdk^keJ%cS@%TU^YYhs1$5AFZ_ zAe!g^V&O?69B{O0h?m&ifp|$9F4HA%(2v0q>rgO2EdF7QX@TmB!9gtja4;hAt6O2o zhrvzTP=({SB)^=~731$tvI(GPEjhWmVe)S}DBVshIW{K6DFBZaB$RAfEx}UYhd@!*; zgb^nYtm{p$_s^tNi}@ltHnz)X8HvODG>=XneDHq8@BJ{kpF8;Lf&LnSXJtyRBpjvTAU`K6WN;iyP-h} zvK8IV9;`ZX=g*LoX6neAc`oUk3u#5nki_QCQvEQj`LS1dKf5_>lhbj8YHDUO>&az& z;twJ);JB3nFtcEOteW{V^5(||N(>`ORnPpj=!7g$aPr=8dn*vzU{n$|J#jLe9p`?2 zUL{_^U>6+3ZyWfa+opS4QtfdH;>XZ7zv(+@6#M;`Mr|)}*EZVmSr$t~`cq0DPsWx) zjDu?pm}F7tIxg=g?r}0>-{6r)VWV7n7`ErkF}d^6y*@W;{|pV z0Eppjf$2O(p40;0mBS*d8;L>KJ82o|HPK!N!c9(kI95E)otfMd18F9lD5_IBDfOgS zltReB9m2A~FZC9K*0)twmD9v37}3!gjI+9*ucFm;@&YTyME7UsWs?9fjBT zQrIDU4?03~`M|U?QcmO43aitt2Caet_wnRn_H=X^_Rj>fjC$5a8~@am+|V^HYdpVH z*19&?mUR=)PG|CC+tKzf<>L<9){e2H?0J$6+n!OEySP7!^St0w32*3X_(eem%>)OI z`SBDF280x4zw|-mVP``oN9tY_l1ux^gvV=9B3jI|#E9;xUF4ei9r`<1uz+x( zOB_hZq=%Xaf1fD8Ap9vGe~gUql|RabQ^^T=ma$75s7aeOWpY)zf9jC33iU3FMS1O+ z$IQ4sQofE2hacq41=m?^+Ej%B=t@tej9&5Rz&iL~O@&YF~t9SRF%y!7rUz|^=1$~mY zbGGS4JzD)+V)SvC?N7aYU`0%QWM0X$te1@>w40eJNU{_gupLto>}RPbcgrYs*81wCy`&f(1ZpAkl)-)9 zpLhS_dpA5l?*Gk_4*mD8cSzG#LM!2*$vQ!)u{fkH#HE1*do@s8A=3y%!g zz3%Vr*YX)Yd=E!jF=ey#+@9o&Py0dp<32}tHOIFv0$W;tqCGdLnr8bl1)Gk-Y}3Ay zI4Y&0U^Cq@G<;&3&lKw$Ram_QeoIKa5MNRIZCS-1JS%_ zIxF86F)N>3$#BBuyuo=qX-{GRIdfSMKR-Em@<%!TH9zU3TQ^96=ou^*GImJWmnZ~7 z*k|6mmewg5GS~x%f*cIL4`3PJT8Q+y4m5nYdjFmZyAX|6Ck$Va*CBMMkx_eJCuZA6 z*VkXb|1(@OMtbF=e!dw!o&Y;4+N>b8%>W^f9{@LiW;!d&G^_bM?<8Pxs(#s8;L8dA z!Kb5Ou=m(Bm*eB^6FSsmK`;0}R^Wf+2fkr$Z(La)xy3rB`#z-NO3YK?F@2tR9%fGh zz!9h>LnR2_O9$F2D5CFSKr<+X69uDbRs>diuHKxfT)Uko>^&gW09RxY09|tC>AiD@M-h_QKnX!lv6DEAMlWOy z$!rq}!RO5sPWJzkPo*X(oRoUOM~ju)j%Ev%zd}iWP|*;YY*83}GI~baaSYAHV%&1I z(@v$w_uu+(5^Ib5gHCB9lyX=4scP?)!s@EpFFc5EJW_zs_)*ZgiPBKT~pA!n?66VDlCdz zXbLyMHjB>3yI=%z5u2a1#N0v&)HR#qC>fZ+-G+ty<6kIm$81fv@y1r{q1{L;jY1c;N7P!pRdJ_Bk$w*YYuOI*vvx$opAu{%D1~=QmisSfuYk`~Qy5 zUr1p9FN@Jz@Th^G!O0c>#9n)i0?{@Hu(w2LVeK@db?b>2XV~hz!|uTGd=*{9?vI`i#!j%Tz_u!Kl6K$1pWVG(DT-6xhL8BB(X2&rUF-?6V;eu7^| zYNL^p%n8Leh>~=uu#YN|3uJoR+?>g>3fFs-*N}G#kSFr*G`bo2H%hZ0kg@ROnX^jd z7aJ{v7Uq$SmZ_|W#L}AkskU7leJ488_tEPyi3?Mtq{F{znFlc0>)Ba?1v27b~4c96s;&eu3)G`Iu6P{BDo?$ z8=0`H_IS{f$u3NoQeR<{SJ{K&A8GJV3g!8|4$4uMbN675{H|q{&3uBf-_j7~j@Xau z#gvN;6}KIeMt%flz8(N_#Ts5kIQ?jU3SLTBl__5OAwbRxqX2@wdvL^7Q zekGmENmTUa-%VJ?a{%k*HI#koW`4Qg?QOoHVTV|4zkK^QrHLjQ*Lb4I^dcn{;x|htWZy(j z=sDVDyT|Zt&y(l@reFPmas7mgcio!8h3h8CS_{G7vZifJ%fE}uVG{o=#e2oH?TI2S zOLWoD^RK5StK)yplxCOR$~;^WaAB4n9K z*?WlL{5+G1fPa};CK3w5rc*9rDVs<`?}efvOY2)g7|)(q2!9I#iAk5;Yp*dQ7S?XU z(Q!lWzGdo15sGQ@XcI_g4!Bhi#`BROXUc?Kgf#v`b~L3Ge|~526Rz4HX!20w$voBv z;Z7z=seL$%B&~mm=grAk_(+AND68b)C%(KZ1@TgHf5{W|-tXn1?Sdy8^e_stmlW=+ z8kUQ(`F_UzD|>UvBFffhfx9YTtB0fGPNxHW@j*}coB*RWA0L|+TT~&K{DSE_zAlQh zVV=%kFEkm5Fr=aM1;?FiTF8<4DCWxR+XG7SL$ffml6RRw3u6#B3Cf5=*L^9peJXcW zg?A;IUrcsy`5VDTy_eQ=4^Z!jzcyV1OeM;h;m^AnHm&-CMU6JisiLt{Kp7Dgl@Q~L z&pnn^;SbReoD%LsQ$TdO+G}Qa8|ZhK^gRD9aFYPwov7PoS;Wz$yhM;qnEM%Vb*`i!+R#0I#`3i`WKT0| zoA`*y==UFxn=QMR;Q@^=Cjm|OMs<~^){BeY)yte0{2tHF{fo0}FT$kz7FwiHYdNv_ zme%Ll%;e2|g|Nr$T9!xL)zd(|_}^{$S|tq+)&J>m)tS%s6}=a%8`guOu7Cpj(l;A~ zZ4ERPt#kSxizCcfSHtNgqy~B7db5e@pJeP2aqjtLfBI&PwN2j5aiWL{0(=vN62}^& zQM&mmuDZDkHo_)O8+`4N<6DC zQoI#!mEMZF)Pv?h$n4g92OuIPX>~VWS?SkBd=yv@7tpQ<+)(IB3c4w3Q+e&*zr`?* zU$aaOLm+WYONJ_?`n79a_!)Qo?lZp8vd>S1xp2 z+SObCwT0R`<4}qmIS0_NkL0Ra__(TkWk^*{S#j~SqW6V`{9Ft~rp4pcu-_bW&&ESBB1Hy&F(NylZu*V&9|*v>|#U5P8=!S1b45oFLR(zHo>S*Xr z)D3*Alg_+hhwtduLMP!|&$MFBWw7}dAaS$isev5A2Uc<5#e78)nLR8}9PjJO)f!t|g?m(+2 zRA@j>hZ>DZvXfN5^S}+ms(V*9XSt56@7+~|Gqd94%ZV&8eDd;VWck&xIM^*QszTfI z_kWA-o9c}w;8`;hHVGt>h`yupWS<4=_`M(`3VpFX3#P!1lJo11o!>-#ma;``5plkk z{)ofZriMJUMWu;)Pnh@Z>jsnRm*l|abUkE$enZYoUdSxf2vE42=ff`3rn?|EG0BYJ z^L|GlFYheHmjE~cDQFBVd$WP;$V^bMcxUBjhCN$SpIS$D1UPZOC!QUT+>8RR5VgL% zkM-rtU3?OJ!IOG12Z(wC*V#-$Xoo42ub?95_2P(@XUKe>?eNdKFd$bE%AfS2I~6(q@LcpH92a#T$+FXY0Y zqUo;`-w@bftLQ$uNtO~JqPL+g_qx;BpSIRb7-1WpCQH?r_2V$QkyGG!QCsuK^!4jr zy|{>lfz^;ks$(Qgvd z8akuQCyDKrJldT?5uvN(kH=1mLMqRt+b}2jRyp*SPb_W&5-FQ9AcLyGsu%#XBtDyP ze$@&Qhf=jCF7T=@sOYEP3wRMf28b)BWs8b)PW z(Jy`BzB{p1++Y{Gb1(=&Xyg8XoZJLk*T&zT?>X_`%ZXPWPPVV_D|WhD ze4F5OzP=;xtIa+U>S$<-A?Dx>!Gk2Mc16fKf8!|M(+jE5eJ_rR$& zKQZ(1Z2Xr2(cc8`SZB`Xl7HvpuL4OAO%sPxZyOs-_zUCI6ltBVl0lgxStqno^81#d zU~M0|OI$I)Kt!_z8gU5wo&qw-GOhlg;v8d0#OwXCaW-~yudPmtS-Y17N;?Ht7s)x`<6&=1(tDE=|6 z%Yp*WDWdza>vv1Hs88i{Yx>^cwQt}bLRYugb5_z^5`j$bPVajK7d;20o!~bbxtD7N zO+&yPtvrvDE#WuSQ!a~4^MtKj_`k@iUYN+YbM2M~3S^wzMSgdKiZz5(8L;f*RAk4{ zi0zzm&k@IbiEvz+yRz5d2EJuw#}aLT`AEWjjoIGnNqPR+(^}EfdN;a2YRh*sxu_EF z3oxJiz;t|OELda-5482&A8^s=Mc;OYb&oCjN&p_U11?34fa=x_npgOF4@%dIff*J{ z(SfDwB-*$rEp&X|-~QAKbeKAOP8GI2>00pL5{NLnup=!$w26tCCSaQo8N&UwS9>6;8jihGWKuKJB zyNQ){Y%%%d;%X zzD&PAuv7jHOzzdahVXeR#jN+YgrG+@uaNn=}frDhD`1Y3E=1=#4vwRI&wt|olu8)nbra6CvQ9Iov(ZEA2 zv5DJXijN0h=(*rVMmDbi993uE)zh9un+1YR}{J$2jc1V8#2G~WS`WdfOJuW zgK6iC8+e3qW9RXT7(`M#RN2Vbry~tdjN3)6QE!VV^C&R#+AJ>yt~7G;8p6!Kvw|+5 zt5qSB*JGIJap0KXM5d<#sh-Ml&3Z>J#F)zImf8IX2J0;?)cxT!xUG|3JGRU#vlHW<;P=+0*j z07kT@EWuj{^1&X`3Kzcz`@r@7G{h9}*OOW9b8^d_iO}1-!{S>@x17j`h(`RVC~#}s z;1?kw8`-&tpeV3HlClf0hNb?{ejT`F@#F21WexXpJf`5iyQiJXDS_nB^#4n-y$uUF ztkG1u$f8;Oj1)2SpU|ZFDV4kJ`TxN|h(QC4Qie|=QKFa8hVPwX+n(0<3==k>#TXOs zZ;Z30K;<$qlc^emaffp0DCFo%U%X5XyhwTPFfHWM4G{2)#Cr$l<=395eQ(g{MZUZ= z6lI`>;#reNy6!HK-DL1-Wf0d?2@K9q9of$kofUrlj$=FWOnsI3@@IT<*lRf8(W|XcZ9eD0^V{g(5`c2>LT<=(c7UBtU>)+zB?Y+Ea&Y5YdR`a@*(!vk_z9?IK5&sLq zf026do^K2yo&Ap&nsE?RpBnA^clVz;726~fIkR`VfxG2WycbbG$XZRfBscg&W@@OR zNH0TlH?6VuV0Ki?-tYjaw|NNYEGEX%=nEaM*>09goY0Uu8BFya<;0_ZB}lXx-D|H* zv(8&t@+uB@)>*H9XvImVzk7>~X&xw8KPC9m7?e?E)!thh;ch)~;sWi+@Ki_}bmk$V zT9JMQy&OJs)z%s~PjmzuA21V3pWGol9<_PDhgAu3Y+@-U*LSsA9KtSo9--H;N!3Dr zCDW3XjQZm{4wkbsi+;q z_@MGQ>&*yniIMtb1mf9GR5Z1xSj$|}94!M)M2M$qEGo(83&a1Iv!KAG)*X7M*Cdhx8o2iH0m8+t;a#iq9qh%e8FZ_&?#r-f zuD1xTCb8c%|7Y6MRm1h+0&3=-?T@I9i!r{iX4U7`5j(?RIg$GaqH>GB1)Zb(f0@N z2zKqp1+AgI!JjRy~AQ+*9D?8us`1OilPF9TzI?FQ`w$@%8UCS`0;~>ap|OE(n>| zBqmo+OKHEOfB$7kY(CfWs9{MlI_G;61L};){TD9$#f0%)JfBU>Xb9&~zI%Elsw+9_ zSdI%dLPoGdccdZuNe)-t589OHeEgCwB0SZ7p4( zUPJsMhM@{lDU$g3WAPd30OQFK^6ljKUO}*$cGe%>0h8pKaF1dlq{v_}+&YxXSZRLa zbb|cq4?#`v_`xn3v4%#{QH4$wE(lh7@$s)IFQIL=US?*^;R*dURWyIds?Ke9(HeK= zi!ygYc*l04y=vIF2KYzKlT!7fKSye$izc!T8e2hk3rkFWS$vpWcH(k65KS|Q4O}0W zHDwGOLthZ-V-QM`f1quv8QB{6b}gZk#3mFa3ui*K!H)O6e~)9}t?O%z}{KC9@? zH&JYcD&o~FPU14j<8csgbx;ACUxKV!jO@OKG1o5&@)$J!zYf$}GnBUqpWfKr!G2DZ zeZ|?aap*572Thmah4j$?_+ttkB*nT7K+yAOR+KAqg|Bu1zC=AuH0h)}qzIy(eTHE5 z^jTLW-Sf+Ps=*kK$t?R(laqK}?{?|BkYd@h&Cz!=UE%nptn%uqgLC(oQ1)=pWz zphTXRzIP}eypFOsIE?Daf6PBEm0@DkMe~~V!fRYc*6*CppzL6mbvcaZs{P9$MMN`i z?Qk8!(^wtjQgD9AeMy})$44xlAvF3R$uOO!Pa)Y%lHgqa06L5`R88(KnG_VzwN()5 zWTW0;}gTh1;X3;K?X)!P3zj_cQqnw9bHKyK;I7|}hg<3xx z&NSByL|59TTuA01T&2(Mfpv6_h}-6m+P)nC5o1~(VD z;zy{r=6PoMN)Bf)75m;YBtAC}~lxcA9)6-9@x2(;{q1u!rmXMGjE5eU5 zzj{tWx0Nqqs&AV0~X)Ge=bDC5QOswM}8u6zWzE9Z=3srp*%&KbrLOR z12I&@MFvPv|4Zs$V1DU&=|Nj__d60*ya?FRtL0j&;)dkCAEptfwjMywfN?vfCel#- zT3?%rPicZnDI3e~w;txrvr_TVo$o>vn~5FaRzr%g7?Z(NzIfEOS&Pe3qg z-stDtX5JB8o>A#V>|q?tVfQ~8CCfVJfd@R<3U0`T`yz0YiD*4#uyo<^ zRM8lul+KZ8v#429%q}2{kwfTqBG}Tf>zOuez-2B)x#mT}2OH5_%RI9Lm`kPED=}=`x8_lKI^R!f+H@H*4p#9g#=bKZC z?w_MxHs0djp<#3-xkM^)w0Qsh*m!DO%l1>cvR6rU?|KW*h5r8mK=?EPIWJioTMZ8% zoD?UDjPdDldhxq_NL0cFX2-7!*TLokSTf`~9}|@lKYc6EP!I{y-A~aVF!IMY0T$r0 z#YAtoF!L)yh~><%97c_@_uYeT>|QM_dsYB=Ff?pEQqYS!I~rbr$|^dSc-~=B{7SK7}1ED&}wp1I7RU) zmQOi&dP5X19S%8|FGj>Xz6WnU$Ic?GtI(P-7WWhEv$BK#^%l%$AZ%v z1$pvFVE^kRFQf#N{JrYA zdT<+vuXc2BDc{ZG0DmSG^R-9|R^L_u;s?e4toPXsowBgeL* zJ74E=+!W0IdHoF(GMDCp*md!;n_8hUBG=agtqk)+6IJAALmz%HVvG{xJ7W=I9XY6u zU|V*DF_eXD+8}^kJoH##S=Z&Sn=Q|`ldFd)wKKu){#aY@hCw`i{EqfRC%uOym-ec2 zK=I=a#o|h1-Pc&*wk6klPQKD@?3)W=V=RWZv zg#@rc6yZL-5T7`OLR$6|bQSx8a@h8~C%^9!Zc>?M>Cz|2x4}S?|$y<+HR#W;Bsz)b2h=~f7PY?sv$XMJo%tOgp8q>z3}EOUmELLAD$D3`~e4D zO=*DXoHwj3>aWG3^BkhCwjpzNkj*+e;l#7%)J%sDW%mT|`JU_b31g9Jemq;E{0^fexZs`uOU*qJ`C-XV=wiye(w4>_j$A3Q};#0y{Qdr@AAVL=pds3BV6TuVRv5Q zr<91XhUvW$Is__gSQ%m{J1U;pyq|{33*>*Z0NPv1fqUR;RJW>plk<1!$@cOWF5giq zoYuC2=|Efl81zE>d|Q`s{BdaX%x_~(T1}PR*aJlkj3q#Yud4~Vys5%e03gOAI~Cu2JJZDr77CP{StCkQ-d9h}nr&@SeoqMR2WXAv@iOEf;y+&<~MM ztqiYfW5dH={}#Kn9mS#ShuL4qxMmXI;kvm6uGoA8(M;y?uSJN`x#a1{FJDN1_rZ?> zn670)X%Nz-Fg^xrbl4gWc?4r^Wg*LdttXBQc&#+g!i=z~FJLyCf6C;ux_Q9Xdc1&; zA+L%iYZcA>mv?)cue*KsdG;n*eBWRE0TLlZOe{WHeKv`pNgzy!h6kln`Lo$TXxc%c~1cuFKVV=cHII-PLsScJS6zh`ld(mE5l(WiKD2w&zhAta+r(m3a)0@GEvv* z#8SHE#AdLXRJH6lmB@`x5Z5%-eL)d{1}X(-{Ss;0q@c zDum}Mtp{Lykg#otsDIZElY*qH=E)dRWNQl7di+$dXBk%PGT^ZRV#_Zi-QcC0Yh(6- z6zn!1!ER&6q7`quYo-sqRZLJijm{3_bl8sy{B1;LbyOY^3Bs>_`Ndc`#4>4Q($DeA zftg2PT&Me?j%=)XVRg&4v8bqc*osL<0T_u6vxyT42nKokV|BQW!_(=-&SbzyPRrS? zIQb3d6*4lNrNhe7wyqqX&}N)GFD!MO_Me(Q`t0b^P<;!)Y1vGQMuKt}_xc?Q}FHsPtF}2s_98 zA98(q!;{K7yTN%9lYmBeDRd<@$R~eVpGr!F#o{`OL|egfY%X*};AU+j3EIn9MOwqo`e0}=x-Zv1M1Y#9uG12smNa#lfKJCRT+aj* zK^74EeC2b&QMAozTNuc0G@tp`N=QSj)^^Q+60+V!PvZ%h@;1NmDIX^PGqBTW4p{sa zx)p_!up5UXqmiso`=&6l-^Hp0-P2t#?>Yo*0@B^-koo!De3P1cN?cF|NpyIyO-(6Q#Dbpt-!so1 z<*Rli6@QgLyYV3pQFte|qCML6xe?FW_C1%@A5|^B;{;)h1y6kIsme?Y8q_?Oepuk{ zxCPe)o*7b!3YL!fU}|yr1D>`RnyZz+o&ROo$^`H(PX-8O^_B1u(V(li0xgA!8jBFa{Nd;ef=5nc8jSueo z!>VbNp5!9o6$b)G@1I!uo_&ea2?F>V%$9o60F`4xF37s!%NB3%@T`xwr@-gdG!Tan-8*f)$GRWasbO#ak%nT2T4#K=E}bd=lco?{1)6MjVht@`^L zWPN+LZEQrU|3^4IqX_EVY^EA58{DQRAun8^_N2 zF6?`Q8HKbWC4};t2NTze!2DD2?o}reAF8YH48w=L&ZT$WpAr{8WpK#cKdv7s*qg0Y z6z$5geVYIM064vV?Y}OHJ*=GgFQr!%l&dOwx}BlqkQ%XB*-j@Ehd=?v^Umi_wkeV&yAna#v&AIL;cj8+FEH7AuSuZ!(Ts)0HGddGelMN%4HukN<^njy-M@q{LW-K5;P7%9kTh-GWo)EsXhyrx zvU~66Gs=A;P}|U8e0dkszLNo{NA7E8&qBJ;=4A_MXN@)|c?<||WvCxouRgE+=LLdI z|1=a|ih;5GBKcJvpHy_*IiGPe;t$;H!gRONC(c*RqqgG(>Lv);;dpYq`T zG#+UMmIWAaYD_{bK)GOk2=kBGltk;$s)?ZnrLeX1^!VE{8C6fua^a zy4T8@WYme#sEprf7^bCQ*GC}WPoQW0{J-nh#h}c)<-p6+kRd^g^Ph45i?+K&FtxAu zqKKv=hUXH+ip=^1qrUp!;N#p4alWoEhy#qb8$~K5yqr0xWKWxe+|mC zXfqa?Z5m}iIGuM64tN(dXy|jdmRBpU{wP7{Ty`%*Wgz!y6_XEkO4dP$El=7}0pYsT zIBes_!(9R!8X6=}42=1srDjZpRqJ{k`<4v%{rUxX-Aju)?!>?18W8#%a-Zh=bZrwg zb+hJ;6lIsm0A=!yV<~~NdN5!#V)=IYXZ)@AaON`tsO$6XTqOXI{-Y5e3sykwtc+c{ z*H*cU)m34~g)X^7babdEcRhVu@`EKePG77eIKo6yMPT6?u)VF(fA)KVZkToEZM<%$XDrL+2q64pbyj>mZAk}!th3+= zb!oq7?gQbPey#MdcS8RjTjAs+Yk`F*^2FZz4Gk7UoW_f4gnYx z7M@WFLJUUP*!WAm34pF+C`IiA3$iT~FSrozk^QTB^=)0kzZ;7l$cBq%xvl~trxYL0 zLjLCgdUv1{0zVK3V^FqVNj=$Np|1l?3P=&Edym?*B`VjA3}3{VtXc>c?ST7kGf*rgXF&O=VY zf8OyW!#1*odw$Lrre!}>G9!3RFP+~{xM9nJE6xtvA)(kb1|v02Aw}+qil;`}M4jb8 z&`WI_m`+1r3!<5o;Oglw#FQIbVvvLT zH#$&`zHX}NDT@R1VL_XVI2?X{q?$o;v5hLZ2v--?cAC&`dW3tNu{-*~drEn$%-POp z|C`f;DXz(T)a%_D(Z}24y4kja)p^WW9sHy5R>2z~uMvp64s<@;%GMOdiQGFOarPaw z%uc)IiDN}ujRw5R=9QW5^~}iw4CoEW{DJ#R=vfhMkyIPuG{jv=@%NPL@lnWwMkDn? zw#~Hd@%w3|&{KR{&FA!fOK)qxjgc5NsLrc3qfm*0%R0f^tvt05+}8b!vRJE;{05f5XN3rvM`hIC>|1b)+9!Qe)dZvf;%Xj@8RN>zO4Pok=13#7 zm&uC+=cLiM^bT~dJR0SUcjg53TA7mUx+disFEsLcvNN@Ju*BEcLYVqX&SY#a8jvvp zf8^x_w=AG*w-ebkO{u>MDBuOST z{&jb;%blDP6v2&$^+W{q@1aMHyQo0_-2dihleK%0e*f2daqKuBAO%qW7`CKoeHLdvmnF8K|EkxY-eI}fO% zzJ{B0YA5r#I`qz5tNA@~5GJ`@TJf+k z%53bBnw&aT4`B>y2%aJe*tWSlO8v#{0QPzVnA5pEqJ%v5KdxGX?$4bU8k4n&Mt2VD4S^DY|8jsjQcLL;vAeQ$n(vfgFyA%zfA>>5- zi~?c-GyxAynCT4js;Rh=?o;31S{gLs-%e(MVDsL*;{5(eTPLp&Tp`8b3yr~B+3|!8 zF>;E?V<9#98(z8Rdtcnv-jT+JntJU8*ashCzCq< z`Y{bH8ffd=0-Qtsw)lHOjQQ?Z9IQ;s7PA5^?a#e|mf_Hc&V6J}=VjLx&%NlC0Eh_? z-Bj=SV)q#K`Xq2;By|4k@8Rv;G6e&LMhU)JBtX#`}i zmCVPGKNA#lpOM!iHmCG^?U6o_J6XeP*jw9R6zabEyeGGcOl5myW%Spt^=gTUXzd>Y zkIfCdxv<#|T&O@yw*hLYk36~z_;^*H*jXqis`+`>3*+&l7}uTTno(ii821O<6+WA# z#f=CGRw*VFu3sCi&thbq>;wk{M*$6@g!VmJbM=Lulj2-Wm!w?wBPeCp>9ZTHEVnK^ zc031U|0~h6J1w7Q!M?XgPvLK3+WX+Ui_W9RtUOR@joXPf?AZab(2QQJiN~+$VNka* zRec@QzF!_p7xzhnbj3zTkwdgGgaIi(?`a1I_z>SeH`XtVM|uD8ZOnMXiR~;RH;u#y z>b%f+P>CCU%h}@kvTvwq3`63rq`s2Vl7f80QL@xqZ@r&tpH~$&86COM_Ue0z z)iOKo?bOFT)P$aD9-h#Vxd@lF!SxRe-yHNZgg{SHJ=x}KliTWZrzczGIjf{ z73k$T!$UZis3?dI;G4H<`dHV_8U#tJ;vHFwIoy0%JHU6ac^DCMb*mQ5M@4uTd$(f! zpb6*Uie@f51vbs_QbibTXno{Mhi<$)wp||H95L!yCi8Eb)8Vx#YcS59YF5i`$6*!U z`WyE?{=&6<#Q58X@9BUSGIqb)V8)=>C;kYUF0aD;`QN($L6_I(Q?ZMvz?P6A|IdeF zA#7SW#hcpXU0}gut&ZOfTuiD0LhP*cO}@M)Ih)WF9jYHRxItqjSJ3@@Xv@~E zUv)el<$gmShoV5-m4Sv1s$ew+!*BH~3Q={|p034wUQ!yHnJ9c*07K8P)}&s=)8HT3A{)%bIJ{p)4%2z_fASA`ht9Az~1rvJAI z$}@j=^SttE1cyWG{pLmLAZUHuMI<4zb6X>U3+n#!sAJd z#2{?nheS4U)dWu9R-o+wU9Zadc@zMAz=N*Xo4y`9X zKL?~!lEcD%PP)HR1k=8w7>ytXvqLE@4_8eT!Rr){-E-N?vt7Fv2i$F4y%Lf~O>LXs z%dr7QQ+Il`T~Dp<=RN4N8mSDyvIRZGoMjL;)(qKRavP!efdQ>wFOu3*`4TB&;)lz@? zmxorhhnAEmqf@!J;;68wJeAX%GMTtX+!l8n21ZsJJ6s3?Oj{5ps0`f7GDa{fmi@l7 zE_5>p-Q|B_&ownVFDb{1I4}GOKUrs<7V*-!4K+iQh9m~t*9JkuNA|W#r>pYEdL-of zajxlTH*iM^0FL6GBKlO-TMB^0UG`E}GO38_6deZ&N4~~MP-QlI?+ZSE8kd>_n!RR{ z>|SHWZQF*@Rwi7WEXdCPUtRM~&zE!#+q$!r=HG`SAZ{BVq6Hbr;TqpydrbNXnEU>< zJDRrtuW-JV4r}!QQs1Lnrop#g8JCPD?UH_M9^$yVeWufvU1w?8q@ANDUJ1lIguC0l zpu!-UbkL6cm9|6wK-TTre8?F#PU}AMx4|8XBn?WO8}zatLNS7D(=Ql*>IzOU$NowJ zFT5XC==5%qqEzkv7Is{Y(4C4-`l4u@)?U5Sv>9*F^MH6^&0z^b&xdkY1IojdQlp1b zAw5B?0Vp6${5~+)9Cg(+J^kgg&gY*CIXgyt?gTnlh}?k9eNIE_$of~u&5Qb-)wUb& zd32TbT>J1tc5m1=2rGXf z!{Mqtpd9I&K0XcI_CXWl<4Y?W&$KUXA5Z>do|sQr7atGzGrl(>des$(?+)R4x$}@K zoEvp;`(n2KvcB#pu0i=xaxMHeBzg@2?ODt(G2|w`pyy)zun4|KMReV z%2M;4q$*1HvFvCpK%^hjv~r+YxVrswR&w3?1NPH{XL4xdXVD)k>_(g8sL5bYemq$g zs*X0r=ywB8P3VxTORd72Sh-C6oDW0EL$N8}!^?xGq;>Y$Vtu(SKuVq#>+W6ADxizn z?Qi-{T1<&Y1ef0yozvmO(b#vdzMU@>CsKGj3_c3km!qziKw>Yasq_LAdKh>Z^GUV z8&f<6v6F^Q1M$-m=kf;lqRCT>UKxEhGp|rq!}-oFLKt#?2sP67p@H!u2vlw zZZCTKvRY&Ch_m3^K$TE*uYy4j4L=Bc5kP-i8hBziT(?5ytC`h%abjv_()CV6nuF9X z8O`?DOFzX`=WW(G8`|&%hE~!uuB`S{G)~KLOcK>}I3cgF3fQoMP+u;8e1+1^;@N*? zSEQfcOycmVuFvy(j0|-L=p~-_PIpQ1z*PCIARVmCbAY)cIPIRV5IFa?-iEzyG@hPE z7Ql1s@8Q)={rxbePLJjnp3Qs@+TgjWzV4<)jV&XzA5X&a3Y?4E6=j{A1o8ac8L<{*qn3t9c;hP7PG&igr#7)qgG60}T+K-uHW$Axd}$03+tUO9{NrTMc5S z?YLe|G;GDW6ddEe=$G19Y|?LCuEt7Z|P0HBzO)o`*wu7&KIU0G9 z%*6oMYDjbuH6LmK_T8@{sf=Gn_bSpSkZdXy0j$EzK1=Vr{%fwwQ+B-cgS0<*Elv)I zz(>}}^e++q%4Z83_hOzWJifE{gkA3rnkoLuS8%@9F899guhFO1qgj8LOpONQb(67A zH#tAx3pq}!Vh=PCqeQtSU+L1OBAtYrV2+0cEklRz^%g*6CT<0HpGh622c5r1;zYiL zVLpUnZ-{`0&}L%AX6RtObecDt*UvsAZlk^1lh5YU?HjT^`yZSusS(3d1OPt0zFr}j zzu&(ze^@U~U55HbteOrQBJ#J0uRbW*Ib8Pf`FK#+w<(qCZ~yxDZO`c z0bAeqcY(vr!B+2;bl(a7+v)Nvp@zy7^OL*d)_2R2!MwNMDuUj%r2l2FwWcmCy3h)! z4~vZmN=N#kiIDz8c_M9p836XlzW5i`(hRvka_9X8cO^zV9wMO&^sx%;pwPu!kUkP) zMU#^$8KT$F3I=Jix5VkTj`Y>_PZxFW?iCc10}Iof#s2VszEO%NpG4k@9LwX=ew{QN z*ceqx9U~w3vmI|RAto6%>(U*7CWUbus^d3=DF>Pi`5slBzuk*Op1*S*m=E=7Z27$4 zuGmp}#07v_`vRt`n|0nQXr^L0x;R4K5KEjte-clArqkZPZ6}?-wU66%OuRk^wTI572CJ`HSw5zZj{kYN7tv#x?ef*bJPtwgRq-oHm_$?~o8Rw5C^HPnG(gmM zQjz90gAIB(2kuRYLB6<;XWzGgj*CeFoK(7l=Ju_F(_7`{m7fHT3=uD{t*_f;pN_Eg z@y$EI?czJan)s0W75}FqzMq{-_Y9HUG$E0fSFKra@LF4$`c0pPedP}!9tVSLSvD%~ z4?Umhtd~a=RAj@6=}XjK4lr8;YUo#UeNHt>7Ypn6uz($((I3S%%syB}z&CG!d#jgA z{rBIV&xP3Jg_CA>PauZL(?*_CBW*nH{$&mL1*=5jOfuvH1RFRagQ>x~_YMLk@{yZT zWP}JV`Dzan?csid!m31 z9IC5NNpIrd(OCUN{X*wcrfbjNl{hH7F0sLZHRgReovA5Yju5`wu0zpDs&u)fHoSKK=9{+-~E6(CM)*_aJO!DUHAYY9w2h74MIKtmn>(kH{~= z%R#+&Z`VCmre3;IhB2tiFv1MPfE2hX_`yWcSytAjni`0ZdB0x6zU-PBxMx0Ss(~=V zK7{t3Nd&u$ zqUP`DiQhn46==K^;AIhcLV8*`mYCsu@8Ku6iV#v=;HZC{?YpybitY10Xmd>hE=dcD ze(l}Z!kn{jzUU|Vy{a80XOZMyaqAmYhM)U}-SclqtnoiOkLt8;jUl1GP}w=*J*mN{ z!MrXq_8!t&Ez^9~H4KM5KO9PBWc^kpw;gy%zse^c5{d{i=hlQdIm-2je{{c2eegGdCFXJ#3uP#>W{By`! zblvj-?DS{ZTfQ5@0=CMC&2pgRs`G>6qPv$?l-YhY+3R_zSefs6L*XrgCLi2PjCSGa zukTQ!mM<;k>yO$xE$a2%tMBxmu(B;?{WaFbqy16XNB=j?1Q;O#7k9hQb$+Ih2s@=n z+#h~MhmE#iJbO=KaB)wJop`wmZx(KED@~*HOYi)8y^JL8f+0neKQ+?%6Dmm(mJ67% zUZ*N)(1;8setNlRxWeB+#>BRLEoEw+UmWh%HkD9hISHbRx~7h{z#O#R3WPs^0(HEg zEJlN`$H4*w!`GK%qghr0x855^QJhs$*2anQUZzASZ~)v}?h9T50wh_$4?JLn!-p>S zF&<#qP1inFpEt;c&~u^a&tXDNWs9%VchhfmNZd@pTVptV@3!qkUkb>BP`|YNi7{?D z_9EKT`D(t(<8#$`PuF?Hqd~)&4T4+jFwcHVCw|v_8RZmMnBcFq)wkEN9QP;vopSpy z3lBG#&7<1$nI+%98m;>Fy9qm?hubhGQ$;-t0K4)LdnkZ9xaj*?#Ai(Wmmzksdd0~; z52#D1?XQ^Y!G~Tqs+K^X#!Zq>c5L8Ycu*t2KKl{b@Ba9VV?V+<;-&rjKz6qEb5`UQ z{oqOIP!C_<5~EuUa$Hi(9p+hxoKe$Cl0XenUwO$Z34wZFY!sVMryr{N=ya`E@%8-o z<`wzZv6Hh0pBMGEe;y0f;9kPEP4al9DE=Mb?t)8xClM7~8=23yI?o z=CcoVW^H-x`PdV){}4!aMbT9uZ-_jEcdYy=^{hf6$KtKA?eT%i{+yr#MftrJV@oF0 zQdb?mqmG(BK@c?X2XEo#o-&CjgjvJ%xFhNQN!ta91X_(1ODz=@ z-kz~3k`34Q>Jol~8_)BG(v$azH2Ig`6QS`wZ$o9kDyIKQkqijwyBIUzUR;F2wr=o# zS@`sVqPp1LL1slnxu{Nb#Mo--_N39t$=78B-6@q0ykTGZ!lwfE?aD8sh3JPvk)**R zuyn08hHYn^pMdOR5)Y^nse9c4?zBWFHxZ6i3NPGq46E*=Mo;vQ@Yo9dB(j!mP*`67 zo%jr&x4-CNYInsq+t)XP-_322h|TbbiCxTBi>{b|CrtZy&G93Z<&IH98(#L)P;W)=@u zOc5MVdJ&kdpSKH#Rf4z&y4|K)=2^ei@_cFY!8T1wKdA4^MCCORxUPy``gob=T-@X; zUU?j%zVF=2YOYn1(w9-~K;+l@OnsPX7o9rDww#dXik`dorFfqGsuLt8zS#Zta&}k! zZ1}QjE+BW~3{z$D`ZwD>K=( z`HIGe7vGY;dcU4y#qag>t1EuMNG)D8v67;vLR%rpl|eF!iOt-fj&DbYH*@wt=zG=) z!dbV+@Q{JN;ee16Sh&lfuX?Gr6U$CnFGerR>YWzZPyfVuZg*%~K5-<~sr<(=Ik+ww zwGWqjzZ#Ss9sHka&tAhC+wDG+Bg90T-g`nj)@LcJjkJMWZ*kF}LhRekcQ$=A`O2AP zz(2j|w;zCKr_l{^&%f!dL}DP5ojOt?;L&(zmYs6AfcOSgG9mbLERNUg{4}g^EXC1} zgK$@jcSky--SaB>G>{hu`mgx9UtcD_(4h6F{8hY-W`#~m(yG^neL89Ce(Fc@1;VE# zr043SL^rZchTw5^z&DHg3V`*C^kD zQT-ILJd&ARlllTamp^(F4_)c4Unu_HCtJfD@v{viW-M{f%6WJ?eYgp5EUxvlU{^Vi zW>y2rL^uT8hTx&FEzk`r&nWkNReg_8C2)ANz`7-V)bXx=(OS73{|Lk6t_Svb6W?=m z1n!C|zz0U{P^2JXq77r&eaV<9ytxAXG5)LIQx*Kebm0^2eX6XU2oIkxa;omLHTNj> z>{bzcqsp&w7BFY+?4y_rd*A0MzWBK*=QqT$D#puX$Hv2&-mG6t4D{DPA`eBy8e=F- z7&J)Bnb>vcQRQs@jg0}%u#yrDMHP&1zxw}>bwHyleU&8XSYKUTfA8EFn|PYw+URJ3 z|K_-rDSN)e*H&N8T7Q?$+7@n`iJ7Qkrc&~y67K}IirB448@VZIrmPZ0ThQ78obePo z$T9oeszu=K|Ck1WJx{P2B}Jg)JB3j7#jP}O2L^4ghAa;zbKMaYMvrt#RWqm#Qk)wl z1DvemvPA{^W&tvhZPR&AOb3D&xigMLGa3xGp9d2BDgOutF5HbOek{Xki0cdnhRu8m zu=qc0&Wyp$o)3KTJyg?u?148;68ayxF+&SI!#m$n(K;F*i+^V61N8a5#*xjDaMuTu286qIZqDzkDH_HCebzv{|nxY zNCbYZ<`#XZLS4+AFE%jC3=C^uvk%nFwe0i89p7Yv2rDz43*EPLp~@49w!zQQ56t^H)21qkF#ypS2|wc!6F}u4jp~>+34=~++QeTH2ZAF zrMlgY%o>j2v;a9tfQcWEv;0T=egMB(@>x#tuf(CYfFJx9qeS;C7#`%07@QdP+!_$| ziNsz>fq6(FiFT%}1UvqM=ez`7`GeqNbP8RfE1a|7-jN4DQubB$oNwLRjZyG@5NZkK z>T-teSQqOS=&_KJz`K)l4jX-)w>hJ_gW?!d%Y44Z1u`E|ShtI8<^^LTpnla$h-|j@ zl4g*s#B3(F&hR|D@MQWzx@t>{#-5=3(aUi6iB%w8@Q1(5msIGcIB1a9))Zlr5^~Er zxsF2(Q*f_}!3lYlFFjd@55jcPq$aZalU5w@?V?WK9}$nx9Z&do4Da0lEV|dhXhFUj z96I{3GKtqcKO5hYke1YrSrQ^-HcrF_~-h@{~p4 zEv~su2~OlKgXIqlpPf;p1%Cain|9vc`H0cqA(+?GLvN4k&Wj|-M|}(_VCv%Hl?)9@ z>~woqPx;_)z^466?6rB}xPC+&=W;Sn{r70_{yWuZ5Vo(@fc&aZW&bfAo{FTJGSrD| zg+84@qL?&b9@U#S*`6Zrb2d z6kF(AlN6H8?4$VL&zo6uqE`Pe$HkpG3@oJy`7lJ4l9RzM$L0=_B3I32e?~bwpSiep z@5G1JXHnTkTc?WJy|z*NEn9Y%Mjw89yZHNt$oJ8teFU6sP{i|z*FaYsL>PS0pDi#9 z-p?3MO#=sC9rL>{bcZ@T4{rT^Z$XG3v5$Y5v+Kv*SYapO?W4?B1>F>Tkx0 zP?W2kEE~KapIM_Nj9#Z$)3{w+B0gjl%o9KlqpnlK}#Y5g}zFhzGes#dy16P*NN6B*wJVfw+r1X0SBbUFpk?XzZ z%liRP1H~zBZpIq2qL7uI#(o#tbx#(y=ZU4E0G1Mfe#$

g&Q8vQ}=8j+rMKp;#C< zG(7FsLcTEqnI|#PBVFd$^l`IH?OK+`RUn>yUHiml@X^mwwG$$ZG@(=>$_%X@NQFeC zGTvTZ>Af!rhR<+mcdK-E*f6|I&N+!`T0=t2JwGcqEcV2txq{zx4Gky<&kC*Hijriy zPSk)3L!}ZW{}k4_F6wo(_O{ei6GZ92cpB?hN^n|pgzk^Ij_ZT#wnl$hLof4SCPz%M zGssRBs%IPh;K%$l)SYiT*4H?P>zgD+A49N_r_F^c-oLv@6#$`g^Q#SeScP%DY#Tqu zw>I|18Tvb)pJW|J#MikB(CxClQRA#XKJ9NkvckFx$7(9is*BUP*6JFd#M*HSo_={Z zHvIWWRol7`r^=zGU;52yd3@#3P3pk{Sx-eP+PMY!L4+$*OvX7dnOT-NBipzE%Rf@1 zAN>ZKdD^Zu77D4wUl3O~mhJqpylTqI0~^z-EcvwsyY?rn=|uiK>n)|PF!1$?s( zli8;0zvX3*3`pZJfL4zK^JqWA2gu;kr+M#*%>NNlA6YZYcMb|ot^EbfDJkf$jMZTxIT_z(%kMdvqZFrPmD49ZSt{SuPZf(K$$!4Ul?;$k8{i42jR6>XDu0b4R4yvWIqgUpOZ*qtmrH|X1 z>HxY*)7XFG?@7BnNfKZOPs_!ZfVgryHWhkqjh4mk-H05g4AMhUR!gU?*q|ga{N?34 z$6yzR%Dy;Y+gV+FhWoWc(Z4cwQQYR(>j)0oB@p{R!UMCBwMs9bd%?uQ+upr@xP8NE zZi?*uF6Drr+Z|Vy)b~2;k?+a2WAdivZzZZqb`Vkmb};nng<2@uX6`?9?MIEcY`oeU z+FjJ2y7}rePlvMK#*nzMvD7(VH=C^_MZIjy^)Ak>YbEmSqu98Z$BehSPqN$bWi0H% zHns_fm&0Kzp|$Nh)RsSrZZ6F0-X47Q7&cz+ecOOiug%~ai>FKY;&gUgjG9$4Ex#F1 zRh1sYrkh!gy*Ob5KdINeZTm<4w*X{z$eX5X$8ywi-Iujepv}S~cB5xtsarjVq1)F3 zyP9VzFO;*QFJT@06++e?)&=;SO-i+1VJxbpn;HwTm}r%sa-NgvNWs*ZQ&@5GP! zb(2yGdI_*hv1|#!eCLUT9~>R;@z>jr zf2W_jxmC{IXV!P}`gP(z{1-sbw1cNeDCBqP_n4+!&hbXjMrg&@teaShv|ICgKxQaA zaofKB&jl;x)zu3EKAQW)onNmnFkQe(g1JpY=0}Gw8W_xyOEaPv>bPFU?UjHY{M_&> zJftV)U;kvde`#eu1>EvGr8h*T{!lgkJ@eXkH9|`uWVFu$Z-PGbpVEti2(iZR@Wwbu z7bhdob72$&AiY6cF7+IIc@6Cq=wECD6z;5tpfuY5E^s&j^@QLZJ7p z0>3Ji3gTJ7a6(Y(JR$lf3ms+j<=U4JrOHb0aM~u-b;zh zUZYnY*dOJcu;Znz0;g}nh+4sMs>y0qX@%1Y{7b>cygZ>1vg6rAAM>2>TywO0;8 zfj8mO!^))*i>u!5;~A|7b%kF4g6NgysyS=noeAR}6xGihr5LeKatnhIgf&OdWlA5> zEiTre65~8FAf!fypdBO*M>q`kck3HneCG`Dz3v4If3eJTdkQK>(vna3zK~DMw@2VU zy0THOt$1>Em15pAXT%9t`Oi%cpJAbdzDPbe)Bb_nowY{sJ^W1Pu2;~H9Dc5rzvBID zO)qp34MqP$1>~TsEyq0_Ob6_>V#f$mA#rta8R5eAk*aEcR9RhL#zbJgFz4JLv3HwY z*^Lw@Dm>G9K{@8u>s`M;oQ*9TKghlfdQC9iOnq!e_GrZvgeaQ9&-y5MjiC|C+5166 zw=we#r8eF4+eys>o>LrxE`Cej>hDoY*vA>hB#Y~Bv=FNg-fF*^M-IJi9JQh z`dUOIg7=SIiC6jw#S205^u3MAd8Zt;@3j^l2K$Ig`l>%XPyPBC=qDaUrN=ikoBP9X zTPbtD+~ufYwCr@rSjIJ~m4-ful6%|A*8;?GXtsdi)J$eM=uz0LQ(q=NO8j;{mt-wt zq!B!g?v(?E0~kp2^rA~j-o%_S-3wn&Bl+gAAqDdV@cy{@w-Kl>Ey(f9OopF$?TQT( zUzLV#B!>qf$^Q>I*npt{!7Biw$k)$xI70>|Y{(=wH_cC3`wm`IYA=e|X7JfEht@1! zur9iitAX|Is^cY=ZG%)M-@q7JW>T8DYR3gF~$XlwjIIeY2?H^r5-`GBVr~S!fO?SP&eZ9h8Pv@0_Fe=xoz=!lpW~UY!m`lio zt9oV4Lc*4UrcooJqhEJv0D8(oz_N zhCHYU<|neQ`m=fyvVn8Hcte^Y>bj*l_=y;u5b1Zy za-S+Nk}tMfC-n^k39`J0>2yu%cH}=8Fg$(b+Y*|`5?6yol`QmW_xKoU z`2F&N@AOOJ!6VZw3iTbgoTt=pckv{iywPdxrbWi^+Wj zE@NQeuPY{wqpCYchpD0ChuoCl0FmAxGa%>dIdnp^e2&Pjw z-NJeU34=imN1G}%Y9h=UQ-)U+_Z7CqL1`mBo-8rw}wGSXWUM9 zvTSzL+4{C{WWfOBK-~$ds=l_@YcwnYC8uYE0=!8Ws8WYn$Hu}uk~`vV*-;NWk_jQ` z81`sbzW=W!-~v*g|O!kfHGen;k`x1?|IhIU1eZ zFKeIlLuaf@)ZEyZT;|o9y?u^;OQL{#Cc4u?t?9m8lC;DYH=N-v;8d7Vq??MI*P4E; z#A^*uy6h@%W9Tcz!Fw$-Oz4s1^NFgQd*_0_m`4An6a3HZut~Vr5x!zBSy{dI1{WvB(aCx55di;}1ppeaFj~?tu-mH6B{_~m zAc$1x}Gx05qGi+N~<2K!|F-Tz4h5eNsm>ssb4F+hvylJszmte zoX7VJM(lm#qC=XI{~`Z+?&ImNol^k|Roy>j4#{-?;*;)^jlC>6bFl%@uNbP1ql`Dt zkZ-F4yi=_jfg??Klfr*_#IJQnDK|EV7_7;9@>3vbFvtr9^r_ZlJ*6%d156E0`8R{Y zX!WgwJtD3LSZEV-I3Lr5!KJ*plO(2*HCHM~_KY7*rfu>>p1b;1i6Dd&w4Bscdg?9nIp3W>iSP=%SN7sUc(&QO`mng=~z_^GlD?DVZn? z$1?7097j3kNJ|Y;VEHDEEG$nH%?pHYsaFY$%38mgw`ge{epdJ>R;XBmDy?9!B{)&pbIW;7)Ys+N zBE86DsdNM9;8J>HTt@r5NS;WML==A{uAKe7*-nz&s1q;PX?ZDdE$$`f6%)T7`yc*R zSNlTHYVRw;A?=1~Bna6oY#Bb^va3?_ zkfL}Lz8%i6R^{-~YIiCe@2^q>A>ZV>l_#~XMJJV=-5b$7hEJ*xF*>5lGc`hQy&kyF zs9?DRS-kB&o2R|9Imc_R*GN+w&#TdAv~w zGuFw@lFpq$^6&AtJWK&QP8v6o&fLCnXjy0-=Uc?(dy8ZKP$u4YY^B}ykd1MFCZlhO z0p}(ijjlDeWxlO1za8fa&4-e~uJM`>u(#=2A6gx0aCZddRMl|LujCmUdgT{2Y4rB) zVvZ;KLRZ1j6}FSTQV1)lPcOlEV%%SAe_Dm9L@X;_kAJ@sa=;%}Wk0?*%Omh>BzU^$ zZ!Iz(7e3Nr7avg<@(<{IcaNOuIKwYA5BV*jDcYeGA!M%8XM5O~(oMuf^$|u)fA^3Y zv`^UlCR`Of)Xt~+6270+6 z_GRh>?}=Y`?+8R~bd8Wsary4943@-Bb|R3?>qnqw!-Y9Iy1TF8Hjmk|zbHLMk8&VJ z%lzLppu%)&aV1HQr@_gb~)#^s+nVkXy zvYr0f+42<+*Rx<0AB8{3Wy*jCtj_t7U@W_%gwvBas$uhHVP`i#oD>c}pTBUD(<4ZM z>CnDc-_fCK|5~9nS9IpDFUK6D?W3ZK7frK8#6hWJ$Fud>EClYT{Zq^rE0wnf$h~IV z>R+*L!5VAOWy5bG)zLIbLESf%MlPk9e+icxrbQgp|K&F@34?R#l&jXfgKzRS##7)q zAYWKvS+_eNtN7l$+_Vafy(Il#DD-YG>1SLYeQ~b*g{VkASx^5kA=kL=2NVPldXD%v z_vsGij*ohqn5xW^kj#9O8*ih+GnOHZ+O3`c-a1q|rfFCtqq6T^txl+_rS7dk!gO9UMqgMTn8snj`TJIX`48E*! zX7mv*%lgPvVFYWpHt`61P<4op(Be-C+nz}f_{| zJexF~a+2A|9wRwM( zr%%N{LLb6#&@K%Is5xBD!+%CL^l!VlQTf^&)Gn5EsmJoak;)c|!hnB3BKQ(S9`VX# z%ZVRtCWDjwSfzWA1DGJglT{oB1Z^m2ASK9g&bLd13fw&=W(-=Ed47iOBp0>XCK@^0|1Fr2D6jm@08FU81;Ne$pQY zF1OS)8qfc<*_UN;T@{-3Cc|-*@M(k5#7u2xP1;Ohj`dA$qVn?AlaR1$*A+4yRV&}X?@(TBU zw;-rG-L_Y}5ihmdcHCM`{U(Z7UL$thgxScM2R%cA&?(^$o~Jx^sA}leEV>w5T(bLIxYx5 zbBH*tcALUy+syOy)FPRVkz4h=bw@2oUV!>yN1j(?-Sr=H8HVYbYOS|Xhg7x{2%#by zez3Ez$x>8IjGEb;Y{{-NJC?r(CncDWx6d+-KjmBR0m%4{7*kYY9eq*Ob@8wPKuC-sIU<6r;fmyQh|uDi#A>B$Rl%tGh~ zAqGSk9cpYQP-ZttTlk$>LPFq(VWnrKjVtnO zD;$GZU;)XI0cShbm z9=gDbgGpN}aYo3h+NwUD__@nD_S#!s!Ia+=K(;Fs$C~`fpLT`Eb#3>D?;RCq%k7$O zOky~|$3~=Js~``UkyE)nZyP??8kooR|3vi-bk{?DL1h(I+}eBM5}!&LUkI3PF)^%T z9x_5nx7E<<^7{i=xECoUsYEQaIv-b>4%3z}>{qIvP0#tog><7LcYymQgj=V%=wlM+ zQM}<@#QK?fFie~QE1mm{>LjKmUXg4XM@=D@g03>_4V!Ku6OSVTR$ZF zH3$Bi-UpIMkK)U;KB1sxCqFgS5D`McHM_iY{|FqVJdDny2ZJ%hiZ>l_#k(){*WL(@};;-7V)?nTy?9*XOWw-~srWE)yYx96u)5I6LCnVzXq# zb#ZHKI`F+P%H`jQ)a5P?`!I~d={Xk@oeHUFMh!~sWs_-K- zq*>*IQ!oCmljgOJpaNm%R%;`#&tgqo&>NoPx9Sc-J2*>kP09Q9uldMWF;Tp!G&X+x z)>EG`>PU0-M_dMSDzm^#rJd@WnbDW6dX03X{1#@u&BfI#d+96tw~bJik@zj6jF2&= z#NuJso8yh3*e7z;FR5g9r zky9^cE(@i9>mAL;+DGqrwz3V_W=R`MDA%|zO3^IydRbZ=+%@(HtWesYj$qe(MMMBR zbRgTyYzO9|M@4}^U052q$?BYv8_Q3vn4SiwDvjUz$B`DpEN z9HKfesvAzevGe=7CI&N*TH*<#LvfL{34E;oM+09jCqRfFL{>Zm8!Y>FwaC}} zS~UC4+pF_!URg&oRr)334v;S9Wowyqlw!)gAi33|UFfm5k+_YZQM7P&tu|~Z=V~qh ztym_f7I6oRjq>9&G4x6)VH>bqWXtAt;wjMX;d>k7gw6Jk0P>XHJ!=#d5ipew0kwdTPe_Q0b}kd^E#7Tqj}O6Vy~~LuJgi zQZtoR{vQ3YIXFQ}Ju`&LAMC1LCQ8r@xgRUWKiP4K_jH_GomiaZMIp(%ZU=f7xXa(e zRaiu8di0O_!=ai%!&mwknDE?6*?Dg{%+RS{wjlg8zu@mSRQ`o!-LBP+mBBs*qC;Gd zDg4;$Se>jL9#hSJwMPbqV6yqXF#^30yIjM&u@U8jqzt+byO#zSR1aw2Qyn6VsI2mo zK4JBZnGi*lIQyQR+;p~1qpQ3g4O3Vt|Cc9fy|!)gdQrMV2Q|IgWPtjQP7aN6r@O~M zzT<=?am?OmoZ3i|h{-9)cJGfgKVAIqw6Mw<5C}qwQ@IPXpbS>LP_{B2OhQ2#8@73puHw6O<MSO|3{kjpl?d`EV!S)cu<1hYo59X zJn>KRxu9?WP>*Rr&?`DN{2L4%q1g@&2ku9+Xjxdm;~ksw*KGmGxx5Vtcl)+ot3j!? zD*Hi>e>Bq>XyeR=;H#GT!=KB@_Hof1LjcX>W0t?YQz>tpSBr$yH^ZJ%rmHU2=3S|L{D7BwzIVw#9G zRfI?dFwEom9bKW*I&op=p1YR}gTz_GZ>h6{1;Z;;I%=2o;~s0tjAGO|F%91YZ)RU^ zo#2~Yya%67pFkV;Q*5Q?Te&04bh@)PFJV}O6^o7G+A~;;U;Q=t4IU;LLJCb->VZEk zq|G^RQX=FNe?&TfnNs=-6}O7nyaLe?4$T46#(VBd zq8`Ed3l_hv!c{be+}V%AGXU+0%}1}OA<-$JD@O+{5pRo(W9+^}yvoetqL&kGr%~y? z@(J4ZPeOl8^n@MnOwC6se+$-z+3xewOpH<(&~)^Jpz|)b-SsWsBdzSs`>$FOvP?W# z-|3_1Q|JlhW7U(ZN0(2We&U)qkjww)@X#$tT8cL?;=52l18_m3Euu2?6SHGf6~?9! zAxFur&Q{#Nx#6~ITV!il#($JiL=N*>Mhv6J+ZWZ;vutaGy<&&txar`{*Rrhl-b(9o zv&-6k{+#vQ_)Y-A(m-jiFYSGYY!183l51e2)NhBFVTaZt993OUlSwutgY@LQQ|Nh~ zAqLqRd*{9a#j-qugo%xSG+#WjTiuSn74NDabHxz1NNFO`!QqvhtYVy#vFlAL>OiIc(?LkeRStvk<64@2$?lw_JS6-;b0r@r2N05f$}g zrvIHe#&;zpcU_(!f5e)dU=ly|{;Y=n(d%VQ6n$5-M;u13nq^RB%BTmCG^o~__9PJN zHHc0#sxHZZ0aX2dzhGO^w{;Woas@!Xfjm$t=nbu(dF^ zMlB~GpBmS@flfsYPlEjl6z9p%ye+R@f%TU6Oxq*ruWPbsK8N&kPl`_s4v>**-%z8R zbqWo&SR#D4t@iaVFNROafbb0k?p-iqfeY+}mG*VuKR(yWhBMcyTh1LM*&(sS6bShy z$ac8E^Ov+ybvp{Zk0ADw$d4EO8tzw&JWXJ!=h~KiY`lEix8ziItIRo0Hp|Q!{uWjC zXphm{NF|{}op4X_;bgD`+%Dp(Bo?@^8sj0{$|oGnw%)aX@eReF$YF- z@gxSpWbZqg%ofiiN5Pk;Lu2^_s%$rj{RugumKuxh`s?iMj$c)8(e{flbV6#+FTymy zEX|w{ajcK|pJ$dVDqGOU;f`$2k*NH#RlB8!afZE3Ejzww+E{q-|HmSPp@t z7vl^=R*oT?J?_K3O=XzM^w^lS$?_3KmK&OZkqBQJeE!07Ix7&d*woeVLf?4d8KWsS zhi%4YKb@}Ol=hvO@SP-Y7S*=QJdI_>t?YC_J|r2i^``z;=<)v$^%h)hMq9Wplmew# zp=i-U(c&81-QC@ayGw9)cc-|!ySo>c;7)K0m%Y!~_xyy6vDTNlUU{bUrvQl_HP(w_ z(K~^vb{sqBO2Ru2g^BAz-2T^FP2^~?fSLvNn+^wzSVLn z{4-}?$hu@Ri3IBUva!f^(`;nTsi#GFi^`c;mKF+tnMEmniKe*SUH zw~AA&YQGi5ERJ9U<__(#61?h4sia4?4W0v91?G)vIIgxDw!{hv4wo9I9xE64-nQo0 z?pTi3?&Z~0^F+h=-S3DEAqq`H9=-j-j>>{Uyh@*AmEat(EgefBx5#Wx4fm< z2tXPYj~$@GK)NeP(2(%hDOKGGW8x9Wx&&1ArswPlOVw-d=M>Yz)zWmA#$EnmvOeHr z#a!aqso6LCb;{O!A6uV}U#!V!ym;})DZomA(nXt;hh{9?f#1pZ+G{mnohjM;v3}F6 zGof6AfRvk*--5hbR&z~NYC3aEYWR;f0-u50n_nLh6{)75x#DQ{|8W4_CMC9$G%s5| z`K{6fm!6H%KV~V@LtvuqK9Z*2?C~tRY%<4Eozfh0YzJ#(B$`xfG5DBZSD64>WY4vj zKa#wl1e~y_l|s|Sh0q2`)$QO?sJ-F#Hi5jvIZxobKstM~EU%8U*3&5f5HZ;ZfLHbP zYsKmb_2|y95A;_Kn~lgZ{R?IHNV{yL)|-G`i*@g;5@~W)c=%*}w58JtKQ-i2d z)BfE7PrRM5jW*4$M%Wfbe?6h?kNp5WB*DCyZ_%}l;iAOZ_SrV}&q%0-%#t*H{n68P z=s_F-)k7jE9W?o{W*3W7PfZQZ6^UI`my36DxKj-QnrI@cQ&7|{>cLL~=zDl}@m-7M zYNM^iKz>5+BfZa*_KA1LW_r}{__uJSrIkvs)RnjzwjOl&i4GoAFec;AeLo_P8mBd` zy%fwygw|NFl03-+)sQ(Re-Lpdymi!p|viE{lb*nf6~;~5OI4+s>;ii$j=x=WC6!kPn*r%=wbysa(d2HXZ4r8OfDJc zzle71HDyTX4<6`VLT-j@qCUqauobevOGLlXmJu6*#~dmORX8ZUYuGA2CzzFFrZhsN zq$UkT(?2tbtgBYX8H+s#u>Sbc3Ez(DVf5zp?L;VKiI4!!Fp!5$Bi$q>PgOji7N3x= z4?zA9R)Ev=GbAA5P(R(EcFt=gqMHM0o+k5}$oRz2DQ7M+NGDu`Zx7MgaBh9>PLJlT zC^%Cw>zlKG;7M5>%9^rVO5=N%!jO%dvSw{hRpu@9Tiems7PS!cDNk4}c8lPUft_x8 zUgu8_D!*uUF#o65$F~q?HGj#9JT9fXpA1z$C0Z^E_q&SIM6~IwB$>Hh2 zH8^C1FP^1$Df`p^GY<29<-)L`ThnaKof1qTzov0~!=PM;OV)bBY085rgEz_fEcdr5V#L`qd@py4tD^&)Z3v2)y;a@dTq+Q zD3-W{kvT4x05CGRt6-DHxrMf~sY&ylT4km8yrhRn_`(E9CHwDhySQZl>UB;3(D5Ch z*tXz`5#qVjj$S22c*!TdqXw*C%3k?Q3 zC3xD}=G?k8O1+$|iD+W=IMk{BxQFB_+XQ^7-!S0=D$n7cyqmJ!jV{!8L*$%}~Ve39?tk@swI3rr{B$hM;7krAg zRMuoaSWujAlEnK?joIJdBzediP+RktjBx6Mx`gFflvmqg_!q!=n8Zsk&n-(zvZLw2 z6n&Lk@NcQJfyAqWo8B)u0}G2X&2mArrWjr;6OjPC)ZDF>?R5t2e&h#Kx8G?souqzR z*w2Zz-F%dalAtcE14T-eIQ5z;unMow7;epVRi3L-kpvl2V3KFmd@dXEEj4X)`IUFw*l0@@ zgV9Z$Nvri<_|uV8>%UKiB{{E|?QIRG!J6LO%Mf9_d#^7>#+>A7LND&VUnyBaLgt^M zqW|^8j%iJpY=k#OP$$l?QSS+M8=CEUrhd?h$= zo5?mERFLP8Ox8ImK_~XC-YfndR9kO`0G3&p#_)}<~ z?Z382ti-ko88pf$Ck^P>Ck}cjwVoxwBr@h@R$UK0VPePb!qb8%5n--Yvg{BdPHlfm zCL5A+G8(e;ahHyw5)AIYEYxaOK}p-Ai%Yrd3SJ#6AneZSGQ0_7O89yG;BVs|j}F*i zOmlW|9v?l_rYc{mpzUqAiB8!2UW^Z4&Tb_#IEa+=uTP`Lg}w79Yuy1;zIrehYWrme zib%}i@8{^>kGHy?-+3xU$kqdNUdG$fOShr1l;k@w^)*CJsd=eD@Oc;6yc|#dr+X>vtfP3S zJYSYvN1uzq`S5FIT&8cTh2^m9MS=z&+yZ6X;>8jssC@qP@j354q?V>AGF12&r%Ajk zDgfb*$}*kybBp2!4z(-UpkuhUMevR7Z{*7Z@?6as3gvGD)|Poj`M*7Vu~5B({e)uL z!0rdS*2nelCjG&8gjrX`XDtz#uCvAZwXC{pM9tqV3BVJ)*)So`x#+UXHkzT0-VVof z8X06HpKzEtF&y3h*#XWkzLOBj9yU97-SPI#k?qh<<&4gKv23(-+m*-K#Nj@TwoXrB zL%QvEz;wvZ&-!{8lj9yYk;SaERN2Vo8GS2cxsYT@QgCzoQ05ErS zcwXlzZh>k!q4}fcEwAA$#!a8to}Dz0=9A`9CLrK`uFBqG?2I#J2;*J|?CADgLVS*D z$}rifO$~eXL+_lfawax%dXf$CEOk&02o|M9#(4u~E2-YVq0E~P-+EEphssBx z$aY1#$zp~nQ>IJ(&F&P*=o%h?TvvUfVV_QR%W%s)2m{c2WRB=tq0IM=bgp`*sWvM4 z(cOGorAOT21!YB=flB( z&C#{!6gnk{-oG_hVe=(w-jBFQY{RxmsDs#0MW!3B$y*Ln`fSFII=tjo%z`-{(D@*) zjg1m=p2fF2{19Hsf#*L8@I``zV_D#@nU}(+PTJA@ZZ=_`KRF{g>dph=anRSBU&^YgUg55WV4ZnITUoRw38!m zYQSNx`*^joLYz9+1B7H(23=&+PU(cZuRIsb6Ke`ugm0xbV%`<+@)}w^D(M-b{emT5 zvFr;L^25Uc)RZHLm>f>3{ycZEch$*;>0dzvnH#1khal(5*k==9{E*3ml+N#|mGjf& zV1nWiGx}AnD#+@ziH?+|FKcx{y%ew43B^7;Uz;upkAB-If_0ddLc7{T{N~exK80EJ z!r7~-x?AK71oXe&OCI}2MMBKHcKRV&1oizDrUX38WW;)l-~f5=6+S& z;piRp-3M|&UgPw=;kk;f5#OZm_B#t?4P< zxQ?oW9IqMc-`f@ZqWn<6^A_F{4|EE@slR-xhv5`1dEGc-%FJvF^`t9z-OY?5P z=G43vO9E^_Ryw~}9lKF2OAZSw^tcOotgJrxq>Ft|HDGrLaBE@nV zZ$nu4ONC!osxM=SUSHXdNt_XPOb)Z?zRT;HnEX_ayXFRDVnz^+mxUUoy-vHMKd!SI z7lur|8UHg_G>cD=p}_;OY3h!9DdYg7(4FJ|ddz4I)Q4e=IL!jre@>ULnKpzdbL*9Cv#pmi=D}ZASw< z3T$BL__tM+4{gP>S85W^iCF?0ZL$uCQ7nN_`_HiSQ@`0&Eb%KtkD!Q}OMVB*jdcX_ zQfDf_CDEX*w=v$yB{d3tTc9;5ehD41m!x#_&* z6NrXvhk0bso|dDVVJo>?EJZF6VpiS|tPT;E#2VH){P!GY`!dH`*CaajFk3+|`q0y$ zV}=Lqt*L!N9QmUABkoa;8Ul@S%w8_(oJKca2!7dm56?PG>(ul_woXl*-pV)-PJR_{R;m1b=L7zw2BbV`fZ;;mnnpM$OscTxL zgM^_OSfaxlgy=UOWr`x6Z~X~NHpbt?iiX10r3!5ThuUFK!B^$v(uR!VKG`i zcj|ZqaY;yLi<3(ZV9|_(>%-!E=FEiSi?UCBU=g75&svUrk5hW@UpOKsb0n#A+>E5R zO#6eh$*=*>Rv*cQnQXt6aIw;zqS`S0+{u@_|1C!DEJOL~H|zirOs@vs0ip*zFG9!` z!kM}OnnhM_3$aP_@3JKpT>(L#ll!ZxgzDsiJm2xJiz4NI0zQq3%=>q9Y!7mCkQrpf z`=N5^h=5!py+azc{};xjC5Og4aeiH5((UsYUjEZS0=h5n%jYUQc-q<(QYAv!+EO}k zamSltDhu@cGR={{uUuV`)-0dQ!zi`H+br+F)*xmzr&#pWtZLD=z7q1U=q@;00Zv2+ z*fV`I?4eJ!Tx$?urQRl}iA|rdFE^!(AGG?zJe^6QX~i-v&H3P)BeCk<@Ie$(FbS`W}uPFdYV00?M z%3BcmzH``!E^-zA1@>*j;Be~3nS5t?k(yvf&Q5$y#yN!@$ze`x1@7c{4ngNRi-fy9m_ti{Qn%2qF7O zz1%m0_*{}FyiRK4nV(Fj+@!1ilZfkxs8&gowS77%E;JnEE2=qQq`EwvX#k&?mj1JG zEyZD0p%#_vZB4!Ni!8iKYF2;wb$Q}n?LEEr4ZUNdJtrACHSQB#w?TJlc1;dgs7U)T zp5oRmb-hok2pq&!==k~(lk8K72gxipcp)bb;#156lHdkB(){{<@|YQj`_dQ)i~pws zwr@ds{HJtOVl?88{L=S=nTL?BH$*vxS$!~6-7#A}zZ{^fAOes!CLJp6k%(DL9?fF;ks)az zTs>PEfjGc{J=jL#XsS`H5sFtG&HPot2f7$bPC69i5*vO%Q0SVmQDpffZ4Lw<#;?5> z{)sq=jTk~BorsM6Ka*+4^DlY){(hWyrV3XtOR6OJ2Fhbg9qL9$+r)5HN_hskrOSId zN@Evg0|jUysb*|KFKyKcH&KsO=_ggc^H`%HTYBUIQK`2QQ}w%rBd`2~7#YH%zJqo2 z1Dq)po%8d#Hk-{l|H-KDWi2#<-#WS62^6m8?(U0G;(?_`Pc zl{|u&k2b$HYfGKq-T4}MD=S^|i9xCEy&6>oP_)QYuFD|%%HHYfVLqRp_ z7Ncgep=&8-K&!!V?%;QzQTYv98(rKDA)<^9s&le*oDdrECkK zd>mrlu;I*Nk^SOrm7&j3v)@p;2rf_~=o3$M5x9r>=)`M%_M3EW^*7QdX^Zg7PfptQ z&Je4UE$R8R`0xRQ&s*I)-TSfqo4;Pp+q;#kJoqitHh(`w7kyRK<@uW24KG0!ZAjGv zj+$EgTc2z}7}sc0*itm}MH?f#Ni9VTfu8@!q=viCJ!9Eb7^CKbj-}TShh5pV>#C$- zvR#cYS-Gl3ZuM_AwfDceu{-wb8CQ;WIqna+4Y~B$X|D41j6;#^KXnJy$(oSuX|AM~ zUlfEc;0B=^GM*W4Syq8y1d-?ViZ ze1?hkbS9TE3>J^Gr*}_R4O)s>U4_<@gw55kNC+4mheYM4IL92iz#BhsM5QnbEt?g~ zG;zQFHP9ZS(VSTWioRU@j+&^OX)^op>OIGSqKD+_ku^rAOLWbx0r(%zK{xPoAOFQo z#mfK&YKr~={(&VoEF4M9O-E?HL$H~~zA87R^G#Rv5^>VwjAH%xn*EeQBXu(yjUE*u zYIjeIN#I*2{ge7u>gqUj$P`0bhm(@WYBI zzh)>!IHw42yC0V$wyc@HD|jq-5=VHr#=@$=OA^>bR_v^}EIOCX)Ygx+3}UVQVr5R0 zibV+fDdX`%i&mxg3v;}N6?KWP>%3})TceEp>~~U&u-~CN2lD5`G%e*b6@Jr1PudCs z&)i5*nw_U?2KBP`E5g$R@GWL6`|3iq$}%Cw`)&(-xJDm0>%NYHK;q>sm`urXBc!e; zh_uYP6r1{|>Wh`C+}>~XJKUH*c4Mwo+{<1QRhImZKTx^^=OyLSObddHQd?lUp#K(- zPpUy_0~j&a8e`&@(we0}nYN>=8}tbuy;ONRo_C|UIYkeSFOD+n24a*uRSC7ACZ&CI z%#+w9TW(O~T7D~Sl9sZ&L1=Z6wm2Q_Ll`~bfoC5T-!{utZd4Bl>d1Y=Y}4(7_Lhtl z5*9Ll)!*r<`+z`%T72V~{*pa)M`+$iKbh4?@(=>=>_7=#Ls^ijCX} z%@{|Ei>;|1jm@0qh=3_JW@F+T40&mk0q>E7J=)uW#N9uQX8F>6#2J&AOIJR96 zPtRNU&op)CUf4AHpSjFJ`yr`02L1gK!3m9WZ2L*hkBXjOX+Spd9~H@Jcy4xC_VJdz zy954O6@nu*jnBL@)g&u{yninVQEMLkmmU24h~?y%ldYh8Ug?O&#tkT|M+U);wd(A5 z;^t}Bc*h^f;$z=MVQACaF1JFWS)89E|F>E<@~cJzeY<76duK_>yfCoXkuT7jV8r?7 z`?Ef-O6t2~sTPD{Jv}YNVP`|}pDAh<=bF9_+vSC)$)a@qMc$lsI~AAYyn6JVsdL_= zpaSQHMIOyEOr}C#Ci=nm97`{{`^p@P9e=AuXXm2?4}JY3(EosA@#?( zAJf2s0}b(dyzpTsMx~^KJucd06rC6$ntFa-g>VT;#c zKgetZ8?uEA*w)3?7p}Is9>d}|=`S85@_7H?iJ83Lh4#N6>i(3OtT_|*Ud-Dp%MC2}l=M}_=64|(zDd=cxKGD5C7K`a zkaf5t;DKQH@7@MR>SU2EVqg12$;d)eL|w~W*4o6D2pC5 zGqTKkBVnHX%;}X`9dI_w5ND=zvh{qq?W{1j33brMN=8zI>%(~7jr7)Ov)p`~7D4;3 zO1FLcunzjI!*Xu6Y>!r`d$|kikaH7j_gdpo<+qi5hCL8ZBOs|&w>nTE4uz`MCLuJOWdBCwCs|0=E*L4PiBod z?y}qV=$t<-ocGN|BC;~W&L{XS!{ZvXi%QHZ2nv_iK8#b%OZ&}W$WZ)yBB(Cmy>&)D z`%A%>s~JY2?+D+LOM*AB2hutA)Np1J$}JLq&y;@AExcvWg zf^9M>U!(4r_Y0U(IOnq_bbz5)|1(37L)Q#nCaWsfsYj_gAL5~FRGf!q5J@qjx!61jVBmQvcDwX(?cdM)=-zF#IY>vh5`6f0s0!!bxOcm?u}P8>t#KkKt)o zPUW|-ES%#GS7<(gV;;O)ZE|O!0bZ}KGrg||&(tHD%;BLXhlLP))9~MNzVrqsy}$V& zSu(moa4E)Qm0Wz2kfvloQI+yN*ZINR2cA=R%0py-4!Msy6khrFmz2X<3mL7L|2}z{ z#V-ZbPEmsNgv1eBLa60-U@C%Z7pttNHd|wm${eL9c%btNT54;a^$oDMdEbsaT#?LZfK?lqK5rdrN#5UM^dx zXiRE*=gFg?&{&JZ?pfDi2bW?52z(Jk&`kqE{6)ZB-(?fpZ8|lEq6i9D7_u93S&-XG z*2N6P8Pk!fzFg)RTF7UY?1jar;*V{Cx{5Iw2X?Yh9>VBK#RYgP=+ksr&vhK`mVxA3 zoi|ii3?-h!?Yxx3K6}bM&DFloyG5UrdsOOB%wfeM=tf9uUBGMyUGb6VFr%we;-bWg z2?`$Kgo)8n9~D%5DyT+OQo0lP$bP-REku;Azd!Lu!V38?I_7o(w`6x$EDvF!JHGM( zU*;c}R{s7W5Q`AJN7H;!Ja!*iNf}*C$tEWtaJkP&xjlW!PI{QQh&OT#7&H~ON7D;T zG5PcK(@at|nG|K4Y+xJLnAe%V#Nc=DxP4B8IslHSU|K3)4;r!SKxn$Jb(vjqBYv%@ z8{G^`7C^F+ELe(nr^tisv9*0*mApbRWu*6n-d|Gars5S%;&hT4-41*Hs&vkT(3J3`!OsE zt@^m5Zx{dIu^j8ZRam87TT-35W_82=xLCWY9KK~@aJ zs_=74>`(NH3FK6HJ~dQ%2)877(Ie7w;ErDWt)f~g4-x(@W<})06^M#7HeO_X_rv`p zd94UJh0+f31f~7*r%}_|rk!1RLA!5C^>K`DVK}=Qr*ol#1Stc)8DmC2QcltES(G?u zo}hhZ`)P*QXw{=uy;UhUpKUbvOYw^_0$s$v{nX}hhfFnB@}}3;xgN>d!&$LTjZL3{ z(<0~;{)NRwNgY}`b0|FzDeCEaa*jk@0FL(z6E2O7X9wxpRGf%99wVo|ph$ zlDQgE<7jmN@>Z*1)2(Y&%F|nmb%TNbf>3V{;*UQvDjH#UG6CozcbvYBQdO`iq!q8t z%2hr%T5{}^Jguv>9w6(*CePh3sOpd!aauOnQvYD`guR?_Np0sMsH8tI-txL^NDCI8 zq{rvhr~9$zn)?+6e@B}yyvV1b58O?s>cL$gxui9pdmc)VXu*d;aF9OmvE~S+ShBv5 z@Iwb^!0UL5oL$Q2-3o6sx*DnkF@)~T@2tzs8ezgUP=+AQQ-ABOU23jMNtkXEzJT0p zO9C{hvYl3+z(b^)rE7Anoy}EU3Q4xC4aR}6+JxbNATo{_Gdg z9L#EOKocSRI)cZyZo0ud)xE`Qz^WhJ27Ry}qBfA) z63-j=5&_ZKeiWa%8JZub7nvE!VoHX zbG&Kx$#`TEI@vi8AKZT+&1B<{3Gtip14jv?duWxe(igl%CJ`BJ3 z#~LIuDI6?r8388GA)w@|x&0ts1B5}mUntwfH2m<^Me?-j!v&Z1XSxWfiF|>f0qNf( zj{~xL0AaP@rsVnHkv<7}EQG7uQ|;W4w+#vnx{Qh}=uoE+?UV1(ht zC*0zise02HfXenx5+Zbp#AM>K#AR^O-1|CD%b+>^2aF*`JgoGn{sRc~e(?!=E?YQwFh0KWV{BJ=Sj`q-i3ZW505*wJ(6#e}U~w-gY1aBFwB*mGN( z$*l{>WOu5)L<9j7br3y_L5C9*Zi3m5l>*rrd9@eW$PMQ%+BnE{GcT~mvjiP6*8RD# z(x`oeZGB(kOFo6%%k5ZanKutNWbHS^_gSA_TCK-aj&=1;G>DzE|;z?r1ab& zQasY3xAne3i_ftu@d5>Bq|)wWyICaTd(LS%ywGl?cv$fe;^d+)NH&67Js5}o0c03Y z3;|h&!nNKfeIe%%No= zW`yT`!mw!wHsac2Q|3dIP4XN-``&QFmnbbc7pDd?1nfTMU&$X+5w>*AGX~Wu&sj~X z6|x$KhOyG^#{O{6kg`SKW^l)k^GM~*nlEBu#(X2)>Jm;r*nRD5Vs#i0XdjPMVFk0R zKM!rJI@3B#k*ST_0{wLeMohy2gX*908oYzp;7iaoiN8ePNdgFMyZYyfjVeG3q2e{! z(|I&W=Q+(!Y-i4qM{J8v>{6$MDdHTiPy*&vQ>Po=B51?dcCFYa8+yz@W6A%Bj+Gt; zs3sH!Ar3?eXNuZZv)Ubqht|%{fxq{mPxLPmoG$$@Qk;NLr`to)C-J;n=*?;z+E-EB zyEW^&(J!Bk{?!-#3xgw}S0N3h7C!QgnBavHYw8z}%6nAkk$?P{1(+d?Y{c8dXepJ5 zGq6Oj%bIVeAF@W~FIv=hep-;`Eb-|B1WGGYbI_V+9N>bnfN;`lOz7-aVoMb#9ix&* zTgU>-IwpeAecNmPJ}b7cM?3aH?FEa-plVU))r9b>R%hpCwLvE~_FU{kL4XQWgG1>S zb(jRn6CNP|HEr(yORoBl&M}$b8{NqJ4~__@kA~n$_{9GBQ;XmoeYJ;X`hnN? z6ecH0THYDes4cZh&+N`aB5+@~zDN*}jgmeryG6!1M#1mVnzjM1vBU4*qhush4M#O6 z4v|8{2(toMTp1@?^6_Z!uf(2cdRpB)DG@AdOz!DI64l`ztE50`Q%Z{iU*AA5-Bz=5G!IR54t=A0k=+TSA9Vg5d=O@M4MrrC*EJ;G3i6%QG%wdXyq0Yj^io*IezpTbW`E2QI8-rR2wh4DDK8Cns{eKeL+o3{U?2b}(N`~R z9g}B;ioIr3$K7|Ms*66jrT)NoTY~>^?P#)J#;VKTw`D3_pXmZAHaRS9KbFbvm~HCn zf(~zyY!K6)fFi^1x2VLZ-;*T%X-&gCgA=5}Kn~rP!0dWmGu)c9_ocS<3q5tNn0@f1 zH>?=GH|Jl@uU+NRX%|e$2Geq>lMeJjz`aWKn^cy`7ic;|%arwoe*5;TD)4C5(&$8M zO3K2T9t1-FwDe$;ESLE_JEAyW!5^h%Nk2sDF5aAyze+0(-|^z&xz8~vCHak-v`4t^ z(~n#RXuMC{AT+|isA8|;)dONzUvV}^z-e=BV1?riIIV}cq3dYtNk zW+IJ421}Z(og*P?N5IMUZ_Pe8udkaoWz(p>&zNz2E$+G< z_qF*aH9WEreXhkc3Wz!$2NST1Y+j5op{9UI#O5qcj!*}IvC zQ3&#U=6Gz}XweEzFdMmFvk8#GwHw!WPI~6XjwUMnbBDeoi;jf@GuD$q@{tvgkS)GZ z6l_04?!pSO)7OazONbp(YkY$Mo(;MpwQe#;PGBWvZ+no7pN6g#8UKSZ*^~ey%V&S; z9lA=I5i^!G;^tPzpQG|@g3)BTr+sZ#cA^=QX__x|SoN$?sh4Rpka0hoy^ObMC?|S5 zQ#F)s0j|I2X4*YJfp0E>-z8_MmBQN=qY>jcwE2IIEbS?*|2!}MW|<&swN5YD%PR=> z(tp7*mmk=k>ev0G6#k^4+YPDu5u39P*5bIFvotvS=A*|EzJ1BssxjsMU^Az}o~s(s zf!o-Gg&oKbEoKS$P(|xJTTt6Km)F%Sh#z{Xj2i>s|H9*X{JY_*z7U%bLmP!0tfQ(B zu0)wE*0Dw-BiR5vFjRlS1BIQqczvnlpfigAQPF-&U9a1*z#W@X?+XehD2*7NBLHqm z&aY47&5R}Q<8+UK+AinVq#it%&U&w99gvy;<7eIW02iw{dV1nfH^$Pnv47hU=YvNq zfIhB`F8~YSkQkhu;fh9~zm7z6`B(heXkRn%4+!)7koInLHi;JlH(` zJUf3k!Ts06Gol3S`{Emxfd4ytg_&LOzv{c;U`>KLJKL6o0Y1DxZzg6hY%<3{UC1D_ zdvHsRXsZ9RA7+s#jQ1BuH^oDqWpB};zD!YT%6f5K`)d1ZIeUn^>QpnV(=r# zBzXD}_3rTdn`HyUu|vigs6NC47E&7W)3Z_97an5UKIhw9X!@hx9`V4`@$bFF)}5LC zM}@6<%0h0ZR<{wB#pb8N07I+oK3}(Q{B^+?p|&Sb1!LXv4<|u2D6~-Jeb6)9^i^he z(&jlgVpM0>bOaGz2H^1VNy_Q}h2|F}eMuSKy)x;er%D@%;0^m=bKI591qTt{YX};j z(mvkx%cOUngrt~8E~-Tc(*ass4;-a_gHuzU=g1-g0z1fQyv|dgS`^;kdZge^8o}#G zrlH2WLu1^ECe;U(5l!vQdDhqjx(P0E%=LCE%*VJj`OV*xCOS-XTck$g?LByjSpTS3 ziIV(q-d8`bU)QOVKY2K9azCxp`*m6QJGxgP;~>?bnf+=S!Q! z|Aj2f1l&5i|0AVQ&@Q_``gqtM*T>Z;EIk z5sk^*bWhIpMfvXek?9bF<8%7u2t2U71Gsq@VVIEi{G0$>zILI^yBp(q@fMQlbx#Hc z;dT98V}T!&R@fn-lbk5w56f+rets?I{_^MYA_05pc4X|%yUsxi$=OU(7V#Gvu!M4dnU4M{b$ zjg$x?pnLopVb`M?zxATrvEi0!u+>pKkClWy79E3jZTCa>iTs@2AGo_u6k|Nb18iQP zeZ;M5uucqqla1dD7gb4L;)iso8^0fWj=yZ(ptRn(0>v5rtt^t6>Dm>&o=0olDyC?2 zh{2tzku?No>e5*n?mz2l)J31ysx39$Nnx)VnxQ7uy2$FNh_psliFjI#jNebfnwKGK ziK^(+gh(1zZy1MdLXPJdSxfG3dFa}?hd%2Lkx1plCbQNT0@roLn`$*9zJjB6`?%b% z0Yd3KddUBMZPg}sw71DA?q-d^laZo*k9Hl{w)Cp24uSrLBQ#8t_T^JYRZJ*UZf#@y zegtYE=O4G9zKadpcm}?q_WVJ3df&sFzlpicB@I3P@yT< zTqBYA7Qmiiu7@ba;P1{mXoxoGvwO8I_7d8KXNDHChxt??C!&El5?}GUniUh9OaUb0 zVYW>|F1Qx@GZ777X;!>3Nwz1SKTDI)x0KxWg+Ynka9N`j5VW?o-sSPcA>>@(namnuKEhBLvqx%Do1t4tnU>>V+Ri*Y6 z_dclw?n67*r9=X4_j`ZN-t7&2%B9yEpMqFR1=JLuOS~p%+`9pKTf4DKj8NwvnPal^ zIHIM&8wR6_f7h>ksLtU%M)wtKuOR6hWinRzNcnpUs=zILZBmhLo?N$rJboZiQZ&rm zSE=p~GPWQbsQ;K3v=_tQd*gC$<)Fm&OnFl^Ub~sS&;1-MNEZ^*s?9f~GvLn* z`bgm!bPX~6U5o1T*ns;d+*u4RUPD&!%nW|=yAleB(ApnKVoEfA8mRnn&HRc+QRDK^TCP$Ij(Df``Kvm zov7Zf!^1+w3iz1bTZPwgkixlQhRsj70S`N56TsRe=aLYO#qL^OO9sKS`CxAYWmpxth8m zSXxZizyL&D>!#CA6f@62l!kYWE*?!zn!Ub^<#!f%rhJjWbt!*)M~_6v8F>WNN3_bw z(}v>E^9|XBH@0QE0pRt;5?>0RPN#I26hM#I)rJAzZ|;#P`#|~;q?rQ&Mw4ePc@3B# zI^^VS#O31IAJk=3weJ3+dW_-0tMYP)bH)jTD8OVr#?G%ltEJ~>ak9NKr9BvELa;Ah z&MG@(+Nmn&FIp3YXCi4pJbz>E9e9pgfl?XY6G|)mL3jKv^ofULkVNsiCNp-qoE?L7 zAG0OR=7S;PyhJefG!hl>2`WAlt-&dyN&OG8WhsGQNh#rq-mw}5VIV{^*jqHJ`hi5Y zZ@Ze|%e-V|b+OP7Q=k&KlU6aJw$<>AMTKGFt-rI*V~M-8pTm;tB6^L5ql(J`&Wi_b z@BSwRAK`~pg@>^cd<|VNk2vL%kOv0@ohuBLtwG|9`7u^Cq$m8lEOi4C&YD)I$101w- zCIX@D?u5}|E8hopgE zzZ5VDt?{gCP&CCf?R2yux2U+DKVzk}cfbHU9bQ1^b;E&QISe(Uq;ZhbH9#0Y)ja7AcBY2)w##$lLCOtS?*DN)pilJM=`{J8OTE$mFDey8P<&e)Q6CCU-ufL%oWs z-NEwQ#%FpQ8 z&#pL34KDw|qfMrEcyslkv~{09-Jc90`b)G)Md3OCo?$^-ssru4*FXGKNXB1w->!T(S?qe~%EkLb6IF@u4a(IIN=A}| zYBF?JJdrzj8lU#?pV8{VEYo-#Yy1@MOC)gOwZ=@;j3vGz&T2NEEMD_VJh`wvrilNq z`^H+A0=y&d(zlOVAnV6Yn3lAsx-yU6zA8ATGTqG#w1>?Wd!xyb(2=~-5bHKpTPbre zyBtD2g{>Nyg$P_WsXz6f4fWjg(Dnh1Jh)Fu6hFSIKK5{_7#X4Q1-pTE?of2Hx2h^q z!|0CG`n0iXlXmw1vXI|>ImRLM*&Z55V@Dypi1OH+;o?c6Wur(0UA(?h<*7WI)KI^# z;AXYIVpD*;Ox8)7!d??O|HzIE;#E)vRHNQ9T^8{of9!b=1R{&dE9BG4ziK0*VkNnl zsfb;)g&`R1`N@^Nmbr#WKnM{0zDaM!WyS?-*D9eqhXH=Z1o(EHH?MLxRZZ|d5>+1R zybIswSK`DtoyY2JC}Wk-$ZO=*^|aD_v-WDpVcb+Aip7CB>@OvR*vZftNR z)@Ya7Z>-b5aD~#t6Fyz^FU8cC-|LfQvY+5Y`O#}j2Ve1O5d{9tUK1lSk6LM7)w2GP zA;Zzz=-%IAz1A>FE-E&?LILhgV8@GL!t^1EDd=SLl_c8XaqMF@(RKBOf)Q+R(_jC4l3U?{FWabJ{ z&{(r4VS24mf-cfV@?KbIYlD!>cF`z%#@!s+rf$U7bBYHkjHhr$%ZL(gm>VD#T&p&TOAoLegC<#e z++c>TNEaltnzKF*Mz_nIAWi$ZdyVxGjNMF{0!MDNdSlW^GOgD_5iw~8cYq-IVF5kG z;`VFA$li^oJ2?JA;;?QfX)hrjveCa*5yWhrEPK(s?5w364`R-dvrTkntrOO2?E z<@Q~Rcoo?6CX=al`xr6L#?frmtk?bj@$?Q3b%5>v@UrbyC)>4JHn(isu4T8(leTJE zt7Y3K+dA26dL0a!r9)9#VD4>$V>9PY-Sr%tH3MML$M#V+8zxJk1U60MT49gmr_ApQ zuS#P7ooy!RzjY<2bkEhQmBz+r<)ZQk>0aN<$* z4;GF;@7<&iYdy~iCQTRMW_>aGmq%XXcV-tL26}?76OPGQwn#b3{^m)kli7`b<u!pcZZ_g9nl$$O zEjI#=p_N~n#xrOUB09!qK(VLW*qEh1Cw4-&bL00#6aiwoklmf_&e!0rjCz|yw$m_D z|CuSn9vh&Nsg(f3im`&8C8{3;l9-U8xmo8MNB=UVIJu^g5fF5%I^UeQ;7E2f!i3R^ zof5_F8quaR{Nv5&4v-ZL5n8&0CEH-=QqVkVfTtZJ@mHvegHCTk11C9AB%ud_yE7x=Fx&D+hwC8o{ z{=zP_JMOD+gu9x(&+g&&1RxA5MqLL9%>(=d6TVi>T{TxhDd70g^FL-mA_NqN*Z{ur zcvcvWkeFv8PL2d3;9%IwSWGKRd}=7u=T7c=%v$zB;^FjO&0Q^9ji<%%U-N&aC>8y< zby-fMdx2OCe*{w!ZD+hk@Y`-w(}jbcWiZ^c1@Gc78lNmw^hohyzh=4AAx>>rASsKU zP*tC*|J&~3R-t3Cvv27PyPIBc%_Jo##pWF+Y*qTUBq|qN^kOruyQ=PvXxSoGdw<>7 z$(}>&41(8w+WSyu<<VBlkUb~$id!N}almQ26#2-B+s!_zrD8bgVQ;^d>wpCwf4m0u zCzoFIykBe$l~)EV8jn2HwQQMDbieYE4wSc2DIzCsSJdF)?eTp(O;ZwmFLVkTH$GdE zqW#-pDWEg`S9JFe7J4ShzCDki#2Vd>#I)$L(5|IC{0s*kad`YbKi6^qV^buAXZP+h*LobT|sk` zI&ZFbR668~9|ZcopjOB4g|ByPdS!{!y^{#J-}*4oH}v#e1yz}6t@zT=i&$S=lVp#_ zjn-#9TsApnecq!X&B>}wQ~sqE)4Adv+3=#K+-&+CSdrPPb)8wH}|7!kgqAR}1zIurfgr7JUicGT?8{%8ZOGZ?>zF-^?jW0R4C}?mNFhk<@B|FR)%cFJ*hu8dgE5bJj z#C1LR2=FtXGmN^vLj;X;G!JHs(?i0BhgvS5o?RoHjl-vr4sl`$R;gvS8G5d1$NS15dmHjFLDX%ImO9^j5No8{#1p+pWk}Tz zH&ItlE3uP=QyXy+@W9ssPG9S&%PuS+a9!RrORJzEu6gbbiOv%|HWS$C^2FD6HMJ+Nw6XpTF{Lxw9zz4HU~B+v*)S-~>3f z7}bFZAqP`7mrw51aP&}Om*3*s3bwZq5b-bVl|6VCyGpmda&h9vSB;o)_1}e`o%^W) zJ66Q6#pmcx9y>>$+pSO3GBhCprc}ukXkYO&LMn=*1c?rAzNQMN*Sz-Ya1zCEY;N9K zEG@lEy~S*qpDkz-UJ?0r@I+Gi3_BvsRD zwifh85j+uevL{wEN}TKRIm83QT(W}#jOh!Ynx9P1+8}1QcLLjNj4L_)I2U+xG7yF2 zv$@A{(DUImLF1FeqNga`HsM)!Z}HgVVXeJzvW`0XQ_zQR__gDJ&B%Zq1wyS^Amlf{ zf*cX&hJb51Y_2M^0)v%E)xe)|`)B<5_0X`GVQjZ`BcZ!m48VKrloV;_n-sf=Tk%MW z>!yPDAsRI52<3iPSn6;!@xtJO=>25+uoLBs;$wpgVAc77!V!!OeZ0?dZf0qA(b?=a zwn{#*##(DIfN;?@*O&x&Y`$HS>aK#NGu5)roh-f>WvG9Q%G-K0n*QxjHJe-i(QC<7 z=8PNc3twe@wFnK4P-6yq+JxFuU*cM!RAdBIst%yU(5`d2_`urG(RlV!SIZ*o2uo}H zs0)|#A>mek#yOR9itf+`{M zGHoH^KZRunPa2su;=y@ym>3Uht?A0{li7=B2V^rm=s+}|;9=w#kktjgz7*32xAyV` z6Um^6VzWR?Yqu{3#q-m_KBdNut8!u`eHxW7$iGmv|2K7~gr=$G6kxsGaB1AjjL!%5 z29REBn|tu~$(8QI2;bSpo3_$ij?W>2(9`35MhI#Bq)&btXS%!sxme!5ZoRetswjBV z0{9X0ZO9Vj^m(KB+x90!g;bwiS^zGr%FY5XHDCvpVF4?fBn{tw4`qJNt>1Xvj<=_D{`jrS#ZCRR$z(MUBG$%<6ml zq_?;X{9z$=Z}7-`;JJM67U>cA0DQyqzmgLR1)V~)G!&&>we}w>vemX6YC7lc%{z9u zn+=0+_uiN8&q45BeRpONhdnQg!Oj$aM}W*mFXHg?eBTK?pG~DX=yz+`epP(%6(!es ze=u?DpuNRV(k1P{mc@J{F_xdQpBNF&ALXCh1Is~{cerbA=FY=@_PwttqaD61K+w0taMt34e zNQJJ4vDF%qTVCMLH$K3j=)7~uR&QEux{^XZj8W^zKs(1eg0^$+-#RwoWW8zdz~>gW z4t(d=P7f{TU7H#2zF85rg*k9vBYO~?0KD7GlP!f+2gCBY0lu?uugkw3oj3o%+`<(O z6H#fKA%nL!y3FlEx8I)gBbpEU%f^VHvk_ge=?v^+`K^Dp{hGvialB*PBkw?9Zm*ev zuA(V7ETA8u{qJTj2l*3v}uh>;;4j?OG|dtRdUfHUIod$2PDY_ z@*}%Za3|5gj14&@n-t(@XIotVP=s~_K;iXZ=2kd;)rqtce7rxBbBGtj&%J)GnaJY?IeOD zNGS;HZ6f1-MRTn>&Sv};5~C&Sm=4*CO{Leq3-5!vD`aMH-e-XoM!o^xB&ZysZF zm3&$IrxqPJA$o%mA%hM`iEf{glHuCCgBtc)CCgo0HB)Kv45}9_!3?6epk@?2q1TL% zz2=1=#ob%+;mZ|3%$x0zmj6Oem+RHCld~6L#Y$HmW}vO6NCv$WQnhFr`TA3TL7tDL zks)Ig`{=kU)g}XT1}&Y~W@II6&@rXh>G!#F6wPKSLD}-eB0HA9IiU*p}Geh`c(s$Ls#y5yZbSDlaB{ z?QT&fVGhBshD6?x8zDfqSH}#8r>L|alDUpL`7o*^eE@S2J zpeeTSdBj0b-eA|=+7^it9&+o0jy*>vhF7#Qzvh~`47?mkl!@^3IPbu3)>D8;Y>)Tb z-K^Iuo>IyMwehci*bMB{Iy<7MTbUX$2d0HKebqt;6R)8tj*-M-T)#S)_UD?Jw#&hK zVI4MyJg_PsdY~7wItsg`g+txthw?dwwgy|0iolh=w+aS=Z>@{o-Ec?0(Z39z&gf*; zw*Lpby{2ywZ+s`~AP`$}MG{@=jpA;qs!d)yPN`Z6;t3Cwd&p#Pygfduve|1|@*V_3 zzLz;g_1q+BgPeK`w($(@;O(2&s8~QdeQrn8+Wmo6zUd;12T^M#!FRlC&csdu;?Tbq z9Cz~;1++qkuaW~hjh3d2{bh}B}-_up&r z2f{5WKg?-f9rCyBPef0@`w-l%?FOb$qHptBD7J~qv-xEC=?WRw|zj1K_Xl@ z%|ltjte#F(AYV$&XlNUJsJ9=JPO(|a1-OF&yu`=)P>m@IBZPM*DYgEhAP3Kb;3=r1`KmT2BO1=Ee)XY;mHQGh%hhG*`?OF=E&y&e@VaBf5dO;p z{mvBE$@;rziO05K+9HdWp3{b}(_ormgAw$}qfVs&<9J;%vB6o3!ok&`^2p(6H1kYv z{&S(TqvjVjBTZ^8>0$w9rc@QNtD%()pRXTxNGy@pN?FM;=7qayJ-i5`%nNhO4-QCU z%BtCUF15#y%tovBktgLxP7!S}c`yPMqE~!-4Z!fSXEdbTvMic|}Hj$Xl zC;7xw8pkUPJlF2I3rLBZe`B}S(Kn-X?Xk5pmyKhV1&MPvaK?0W9*nM!;VK%7+9MrsbyZiJT|B45079bX0@OUZU z%N3g!`H1;iAXFih)sNHtM?QRXhX+*wgFZ|>D*JcnzXEchp$|m$$S1|buKikMIt<*3 zNjsrPflNW8C_P&HRoAHcCnEvwsFEiMKKf|7yvV?(RRsglXG+l-&axkJ1r0}T#XN&a z``3wbHAz2zCOG{HYdt#U@0Q;m<4VM8k0SK~V|6K|mKeTD>9-iA=zXBnA*T9Q@9p7L z^J7^)o5#mzwk9kLCI8yWNuZ1H;O);*Zc|6l^RiO_=2nTu%fwZ{Wuh z^-Lf8K)jyBozQV86?W8=eq|}U;4b=XxVtGHz#Lxz6)t~M4Rd{YIKFYqzFLu@bMOVz zXdn+C)lOd|x_#E3Z3TI%XFm!((&drU82g&e(-~mw_ zR15vK+1}hWjA$KoOK^Ol8)2}J;NIV6FJmCsP1yAq8-pfniabTBL9S0al>n~d*}SL7 zY3%W}9NG+~5v!lC8f|3sPIkh_IXRLtdYBN2kj74)T-NTwJH}^}M4kB{;F`ZF-a&Z2S*W`M}!CHwX3}9^W7u4$S_v z^R@b5b&rqpj=0(0X&7!ryE4gP6(X10IC!E5ZndhIl81&;BkUbbHo%SK3w5nKnA8{* zjp8cz*HN%E&-WcJV8ciUWg=b1H3XoBZzJolI+%>@AGq~uR~}~#5zeQUPGZJySQ>zl zZUJRkMmRwWT2~PtiYWHJ%2Xo?-_Mx|L3{U_&x~9m1Vx{yo>Un!saFLP2+Dq=dA63X z?W0%4z+&a*t28HoF?D~Z^N%sA58#1c)uh2`+x4%v7 zp>-PK&P8J~eT}D_;DSCIiKlB6AE!}xz$lmu=$u}i9C_U(O^&u+YG9eypWB_SdXQ_| z8WTyhJwvHfaw*#28Egz4dD@u2*=45w@$@(I8^N@|b@p+Uuh@xg_xuuO=`+Fs&vRuw z%m=J93OvxdKTcIy(aF`XW%hMiO5>gBO1+NpaN*FMo=vO%vT=Ib=JS> zjAJbIp`cVn-cxV+zmws1pIu{?z(=jY(`5&8kbF>gfG0~zz}zaIGCKe>Tsf@qFm&oC z{tNuX#?-%?PO?eSrmk2%&t|t5p(5LhgCV>U%KG5f%V3#8>Ds@z?x1Ayvqe8!rYCN} zMdWwh&jZ|xc^ifce-6HjP${(g9a5qTk!AOa4~QBkY`>kgyHq9ji)~1%#{`o>vSVE% zpqmw+gDf-odB<5eDePE%_wiHllN@ zeB4ee%z~)OG>d|<=guah0tc4N zhz8E?P!=&W!U!P2H^?@sG*K9Y?k(V`g`m%Tr{qHUmu=4&9XHOc6}EB3?^RgB}Z1FQ0+ z&Wxy>YLfXGpDLp>&IdOyy(W8+OBT;`onR*x-9vdB(r`&1I12me@~LEXslQW+ z?D>Q?xkmtMh1`YN7OGMhrwl;jXX5B9Pd_dMI-};JX*B#=aMVr3)A6~6w0u5QJNmB z73WDDAM0EypZj;lYsMB!CIpM{+~avugWI$`YQ}?ER9}2XbyVd-Zn1T_^^Xg2g*WQa zHJd_~MJ}&)LVrKdXY!Log|fE%amP)>;>ny8q%&N}nUhdI)$_B>=jay*Dcax_{=KTc zmMQFxl9Gt$3Pm=7p9>4Kf`-f~sS^)`!;dOLKq^+^>+!t1D!3_+?k}F`r${w_sj~cT zdHCKp-j`3W{$iYw6Q5-4W3(+){*QjwFbGMom6Q5anf&sox@g~W%{$}^cD;vDWHcL` zeFQj4*@OQOVH!BNj}7rT&M+72&JNHAk~ARr)gidu9_^2$e#N_Yw^$#Nm4$>xs!fE} zxrqDhAn5w0NGORMSlK`Ev)s{+env_g3<9pq9bTg%oSo5p9qv6AB{5-@>-jp z7PEP{yKJJ+^Z5WZ@8`7rmWi(llDc0RfNw4XwCw%OYshiwTkfp;NKY@GA1;?(LVLNB z6G4|kO-Z7K2JLSZx7^3*p*GpZk(Jl^%@HM}6(mCODih98fL=@X>By~ml`H!ww$9}J zvZL{1VoZDhbiaN}rMj5rBx6`2FIf!{`hL7$FA@}whtkz-wRh-gFvV|dF5n;Vu5mvZ z*5#ENYyfM6?Kf8UFMhJ9{zd8S;PzqnWqZ};ZK<6eOLuvRw34rJ1CgGhpLR>afw%fT zy=nX6_oVjcKj|EwJ2l_|nfU>-*ii7h!fD_n>^d!Fg>M>cTO<@k>DlpDlT7ow8F-!>i`Xg$Hnj#>G>LxcX2 zxt=>`6JSn~bd~r>t|h6g;pa~wGF(}F&h-`yiC~?)?52)~dprLNch(g71CBW*-Qi)$ z0G9JS2sX*dt7Is&Fsy93*NV(sGW*eN}adxM>_}U?*kn7a(XGryQnYb2bD5di?n%3 zi4X+VA@Z01K*T``h$Wpk^F25=b?aU?O3ZL30dwbr^N(xFt8}FO{GZCA5ciD~P5*)V zf*jwJDFXJDUPy{{pyT`Ab37^|=?55N4X%~WF|*I2q`olX?xY0H^dcRc!|)2JJgsk*v~kC>E+yI^Khue%f_d?0_TN2#V()erJ`kfYj`BtHjc?{p^{s-lqBB71YQgnn01=viCIlXz3K8nJ2hQB z|NK{~_K(b-fZc4B%~E6}Biaej9RyiAy%pcJB?=0>B(hMSFNL{26H2 zj1=-jvl&}ktzW>%ZFC&pVp(V57}#j&_knmUFcC#StPustSp-@RvezYdvas1f-H9iW z#BCZfl8-|Yheck6{_B{FwS&oClr=%vNfNtnq~s5aRHbfXE1wWM>Rv*~f4hLr5&ciHbOb5iaX4D!xR2ikc*mtnIKllo$X=?Z+T(eZIuhQ|m_7`O8H&_(FYop_opdHG~#j zv{Drcba6BlQbxNA^Bz&+dIW(S`0?tWwL{ape#^)$Tr(E0R$h=9v&3XKy4T%yDZ^iuv8co zJjC+rv&nN?%GqCHPu^NB07IGy+dWpO7o1oWvog6zCSoU$8nvLKtAGB1)rETeMi05i9P$Uj^Ua{nfS9qx5vkQ9;8}o zQVO14Df+)oi4U=2fNiBs1joOAxD>V-)fg-A0NE$2u8a>5H$0yLxM0TuI9wA*QTy#b zjv<|ELKeiIzUQE5)F z^v~b0cD~RkS}XmR%TXyp2p7gF1Lig^o>7Wfy2rJ+BsE*>QEKVq&M6+1O0)bY==gD} zqwF6Fb`sWICPN6pt%Cb)A6ER;W68qd0!qasdD9SL<J*>TN*NE{l|M`cLNel7o@bPukjXPmrw!bc!GN!S z6-hvsw`Hl`RR%`xTQ#rEbQu}+rGfDQ`}}~8a<_;~nw9Zi38KQ!jTs(TSZ-kgsT$Rj z+#m?c9JWn>NBCo120VTuI7Ix0)DJ4x9n$Yckwm-e#+bc-i-u&pN^*ZFXX^(HSZpVn zUg^E`w$0A@mQryu>@MH4I7Xv{>qH6euGyL3ZLi8|VHW)%H;^N78hUN(EokwD3#9QD zE<+Y@i}g>j8KhWhOH35JkEXkObeTS>ORy{r`P)%?^xB4Qm2CG#KjgDYu#Sq@3s-O( zKi$XoV}#w_hFZC_75RZ^5LXRz%O#ttlTz1!-az}1tc)l7iTc2v5Bh?QmY^tSgZuEg zfZi_Kt_4ROblLGp``xgi8H3g@hP`G|CNB0joP@CfQ~DTkO6VqZ4`cSYw)1z*3>_qH zSil3p<~uw>C#_gqocpmPk$C$y`tdEYL7eDGYAI5-JM zXlfDE1?@j#Rg)JWGdz@mq06mG1gUN|ZiH)d{}gg0!VBm<2Mhs!`@S$8Jt)XRY+d)> z^(0Ry?51ib=B0EyV(ifDs1rTfa$eXtoW@ef+3BxsZF1;RGvILnUa2dzLXRX&UGh>v zqeXH$8w~%R!-=pEMobSYo9JE(f&=Zi#B=|7dHlI z&p))_0eQ%1W%YTvJ38h>fjjP^fl^f=nwwuuA33RAbbb;MIqvSKUwg&_&U&C5v2<%i zl4%H^EcrdwIumwjNn46@bBr1p5_bnVr(X92BC^qoG_rnDUM*{6$psG0Mw0jG6PAulg{pk;KM^-;u%AtE~r`9jz?Yvp(T zv4Q>D#xB=-JAcxNJ-~~lOdluSYqUQxsDYYMxc|FxMCemGDuTogUn;>Zd)^bHn=;!v zn#A?95wk|piW+?%*#Y?A;91ZpR1{)ub^87Go9~@*z~S3cUcV)o_JElc*|-41Uh7zr z0qJ9hX(_yKp+y;WP1?*ldwytLyQ@!a>8!6jVL)bEG<@(31DLfV+zyw1|};A3dnl8f44C5e*Zc`8gOTG+IG6w^dEVl z7VGt-OVB*P|GX|SDa*NiRP;I&%i~e&^BD-%kz>(X_U6||D+)n22VNyK6S<2lG1Ku4 z-Pai*=Qy5kUW7-W*}FZM)Ii;Mvtyz_N``Ghs$-vyP`2D6zW9sWjg3b~!QFw6j49>* zhm348S&&Hd$Vb%pcxpAFo(J;YRg@V5JdlBJHO63yE&M{%KJ-y_4DK$k+^H_%!e&v9 z3)6|p($Mv^g;PR;KZGdGYvKI-Cd%n$t)h}B5bB;ofhTW*Pc$rJm{Bc{w+gG`#V z@TGseyfPXK9{M#r6mL#uy#V<4a2bDUbX}KVxM3orrYq~A{X%>%um{W|jS$b-zb-K4 zzKciU)Hz{PTbj6IL9JUNz#oDF8{H8wwwgh|3lLtZ4+Oidh>1Z%ZRmeL8J?7i*p?zD z?k8ml{MIT9GjOhbI_fclE4^&^bMgMcq*0p0aCceP&Y_bS0SmG|KCV=SrDE%Z3v6*9&6`%V7Vab)Pv9l>Y#$DU93q#ZRT;Y>*pd zVj%RT^<#A%za5;;Mob(3f(g9;H}q^#{KJM%awdwTuI zEre`3wNzzQgQMmAcg_a!=e@53n@pBY6esp}GKPU}h~Pr-GbAE_+{T)nqv2&P$!tpa zz2STQ%xS$bo2t@)QGa#}A{(Wa8cn#M%yExu5&;|iC>gimk@t4fAnea0<5iwPq^edo4F(rDNSJm_m06h(4WdBw03^8j6@kMtqTZpMxDg)LS zv#d!PLi1mZB0ZE)k@6}dC35HYVy~q!@joXO1kf6L;Kw5UsLlMYpdKH0J4%m#Zg=zX z*ox-XTDY)wGeO6|0c=qD1O@Tk%;f1`H0R+^AfdcyRsWxe*TovGRw2Qfm-3=7id{MP zI@G;Rz)B>pbTZQ0jyu9YqZF&Fvbm80DZt^3?xRs2=S_2Zs%rqJ69@b1w+$gnbJ6v> zKROgs_vBOl;#U&ur5*{&qMpL%xGSfn)i8n|hLvvjLK)n>5DJxMnd$&Nf8p5~*9>Xw z`%%b)F0RF=`TUWz5$^Nz^d%{Vaf(Ehr2z>y!mnK`zs3{<*KFzzz%U)ANW#Ez<1}C8 zBNUGko_@u@6E;fHnb9*L&e+T+thZP_P2unyAez&VzK);5C)4~Xt|Pm&Y$j>n)V*N9 zA4t&{JUyg!;H;3|&d1@Zf2)Vf00>S_hACRm#h$pEC0jZiZ&or=7@v#1b1RaQ)D@Gc zvT*vDNLmj^eFDezibyLW)zUF@bDMbP*{TW)-wqnpTM8+zPTK2Gr%o{{rt^BEU^wZW zOy@{bzcq~#ojcydH+2j-DR8t4c#Wn0vhw^AM4Q1>TH=nz`LlND^hM8#hRJYt4o9Uo zrA^|D3+He`bIj%70+kNfssL{%oX0j^l0gact6nSV0{4uSAyFT#q4{K`i#T}Kv9+T1 zWIBTW;?lbXLQ_uMj%v|NScZ&^!>7l8FSMmnt1(QQsQGEy;|vd;l6H$B_eeR99O1aSxY?7mwLp_=KeGLo9<(gNh_HJXkW6Q;}{lE1?O-Q zH(=hB^`jZ2U(Cyyg%Ime{!FDp$SFL-nTvu|H31qflw)GEwO+3b=n( z9{Z*o)kSIO>t@y+KF0Pb)fbTR9T~)lrT-W|FSZJhS=O^R6vLGI-h+-4#w{=^-rl5i zM_3sTqMhHJe^|kf=WDzw^20XugDN)=yO7ujH%r_qeYwAS?}FnnHv^KnGam;~F(_8$^)2H3j((Vaaj8$QQEXFzj1li77JyfUFS4DMl22TJPzcW}o}o zCaj>vUk1*NLqD#PjLa9eBUYoo{Qr~2idNgDl{Jo-&MwjXN`~boS!HvXv<7`lG5zMZ z^ov1RFZiAE`Il)LevByveOthakux~i`8~RCX4~}?)SLQ@C=o0Fus>f#3vEJAdta4sk}Ypd;7K3$AjKb-^?6MWEkbHX~pJ?NB15lmp$P> z&WfO&zD8pe;i)o~yYu1wjfqT!tYHB+P}u9UID&N{qhau@P_o2{lwq>y>7Q%-;O+2C zw2fmNpYOY+li%}vW`pRS6%X~17p)f%zA>>COxZ)f|8#Ux;W5jy=sclWic47}-b(o7 zUtGZ+;ilVGU{Ify3(-Ct*p;cSEk~V zMe#LPRyFP=_v3eFq`4XB>JN*K(fub`T7C*mT5Z$wzn2E{0b8snM8Enz(;4G%C)T2f zkX_Xejd&S|2F@IEuC_&Q$Zz2JUu*42Eqy!0ad6)M`SrBlgLV;>(^sw*Sv^JOsu}L| zJlBh!DMhZF`qJ=;w-EE!wn%zswgH5>Iv((ymIkF*3`M=w5tc(z5Fti%1FqKTOyr5Y zGE(TD@wpqVqQfTu_JA%EC9>+9USlvizR&++X$t!h<1eFK zd5Ly;+Km4;X#a4J#h^OMD|7!9^`-SQVj9qT?$zUfHk+YR88H$GaH7W&G&lF-MB^;#L;AH}$(F@~?Sc+%1RO&# z%bAYiXOJ(&r%o*4n6uj8?5tjxVQTvApXs%di8wn%VH#uC)r{_QSD?HZq?Th!eDD{) z(u2jEc^)hy7EktGloR|ZOosjfiEGQ9Q5E!~Hp9v{D>nzovLBM;?a#acOR_0R|I?Za zR!nX3t~(2T$Y5d`AM08j>t}ZcsH&YJ$&W>@pF)d$(m7*PR>=##nKrG%mZs+30G~1U zZM7!__mMphcB-+Ty9(b+(8gip!e2@QdgGh1gB#i6O)$gL9E~>kIWDCHHEN!=DfFEj zTkaAZe=22k+goL4Rs5Hvx~6aZKx+P$66cjx#n(ev-X2RC zvnm6A$kq3^XUk5S`kN#vhUwzw&pANJ$CURRxIGxdl8EHfy^}f0rNth|dZ62OGxv1X4cygJalzXtBftNttz(T!(Ee(J*23cJ9-`?a@7M z)*{m99fw9FN4q!Hb8c(TY|@qwEO&~REt!y(QDvhQdLs}>G5!xU-koHXmz&Ef=C~gy z9a`$T$b&7@rY&MAT16s3n2P;_2g+06x*OGq`G%j4JeIC0OWi(Kb*q&bg8p-mjul(a zb_zU%7aUjni+*>B8R+c26c+u-RVRAc*7K39A*AYMnqs&d_IO$=G2KGXQdIpx!Es#7 z{kIjE;gnIskh0IIbS045p!jzAIyL~E`h8w<+#8DA;9IP(`FE!8snQ@alH`7eEoAy7yQ6BBu;VYGZRIKDe`9SCkUdcDhSrw zGOQlwh&*#B|EE-@-+NnMn-18t=7jm&c-3FTxHbL**M2yUN6=l|ScpBoTXv~wLgheh*e`_YjMt6cE03?A$EP8!;; zcAjkPR7=? zA!!XCH&2M_=yX+c{pJy9H#s*4+MHVeNLqe=@qt!Jc;GXSIWw9#5SP@OOku7*%Sch^4t@}j-n}dq?bEuDD ztgWw#Ae7)Fjtn460Zb*U*vSbhW^OI!2#jqdaT|N)ezntA zZaDdFDe3yCAZvc57(Oo`KtPiP;?6zEEhKV-sQ)xU|7Mokv=EotI@57<3tCzhwC)1C zG+hQcXHD;2nsSuzo~Ny14#ekPza`!bI z&mMs{E`JY3L2bYyJ|7}AH~p$TM(L8a@lu(oo~iMx5Vuk`@2{HiYJ@Y)_ztT*dK@*8 zPjV2i@XMpyceQ}popVs|Z!CV-3PhbDH?8$Hoxnc8go-e0#CT+YS!YP5*Oa6ma7&;t zw=aVVb*B++JPcb;El8rERRYv-qW<9|hYDvWcd!mSkg&!%@0E9jP^>hDrsk`uAci)6|FXGPhbG=*v2vIi~h7uG2%ebxa%fJe=I`lUIDCL?=3 z&TqvkzmEfSKCR7u!m}s)AeMON!~eees|E@oqBuq(Mp!d!x~m}3b@~j8d_vo~K;a|b zPG}L8{zwoLmHs_dJe(^e5dxnqtZnGi%;lL|iM5NI@N+6=Z>-{&D@=$M8ipV6m@DG@ z@fJ;8fhCLBpqRrP<#)(6M`8&q7GpCk+m#3sehQ>orsq_bwEYVQ(X3ku*)#|6(=}QN zgBY-M=XAsx@2uh1{F~xT+(nUHuabdUu8OXQbk8Bi!52R5{-#>{o8wG`nfo0C1CH$j zy{R&9xaYh+-%+Ii-(M$ZL#H%}Tf2-2S{VJ*@db87^r^aH>4(#0+3sK;op;%@+#r(X zyuL40R0+PV)a)!ggY}vN7G+rg6zzem-S~USs~!+=;2$*eg>6B$u<~vg8k;2u}QM0tLRNAB~eJte3k}29?Wdsip z#=XwRHNQs)_>Z$mc?oF`?WF47ce4|-sq!i6&t zoM=eZi3*}Q5Vbr*n@!(VDX!>#E>-G`=WLc&-XNtrww2$pqx{wJSwkNZS@tAbk``g4 zD20Cj5ZbHo8Ln{ZO2<8?<}x7oFA?-XqJOrH+sb{y>Gbd?ZhK`z%xTyMeDPIw(+@N< zi021yLcXU|7CVgKk+^Il*8G()$QHE!Ivv<|*s|gHYTzCq;cY#h1abm7?^E64mJMqn!I3+FOE{Z%$$szz&R#_guY zo8#5uba)6ZeA|P6p_kS{`X%|aztZWRFnifV9&8DcOf*0`IdLXp24p0rEU3(mzD5!F z|9Cpb_Q(QlTX&pvP_fO9*|BZgM#py2v29x&+qP}nu2?sFpL6aHSWkUftDZ5(9OIq$ zTI?iMTGT1*Sk3ACfpe!y&xF0b0Zq4qLa&OhY`lU+VMroBsz|rFcsIn_itLCwa6wM} zi*bkEpgk)Ox+UEw9M_WRAv3&E`5pon*Bx1b3k1h&Df=TrO9Q_Wif2-=70Au7*C(60 zr?E4^J2S_zfVPI?2VXDOU#RVl-*J@1ByO7k>cgqywR=k1(w;2vHaZvbfMwy;7w$lO4~2J1NiZm$PP{R zd_xQ4)_r3=6cq2tEWU8Gfre&!@)q26Ztq>}Qy%7&xe(!#9jsK%MtbV_%$);4^oI9} zlxLNzD1-mLsYgbty^@kMd_IG1L&3jPc%5S}pl_?|iDnprVAEUlU=>*kh`h+?++YqR z3#iZXVE!rw2s9otbQu=lHJfx-I zLw`dSiMwE>eM4#QQywTom^Xo*&ZlBD?`qi$L(i?MvvDwOYN|(0IMmM#PE<&!AcGL_ zA7B8Fp#B`hDr2UBq&E%-2^2=|d@CqY86-U(-kXrnI$k6Re~fgWoXGEt65nu|eNH3g zFW$%`hw>!XFxB_$0%0SjDw2{W4 z9o)#o>E4nN{sMslY6{xDzUM=k$Jl}AQ38ijapH>o%Yzwsf;tH=3zw_*ZcCTtYAtW8 zguQ(&vq6+TYW4q>No^1-taoZIqdAvRRucV|BDVAHtu zY(UD%GMlOxRa!HY-eMvMP4-Gyb+mg{eS~zalp?!IR+=H{H9!H-WJPRweT*XsTNX89 zw!D1BJ8&g>?2q5Wd_yF?*5{R4H_{dc z5-C2PN?oDx(m4cA=!w`KsE)MLsXzu=Itgw|C^-p4e(nvS24dwP;b3Mj!5ERAkA5g5 zF>3DncOQ|I4;Y8hmtyDyX=2DvO;|JzlPY^jg&8q<8c zX2e%xbi0p{f*#kc&GZ}~)Q6;^ZF38RQE^`!NaY(i5r9M$ajpZ#C4hvb^o(;+EFu1g zslflskTXR>5}?!xze~qYLV`LW#eTV#j_DkrS#d0VA{$97zlXTs*di*tYboC{4 z-g%EvPy6q9tM8oYpXGnHiAfM!cIfhNVXDkganOxPur;3UQ-9#42SChlIsz~MI=!lc zY$KKl1pi(5yz5Y#HDI*_0hl{L&kIr7C@S9}WX|Y}hM*W)^&1&R+M}9I<-1c9M)3N} zR~2Rm6VaULlIs?tzK*V?o$5M$)@DO9b8=5RLlcnV$g#g_?Zop8 zt#(MK2g~raqN_b+!no#$+g*Bj_=iQQe#Gotz3AIrBRdD-SeHFWOV`zg{tZBUDgDuX zYNZ>H>wiA+pWq+aB;peCVm>%X{vM(z!_04;h1S1>3v((=xqW9|(5l+(@aZQhka8ZF z%lec60H4{+ZxuI*U!FU>G3qE#n8KzgKVV`6N>T-~9ScFsP7;8$cV-Qkgo?JI7j}c| z;%Yq)>7O(pl$*SSCNAziX!F-TLCWazW0YH>ZD8p0IkWDno~y+8-SXjqK-gfWNCv?+ zrjM7t@jJobz)t`1!Le`uBf2g#yY3y?w=%_S$bji;S8ny&Q#`@#@51@CT&C5g4?Yd6VLOj#j@zbvhzx1QDe5yVy|Ts-hO&Y8{iBC3+1@w;|S#r}PHnXHX3yzo$NP zFaEdnzY}E((&RkLe{`AMTx2C@uX@geaC(B@*N^`Boj(j~F(+9}shKSTTL@>54tutP z)bz;~7r?&&6r1b4{B28DYLBjorB4T?9Jb1r0IOo+8Kn9hMsXvah99XmkR#SptjB~b zR9js@ml<2W@mkMMTiJ2o=J(Rl@Qy{hwZeYAyX;uNXp^ zq*auWSZ{dRkQ$tU)c(`$WBj73FoJ|E1-)R$EKruN^XTBD71K1|rF!Ov`6v;44% zw(s1*tj^S-#mcHtB1gaSAvd`S%e)f_e0o{|T%I@LXBOQSg`~5(Mg#R6KXNx$+RdNh ztC+{6G0vQLPJ<`2Jsg(sbMgW6L%kJ3TVk&;YL3+N+i7|Sn$#cVqtr*ftXNb5LO;im zeO~vFO%=3_l0nYS<(Z`7=9DFs^2CF3P717w?V33t6cIB6!6*)BeUKoo!Q>WXdwx*3 z`iBN=NjM=l&kdRQ$0iU842+rwDv{fs5Hq+7YbwPl)V101PM>te7@BO9{KjI70{fHV zvY*M4x!lM7!1+^SE=X{GhbXKBkd=ZV8mMcprBn;s%Xo-{M}xSs=C?%w{xN$h z3Ll!97xLx%f}D1XGlX_D4pA96_E2@8(r~ID3E*0(<0t6m%>zN*{Pr7(mfHniMwE_8 z)61W%!~GmHxpMXq)z<%QBMH#@3^;BAn{zE*4v#?Ob zsTWDKkRycf^1P#JyRXf_o=w{vc^`j*8dH;$0u9>|pP#^Y0vhl0+v|6?1;H55 zufI>wy`eEZaqRld+ApX$N2_!c)D%-YJA}G1tmZE@CFe^3oxX<8GZalo${~~v|3I(QO6X(MyNo*MELvZ!S@!(khr~(+fe0KzD(;267hq=1*`< za^v}X4H)D)Q0fyaP;Iq{O7pSmYB}w{^J^v%wrh><-~a@Wat|*l(y3HqRbSc-?&cZ# zl8zHgRh?j^Ft#)aLf~GS(DY%jpnKI7k)rWiU@^P>rcu-V*ZUgQn8Xx>OJ3} z)8#b#Hy_*t`<&XM4VF(U(2rG!5nEeqkaE1|bx_YqahQhz$`_d#Y3nuN6b{Ybfip z+3A(Ok8WF@>o;dB1v~{TeY?Z|fX2Sts;0I4?j6IQk$RoWQp~LR=JUG{AwDR_7F4(> z*1SiC-NQc0mGo{Mvt$*`-d6qe`QJRDfbcw780|}^E!%&5#=;l$2mWg&oMz8t@t`ek zFW);5BsLU3+Tlmc5aX)@p|*zy!`%(o2=+`zBL(ASnX|g> zP1X&SB%h57zw|(`x{@uYaZ`AbJ;gq>`;Rm*EoM#I+0I2sV2St$5nVK-rX+nGoYYaD zcAMj0=iUNs5YoPAFdCN<7q!eHxkUf)2vw)0QyoVJAY)B4>A4tHz;~2}LG=S(2=P5? z41VZr@dswXoQ@%HuvHz-e5>dkX@))W!UW#;m1sqyD39Hu&OQ;c#$@zbO^;UVOU>>v z9QULqVPwi!gVNFnV1A-GRgDj=d}gP5w`Nz|Z=<3-sn~lGEZMJ$V5FxZ@?17JMtK;4 zi^x%Nj0uoUwr){lq$^}*h+jn5@DgBIpD5lt_8I08-0s>v@y}L!mF=cDkZlaneQAWy zx`A8XozClx^~*MEs{t{E?HL?Q)=&5*`IrhPBQ@!r`;rJ}&xeK3`nsGe;SZJGCI5fT`^EtF~b82Q7Pai4hm~jOwd? zQe2%Bo$iw1t_MO$?>JKriubnjSDocF_qVtCvIKp%Rp)TI(Kti}7ovL$K|#)8yfGt@ z*KyFn!@{;UTv1eaVgl`7TB|`3dvR8Z#6GWNl;0+SMJy;cP6c>&-{`64A#m0}fw159SX^dn2_)x909XeRzsY?t1sp*Fd!uM%5ZKPWEgrXG-JRx z6Ym{)Y_+5DChb9_>#%P?TSkwv#28ilN7hq``d?<5QYdL0rDLjqP&_*G5i)U3*`84n zFUr7%Q!nW zo~o{}-DB^Et2bRw)%+L0?X+(MRn9p^)3(QB{~H)ae&e5ArW6mZ7-kIb ze(1xzANBcbkFIx671koHDQ4Td2H0FYv$-J876_Lk&%?$|!PbM-ABbH69xc!nrP%HJv;=ioL zcqG=yn!V18$iFuQoqaLDzI(4<<Gz)Dh3To8vv9 z)IX2@p?fd3&4_-^_OOm^H(0a3Q!J^cHz8f1o}OKGGQ5~9j{W-<2<&>IH-`}22X8AI zxmlvC$5#m|!*L2zf&qJ-W7wd5a<>9u6Uqbj34P;d62>=sM@cCnzF|xY;&MZgp?qzq zrw-$4(9H};#|9R_D7!dF?@`m2=j}n2{^ zQ_*vIfDv)*1Uh^WZb3*0Dn-Sh2<(ZUlaxO~kTe4a?Fm1~5K>|hPk%)>_Q^RQ8!WjP z46cS;VbhB$^rc)D9hY3p$G8g}jtPiMP8ahCt&(m`)MC9Qyd6ct9+R}j@Nrqk@G+aY zZgQBuiMChR-p787L_@q?+zo}ZO2>@4Z7 zH!L(1Bodq=Zq+nj0YUCz<)$@J!oD-M{rYvz*|=~$IFE;J7*gblXe9Jo#B^aX)cDDv4ibc71*Af$a6Q*KK%PnW?-L&Q*7q72{y}&W;gCI( z#Pkx>AiqBILASqL-PL18u_db^nX=d|CBNp-YAOgm4v!~%XbTtlc6O-uA!9!9nqXw~ zZ~hAW%+6>y^8=tK`bpu731C`X0`N&CUrP$9E;9oQP#oiXZemM196+C<)uG84VsaqR zzS3U97RH;m1D$^<{E3BV#9X$#siW-vKFNJ`EQ-Skg;VRdUU87QfQ!^XoLnh-xs)PJO1}Vwt z^sAWtT{rAc@A=&~({DBe21YizQ9nSW1NFjYxrWIC+hsMG_~gLg#oqfne--Q)!P1px zYAyY2y|c4^bxTwAHSp60Ri0{vgmkS0xX8FmFr>t5&0_!#cEh;B<_=3Ua55sz(s zZqLJSz^-~O<@u@?{DmhYK|j0iMI&ppoMxX~C1A|R8Xv{JAQjbGLkLX6G&G-1jngAq~q(LbyY8vjO*1n~y{A8S^t`Ln4+xPFwk0-p)Gz zzYLvUS|?mIiVX{nI1Mc*@MOo715GsbQ$4V73eq|J$9FMW9hYY|#|LREtnEgSK3@Qh zII~Q4^p%HTe&L5~q2JXKItGbt$IUX+G&xTkDazaHi zr_6?lq?SyLcDzT>J%hS1QCH?E;iD#p6XV-=u%0$&gM61vk*t(HU~$si1DlDzoAS=EpXou@z znPQQPMgN;~BZRHVI*7R}+;5xjCt4plJZ$hrT#s2vcAP6?-SGr%@etEXpL&Iyu{41; zo&gv|?glB}w`9CBxVSB>$l@ zdV6=OXDvkdKqQcaV_=g%j4J9+-axAjjfjU4_48ES-{!Do{@YJkjTP@DaT#0u>Rr+3eKjxTN^=X40_hJQHTtA9+@!kVqu0vcV; zH@p_{h>X9v;$la3bEt}*CQu>egz^m@+Ho;0OyINi*9TWlJ!H?Q-!0g^lZHQyG2a z1w4kOo*v)iGu$)@;ZF#DgEL$`{X24>bF>>wpp0L61udy^-@i`hw{@5*G97IQ+ zx!&Dqro~R9U2X1WHgQayq%NV4>9*GE@Ev;Wg2IuxU2^jLitl8xe(W?$cbTW{WbG>7 z9>TX+3p2*KyXb8;v=KjU@a}EN_@;C%Ic{{2dK1h&2BC z1_zUk8RzR}2``z(>Jpo8^k?O4$Xy7rq8av0b)&rmVkH_AGtE5^alvW!B#sm*WHm8n zgGmx^CrlY7VLuBOxJ1a#2t%4VN@a_f^4UH( zutYTD^7kA1lhK6$My!vC7*C%CO@O`50ERuvSgqWQlteeYHlpr9#;{wgjTO2URqa?M z0d=@Jtj)a<`=`z@>NTzt{+YGquA+6$=vMoz)Y z7-V7;G4J9~n|E1u(M?i4n(qLPrC^gUjbyspL(%~!qTJSTbZE83$Idv*RqoI~x}Mhy zf)}T>|F{t98`R1DcYoAHZ}NzNvW}bGJVGqSun*=P$F`=niwFk~ubz`U{`;&+#|Bep z*+kVZa?1fEs!Txg>a^C+ac~omL+EL-n7aH<^%@w8{#SCq>XFTg(f%T9owchj?XM^9 zI%QS<#H6G^4C`U*x5^+9+^4xAs_&j~w2E7dk(v*R^>9zmjfon%hATSy(XDii@k&k6 zouU|D?&a(V&MIgPPYg5kF8W!4Zt)8KUCavmYU6n)_?M&%I*W$iWR>V^4=vd z{B&o-&uZ)3#f}l#=t%X6j~wpoW@m4Czr~bN;9yM;7`h2!3>cFU1bzadgdw|5+_HD3 zD7&b}hkj~UG(q-c8pIpD9f^}2C$wDan~;{M~7(C`Iy$sJ2v^Hv62FdKtaC>FLo z1gXvr?6B!&>r(Ee#e&tc@Z?p7)%xvm@g;Oub{m=*W{0I#%EzJs%gbERu+8R?Z5;wP zo8{BwZa)$iWTly5&!fxa8j6LL#x}eD^4W7hllPNEcG|87=_IB%dbf>a$_%Z38t*$; z@`|$-+&5Mwq+`O1zwS;jGpsk#bPNnZf0O`BqfVr&7`k&&4tN2Ass{uorqY%K4L4aS z<0lM=L;SO&7AXh3fxoT7S#*AlWXIY(rOVVkbtyvqvP}APaU3hoX;Wj8lVh{i?;r>W zA@jm2gI&jDXSi?pX^@`hYwEg2;M_ld0^``up5rAMJa$4P?l4e&ve_Yw1_NTD_%0jo zk4RsJHDco8M2YWLND?OE&kqp&G3qYF{~?hecnFp0)2U@sw^RnU#QmB&xd(;-#^$Sh zJjfW?{eNCEyctMS)Ry{@d{`uF%q1B)NDXoHCHeKA{;tXhGsi(=C<5U{SwVCPa282n z&+*x@5ugEAa0;F#p1%am+#qaqPd#fgJiAU0Q&mqyOJF9ku_ykMAIAD~p?|)pX8OJ3 zQc&2)?CSnfL|CN2dkb1KU^c6&1w3JvPzIYm+1y|Fz0B(GGW23Car7yT zD1>9aTtEFDTqa^PW6S0y$sUn1dq-ICMADw*dn`H=S?&No$8qxUKk^5rwT;YB09#y+ z*X)>MeR?Wf*dh~;63NmwuMtTq%^X$$8A+n@7|_{L%5t!`j&0j~jayZ#$v*a8wW5tH zENkX|q?)$^E~4(x<&Fu%L&6?1aL*+331eSS7AcsKlHz9M*DGV*Xf3bZT(Hgh45$@z zeo|sUYN+J`BU671xCs+cO#26|iF`0Xsm6;iM(KDxwT5R>ruUm`)bOZPKeFc-iE>q#!;;O_44B;nX@@vk=H|Cco!Y_pg;Re6^RzyL_vmpm|6KKWf4k z75&Zse!#*S14AnCGmJ3j{JJJ18-E!R7JJ?B_`j0-M>huz+|#4h8evugKqRZ0Nt zJgapOUc`C4^L3{xCV>@yxt&;tb)al0bQ%cOH?5a-`SdqKiV0C{jcYeix6}ReVO{v9 z613{lSi+XAn=i}OjovWGPNH$VuFv{zEj&v{gYWz1=#OAqgf)%iQNa8vBDMcU67CWv zAQX?~?&HqPjyv+)dUbOPZzu|r|iE{CJiw&y(ln%>Iz zVUcX&>P_*)V_9AG@2cwT?CSK?+XH>F*rc#FDwaQ0Kv*9OcL?)K@9r&)aa&7&(AhyoXZ?5W`l0d|A)NOdHwqE}}CeAI( zo$j7$i!~m^|0O}_=(zT*a2zg>`(Ca}G+p0~yIl43H#Icq&wI3r`<`d9#OV(M8&2x- zs7v%4KsjV01H)jJr(!&R;DQopFj-T3;EOIVFIz1is#z(HNm~>zw&4Dm2*z2&x7WoE zwhS~ViazgREW)y9PN-#Zb`X?O8m}mRe~wGwKNl?Iwg*UOG}Tcr)EHQYN&2G;Y~bv7 zNxAP71vOD^&$l6j$!JHk>AY;N^oEk97#a1q4GvdSJ&b-h40IY4;G|d^zlyKuDVk(~ ztK6@Qqjf&lUT=-8g`SlfJ^e|12QJEJsdC}W!_~yrs*BRt9J&m_O{j1nmV~HX+|beH zEX12;n$1s`&kqH%C*&WcyTGtl);DpOQP6ir#q$FwTNqy@|~d-bMx zAB&<6tJ7uJ+z~#`B*bEoU0H~4zHk{oHSD|eTg^8Q-dZ-zni|4UWdX%*04ygk7V@%d zQAN})`iGP_$wCg(cmPVZ1*+UX{hXuc;A^baO*EI~Bjd`o7f{VhSn#@a*GS%h*F~*J zv{|}K6EeEGJrV*f%??_^vQpT4uGnPEn5}N} zcrWrYvgu#X)+zVBm$`eYxPdLz4(n;vw_oGc3Oz}$4Zx~d+dK4aHPUAi%kg!pPa<^S ze{{-H88kuJp?lz=YS=3le#(k?oVB23OekruV`gVI`f4$Twt+LBvl#5b>DUSUBIe5L z59*a=dpt;DG_itEvjBa!mM51-uv5`pwn$1kx#?CluU(Jt8ZEEqwsn*XLAe@?db{{O zvBFgfOQq=!=SPi$%5jfvPoaz*O+kIgb?uL_ATEWEo3sbYnY_cqKUY@bN-3I8AMET= zLMUN21w&$*wwHy29Qu2`*@e?eG;>JIQ_-9osxCtvrLC%APrs z(XLisC;pwS|5(do@NW$K;V6BFpoOiMlEv9+AxxsO(8;GVrSAiOHc!cM8@A}yXI0XK z_7~$N?Hf5M6a3MqXXD_rV|l%hYAk4LR1h7buIa(aBIEG>UVn;k6)xwUc^3s8#vw z$*rg~$v@%nJXB)+7ZWVK{lMF7xmMl|PG5|>wpL_Fq|oGgeJt!3{dpS;vXm&UnwjO$ zMv#$~Kw=YPYEC!u`7pFK2(A*5m$fNr9hV$F5ulmuaXb+ZRNx!J-1K$t9H!AX9hfC? zfo!D6gA?F+1xOWRfj}5h7y=B&9OhzdRA7O4_H5;*V?oNVk5ye%)kMPObicaSPDJ;b zgfEs;0h(<|r76YaC6QT1QJHEq_AolBhySLcXL6bZLZ_)bplV>WLp^wTbJO^Wo}?ec z7qJ{lX#CieaWs6&F*JA_2Yuca0zXRDa;%p(JukTQc#XPUC4f#W9778-PNLj<-U1Gj z?2QT)&AMV5=`9fZ| z?i?3an?@*_pEoPdCS2az4P9`BH4WyzcDzkAUAiAiZU2ThM15}*!9tLvSRIBFC~}1g z?bmSPu`?(xeiJHB?Khz^MuLa&u`O0O(o2vxN9~^Ah(Z^fOI*V_wl@U{rp%~v+MJoFzf5&w(HKqZk6-d zgRR9)Y6ixSKO|SQLV+u)JVFx1YD3t{iF`9n$3Ylo$`=BMIHYiTwM9pV1&$T${3Q+& zs*IaKJd{S?`}Jz??KR~d@9K316fsWn=<=xGIK&Hd0ylIaDJ6R#`|Cp<$QA*seNx?) zusLrW^?-MiS#u3&0Ll&oSjbyQ7sM<2iPlb~709dNwcII2CxB+HH0PdIF^zrC>f>J* z_&>SV$kLbv9;4^K$?3YYaJbgJy$&+h>eQydlJqFz?0!zTa}b-amy!!09>EOQR!8!N zg>B76CS_rw&U)McwEcEe7sq_MeZ0-EP02esz5dO-LErdb5}Eg=lI%c$BgOA>5xvZT zTJmFPKvqxS(TZ`BiTM~#mvs)8vK=CPj5&Iq)xa?JFTyL1Jfpq=tsoqH*V3hUiuXHw z-OXmr$Im}6wq%aX*5Y7>nfBj7_SfgXrYO-dYlpW1=>{(qmE%ZkEWkW`taYZwbB32; z9zN3uT3UKb(eBmTRd>((Neg)2>a|4Od=*pcQ#@{)D3@S-DJe%}=lLtXnOX_GF>a(w z8PW2{iVL6%z`yN%*t%qU?drcv8;6Oau6u zqBdENI~26H$#5M{;&fY$ZYowvnd+oAOyJD(l1R&;Yp+&TL!X=H>7$#~Ym#%%9EA^G08r_@wafc`~BVkJk`9@-ztYpDBrx}*#V@dS#7B@ctJ zP3#Qiyt0>4O?XSwPt3}uKn@WoacHbzjW4$dEy7K#=9bj$z`aw!TUhaBxuLC+{67VH zrJVY6!21ltuuezU76XDQSXfhK5EQmus+=N+1M0^d&Q>aP+=ju?1GgZ8DdrD<*1euf zI{NX+mM{_*d~qQPA3CEjrLAeN_xvo!34yns=IbUM|G3*d9H}ENo)gRB?YEVarPp2z zuR%bUgz^g=UIe@d=^?p;|qoLQMo$(3RHne z=+&dJ;X(+Qk7DlSO;R?uASI^u7rn(isu$C>$E}ju9(iVm=QB~;g+>r~ge=dq z$dX*dS z+~n+Vd^2z9BkRpm&g(iAsSSLKRz|>}x>1ZZZtrq|OZZU$7cA?KkfSMgUe8wL!z1pmdu0|5|dZeYcS;B&Jwo!HTtQw?!avub-~Ncoyv7y3y8% z_1Ja4JI796+0>tRKcKWHjxLvSR_!v&i9-w8{@c|3*fSmK(BV-*(X( z`F%w0wdYMjw|(2PBXH=^d_U|WAHSLl5` zW}*~P8_FUBOXWR+b{ZGOZ&Q-FYEcdjv_xYlo&BJD+$Gny#y`){o7p zAEBmQ-)W&r*eduu)SO$hJ{vpdTof|s_?LInA2P^YIfT9lry?VF5i*NNGhYN$@RUBk z&BVE%og&)-)$h{fsjPn|%Ac?QZWBTj%BC}V+^Sl8e%NS#J(yg=JOR5O)9ffj32Gw} zV%91$?E-7d>@zfXX&aLTZi>gvhRWbh_rc6!6`Lce*Gc%G7ohp&jk(HCxCvP2;Ej*7 zP)UEeX$62OZ8~2OpU6LRAvlpuPZLoeLN5HuL87?}PmAt_;;%pBCa#oMO2xPijSUqA z!l|3#zmvF}GV|GH9e*a#UUp9hBXsKo&=em-ISfrz{Z~kXH1}|A-6HHS18mV?XKMd# z-`mg{8ONjnfa8{6D4Zz%ndAXND4M24)@%p_GZ+j|)(XsVrj2-zFS8W)l8Ee?E+h^= z;L5i%6AKKk%cj>ezP-~Jf(c8tn7ct~gF8Kh-r={a6SMp5_$R4hCojU+i>sCE-!myb zZ9Fd0bCCgJ_i)A7>*8A~n|vX^Gv0SHrk|y&6gEl34)AW>>5*gXi{~Qdm@7v~#mKeIpN>3oQ1Nej4;iX*_>)#`JA%*nBr^Un^?)*wHeN>ls z%FrAJLaUI0p=A9Ycj(CK84Ekg)~-JF4)nq&njPwDV%6|`)@-W*i>DGhk9ebp-kJr-(i zU}jh<5A3?5D9(HR7S!zXYwz){M=5_0QP_i&{}N?UsJMRLHtUNZ`E|-M3%1 z(e83(aJ_8}{AD+tFRdF2{e-oqT`s+xlP!9|%l@1yvJI;+MJs1Ji#6gtKWC$PzV^lI zELOC{DbU_3>$Ip=shc00D0JOlBmue-lDf3ahLU&%kLM9nH#^SO>&BCPSYD!)eAju!EqS~m+@xP?k^LLr++o3F~07!P7rD% z?{r=4dZxz^dxvUOE7W8B#|vEhHlIYcMSM)(cyX@tX+r;;WwWsP;>i{bQxum>Nw&jW zxGSMshn8@O>G!{J2`niXEA=EU5OW$57i`CzE+HxVO0qSNO|aTG?QXHMBbb;Skr{S{ zZQ6g=N009!^s)%7S)Ih_n3#+8#iG?~=2Kcm$4yYrUgsA7a8Sv%P`5Wq3*daoo<}GQ z>}%e|zQqh^aR-CAdj|UQEQHo+;~>}mkh(ZbadIQ8qA=kg;~N#4k0mnqFBm5t?8U)N1AgoQ$F`FBzL=Yed6-kNpuXNA zRK8Z4`L{v6VLN`)?lfaLyIr!sI`e3s61m8{eYvS?!G(lpT6W28_V-%s7y4`rABBU_J&VwR~YKKCBtVxj!QCLN0-!kY$E%X)4Ak znF*d4r7+Spu6@F)UsfGP;CdKqJs!y4XkPWt?_d~Txt@z^5SQyP&*@fr=gjp*a-xM{4C-epnsIoJqp$%gaz1Y479WhY8r7j6+vBCiK%96=$FOP6t- zDPH3sdWzM@Doq&BSxUC6E$n;fAqXSOqmfb`?}~srxq4cCl4_$LppWMvT3etWSoUnS z;pth5WWmT1YI-Pu5fxF8J1T0moO{HJTKSeMuUDUptG21UU?ts77`9e3Y&gF~oRqmV zhe&k}2#y`q;U5sBWN?g_6A85Yu|d_Pyw3q|*3H^~zD8XyYfJcC6g)v`R=#d6O?`LY zp4P3;x}!JhmwkZQJWYl4srmh-U8c(30ippxdM_&+p6(favfPe?blhGuW^I+(oqzPy zWjc3~Ug=lOWL+*|wt7wKZE`#uT30yls@1+|7dKcLfiV|>8TReG>32)gj)b};q0yqE zgvE8JEq)xs4%v3Ss8x5Jp^3d^naM&2NiC#KyA;;(o*;<+72hEr$~xtju#%P0@QLi| zz(l|MK%PhN&5?SzDHJ-BjnL07iVfz8z>NBYf|#|URoAr%Wv?LWt)TyX!@PccVEfHWs)i7n^$=>r1N>2i`Q4BO)r{dc^b3{5!rhFg88g@)eY89-l~})? z>DLb0zcRZom8)CbG$hT_J-)G+``idF_M`_^XGi`gytM;%tHsp7W% zimGf@{a_NBEB=fvN+{lfAHIt?_e<1S(NJ8nMG>v#`0_AV`n4{p*VjT^5)y8G zDJ-Bp&TWIuqnJz1BL1jrt5A~55zeI0Z0S>JbG}yn zcNhRC4g=|7=TeGYZ``FRuc+SgRlNUTC!D9|=^ri|cER_q(DT0N?;Na_@W@KgeBOzk zLl`Hl?b_X1aVIY@6c5kr=&aO~+j+1zmvW^_%x6dqE68N>SM zyVQ7Lp*{Z-X4$&q!1i8P{oc{xn%N%iUW-7ZS!f=|r9BxoM{~7$HAd~>7Ve=$urq0m zIG&zf+aYa>)qBa5;$R~gB~GJQTdNB@aB7+s&*g05G=TaiY`+Ju%3HqwZ25~LIG)ED zAM@)it+ihl$rZV(+~Uh`W5UXpafBI#4mG4i#;`Dc?VXOY271Yv45C%ih!IQ4B)hOI zemi70kbGj*9SCrZLlI#JwUaS0-Pv_%fS@*-btC0RBlwbZn(u88<=cIX1r*yP>+8*B z%0-IYS+Si-ug}?+PSf>EkI&oA=DtVyzr&wK8pK6bE9dmO4ZfeJ5ZmLlKch+1#jjQ@ zr5;BU&K>IDS7E1V(USzLdwvc~pHi3m0;4xgQ>xZVjw1TlN_`zR9T+w`B7#%b@r52M ztWWcL6;l}`&C{93;Y>46DZu5P*ctXrhbNm>O4j}DLE*0t{_+;6}lCDAk^+Id*qzx9F;!fj}D0}XTuzyZHv0EVbn71N`|%# zBHp%bJ%5AQxVS9YR&2Q2>Bl- zx6=rmSC!9&6^$%J%3+@iyN0Kfdx*(PM5WD_UWdL^0c(3wjh=_w#Q?fGaD3V^C0q%P>-_GVedf zj`VxdtsLIZH|5e9%GaZWh#mye?V2`=44jz5@Op3e)^d2hZHefJwKp_qOjrJiD0m$i zi1p14&ZY-_)NOa_O9tja;%t&+8J_-YX0DP@QD2HulK1@)j-!wXjA^cbc#Uq+YWgEs z^Gb|T7!eOU&qWi`I3dd>q}Mi*TSn;0d3dxnU;dM5eqamA`-ViO*WJ;8`jQ%k>#hK^ zwkyA=m|$`Gkk%p*1BiWLMKWD3mt7mfyBbSl@$hoI{rPP1cFBj?z-5cms8f(Py>`~t z>7WYM)h8FOqx2=`y$7$w&R|uYInxCdW!TokoN`&>tM-JU5ubQI6w?HrN%LnVZiDa0DqZH^bK4HGEY&Pj8-?*& z@=KyhlHLG+a-nOk0ctACno<2nf=U#z7t8Ly*WYm}_;sipy=sCfUWG&n!!qpPcNll6 zwRvA{T{o;x7u7G)Ehr>0D~D;WYYmX4UYQ`nNPGvzH$RVKMWk^<px8N)FrR0XaJfm?G5F{YEzqG>4ob4X zRv97kd02l$#5N&fPVhwG0@J;pV=TK{a_T3|d{>tc`1HZrqurJ!IcYMv%*=SdG}?iO z$JU=foF>-lB3;e!S-|!Iy1W+dTo?1c=i|ymmYDTd0ARg$c*~@~8L!WmO}7{0tR*3nSd)E3rD@5F5F5;fxZ)|9o3D8(zQf682&-dN z>xaNO>9X27Ae!b!OOZ%7&5uyt*20GCdUwDG=1(yw`P?tpGym9*aqD^eco(ArBTI)h z!uL2cpT*$c%r^Ss(QtuC@!P8443-G%3q~m*8G0J3tr@INPk3FZv8#6MIf(1lJBoH= z-lNxDx-PiC2>r*Y7)<^Q3lMnaLchQ2ip$P`%SHQvvDzidNSlu*w8X7vTnp#_15J|o zReO(-`JNDyQ_iIR50)c(>m^R;dtV$#o8vAy4s{qxJu=n*Ab$3XIEI}EH>B229+?0s zwX;7`CX;B`tZHp=5r(V%AN~6u3E<62#d1bOxlmKk4lMncQpL@Kl^Hk2?9~4Qu|Q70 zDC54#leTYSU2R4)*L)u?)d&{~GL*kKtB+m!7rf3wY=(1jeUCfd23Zy@elX(^jHXDO z)m3x5J_ySYGOnR!N(~40_Q%A`xSO}mq8h5$@u;E(%O;%3Ec4xvGj8{V-|?i$V;>N! zV3si%O*q>Rapq;3C+vjZY>&af`TO3GGQ98kH!^64mY=W}?42fGoG;fOEFX_Rx|`J& z5qDjaJv!>!#|KvI(cN$8<$?Bu)Ai$D$JVvf9;;)U^@a2LP{Wd49=aa)Y&{QBPO~z^5j?M>DMg68ujw?nb+v&RLgIDGYQ~ww|1b2SAT_!&m zd36*5mxHuYN9YhTHWY6tLTb&uDkh6-U~UZt?^=KVFvGL|#+`7?guA-y-L`l4Zr>9w z_l3*ZaNZ7dqs{gfhNPTGOWCsI&;@Dr+d>^~%>4WE4;L&@9Ul^GM7n<9rVVSjrgt1{ z=Y>NfK<7^7iNaN)mszk#@%mr{Ta_PpLGK6xp(6x$2X$~>s_PIj+Krz$s$6#CyKMUY z@6*?3f5jF){nNJo9Y5jD%$yZh*)bV@4P$IaUqiV3>MQo*$Nr_g`K8+HKcY9@W7?p4 zKyg2Ovs*xIK@S;=XHMJP$rJX*CqHgOs{1=W^kKW_`+vYjwfsD1C4D{HLA~Q7pz$mIG31xM4!Y)-V?{~u!WaSXg1iIW{+tZ6P+mis|fKY z&<;oRFgK`|lH`wQ6ZJK{DLnf0|7Wi}_Ghf)K4cusQxs9jP>SCsrOtJtYa9|YV^EuL z+%I$o(6U)~!b$Gip8eOhf8n%VU=5cUar#OuGeShm%qVoqF%0}z2DhkLfqHVase6w4 zOt^9O`Wj+JTU&a`e2!Be^1c^wq&M;79@l%1t?{?GcUJes zUVh0t<671_rIrpYfrW(jL*YwsRBZ=X0bWS2X$dWk)!c2;J021=tXq&^uVVhtDcZKK4@ z7{n%Aw=;c6V41z#*6sM@IlFhaW*<%OvZ<*_&4}Y(u4_cfxD&@UtJ_7h@YJ6);B_ra zB`go`*X+7oHnWQvO;i37)@Nve*INgLgL%s%qp{R3OmetyH_yX80@t=b92L`W#(uTNW~f^ovFYYUqZ zEg663?pJNk%!Ex&jHx|NRqfGdXX43i1XhU-Aw)1iKY7=KXpH9?twLIMA*b zV7~NgziKc4`o~D_zmwxdUF*ZzY(0J;&jshZnXgs7Kk*Tn~8rRg9@m{`Hp_S$d$hMaPLV2}RDU$g@cJmj`d``<)U_ltfD zSb)pf`@a&G%h>w_u5BD|9qkcE&$_X4Rp~L)i={|742E@&8My0+ZNBiDEohIl87)=K zu@bTK^HmZ0imJ+BY3GP$Esd=%+wEWYr*`t4e@467YHX-kLW3i6B#VFZp*i7v$ZdB& zBJB>X3%uR`^w6uX+Rz?l>nr*=gW$c-|Ay^4_p~08$0ZAsJ($I%dj~HM_4104FAw2O zSYWFM;5!eCP7kD-jf5kG*;#H~MFrKbP2jvf+ww(iz!%yKO*qpc?(J`bca{fMNSXK{ zOWprPU!A%HR!z||r}W_d509;7$`M4?SM@T}@nb0|ZHR|^_CmWlbjZd#FWKtZXKdn* zcX{zq=g3{l5poqB6g4drcuK+1rz&Em=FqSl#%iDJ(e;j9uh|nz!(EtceknF znYEWOv|VsH$#&xT4TJKaV z2VJM$_S7v+jCbkbLKwe3#XqkhQZvoV>6wbgNPYm{GL(gyI!_E<9fNmewb>DV|odZFZIy1!QSOrz4WS=wJ>tj$HCYx`!vsJ9l9;9%n-GO z)jz$}EYNqv-D)+`k|(-|xahlPuwCN<%bR`AMcLx*{ZIBDy@bnCIvCQL z`8{j!ov1#v*(a~mkzOd%YN{H_PO2)t#jthSqh8IaKopilBqdB-LQ}u3?`@Kl?d2{ZmlNMdzbHfUQB8R7CC2obUv>HF)M5m6r z(K zDJY7HfdaXnPpZpN%BQ4a=qNSIFB}k6BwLjq>VQD-(gJjvAdDk<4A)#M*U8I-frnZ@ z*pNJR-ygEI%O`F9wcoWxeTi%6u5Sy>B5TTA7<^R`_T;y^v}9lW*?(kje(?!?iEq-5 zz5M~(bK9X1h!QG!udld7&$x$0)2N=QL#E|}yBC)CXwt3|xz2JJ%#_xEia45UJFPDUtQ04YfiA`I@?}T&DDI8&Aoe!gp z$~n`q_m7;g&s{jIJ=AvF`m&aV(#{5WIT=h)@9WvlHrH2dV|8A_ZQ5Pqr|sV9^EN%T z%iGD+)P(jEo78ML%_w4NE#ocp$y%T?5R;hTvPSIZ%2=O2@8H19o!Sq0x7r~4&1qS+ zuY3@R59%0G6bkVQo#T9`V-{b?Yk9Wem%MMJyi^ZacA{^}<84y}{*eYbBxf&w+6E{0 z*=X5!(62!eRM#T5ps>b3mo}Fnp^U3BqE`Y~PF&*l8K#N;g}Pi>T(ckfs~@#jUO$!W z?d7Nbz<%|oe%yZGzx`i~=DN9|(fH7$+HaH?x?g`?-DsbOBbfa@*;jCEVAUR*dC3mX zvW3l>1pT_68T>^H?MHIuHHi|wzu|?Kw72;#TOU85Hq27i*ds3c)t%}uZCe?o z58RR#=sV(Wa<#NCB*Y;tLtaS4F?fj1Cp*P>Ye;9po6sQiw}#FjZnV*bkJJ(`B_HriNRe}f*VR)iosHjjKQO`uJ9|k z#_*&RGLuW3l$ngYnFdM1&HU6MSvKR$yBYdto8rxLIB<|U2JjJ=TJr&u;I4s_Fp83? zYsNlcgipEq_6``ktri0^26VvzzR|A3QdK#nfSm@UQNL)A62Yg2lXnr%^~f|Z^TtQ) zVGI2k*v&on8?hW!PPpYm-)?&}xZS&VuLgBvHZwEhF-s^3%uwhJ;$M_?i|9~FtZeHv z5-vqRjnF^1p<}#x)wvTF4yc>_L?1S!?US6P5IaLp@i7vH(0|%IQKM>tF6<-HpcZ~f zciMon&XeAmRutewJ9Bvoeewa<`!ZoUHx7EsJp1UM(0*}0Z_7{o-!`=SsEtnVa~oq1 z1|wf>$TB-xR>MrVbi&Q<+i&+i`k2o^@rTTmx71y2tZoT*YNy@U zl+*2%GC-3s@m3+zGX|##w`npjWAvsS_w{#6;V3zB_q{f|XPzYM3&dj*CeXlJZe%LmaHETpO z;|2z_d{H#sH|vnS$yWPg#l0XffSFyFWqU!+B(%?$WgF^W^_fXA){Cj-IXm#=zp%Xv zXY}wnVl%pjPRRKn;*Hf=7ylzRW&Ha)9{SS{<~L3TtRC16=9cH{vY z`x3A*Ffk$F%#U$1`1QTI>T8_>!jOW~o%lIP*CA;`px*mK%`?dYGtPSDOIbsqYso`d z%!Ko$ZbHXaX0tYTWv|U%J>|}%;pv0Yo$&;Gi;Sv7PxH#eX3@5)d~n4H2bNn$rlSe2 zuj|X9a-??FIyRx-dp6&&=dZ5XYs&{TX3`AEVeM%+I38t*p7BNM`Za53x;Ct%En_xT z^x=|qyKmxkyJyd;+Q*db()f(c@cF=me=QZKubhblZ__uRe`ii#FW%5TbCVkT$qC0y zRX);C&1#D3fpHxgFhynEdHKrAG(Rs?6-fdaD`t6HdJC;>69kO`k;(Gp=3N z^}MEQar(@^Q$4QxY&ev46Z%1z9y_N32N#aidz=B6_ElO#wnbtOjx4w|mZYtoZWT{x?7N-`f|y^jy$2jwfGy!iHb}n2q1}1Gdb@-*Sqr53AqPha8-r4#b47Ra^8) z4k}0Mn8kd*_*?6 z3pe(jrmgw)2Y%WK+0&>k7`^)8*0(_45qFd8C@-LlbE8i>hwLB_(a+VvRTry6arKc# z7$1uHH%uF5P{(Z(y10FU0UiU7!0+h@l@s^^2olO~!bP}*^TrHHTgI+5=;)MFIl=yg z)Y4HW-27^4roksA&7o4SK3%0%vZqYF_}{wvvq3UpVwiOU0?NQl~Vn9E3Z%H|UZ^0#)E`f)h@ojb$5dx5?z2@lN=`wLYd;86&@op7NK3 z?euRt;ZBY|Y8N#V4kuhLMa2=vO)}n8<&3nt{B8*8ORLnwm%k{K9C3MI7zMWtMii`! zrru#ZvXm*}c(bgS7rQOov}3{ca-sL~-55mfhf`vh7@|wh(jo( zq%0%KEi9st!rxDXAM%S_0lcB<12+A(|IFr}|F^dM_|Mzm2mh)&;vC^?cwlhJa#Uu* z;e@;Io$oHPsL^ALID{^Z2T$syX>-S{8kwjHoN?m1t>CKb!qd1uI4|X7>}d^tZeGfC z?BJhiB(?b$v?^-L+s1F_BTudnju~a8>+rsT>o}cRIOK*0H+=6nU}k3A_N9Mmm-T>m`Tp;9vQ2FqF0XoD}i1@iWib0e#?Tq_eEe{I%DS9E>B#S7s9V%OCYG zo5)GOhf(wcUeb7WbzPgo@0qoU+qF%A9BLwi1OA|)L8UI(RE zc!UU)9~@mK=^im@3D^9?HbKzazoJgfFXT-bf?vp%{JP`7q!*5W9MLDIJ|T0-m7F+t zWqy^3(}`ujmxm7PtGgF%>C_i&TC>iB9;t(?>nu2R?#=Hm6P_EIfjPztNQuD~9ng&H zYrk)wIcGBiSM2%aW7&K1wAF1j6JxMshN_}5Y8z)+iEHeW6yb(@O(di7t|M@3jRetF}N&uSW$ z-uYs*vYD8=9mAwWyGl8lbflFSy}a!e6p}ryoF`07xY7Kes*fq#cJl6Ap4!0bW!p3N z8TnRs+o)D76jf?Pijq-Q-JW`d7o{syIJlC6i?S|@4&^lLq?5EE&m}mlf9!AkJ^S~+ z|EXHZ2J~lt=p*+2hmToj<4gAX`dxN@)~*|l z!*z+5eE-;zomkrA1FiwBhxetD%Br~J6#M}ERN80j5ssFN1c z%y&f~MfMC0I8+f$$ws40107x|6X#AUp^}B;%L2rNHh$6)*P({LmGtM6*;wkpxCY)9 zn?0kytm*9_ZCEur&ns4Iz&<#GrXbP|J*`w?!rSG0RlzUoJSsfebt$frg>&$(f$!z{ zbr3>&8*R41q>b!opU5A|3ktX@v*1zYhBnQfd+y&Vo-;nou(<#IwpVXeb0*v_ZPd#R z6=$3;Ez&i}467}J^UYFp691uFL(QOd%{RhGs_#W zi96q8s~2Ck)$`BU0uOPIe#Grf-Q};azV0adCT5r5guC#{OZMt-{F*OEz2}if{o%ze z-Qy^WB2d$VWN6Nq+i3n9Mf2pA=9WBk043BGDD)C2hTQrtGu*B#&Gp( z$9W6x>+fyMUn+#P-7;8vOv_2`dE_1T!sDN_r+(pQ?LB|}qc)*_fi_7$kbO%rWk2d$ zpt3;wvSbF^y83OLO3W%+8Q!OdIs6KjY($&E^Ri`hLrXXH@Wu?ZWjkHl1*Ruvc0Jn9ZN?Zi3d|2NHuF| zv&@k5jy>4GI+%MW5vT2=*s9M^shd2AuGoN<2*P7XU#;Bp>~GlY#TVr?)C(&)Mt1S? zN)EUQ$u@y=M2LRpawhe?Ic1)co}0oQedIpP4Ak;MZMe^pE@nB^AMyzsbjyXn5x6G9 z_K}t_F`*qd(;UGK0u=NiYTWArH}88je{hp;;BeeJ6xCJVk?(!6%Nzp6oj*qa;N%^? z5H#3k|AW&BHv=rL;h%_J1?ixE~$EL5S++i(8os5}qYH!RwVs;=-xfrKKT{$mr z2!(m8>=Ys6&V1|-w=jIr9dT@)&vhGpzGNy@k#Wl0jJ-JFWxEI&YFcm$8z)OR;~UL) zCe1@K%DBZalQRxES&pjyLmt&t&4!yg_3Lu>cWmnJ?@;|yhhd=}-Nf6|O8E%~&q!>- zJGBLHxi0dzs(#^b{{8>me)(hnu9VOn{i#3xefFRK;SZPl@qj(Jamp^QU9>MR9o4Kx z;jYR#?2chM<+Oa-2Y;wiDPQ1h0Nw1pzP-<*(Q3x8)?!EhC%oN5SM2_&H*9)z&1ST{ z1+y_HCMU{FxDlVt%WP-b4#C?@c^0C4UzU3Q1zWZ$Tb(>+(|nXhznD24<6oz9>RRMx zTFL51ZvhMR9dS3YlKh(IqLU2SdDGb>+!T#8eqA?cIwkJ}AkrSTQ&;Q@mk-Mszvwr* z7<|Az0ypDGqZjH82fX5>d2DYd@MdC@t#5*cgZ6@nFd1mBI@#-RI(C#ehZ)L#C2`RSEVT{d2eWg)pWwW zKK!s^a&?EX^Go}GmSWD$OlfDp$s_U$j`Jq3l70A;Wz^$ zd3|m4pVfyv_8!t+NT=VhGoSmk9s7W97s+7_T4dYe?U&Ny-Cd;K0FV0mb- zC*z14-n>h2ICpV&_zVi+4ry<+jR%g~&=+5@D|#5+&6h?+3y*y1$kn4avNoiCPFvT@ z0jKL(zd>CGeA;1DGkXqv@n6`|OTTY(_k75f4?iUQ1(miSJUMRs!Iar{a^)bP^NT1! zmc05ksB&xK!6@#hqRmvn;=?&NS0VV_A2^31?Wl@E#dH zZMToVsW0woRnVxGi^^&1OHd^b&g~I7;nD`-s1D{Sf5xZ@yq*5*b7!4!esI4#P;%^- zbr4*;kJOg)Ey->_(yBtb;cn{QGnY@amCOZBemmu7u^CfB6W&VRn6w|)`oPKs+kNFz zHaNj-xQAW9>y-s$_-CNpERQsoy~LbnTiZ%E+Sxz+>3?iL_4EG>)SmFYAN~&eFFx}9 z8mG%qs+SB%Ke#?%`v#Y7cJMhnw>GQxcvud(Y3Z-$BOSfp(~?zp$}ymI0czGMD`oq5 zLVF_Ofb*9RMYGv~%XUnA6Yd#VjG1pNyXSd9*96x^%!Df+;wg$@2e*aTX1sFld9hxy zHwNC}&bM$N^SVM;n&@uC=CuZW=vK9WT3tWfObc*H6Pht_^XK`x@?zEn9adoTwv>Sk zPIU%Yv%2XW9eu-ITfJQuu{Md+T6n*-xw;jgiwiyJxu9dfkonD6nITsxxa5s_1= z!LEV}6$O^gtRT>V;M6M}P>-We(lz%TN!}H|q60+}l2qK*f{j27 zzM~EkFJq9JaE0qMaEnThI9N9E6VABJkG8a-B?xm`63L7>X2RiwV;0=>wBBs#H#5i99a{RpHS;i8;lPj>rs(HzN%sib ztF!R(DF?p%3NfGumw`kE^MS;f^VGoJa|*qeQRsu%siKaA8!`lEisx;)2j>e^@**|P zl$16a8kw}2M?PX#KKF~Z@R^^s%@2Ll24?rV{o_q^&=>OSiqM0Zi*sks*rlhRvMJ4M z+`WH)(JxrItUSC{x+TzzDLc=B>vkKh4u1MZ~fdb!qv zS*fOe^et+Ey#6w#ujV(v>?J+4t&GS~p|HMq!Lu0KjmY`LerPNOTwRU5g*t~1*y@~? z%DsG2U#VQTU2--w1K0nsI@>|5Q{};Ja73;P-6MyT#?~bZtAnO7t&)%kSrDk~0ohd}GMDGFD2;2q2iIyo+V{O^S&pe|UGheXri?7Q0rmwb2 zu2J0|$K@1Z87<{c>46lP#-y`Q8eHjv!ZO5j9qmb_c=M>f_^BnS)05*q(`iaGOZ+1Z zt(Ra)8xp03BgV|p2bWwWM_OW+a=UuSj4F!VXj!$HO>h$(I8c5&d&0Q*>CtU7gP1GCS@W7eXa5FNG8k;Nt3vvJnumfx^*tGnfp+i&wYL6||Q zc|Q@NjrFv%Y={-shV34_Y`2YGumdA=Y6~iFd{PdNVa+~@>tk9m?IdQxY00eQ1V%^f zvQ^kg|IXYQCtDghVtXd!6xIwS1~ixiS`V-WZWNJO{6QRvd=bqKz-z=+w zOMc=eyRNdfIqAk_a)wq9=mq7Poc>d%e$DG_^7wbkj%6%Z__+N9SGH)u*|vxYZF_r~ z=afSVlChFq4z;I$@=L#Jf9HSwsU~&>fAHHLvmgEQf7(9+GowA(SS>av=RNY%_bn|i zYcIoPJGA<|U0$8Avuk_pLT8_?NuU8HEtC-LFmxm-TW6gTh+_lGc3}8|COTZyGx(aX zkQvsq_YC{M$q_s~9W!_*^a5o}A8O!b0uDL!r!5kviWx7TdcxDrjz6jop=b|A(cwr( z55Gunr2dOO{pi-UK;IE}GwUX=x~2@IHGf@sEds7!1+Gcn_;90nXzF#l)S0%c8@qJF z)s36|Km7e8m~?(M4_~^ew0w)&?ismx;h3l%ERwdwKCq#C%5)@gIC=?=fkVkPT;q4k zqqEW>g^Y}2yz^%Kq0rNjlcw@gBRWc8Vh@>-u@akQe^WDjs4ljKksS#58tjdFBvha(1eR>uA(D& z0)hs*_j9gBVW7yN`qXtSry}#3@)!=Zc373JPLS7%$2z#aQAQMqY}p@@zg5Vx$V)im zHh$`wvC}k=DV5VGy#Ou*n5agZi8vBUnT3Z(!bz6j4Em4hVRM%p^jA)vu=#WPirk?? zH{g_aEA(q_HaI-wYtW0;SkeSJA8#0Z z`(3uMp#6x>UUX;N4EJ^M$%1lR#~4Wy$ixZfm&S=;40E5B!%u@Oy8SV2bU*&e$8Fb3 z|IyaRrfq5Oaa-MeyLG1a+secq>u5ivA?vsh(pYy8dS{{DPJ4{`tU8@z3S>g=dG*I@N?6~& z+s590+$QCSoYJz?aqTb1S5{dP2>&W1F|mj41lQL4pF)$c;s~TCjlkAvm6hm)s)}46 z=#qbd3x1Mi>~zf&!G2n$6+SLVux0y!F7{$lp#!AMg*kU;FmRG>Y%)tsUoRb<(k#P0 zwrlRRtzJ28BeRDcn}e$$K2;e7P7@{jl&mUJ1l+H4))`;=rEDAy1lkyIXH5>d5iM=h zY|LXDmwog6%X(J1(wVZw^)Z`U8@2_lG#$~bwn4S^^|ci_PnK-ojN1Lcl1&Y*$l*M% z8K8^qoaT9FSnU$$pMQN)jyRkiLz>lu6E2pb#$JNRfgY{{f^QLa(!X={wC3MwHmbg0 ziGKcv?)s}j^3J%*xB;L^n{dW0Hkq6;*-f~G-g63W)+?C_hn=`zYeuacU+bFvwLW*! zc3t_54XIBUKlHGdllaNFS$5l=5Rm4CTM)GIy}pD`%=m)enq>L4-}(>sSN`TtCWo&0 zkq7UypZLo^;>(hm3CS{c-&0cUZ$*wc9C^%8TU%T6W&Yz@Zo9v8R_CW|UTyHw+O#cf zO!x;m&R$;C$2jB!8{-*TLdZ!sGPG%vL)x2DGwY^@7B%y2!Ow@)_85zf>f<2v&Gb*+ z78TE^O|m!Ki26US1N!>P1`UIy{_9H@ZSB%)cEL0*9Y5$>Vfbule{m7lRMIDT#u%ds zcb{~bs(*3|S)lKTyGfP7#lkQ9CE{y`q^PmDaJeiQkZ|GqXW&N$?S0cv+2`lpW*0ZL zFT>gu4eT_E^Xa+0rF31iHEV}5FXL8r7tXcvn)FS$O_O;UlfMeSR(6 z%*(h<+br{1c}@Bz+@{IAjNAM(J>%{2d7Ss9BC?*|XW6|&&)PjxS9PP%TNPi@!Jt1| zUXxpQn}mO2^|l;xdqd#GAU|>w?k=a7GE#1oekXiN1t&{SIOw{i2q)Zc#!NVU9hDC= z?A4o99B)&Kal$bRjvIMS74PP7t+Ob(mO~F7MFv#;AG$eFhLMwbIFYtl`#dt{GN@gJGj+EjUDhgBzxeTFCB~%nsPMuZ{k12=SQ%0!lRp9ONVOCf< z-;kw#2@-p@A!cG>uar@eeMS7Ymp3Ysvb-Yu`j@sqTl7$(nLTpEUASNuzx=#S9gq`6 zZR;zYsTqv>$8Sap&|LCr${0PlZnM4k%bnp-E!x|qz11`$j(c=zXW;o5PqT37!^gD; z7k$Cf{m1ous98mquIfRO&FbSrUGU}(LI305PUYbbb{Yz9sP83@E1MZ-zn5CoZ z+S+9JjVEp6)EBipbkknFe8qm|^IwEzSNJ>M{q6pc%gi^Hp8Ct+q7O6RD3^Ft50K1? z!x@KDkuoSdWeCAwO`m4ApaV|0jos5WsISE0WEt1Ka_q&*11%01OoP)AJ{|xd7H<#b zURK>?E2PM-@G5+z1B-=+cI^DYY2;^GNrt4#7U0=cLXiy_3E6jhtaC>$vFA6^1Xg4z z7q+U1z!ig*{!}@RPTJM5^4QainS_@{?yz0jD{J}0XKnPMAE;GM(W)})a>Z$jGK8pd zQ$=_9`3y`g!P7VsrvrI7Pmqn7af3^YJ>_VYLu5!ZGAB0{Y~Si7?>C=0@v{Ac)+{G* z|H=RPlV0bphjd|?A0MC~9;1D!O-wRNQs?0W(VmKO*wP-MNdfb6ijjy!LM8oD-Ci5l z`fqere3t6m;2n~Mk8{bXDc)R{ZUf>iLc5GCZ)@ycnR@Ws3GD*5kfAH7Meh?bmF96l z1cgA~;o}#@8+viM`^?8=uMO=l_#y9`JTKdw@LQ~-IXWM>m5|2X;Zt`AO8#X@B?piTKI514QQV^VjO=w zAMnC}K8iNUH9_A+-|hXDXrj%ey=sG`{!6cY#>+c7{7}`_&;=c;dqP!7LQTV0Qucwj zng#lfxSLdCTsz)()6rb-g+#{)oqqM@8?)hje+CWACiSUuI+%BLp0d-cd+cQAPP=M* z)hV(!SU-HLSYX6fY@eO6d&W=N?5I97KB3uMdaKM}Wz+|<;Q;Zrq;^GoJz=Z%*QCp~ znf%)L?ffP!@J{&c{5E}?zu)xGuMCtJJOKM3Ld1!`$)0e}|CW}eYB{Q$aLb3j-M@~? zl2jaT%!tb`q%y5M`L)Zx=6R6;9Q4Q#GU1paRX>-HT*<1@qbKVuY^tO45nlDMp(Mx4 z{vq^Zh#WNVB0w8Z^9cXYhk;j6g?$PvWF&)>ulw7_2v36+oq>=HW-8Oon|jWf#POQ-8kFI_F8;;NGtp5o)=n3=MD-P#w(@B3lZPJOqg2R(f zK54JN{(2YX(W6K0(MP+Nn{^@dPi~q8xR&{qt}L#_hBSkEP-8|M4{Hm9+6JK`=Y+OK z7}V@E+7a9C)5rS~RgLd(WQ=NdkuSA+?0)M!{)}cg&1)ru>1&nQj`M=2?~T4k%#e!olWWK_!+Bycovu z#>}`;@pdPj`e+<>bU-N!JhQ(FSl5xn)S_5h*8}v}yB^X@&7;?}vSsoEIhC z;~p108BIiO1yg?sL=kEq8JhLyb)qbst-RO^>?Drbl?V4sEO;4@b^0L~koN~+B zET4Eq?TEf}T;r^*!jxWs(yxwdY2S#PV6oSh9J{)Ak7$+XXQ zUIJEiT;Nh^;BKdX?hm&xa#&@?KEcDXDHy7q57$ka;O$4}Q{>m?&Ou z1~+8!w5^zvr%cBw^a*dJ1GeC|s(P}^&)dx07j1ao1~YP@_GQPPQpvnlMbMrINB!&}D!eMj8QtShb}LdJ2n{gG>si&!0#$08J{0S|jL z=uM^uSG)SKH#PsXy9|4QK+re$0NG9A-}f4Tv-# z_L-BqnJ^ya1`!(OjWX)1ee;B0rnPO&`BwGgH=R=dJv21f(uT>~s{TWgLfOLaZvQ>p ztGZ4e;CuZSd0ZDBvWxcUx9NMrg%gesGhEh8xIMaA^L8tpZwv%|#zNlIdyC#7MN+ul zc&s8+GN2Wx8SyXPTkjE zj@)>7?99)*BeQ8g?D_Qy=d={|rI%jnVo!h754WfVxX#M8R>cibxsGul;EbCe+HbdQ zyrhpzo|n^MIjPu+Hp>bYEUxQqI=fQ`5S~u?!Tq z8`o}(7ny}d-{C3xVj#|UiB%kN+%p(UuIb|r8_HW(d3?QZj~-xgqIDX57eZZGTCtdl#;A#{<3K2OH6PT1f^Dc1jljoU$ie?n_r;v)%J_5JD2T?z?Wwj^O*`Vs5dZc|5U@=0&Q|Js+?hCfugWyo`6i&yq6+y$QG3XI{oV_+(Hvv3@M&qz&vl{VSd{CC6=s z?dv~G25(X>x4935Wafil>OkJ{VR6mDqr614tsXZ+TZybKW6*) z?eTU%TVRG7lT&brgh<)Iewm@&scT0==%`aTu_#!Sn6lx3PA% zddjqQeiwPg3YOOsQrEYl{>Y5k^e$32mv}3~RDIrr&S<(e&ZN9K#nRNUm5!BZpl_JJ zg42G~3fNIQ;HrI)mI!caz&qh5ZSw1>|3;gKwbxHci=tZXQ;aAcJt9BgTBOu9VwR)M zzwp~J6HXsynA`Pk+pEp{nF+^@c9&oVeLUpG;4TJ}!Tkn~DUe81OU2^4$~7=|n-^iD z-Gzm#HdxU#(j6@Hh@^AxGrXMWJ(3>QZnr=SI9NH2;}v>jbK$BAfx5_CXq=Y}nmnBz zB=YOX^@P&NBv;l;lqVUB;?-=vL_ih|-s#Vvhp{{9ls|Uxew#S@UR!$oGdB0cKeNGa z`?G56u{?#cD_J;c1=^^8%}sAWdF#2MeNP64<-DLg%5YgjiG5t$lIG{A0co*o5C7sE=3_M;U=ymxxnOtV75aod2359986n7z-Zc3& z|H|q=oeJw~F{^`il{03q)fTaNKlClYHBMhZTVW$^9=eu>^}z;B3tQKhM{Vz2ZY00# zb?`4h%Hgn~WluOEn7PY?C-R#O zmW;?rLl4l=xmLG&-?2jfsQz+M%eGe3=k3+Zl5^}w*9i04!)uqEA(N7WS#L4>EtUeZ z51i|y^3X3{{KYj!S%e%jFavJEopf@{9Mp$+9=hA_w|jQaxbuZQmck*&LoY89#Sula zj|^s|zbw2-Q;4%(*v-;KFC1)>1nn3WnFhR5a9$!H_K?ISCTf<~#`k-naEJ*l@}N~> z@^9fc(S25&oR_qdNzE|ChNs7mSZDQwEuDPaCXT&d_SB=G>{{KVRb+}+6(J$0%e?P- z{W(b~je83BG1U!!@QQkh4@aN|`?gK5Px!rRMPpK&A4^NiwxA#TdG!Rb-ZeYz`)o10 z$(<}3&vKpMh(k8UXXr!#MlQd<72B1_6>_*%s8$l$BOm_kS-3tq=FY|yT^E!!p%rD!j(hgYy^grY9=YFk z&(7#$ZQ8CzW$@vJ@d-KL^ykY?)h2xQuU?c;9_6Ed*1gN1axM$?Ti1QLgNV;}ix^!+pB@UPwKHP-Z zmm$6kG@+XmZvzP2?087?he&k{`4H!su!A_*dek^|l;%O?Dm(huW_rSzm*TbYcgk-I z&=N0;**0#M*XCF0ijss@0j<-(X&ig_H}U{?(r>5y8S zaY8mPB+7EVRBo0WG~CEx5Ccya3x?eD3(pElTPxd(hf-<&QeK4c=dG@z!4hU6oX+Y% zd7kXqg%jmNpM0P#h|VU^M2Eb{8o0qor&U=Y2c=~ad{XP6$5vuk5T@??L$<2TY}Zfx zM_WGph)o{-Ht*Yz<$CL>%b|u%iV?CcrPIn|WYb?dw~)0nqEoaJ#1ddI=AjwVqzWy=@?VEOh)mo0#o$4R5 zq;&@UL*D||VFB6||IDWFJ4r7BW_5u0Nq|e%n>FLyexsl3Y%;$zj)C)3vX(P@0>nd zwxn-YE!k#AU+7GlK-J5bG5zG!(H^?MgQJtaBzRCCHW(b6(HCdqWsUQ0(H0iHQQ~FX z#%RgwGG!XL2^W2O@{@1I*TM%!`Lr=?IJ_>$mGr&1cE}E{owSuRPusY@mg|jxvP;`Y z)){+N!+2fyekY52eATv;=Hjbvag>oRm#yT`EGxnMIs@x?=X{SsX6$VheCDUOiP&q4 z@htpsbTHn{dwz^%BrOe(A9`f$vXpytNF_}qul|TVfKMoKIrH2#IpX9h)T*OhsnKlmJ?Q;}Thju4N8GJhFI!60yyo&IK^h$aozBpP&g6jG7~xAv zT3R3n+@uCHJan=dFK;=r6WFo<67$AI%&C6OjvV^zKpdfcXwj>{nkW2}pFAKFwkc-Ac zOAJ^k({&Fj9ZP^sk~UzH;#vxNaB7>yfHPST=b)t9kijsoNr~Xuq!K)KloTLR;pH+F z#AsJFP7ZWQ0d{;ygQMc2tLVKhY80ldp%Mw*^b$kv@K}5>B9c~W8VVXT0k8i z4~UHAR)=?6M|0VR7S4BF!d#=|6GruGKK#X!Jk$P3!!~!vZPw9}(LpU$UDfi@HT5nN zdZ^ZC9z+$r0KE?U_>U$WPPoM2KEz^D+O(bo?KU~$o)JwmTv%Mv{&4y-DldS=hgnKU z=MF^>#Td90KtcuTd+xR*e0tsujZxPhIBp}7nGMdT_2a9T)0!d0Qn4}3K4I*FeF$6i z?WA|+&)MpQm-KsC+YVe%O0&lU<0}g|=msY=Gk#n@eL;3;a<9G|sh?nAVqBR>~=>{0LR**3p<%nmG_vL$`AVdD13BqH@y z-}4eqd$;pbw2ZeBF=_MO+i9`OuuKG`c0h5?Evpd%?hGD{xaPsg|`vu3#_QJjnhf2Z1+m%I` z#yMa)_tqx3v^qy>!iB1e+U&5GKP&3GU(?=wQ*ZpbjcC8h$-6$N_P)ETeY8b@X{J>L zG=Q{c+N<53oL=RVPQSCfylQ{?FaCA=!c)&RxCF@a?oa&K58JyQeL!C))qav{6Eo^B zm{rO14wDWj$DMHvYSOZN+9d4}XBrb&SV6`!@)KWrzClrg?|$?_^+{9epC;YmH>Q5e z`(VnUUzXh2CTpr1qTJM()P#u_U;DIm=FZq_8~51a^u2QOsSNE8$9{y-=G4|j?|~={ z-HIJUY1gXrVA4O{N*3rl;%;Vb=xWh5s0%D&MB%#1U$1D&X%T=+;9`c-2P}gdrgx&g zw178>yxH`%{-ETvi0i#lTjf>4W~bmhy=Wa2;G1cjCrnIe=SOON+_5|{Y8tGj3@G!i zSH3oV`+P@vM36c`n=moq8?HYCR>scp#*iBYZGyc)>KSv~j6%Hjkq2`(p$jhYi~iBR zgsse`qpq!=a!XQFc&Hy~l$B7%*vsLRr)_T+Fn9$h*afWl2Vc+KSw5AszM|!*4JX{( z-uK#+9B}zT2ELHWTX}BO`QTHye-)7mcB6-)*t5E^7M=Y((}k8eVZ&NKpv;R@S& z>m~p!I+GTs*duT`C)ITnJ+mkxDAhCc&^NW;rXTozn}7OOZRvAAYa`$N*ZmEmKXg=u z6tXR%<^ut;>@-7-ol+>8n2<2>^@J|-IDxl%LBTK1!nR~n2c&h?ozul(CwUvF*9Gpx zudn~s^)_ z7Ewf1pUx~1#uJ-%K(oZ#;mrJk>*Loz>2sa6=v*(9)Y8{L3TkO%E7Ff;u-tMwM-JNN z(L*-L@>9vqGSn$86JxdsGhBE`_L*>QpCSQ;P=I1&Up@D{Ey|g+v2@81)-|K=&5pii zys^tJZ_Mf|th4%b)wr#i_DLQb7b!PBm(cGmgX_u}T-BFFS53R^33JKD2bOF?zsZ3` zeTjC#W;z#bSdPB#pciy`?xW3{5T|8*0lthrGhcD&69f$-2d@&#Q3$5t@a$fG6JY@eo70t zF1s|%24R3z`*)g5E6V6J^RS8)qp_+9F5B4RQw=1BNp=bOvuj%lSY1#8YilyYKE}wrAJ0uAkUzk=dI$BNJ;= z6f=Ll{rk9By2vi@44V;0+idHPyg6Yn41d68J_CZY`t^}IU6o6*k&mSM^1fNmrFUX^6JVKUbA;en3z!H>Fqf(UpUD#2)Yw` z_;qJjwA;wzjswY%PL3EDLS-E`rrS1{kkWp z8~9iEyQgl|Np&jRd(Xdu)wyLxysrPHx#=|d=wE9l+*I1ze`7i^dUM(+#~VvgalkPn zjyMl?9_IGJlX*RBV7kJ^>!z@>v5vNC#?Iy_Wm>%`-DVtM_lkky+=eZb2d<9_pha8+ znVx#9?d0@Ia7^6>bCa1XA93K=EQd;dRstGFFu4}ZbdV8@m)aeq(NFD$%&ZTvVjXhD zj5K6ZP{ugm(C(Ylf@VO>9lS5ieBpzs`-X3{GLcJ&F~INqN}1!26gZmQEbdy9UjJa> zY2j!LpP*~v^oDPoDkH!^e}M&t+cqeukva`sb!WB(w~mKyP8U4kP96J9KEu~ta5M(b z;No?i2`Q+l&&ADwfKK_{K2ft}{qSO6-#cFC+OYr@VCdq{`e=q`8OFP8BE~Cz{NZDN zO3PC>E$>fDTAtcFx-(;VzqcD46Se`r?!YXV{l8*X%aESGoIJ=(ILgILJ3?hC z*h}ETsRL>9p^v8J*^_z&=}J$`?@oIccBjKDJLKeJ7M(tCgvatO+qX>R>*wKtvVI|= zpe2XWnNwrwzkDbI>Cc?n$d&}$56D3`)^$3K_nb~+-3~XeOr(A}0OyZ9?5uR{*33ET ztao&W{&vYpw@tISy$lsnGZH2X*vxOFE4ve0Zy%n=sk$rr<40ot zGFQ(mu9VFk!Zoz)kWHNu5Eg0qHj;sDLAWjNrVV;iNGumlsqZLE^j@SxNPIS&$w8Jv zwwA@4CNq}cWF*T+O~ZEDpdl&ZYvqTmfY6k2?pp)7!IhkdyM)jyAQ(*Az(Klo|La>OLeA_=s zU;DaOsSR^Nsg{V!X~ho`#O=LZ^yMj<^?H z|3Vu_7+-zKs2sBDPyA?73&_2&kk=xcbQH_^L>FgIr-^$%U}e9n`zvkM?SN*&^{Z?q zM`IXraK;gWb-*&s5ZW#!W?N<_;R3Nh#}Rj)D z3eKXs4O&kKybKLj@u1Pet!kqXtXlry0huI=Ha-qKE{PRCcGK*smD=Vez_*#0N{c#(XHiK;nmHk$B?R=l)@$SiG@FFs zj93@!lzBDF0}5%4uqa0y%e3bPFHS>hOSZ}chaCM*4(nh8S;R{=CC7*dG@4RgsGC)+ z)XfCzWan1krF6FOKIROaKHget!BGb|;1+f8Q2)WdO&jt74Fg-Ra>Qoguydq=5%IF{ zwTQO%V*O4$Vs(NqiCs?r>6d;z{rdlWe@R)z-}Ke5P2c|3H(C4TU>#b6O}phDJ{c4*_Btasw%sr1mJPgr(&;h+ZFe-0Va8LL{mj5329@v|&H`|nNM z`+n_TIh8)Y^s+R+@k(3vI&6nD_+)@CwKLyi)3s@opxHcDt^cwdo%n@mfsP~YJk<*w zs_axTFy|q{@>*1*b4Ldc2tp7B%w4nMqd_Ac(*EEq1Vdjc0Xy-D@zt`GA8FjL8_*+o zg3L@T(qcvyY3-%6)U&(5oKV6LN_xbBtAu8D=$hF@S;NF(3*cFWCjawP{?NBlI||g$ zFTO1es(^+jqCj7@{H19lUP6S%J#l8og(AT?V8f_E7Le>RZs;X#WCmQL{@n@pH$D?i z2Q!=+za?$Zhf5sHfFrJ4j=I`QVMEn(yG3WS0gB>y(*LlDQg|te2fJdq4?^QFO;CYu zc;)~jAyO5Gq0;T;26{`XKZ-1$RMR8N7fqCtF+!gwmhW7_MMjqE(MrA-2Lysi^Qh4{ zpI=McXyZqbYX7m7yOigK0LhtP zGhP;}`pUHCtP3FpG+u`9iB0*Gzzpb@PWdqtb)Ie5wAY&TpX(=+bwI}gt6G5e<87#h zAVwN9*KPK?1~(ieQ+;y8DX{Mu1CfvW4oSEEO?%b*h0TIjLZ6-oU5nlNNYxU|MLEE9 zcteaE3w_%EX!CfQ8yQLqr>9fT!DFfK#Dr#nX~}Aj9BCRzdT_|;qZH`R5yl?I#uXjb zpaTn53H!A?b+`6tq8lt~kK!&lD(v)CIiHrZ<7-7aF39n=ylZPp`NTnH?Bfh#uQHaO zvX?C9o1$MB3VYEN$&Qkv5+`YJ-H(Q&8#GZHP zVVnvk5pBXyCZN%wpqI(ev%W2H(#mwpEG`tTEMV|2(+cz|3SQul7;zw>k&g=Ggeq2U z%?M-fu8FR#niV!+eNznzYQr;@iU&*5%XmA_Ob4PuTfP*qw5%dHCBH}xX!%^$2#q4P znIOpEhZCAvv`aI`(~g55(Ec9-Y2@lRDCaDtMZE;9tXa^9aS$Wc+Ho+B>jtj_D{8U8nfW^zOMexJ3Hfs%}$eCV-*Hlukb}a8bWr*U7xQUgQWs5 zy6$Roy5e}HZ^nCAiDT`F1o&EjEAjU7FE7reiF^J~GoBBp$Cj>6`}=N48|`q0e(RIW z%;SCD$AOwp1a|P!*_(xCY1t)9#>i1`gq$6|fv#LLX|erMzTR z`lFeUb>BOFaoTXz>(bQ2f0-sf`E z$)trb6gNF{J|Id9f#GM!J(=P3w)+OXp#*P7$1=v#@FmQ$vvEA`G9%xVt@2$Xx4bZr z7sZ5sE|5WE&{~2{`OmKYqwSgrJC}|H&d~x@msf{ltYelz+|*3kaId@Qm&6$jVc{Gw)zXy3EJDLYklTFXh7HS26vjxlD- zbsd{X-A7NRzSC102fHWs43u&@8uM_FWYYAJOcw#HQy%L(JUk)rS1;G-D2{kUq+}Lix-L?m+J^NH<0;6-)4u~to zpUjq{@_~oV9KV@Qd8~c)X!)J&i_ID1hqV{qsf8VBYU+uU4n3HLFZ&7|MzIkACeNQ) znI{D31I+oRSk@64$aQEKt#QkOKIa*o?tvALperEwM#}?P?cE+^q$!bji$0{GM}#5{ z4|Ft0G}C5&E4h3I4Rs|gOdK#gmZj33depXTskCr|b3jPWH3V%y=*xJMAF7idMQG+n zUhoK01T=bO20hL<;^Bl|QhVLF_xI(DpGllf-#xHF0yDR4pH_RLP$eyE7WZgZ#GArJ z$3JG_u^jaSfBv!buAlw|h}y!m9JNydu-Ip*l!j6GJE zN*~xXV0`pi9TvM+tX?zDZ2miJ2*CrsO(hH~&5n<$e}!Oap1 z9u4`@ZFx~M;qH0A9J~k8B9>+-t0}M9+$TYY1ndTzs|X2XL?n2nWWeDrA|}_ynD$aZ!kP zzBOVL3?>oJJ)D6Q_CNw?{$Xyn+~lz-Gc@F!X82BBy>uBrPje26M;SBA77%%z@|#(( z93h~^%NweJ`&=DnTT9Cc5wG%v9CZ$}i2psQe`dl>J@)aInQ$CGUe1KWVYAvpkK#VJ zftTm(UeOzFIA*+SD$VpJ-|8;w zVO)i09oL(M%W|raW#m}{PjhVy3tAT?45e^yxFG|M&ebv?vIdxmgzx%WZf3=}5;Oc@ znPotk>0{)9^YG=bw$t(!4&R$*9{pJAz2>XUp&B1`&UuZi81gaZI`9#uAaU7Dey;?# zua}LFE5qj_iy-|hKbJk?Rh^KZX+ZZEU@gn=!nBx!d_v~zHw}G4w;*?2ybMl2IndS1 z10Hu3zLiJ#mP&f3{LifZeGe?RdFR@(!0HyD-&x;l^y8LcP<_$HE1*8aCer4#Q5)&& z(@nz*4!0td9a?c4D+Fo7c6zDc(vm(efDT-GWIcqISEYwCmngrjKk5XGUIg9M?C55MHdbY>>VHYq%CRI+{9wPoO8RhlvZPg{F%cmC(coqgtN#tYol_@j1C4@i;Ce4#cO#4`e_ zhuV-v+-PdFKnJBQRILgvuT+5zKWldNJ zg=rz5WD8jl#nFKl%MYG^_o>gN?|awJnv**tR^Tcfl<@C<=s0pjLm$jmJ6`Fe-(GN7`8I};pw3fG#ymq#gCzg5d)ADf-> zo8>z7U$uX8!s#?34rX8`9LrJJ4-O~X=FOY+VE3h|OmFAG_x#L+%dOg)XS^uu#{V0P z7Vj;XyBwn%Sk&*KCWeD5P+}bLe!@;YAQb)4@PnT?7z4jI+RT9NMbvUU8%`c_Iylh7 z-4IPq#tB`80H54lgM-145ws<9>)%-yXD=oSc8!ZMs1rTe@xUE>(Kn^zpZITS@qs@| zGh41oJv)7G6UM_e17%`RlB9s(wmHhGF^Ph@Uh!)(_c1;NEYpbdF^3vJg2h#yS~X>w z(Og%rYerJX$M6=n3WS61EbjRsw}Qy1sTJ3Um+r0Zj5?P&JL)>+e`fVhO>{!X0_$3U zZp|OvHDC}ajWG(xRriXVR&pS*MC^3;wzP5iNLrlIGE$uq9i&DUJOmj->~}~V!~hZ> zjGPS0a;>EmZDKA55VJ;@={2k+ri)qvHa#nZgvlH z@E}2sxKn#AXL4Xq8d2NA*-9I={t9iN0~H##mGvq$nj#>oGHcUSMHEAeXZ}rw%_6O4 zVnmcO%DA^_9L#dmpWdIg?fZROk~(_b8?B73!j03~m!|MDD2H4eU%=1Zdw=?tfAJHV zy}QuHd-tv#>EHk452P(yG+Rt#%J}$L8q&;MW^&@hXZEkngcGH`+n|HhLS|Liwxtaf z+2{~s;uju%Ea%kV)|;-=c-HIAKf$W*YZQ+mJIYK-#DNM6C-u=H0WF|R10(*ap$vUVvqmwNdNUd zZ1>);`FP`_!HMcLFNo9EA9#Gc8GaspVP|QMZhV3VNx|C(0#f{g&7nUO9G^S|W5@th z+#=I&&w@WPGqQl|R)0cJ6P225v3ME@e9w4sBd77~aR72DG5F6i>)Zlo$A_ zcOoqt`JJxL0i?D$NFDg7exJ{*w z=^6-Pm=RnX*CA5`lHG=00c#wORRxU#Q}j;xE3-tKYpegVPQ#QP*s(y<0<@E8D*=^M zwkzU{Q?}ocoJdo>I;3I#3u$TUNa~d%&c{e6K2v`O&xQp(1G+Tpsk@i)T0KAi7{GGN zUUQbYV~8bL{HQQ1hue~H*!yf=AH}5M6yyVWNe-KBySAnD@E2Q3c2W*HmY3Qrb7rc@ z;l~eJcH*gI;ArC`mOnODXSN5kS}3C}Wfio2m*5)4u&7EP`7v(KYr}jt&ELO#aXLMC zwN5x6(Q|xIOHxPe!?mxES$4ixl+PTKQ%~h*fP^j30ss@{TjDlbuBel0jD;XOu43*m zd2(@@Zp$hQk7cVda}InQcsS;kmf0L%e~Vh?y0Rf1 zS=gjgU$07^*Isiy+EQa{_hJ1VF~{5{9ZIyQKh0N^(sJ02ZqGC3>IQ8w?xOtX#Foc((x8Yyfe#0U+N}zF2O(FgKgFKJgx% z#5|WquX|JK8OqC(vn}A%7DA_hRv2LgjmeR~BA+n`RY+oddF=7M>22@$(R5mSO*X^k zP2=f5{o8+)wr}06S+qkoe%NWs)vQ%F3{p<;Agz`aR?0V31pJLLc23Prdk9 zPNp{*A>64=u8mu~BvnFOeE9cl&~ssS4Hl+@E@2~AGBb9StQd$7Jxo?+FtF_^?Q`?0 zH2wH(Y3i>3mAYQ?-B!n$!|EV}H*<@0lcBOW4E-EJMmhC5XOR0Ktsybd7df6680681 zy%rp=#23#%H38r$Av^LyHjvQN$2H0{8KX|Kg!W4zuTy@Eu}b|{<8Hh8Iw>6sR2HD? z>5o2aV^{85(Wim&rlTo0_Do_%9AQx#{SWTA*2W$ij!6`O#e+dG74$J%4kI@F7T0cY z8TZvznR&*?_u{~!IbYZy-DboIXGTsp4j;f#7mm2OxjCCX#Upap4js182uDwxve_v( z;zqPI)wX%i$Kz90ZK@+^3zSZ6~0_p;^z@wkK$gcpH6TN^0v9maPLzMx& zu&d!|*)Fh`i=}CXk(`;}3}~hiHeJy@C$QptSZ4N-Uhr|unS)O0vA^8H!kjrw7nkJx zTpUQpmdDef#Y@wD^Wr@l=r-gYr-1*7j7(Ua7Lo<%*fRKfyeW~l|JJayRFLTEpWx@zhD{fp!Q*hf* zPj=h<&0f-!r{|MD%UlQNk@?9kW-qf(yuW5%B|w~H+Bc3I%kiy4SxSBqgb<2p!Q_LJ z6RbVR?$JI1%b~95Lt%glODqdtoJ}XXE|D`^ZC8IdZTXACL`Y!tELRn>tRz$v-9Ja}*?euBn>3vrDJ%z)jn|3hhL=J7PJZ3{41IfJNfWm*a^zBD+B5F`7ckWdUxy0no%3ShBs)2pT;ou zw(QY9oA!gcTRGeNS=N6T!%fn;UZ57}IO5J{#b^MqP8?9W(8sHet^0UrC|O;KYcoWu z=h;9hyciq7F5^w=)$dC=5qHSdWjJNTLxv5$!U(Ov_IO~lO7zrPUp_{C@XVumC2e*3 z+A6LeSgu1YXD!MaqZx}blOA!0TwR9KL_B14%HR6%`%>5c3VXtR@&A((ZaOg&ZgR`Z z)5Z-OY)L9hQE|lau~43n>NWX%U%lmcJIQ?lCyM^_FsIX7U(XNuyuQd=Uv6~ti>lQL z&n!(+rm+`ID4boRKqTTABOqUaBm?4vE_%WImEZoOG&Mb2jXR(@RsiQ<$(h?5U;pZK z!!=g~GqHtxaE_Q@Ij}`Jjh!yK=hw|juZ$c0DOeWhlHs|oi_^nr!_{xlUOSJc#iu`+ zrng>`daihtIpfSeA=in_bo?S3a$NVES%4Umsyc9$y>P%Q`9(j{Mioo2(d??1-3^|Fq=pFj9in)uwO)8x~>R1A6HjB|cwNh?>}hfdr_ z9JpG(B^DRQuN{q$nJJz8nU{a~y0rVBzcKYsK9HXJtq*Hk_!##K;fhAxMH}J;+S6)# zpO^AC+Ec66ox6?&vIXdl`mLOH}iM>TfIOG=OsMGAOlP4$A zm6z^u*~RtL#H1a%uyK4;^gc6=86C`s!}-@Ir(8JP_;9Ar+VV&#=LaWbW@ZRe*b=OG zO0zP^DK&rO(R5lTlg^D?E{ETU9nQdvxPksbO%C&Y=xl5de=m2oEiAEdh!Ko@OTtiu z3@xw3&vDnJrmZ%H7eO`4M}kb2(z=&$f+{TcX>1e|{Vfe>JXhY5X5BFMGlrSNGtY|S z*|Es-RXvY<_M9&pUd)F;EG&-bq}Xlgr0VM-oJBe)W7EnZn=!|7)~%Nnb4D zA;#W-q?Yl@$eARbEdiVOkq(TwOKM)ju6lk-=Tc@FKUeupDOFi>nHNs(HI&nRd(27F zt>wYYiu3oVs*u)p!c0^(aiE0;G`x&MeiI)%Koa(dc(5@VE)gOlL&rCj3A(xu0v$)(d8`}j zJB^MMgE+7l18vjLrJ43$4$hNJZ~1*aOo@czQfCGYX>R&hY7bQk1qO;gJM!M zp>V?O?Yl9Z)J(V$J><%R8Dcpq57hFZ{Ja&UVr;KORcUz8;`<_?2QLNI;jMs*D!n?r z3moqCXID_s-|ls>fkH1jN^xE}O?c2ao^wEq1*7py(?uCL<9_kozn2cL@(X3AS{3Zs zy~7=Gl`!-G)n6by+2j6;tYk@lFHq?0FKYISVDQPhTFGoG9XRmS#RRmdtx zqY;4RloHsh871N<2b~|Jz{hB>oPW%QBhD-tUz)m@=Cqd_&(KRQ-q}`6c%3+PI_=uN zEe&d`hao+0`DnER7|cOdKWCX42fmasW0dx2>ThGdX&)z7E~SHAm)TO;0WIm`InEDi z=7f_xGhQAmjBlH!o?s|K$f%X=O)XF`Uslm5BA%IM5Y?A)-R4G~qv0gr8SdknST8f5 zhSjdUW(W4=!r_X|`}){lPL2`n4<*l@IX_vtO8VTq@WHW}bUw3=8Fx#Iz3KSEo^)(w zce;OWQHN_Rr;SMtIZkDjV{V%cgBX~7Ak80oz~pxI>VseV%<HowmfZ5X=v7THOkh;{LdPh8M9#%a^OWqnibc=A0afArcnsXz7C3>vZ;tgO9;&B=@Snd+Z9 znUMqe+rIxN)8`*}q{-53{F6WQJ?X}4^S+bjD6SoFj4kjlWx;bb`DauA(BS}|*9+8Y z`n>ph{gA7=XFS7pW!cu^zp^x+rgcEv?Ebs;vtoDp{M_~Fan0oI;1W`f6Chl7oG(=j=1w!C(+>onkqIZt20`MROi$HpvRq#Jl6gl z`a9z&dmhWu5&e(X0{-$s`TZgC>?}cRSJ1}*9}mqCFDL;mUwsBA+|(1FFeltT?Y}TF zerwvSgBh3s$G&j<@#4;}PdrT5cFHs*K8N>o^c?)v!EgM5uCiL8Yyj{mJzo#!D63QJ zYJ7y3XVft!I36`&wcsyF)JF(32yIJ(-N4^pFra=0!G-$RV9k z1JmS*jR*f#6dK<{2?5A=K7mVq%z7g~;y?n9EYr}GuoUqiFY_bU{yT#4kFpz9F{3TE zDVcvV?fsQMNDFGW)d#}ZmNb0T&1v&XZqy5%KI*nzn=byYp7i9eeL%CxTyLZkLU+3{ zqbla=*vGb&iGD`%`6|DbT$x2QS!AiS7Y#sRWkpNSF6~P*PaLV7pQO_+T4umi_)h1v zb==C$?ekQAjizF&JQhU#)f|= zau@@%Xv%O*2RMxAMAQxX0ijxcst;&*heHZ{WEfK;Mt4F$@{4%RgQ9kbyg*A{wV%?^ zuxuj-z8pt>Y9h#Php?@QQYvMRG_WdhwJ5`}yfyMG8s!JC;BR?|HaW5@INWB05+8e% z`mC(r^k5mQ#^5{-Ga9-hl;x{=cA}QB;;dt7D?bP=uz#P;qFd0uezS_lvR642mNurn ziL2>5{<%>o>!16w{<#lb0~%k` z*>c|lVR>Yf>k#_tKI)oE zi1bMFq#8HhvqYRjZ0^4QK^rL0&7-L)5=zGMSWz?g;&jt*N`J<2eUT~TP zcVcNgeRknx8b9}>aVIbUfsP~Y zd{$3%xPa1y?roi&887;q?Y}5}V$+YNlgE$RuaxuJRMS5G7|{)%heti0M?A()up$lo zh(~@iq4}U#Z(wioYm+C0;EDY9CsmO~G}|H<4~mwst~_Cf4iH8hNNCbG6Pofn<*(}m z`!(%D!pA=QaN0c5onG<6adUXZ-fKJ{tA`}SX9zr~`MY-U(`lPt93!k}fAC5t95dlg zX(k-g*!k6*<)}XOTN^pcseS%}TA&%Pb}m*geH4-q&>E#DK|*kMg;T@}QH@VNLztX{(Z@n-q?-6#Ow*t74Yfi`Ayi9Kl~Lyr z*PZdtLNYTeWud>dHS%&8&_~4Ju}9OaHp-p-!Us~{i{CCiB(cCTRFQW6|8H6h8-W|~Y%lB+WwvZ=^(8Jpayy}8fmJO)JJ+9e$ABc|IM6Z~KN%Zs_EcAz z?9qV@3-_hPNe*n-0+RlqTc^DnR(XlB06`JTcsS`I&!3e`TGGm_IhL4m(&oiBK(p;!Ki9JbIp1K1z#%E(K!WExTs(b{(Kk)?U7R*Z7xdvC zTX2QMCX?iNr0KTGC(VLoDwY$V4Oy$@+xQqXAcV;>1jLUpWCt&xh0NdRig@Hx4jc^2 zN-!HpA$$MVeV3eXl#p?)r$;juRc?N)=+TUCmJ`QPNt~vOy*l^-$TLHSMgVc#&C1#L z)WXK}lupH6)&UT`%QIV+HvK0 zntAH>GlO8OM7;2ugV)vxYf$PG}fr%!u61_>?5jremTZ#{imr7=Ku%73Q%$3g=`BWZA0 z%K^v6><>qrEgO-sd03t)#7h{bjPWAbb6PKb{^OaE`Ti?fZ7~bNJeeVaBgDt`fx6;}y)yzN&U;GU|)hDLRLAj|V+EBGYxP=^_>%BHkZzs=kGx zp*u9`mvy??$@l+p8vdd0OvCz7wSVh&;XYaM_dcSM<$i-6p(EU#U)V0o%Vp*+^seCZ zT>eUfT#4ABYvYU3rrk1vmaLG`o z#>{5R_5R3s$au#LoS6a5Qqe?>B{||&w268QwK0wawEm*Z7)MBtG*qHnL0gTHmXG7i z$1CLH*l|7RYv=;Lpm6#Y`lL^A3(dtbo|&b~=LW$BGeCFyb%z;=@zUc0;Nx?y}s2;^M~IIoT3^#IV)VnNY^yRvWY1~inMR>tAk!k5fdg4Z-+Y9?j8 zt!{-DX~r(dG7Q>);6#6l{ozjb=->J?^@5%Dab zacgLiR!{|XDS~)eC$x`c)j22irepV|jR!uIx(CM6=(S&0l{?CBq@hBFWg79OE~Tsz z7Y8cu_W$v}{#yE-Km4GPHZN~^jbPJY!NtK68)3qK!!gD*OvWWe^@D$CWk zTlCJiL|fujJGGq7dKt8E->mZ!<%+3MTm|7`hHg((Kw3%dMz zNgU7%^6+rCIpJ`A;fTZWnxZZAIWqvdsjVAMq!kc72?5*U*&tC?Loek;9681*UxJQQ z`(Nj-tUR42o}Ns@mye|G?R!#hx+5)y;a!266ST!o_oipb54NF0O;yLx>-i|Zct(lr z&KFK*Rp1!@xjK52=*CToIGUXb4&X!gfX^~SC8Ql_deq3k>-6TXkZiZpbrO~Enh_^ zIUv|4=^{Den&IgK%wU)1lJEM3%o;Ov^Za&sV213LxcdFd{A8Nx(rKX8QdAs%`YXr$ zY)^&+o&zy7G-2cc2~GTXhQ{-!s++a&v#hoGVHxRU(sHCs;9+e4c}4~<%kJHVWOjFZ zobj<&^{|YiQ%+MHaX1P%U}HJy2YP12X-3-voV|Aq zrc+B>($vx}(aoh(D<{+a2T!JVZhkDyJb8QS*Y*e04P_`*Zzi;}K4eD1wp>hxvRJDeA%j-{PX{ifCF z==E>XjK9GUTtn1EPPQ4!G(@!}H1T2POFx&=yMO0>=@)ZNqg{SO733ceUe zoE&ie{(AXMhR-*CQ(p7_ zxt9N2^Lw`XHtC!9&$Yfyemp4#bp-eDXiwU-X_L)D!})bCjyT%%^z^iT!PVx(2l(|? zGXV69ZjYY4_JN=m(DEgne4KDo*$MY_|BKRzv76H-?FWbRjeX(55yyj%hhe!_4fJ37 zfaUZTFJASdw0f%A$BH?|{DaBL?o_q9Q30eBaqHKr3!IiVJ}4n<#>bt%M%QfoVUIpJ z%^B1P=K#1sN52{~dMEPniG+Ngg*Zy7OD){Ws@r&3!aT&ycVgvoYNf<~D<|A62w7-$ z2UknYFHXH<7p0MFzc!t^?}KUf?%z#4H@~BH#6c7eCcHkGo$*o}NFX`*bH8eS#gpLK)vVga4<$Ck3ptVz7ELn}&`kY!Tt0C+|R2zCvYL_n#Y%phZ7U1Khj~Sjs zoR|yn{>fP4xy-zZG*78<{XQeM(2JXh_~Rp#XXu`bw%6o2ws-#_&8%WMqz)9&v({y{ z$_w2bI$z@TSl7zd^E51yqvh@@LIY_!Vy$#7%~P;`afQW&D{HnWcn{XcN%0pp9|Tf1vRCu^%y{ z>>?+#>V1j2;4lc%S*A*%mhwzFoDXx_;)Xn&bnF$jp!Y3o%X`D=k_amw{mXx9n>@>T0U ztDLbbU~-ff@+#(`2bhZIThr5IkgR&;kdIA|zeoGRP58cWdN-;<5M>y)E>$Vxm5`t_ z!C-g-b;>HP_kZXw)4%!oU#i5Eky~%NF1_oW-=P_2eKrG*6MRRs*CZ#Ul7_>I`fJgp zl2>w)R2k*0>t8&;XNDk$G)$bHYGtOoTVHgY&4k4H&8jp4dMr$6Ymi6(CM^oG4E4d; ztJA|PH|j@UITAJVZG#+e%%U6~(Z1Aj$Z;|%-yy{LamCxCThd+u(Z#saX7l*s**Uca z=UV?YWhZtaTcG2JJKwcK52VxOp6JgEZ8okBOXs&CNE`REzT6(j8cc)4R1ezRap|b= zunr6B9CyB1PSCX9w&Ro$w+Mp$%+9M7}+~&=j%?XDi%pRas!*hA%mG-<~%O&t4@Jpk9?Uh2%P4Bxi z_3ye_zeM(_AE*~7jeKZU7H4%j>Zi4L)^y@vhUu-ZNSlX;O*YF=St83v3_3u=-}w)! z2(yJ(6y@*-vNX#TWcQJ7^_E{OgyK|0R-soNI14^b;x!S+l+)L*jSYIt0y~I#S(crj zacF5sSx3`_&xX*Sddw1B;VQ>Mq5{i-ywebmjaW#Ds%zL-va;^S%Zz*M$Le;WI`x1{k+bLr?G z{z;lC_HnP}Q{4VE{&jClqhqJi(Lef-_98O1Ydh$6Wx3CIjg4uRhdvth^`~9p6oy`q zc74;^(-I#<;7xon9r*Akta7jefxg3wf7PeJvmC!fJYbP!(Pv!$QD;qt&-48&U-`;( z>#gkTQwM(h=!6U00yIEt#I7fOj)rPbQN|qGJ7|S|%KKk;n&{r1#+Hw1iRK9%LbKTk zb!}MDp$qXh$ z!OuJ(BLu14H1I3hj07!d28lMGLzxL^IV#7E?(#4m@?#wA6$9e^Vj9jlX3m+TOUMh^ zDaVPdJpFLiEor6{%N1v5rgad+yg3qPrl-=wr&iL7Rz9tzcc0c!d_`MXquimpzi06L z4Q(Uth2@%y>qXHGIzaC{vQfOG#T6{_YDngp_yuw3=J%}>t#>uUrRkIA06y7sm7bT( z%#s7e+MNF|?4`&Y#RM7DLxdt)7np(*8bVFK%)_!*ssA7`diV$@OdRze@H}#GK zpEfbCCCVEP{At=S^c>az6H@A|&9ahw$_dOsK+G3VRx5KGV1&*g+;{DG31r|87jFs17YULb&5#Df}p zppPx>URp?>`s_WzYz;UAueyAXjW--XgEMYPFG@2{+?E!OK5WQOEnJ))OfN}u{n{H_ z&NkY~207n`iOcaeI5=#M#9pmz^Rndc`6!6En10!=FqLXv4(eDTP^)4PIfwc;gBhpJ z)1q1ey#^52RLeQ+H`l=nb7`@=(FEAt`-}wS9 z1g>YRA2R?l7-sw&uOGHfR$bc^sB)aW*p&K5yb`~Te#&m4Z{@B$C7@|8OTMdkabk61eDtxTEv=vX+Q2joRB){D0<&9-0lH4|>~iQCNy z$H5FIM{iCedg$>HYQH)uZZGTAFHdU-J|D@}coFO`bS>>vsbLFiw3Q)hMXe7i9fy6K6mJJ%Q;&d6iDfB}hhaQ$f$sOj?U29)BS zd|1U86UB1Ed_1S*H#9ZGG)eLs#6A_FQpOI(^%F(}CNba2+#3-^;%xUG~5EEQpFm zeQbE-mNYE7lOOuO(&78%j7$0j_=Q@I(XYLRhF6llVJw8MQM{Fv;WYfhOSPZfy!MpT z%tU=(n@z$XPzh(`x+}F5$a&P)A;~Ot=}AZBh?^H}@Q0bjOMW$jwphD8vp#_-1m_nC zt+1j$OL?u-&;E|RFZt_)zaJK$nef-__crz@u^PM7=!i1|jiqpG9dKHXxU~F0T0DIy z^$cyY0>}6n&(|hw=j$>8Z#}D(^tEnX>Ed1OOH*fN<%Bo9-oTDX)yHlv05n z1|&q>dY$nBe_0>Ogr%&<_^R!YZ`8kwzP7?T$#(ilk&u^`#zy(@pKocK2IRmo^g6vyfggtWHE$2Px z`eZPcuy*N3O3J#lpaUSf*`HT?`ss%pezY8%9WA z#Zbk{W2xB1tqiO~8Lv}47DfX`yosZGSm@6<13m)(bn;2nnY1Us;=5t>Ek{$MNbyX))f|+T(6Az?KhyK*| zh1+n=*BV~vXDSC_KzOYZk2FkRo%eRNJ==>N>jN-5 zWWqMPgiowTpX^i!Xa6K=+4!cC0dnl@`s zI+mjH$4odL>gWJp9C8;-u=dM^KYu)0Unhk|1+w+(yd)clfIb|B+rjS_}V zj~owzmge-q-^52p%$Vl{LS={wojo}iXDzZSrU8P{mA4(Bm02K2EXs?DB9wtO910m# z>5P^BA_I|0aAd=)koEDwzwZ7GX@o0xuxRrg!A$j4~}}Rl3Ntr9&7tChbx#*R3|=);DR-*v>S3!%b<^b(f*Ng@wod zA{~5i(rh0>VBe)B+n4^(o3(q8hq=dYPe*RQE6ttKWV35tnznt#t!Y4gWb{?vk-8S% zllFajpUd9(@^o=_!YvA&o{wDBb`OXIt=7_Y*0E50bH50<9=_=keTkbQz{CCt zzCa()tA!_@KA0}Kco#j}bZg=?<&z(}H*vWSz1aYrup&pC&u(XCytI)KOeb|^?yWt?XxbPw!)~D}JKKqa{;|r3(G-GJf(i%?xB&yJGz+rf;Sh|b0+}ik zS+3$usq-hFGQ_FD%Tu4;Wm%2X!(L%HR9peGFGw!YVdmZfvM9qN<9Vz9EYi=u@t9S{wP4RY)#(qD(jyaW-y!AOt?oLe=@!GpZ{2z znw|-E72LXIWBU1j^Mh&o)=k@-8-Lxl zmk#di1uJPU5Pl8Sp&m-ZDnu|-jutRy*kGe%ieYvV%mWW(@Uir4Gj7Q@x!6tr`Ziyl zHeCMdbmHE>k|#yWywqMeh0gdOJo7U}h=)S3HimwQc)%j-Sx2J}q@Ig3o9&0rz;8ZH z-TC3P_ak?=<4E`Mw|!l9!ktb>{@~r|*kjrUC6{O6lOIfzpS(To{DF6BU&o#_^4hOY zL!bQBG%dQ`E3ft?p{MRkUwrrb@Bvtux%cl9f5)zQN$NdzO!kxW%|4>YR=eeX&F14< zDzxG8%d`cjUp#&wEgpAUcTJwKt?%gbvxl@-k?^dJGsPK!$`Snsyogt}Z4yUa0}0U< z)~o;cpmnOwwPS(REkLu!7S3awdSTY??)Q3)E=jMwyBrtz=qRW0-F1ay+a*&aa^dElm2h)yiTQv(#Gs*bTUJfhbINE(#ekP#( zz&dvpL{a}CzAOV2wqi#3>(@Ve!fZtj!@cR~v35Og*ga(5A7lNRJ=!;t9$2_O?eBS^ zCLKs$Ic5j==1@QBAK)F9*>7^bsjc9Q;~*83EZdv)f2FO`C8{*a&@7WuC>c3qD?}XF z9xN9dm9)e4|@n?~v$=Bt$%Vo@1?fwT>=c zHV{FcQ>0ZCtg6;sE#U`Ut*aVXU-!=B`G$3^S7o`cfL2xAG$v+?A3{o`UUIo z@Qc$4El1s|nQ$D;z$vXP(ZU&rQ_ddd7o;;z4Kz1)YreWflO?$d*sSPd*e~+U*3nUo zICIGOp5*wic*_r`C-rgkr3gRt@BVY*?_9(0|JpC4Z}_^`t9)*NDAJncB?J3N5Hq1H zX7=%PH>l@_E{4mEBG?Z&e8p>0-;??kbY(%nM#h|Ha^)n zs`eiJFZ1G-cQE#81+sV3t{q$39C1(WKd61iL{I;*Ki{v?`ZRK3M0&;Fl&LSKXsv&Y z)tYW%INCKbE%59Uo{jv#{8uyan6b)N zSQX1QaUnOucKyT$R`IJ{#7p)ScaM}=R({o8l)qd1{1L}jKXZmyP75ACcd+i%lzL4PgVLWeB@QT8Sohg7MiV;r;&v!JX>DJ;EU&k-6 zR&3)D(!@{rremgSw&|v0Ci+e2AGEHOn|X2igq_lRtm_KBPimQ=94suC>{UDTQ!^be zbPJUOUMTg?532m}PCdNugK20|`@-qKjiHNg$yUUUK9-c_r)H|!)eYtHdLmBSiSY*u z)b&p~@9@V@o=R`|?sui9_8kPFCE#@X_dovq>55A)PNUj`ZG&dOu@~F$(4aZ9$S2e* zS+W9>LkQjcF6)!aVEI|MXn$+ff8Z`^_06Y0doQ;wVW6)kUE2G!erUu2cWv6AZd896 z(#$wkEgyV;0mZS1R1&4aX>AKh}><1(?%aDua8Om?sTfeu)pBY3v z-x|}#rOb-DYoK&)FE9&q9C7EXLewMaB+ZUjrmd#hdL4N+nli0jULK5dkLAH@gKZv; zO&>AH+2^dUjT@9REAedkb;|Fy#10lB5-`%&fX{jT^R4c@gJBQ9RDo&%Mh2Nv|M{5q-^YkRj= zy-_Ds53vrS1nN^6)k}r-An!%?;$$!UiUIjPX~TUU)PFu$=JLA!E`U9=%h!B4do4W)!;*bvUbcg z@yRw_O+Xzl^$!|jSJkug_n8Su&>XW6IrBhz{FmQrv%*Zg(z`alFl~R^+tTLkdTHJD z`gGaX9!p>Ru=a<`^$#TU?%rK5T|GUPHr()HeMl%Hc;`4PzbKWOP#V1K^0ac>K`qDd zUx>T5zbak&)|s^TeSaq%AUH$$m5bsrBQ9)Fmmh>2(1AjtYv@Ck(qi76NvjM&o@9fY ze;tpy0TPS0z6IL>wKwWxcxc2?zH*npQrA@aqZ>$)me|Q7nZ2&FXBLgm!-&8AtEjEeL1S0+j>5;3gd0x`{U_> zM;$w@&9o{ko2ob(LZo z8N%7tf90f=zh=*PI?#PZx@Y<3G&HJHb4SLd<3Jk05vLh(!}{SL$heFH z4!O*PBk)57dfU5}ptY9_1szb~d{E zXjaif62q92$6badOPAudq_KyqgvWqygQE@51wqJk0l~`dGFFvl1CQ*Xdi zl#V>X=?DC`7Ld9(#?EkB=KZ?2q%jwA#%ouv)2`@bu;a&WNt?H5DcUt}NLT;VE7HW> zpH9c__*9xXEIEX#e103(GnPM)WGjEXP{f2(0qKz^@=TM0fLqfNBi&h z#La1!Z>^qkojO6T>+l?6|7TMF#;kMcSm4YSV3^{M<{A)+hyyJsAEy`Xm+{Q^L+egc zJv+2X+yiN0>WDew?AZvt{t+1Nh{dqy@yJz12nYyK5vQ9N4e1II2l59l&KmZ~x@c$n zfelX|JmUQmp20UD^vz&Ql^BrxG9HG~k0yj3Elf9dnSF?96VWV4gKaG&3j>7l3fxeI zF(;ga%9e_Z6O241eL>h74Xg@Av48~hk2Fi5-c2XFtKx0t^z8QQb5VY^KO84)HqVcn zI2#^4ePen_*T>WJ)1OHrSG_LF4Mp7+{ETAuu^()*t#TR!&3sZw1D-jMAGkaxp@~q^ zMm(TX1!g;J)gSEn6Hgi1sop(lgi|~9PUVhe<*K60aVdBq*FToxS=KdoB5m9I+tz+Z zuY03rb@5J6+lCtQd#}Gb?l^%N5W|%5nw}F<7t{Cr>z|QhtNq{u=-&0t??^AX=~{ET z^$*aldTbV4Pp^-0*mh^o_L>~gsNqza=g(N=r0zedT7sm!~;rX)#oi=8Ifl6IX~2nvEbSCf8*P^ zT+kNiIO5J%^{B(q0SN&uZYjlp7d<#2G{iG)Mn3~UP(cv!;*HJZXZ{f{b!uU*5y5es zAmSB%=oK(TXa1&lq-Dkxt?q*?VEafWqy~M9er-466lEqa;+^ulrDRL5L%)r9Stp*8 zuWj{T)l>`GGrfriCodBIVJuFPIj%k9PTlo;X++MrzC(YO1~2^qJs5pCDi1^cF6dxp z@=o9{tGTUb^YNgy7krQ8Uth^|8jtZ3no}YIB+Y)U)s59pPV}q``<#uN8_Kqq18#Lq765l1iwjqqSq&bU%S z^n+Y2_7Y_@;6h-KXpRJ|@*=%~9|g_zYg5Tgh(0n39x+4A_fS%nIj<(tq5u9nY2c^7 zCyl7xZ+q)Er&GW1yLO7G8!4rsi?$*#t(<%~9l7U_PRY|vwx4CRT=W{Cyt&V7nT9!| zj-~_u&wos_-|!9{TD)6d!J}#H);HMS!v4F`Q@`_mZR`yL${z;cw!9~vaSSx^tTHlA z&;s*E7P?wljo4?zL6XzRw*Er4h0HJPX}TGf<+j={Oku9POrG^U=lU?g4z9rtwP^X_o;5mEoPVE!c2>T8k*47d15h+|~lpj0|m9)^AgotPTU4lQ~ zJlo^$JkSudtPSUlw->cg$=KHy76KaDD6x*Qk%x${#o_%=1m; z0g=bQ#YL@R(GP1s@KgUs`t!f}1)37%gyn|Ot^l{fEyVewgVb)$k~!k zjRWwb4uFJ+N1d;izpC&ek=c9jLf^TmQ|X~kemEUCtPN=!;nJ&aN?W$>NaGtfr!oDF zY#6KdgtKMJ(I(U$V#Y7p5=`@`vq*?Go3X$P002M$Nklrn~h=`XsAy|lD3HOve1l69^1 zkMg*Us&NbLL87dJklC#wR{y~OJcW2^gCJhyAeedb6KVeFBWbE%AGVrh0U5X!aYC*Z z%9j{#V_6ot5Mp~E0fq9cIJ=MZ3TA>9riPg>6bL62OpBm2XA!bJe@uTPJBK+7!9L1A{^iIgrIPD@7~NJswWcJUS$Gg?+C;=omd4&eB}e@iDm zenr~+@>ixUw_K+eXw4zilHMzRVo%!lKYk^hJglAK&@Z!Gnd9(=%rB=^_7PHrXTL^E zC^(Ub4my6drngK|9Zxw?2USvJkr1}e<@d767EwOrqg$32PJw#q9!wNMZ=x-6ZlVq{ z9_%`YHOLs9u&xE@7W{cLs@qw#v49|0+t0V6Nyn*0MVV3Q&Bg*~G#oAgco*2qvK z3p9LqkfzXtP5R8=Y;TSRo&LxM^jfC*c=wCa4a=WMv-|H(!|+Fc=rp<=@xCbg>S|+S~9Y| z+UyzyYRFg@zlm^;^^aJ7z#cpF;WRw;u=a(!Dh=+wSzNuI0$Q0Gx<#Lh{@Z3RBdRZ! zueSNW|H6My@BQPC)EPzWZU5w}(l>nVYi-6@KTCb(&;<77ZDj1w5a|(moo!trB5(s1 zVpol#@AA^T-T`kw-$DYdJkCERcX^Q zUgDH>w^seEhhOpt+*01NjR#LX;O&K=fc!E!{^D;={X4Ep>DYs5_Ksgo3v)B}3m3m~ zUD&}5=Vtep&**EE({im|71Mq6BhtAA@*?3u(Cm_LGlGBYWV_|83m;H21(m_|_wq(r ztCuV{fLS2S)M5i!sC;n_dMs!E#D8Jp=`{7wU#7*bp>%Bg6)x9A3q$S4>H)aQ9~EHB zF3}^aTtSj@`>SZFpWsQn?N2n826XxohH}g>sb-nv*(MOcd-eMw{Suyq$>1Ya%1WThq?3f2EWT zErHs6`N}OdZPu}tDW_co)1xkbkzCV9xr@xOLzF+D=}WoMJN_Yqe`hLlnYUK?vn|BR z_yg?^y;{m2igsYf0%x#5F@%@Hc}2nCMi8`_IMr;N!8Db|H6!k*GWn?Xh6oH-M!>4w zE3az*5eMp@8gm89xaTsKB1KE_G}GFa#|rlL>QqB5!Fs{PS|-~F$l1I9ux8NVOwYPm zQ9K@L!QZ)iUKy|Qv-~bC&rdztiwQ?u?yttaA-}qXB_UX&1970~Q$R&4UTfh8ZRD3U z#7mxKoV?XSiBrY{x2XSI3Dy>>@_>gfYlX-QeUU*NxSG6G@OgH!1epUH`n42qK!;)t z4GyPcgEytQl>z<8bC0HGvS%M{uWogyGpaV`*8#y$63uHGu3I2Zn7Tv*O*Z>& z^Z0n$q|=tiM@Q_ChC$8p<+RaoXz}jGKZ3{F^(F7w)c=frz(0QbZ>7m6K9wftyVFM> zZvSR@!DUybojdoWZ96ZjX2p$iAe;`$8q^7?=9JTzghSDta?+MML-p0_6Q#>g;*{~g zd4}~Lo_#*@)9vuyMHu%t# zp-E3(q?LGU#nGLxR(UMXF#NI}ep!Yx-YI`w?-yeI7ghXBqCCuau$67knxQTMiTKPo zcgC&A88`8XUrO_uhHnQw zFWRNFcjOhRe}guh&NyAr2`Qiv7!%{_F@3laV#&0>yz66Dw5E^c)%bkeS^$=DPlZv= z7tHc2=%rCUFT$=SGOP_$@?7qT|M|W&W1I%kj&J@>Io*sbEgU%DhdlIOo-TS7AI{29 z|1emTl`==U13C^&vyQ}8X42$Ge=|M&p@+ywJv%RxEHA$$=+Zl)J@+D%WhTD)CcVr? zrl6I%fXK2mFF;v+X?R;2+OT^0T}No)ogs0`8ZgQoP#ICyvoAlk6H6;|)O&0B4WV=C zSYR~^(6;SQnH5yrI(3F+Tenr9`kH12fwc8z0mX>n#kEa?BR@^K)>1|WZ20+#W} zXUw#?jiHJmPjTxr@P$W)#KGmt9$DMBY_jELmQ-AxeENX8T^ zdvY-oy7)r~_h%D=X9=79LssCdmydLCf&49;eA4bFhy27;ceFJ29$v3LgPn1LExa={ zJ4GC*KUdme8F!V0ppaQu+!b!YSC2Et*b)acOdU?!o_vq`a(7~1xE}2b7dChHUMJ3` zwftTeaPxX{(PjO3-(J=%aNf6n?@vCGe&#>_s;Skyy!@6M(~rFK+tQHs7aShe{%Shy zeQ;2Y-G26mV^*1pps~bdIyI6BK>oSDb$jBz{DanUT;GTXPUJ649!w{-t8 z_!88A)rQ=Bt?i{yr4zkyEzohqov*s#rH4O4K)-9fM$ZS9@m8$juI!D98%>79$qq=o z`3EP`&NhVZQ66|(9GIB|-_@tcAjC;Fg#$B71k9;AW{^T!LGvmT3at@-ubf|@`N)wxtg}f+Z zj&fg+dFHa&$EZmmK{;fatpq_fK$+bc zSJi)}4;uD`TbenR4h>wFmPTG+C+(V@#KUG&3O^urWp-6SpCRUH_>}xlFMk#XuJ)(8 zuyh~{Yv^%^1E=I=$yP2d=)@nq0=DY&h4kbH-8rT=@3iMl*SOrL{wy8Yw`6=q-tc{C z>(zQ8cQp(euUzKkB#f1|Fp6TyAfD30vxk)Fi+TtSY#N@umTOUUK-cA`hu?qp6>T=vn}KG zR38W_fTI}3KD6J2!lTWrvle)#zTmvxPbfu6@_-SFp9hxsD!HB8F5pQ^dTQT6j8Uuj zk>1|#XeITywa!L7Q)z?fq!aXjy#r&Ih{NLsJVart#&|!|k#^}%LJLLAmU1snvI>%$ zG-sN!?y^4R5DtPNgm|VVJ@Yd!X-b&9Yso8Vg{Sd^_!ECX;>=DMB$e+xXG-X2YIC|5 zgPm-9oQTj0F-*wq?Y{*Nc=~L1@8?wyt#uIsQjQygqyJ+zu}p~(3eQH61ZN3x#SsUg z2I|Di5+L`zt@b%DeJ2~^9b^q$vnb8|EoLgJwN%ODl8>(O+1<3^Bha0QJ6}~4{YL7G&XPYq{6`K-^{Q={HhsZY+T`RUZnnnA z;IV-aPTEkIPw3Qmv;EcQV5M>@IfFl!4^rqUb=C4HEC1Xq9V^)^OPxvQaPdvBd;a0y`53-}$24j?zU$}g@^5;L4Pb20CJ5BkKW^h+{cU#j&;E?)d@=pv8 zzN?t@UHnyc=?C6zQ`KrXe$c&$Z`LXKEB~nz=aAzy)MrDxCMy32uYI*mUV({#7W%@@ zPWkkk`KAAvhy2o}5?@4?eoh>tZi<4&}Ft zW(RhNo!^P8g>)92;~-&)OTZZ%N*>9;SlOr7RJoI6pY86jbL!Lk57{#Mwy%ogWL(af`fXN8HmN$%z8~R4J*uI`? zZDsDbEk64MEW$OH*IoRvx)T!j`@w|FAdSNkcRDfC!}bK8q$y!3LmPDPBlX1}TLjO& zWw;&0QwET*2b0Ishp^Ib-j4R{*7w2sxlsdGLcxDZpkGQbvx0Pmbri6y%b-$ONnhbX zQ1YB@{j)D<;u|aB78Vw4^vU11(b*?#@WN|tc;_qR_nG=%#v(I%+N z+m6E$Pdl2BzA>+?^Tg+$dBl!?`rqk$h9{QC?UQqFvU~c!#`?y0+W5qj?ceug({$+X zwVQAG3ao^~#6Knx+3xuOg#BP6)_*nbsZDw-_x98}{aPv0FFK@`9u(W}vAv|F@ zz@7gTnLYzQg@KI*%M@8md;AN>%Z68^REAh@ACz^kb;?QOi!Kqhs~BKjjhbv zSwiNr_3=o^r&Cj}{3qB=^>=OInFs9@R>n=d_D40LS6w*Cv=LHv2-xD*(!kMZgL6R^Yf9Qd_~P zk4I6YGN-B9uw3`Hvvm5XP2cy&_%7U_?VEZdcE1|YYB=6l&WR4vlQGUa1Kx2{PyRY; zQn?LI!!p)E=J7mv3F{I+^SX0t)uL#0&^MQoBW>V$;%?93-~40S{FC2mcoD>Qf7f@} zN$jG!bo8(7;g3w%HQ)Cp8^kJx@!P+}uDkskZE5;6xUtG%VpLX1kImTWgVTyd58olh z>W2%z_S^NhJaZDSboATM=A2ENw}XH1$F>|RRF>}fE!+1MSKIC@CarJ#ZFcp~y}}lb zoirQ5D`j{sMQH{fb05MhZJ2CVA4Yf-p=z=XRH%pAdz2o1#7<+U$SKqek8 z0&MJR@C6h_nTZUI?Z_k4e@uXJXF>cR)Xu3O;fgr6UiKA2ro`+1`j|})r{_qqYJJ1_i~!Qm(h*Du zT&_G$LTJ^X2f~d9O~xZ3`AS@7AtQ(UGTfx)wZN}03>bOch8$NUj71@o{7ebYmY<%w zFxS^o|B@WVi8sjG>Tx`IVej|Brs{k+CwVy-iKsX<2Zkm?{D#+0Ub0b3 z=J7mn!c3)wsXGVHPWF`ai6=~%2{S)o)1GP&^l&uYNih}d`rmqE*$h~-ew*=bmNT$f zH^kX4FY4W6TbFxn`N%PwTj0|Eep|xD*V}SQO^(*Y?bdlC(RXs%hUe$3ugBN$s8OqN zfj*&7`opYf0I+8A)_2&+yMGUpaSzxDtc=_AhWF~i>lb{#ptfo~4`(u$vGAWYys$H? zrN8h*$)S8iUY(mi3dra^QY=wvDYLbPLgYpTa3JfFzGRrv>+%Pg2dk#>_&LAVHJp?H zI*a_3B|C|4gBV`fczXPeHoR%4CgFyMhw(Ib$)-uhc7(qcJQ%Ub2FECFPGb&kb$XBO{k?x<&ptUH{W!0O1zFwo#*9@m`hH~>m<3oj-8)*Wm>cTJCm2JUA!9sZ*cWMUPUDD z_Y$%>5m&R)*vJfD6}LJtiH8TH);p80hi7Ycab;6XJRzID%PeR!&$D7=iIW2O*oqzi zd;C?!KD%U>zb0Fd+1>-s=(SY#V~$&RzS1JIvIR^G1h(!Z9`$RZ20kFpv=Qo`6Jqpbi5q5cwc6=}w*9Gphhd^0U#EB*Y{vHH zO0A9Ox%_u}RzmV*e0$KRaczdqJ@^LXH~-W3+OZRO$ks^QgyCQO zu8Ql&Mi)LIhghnfT?b?Cv1k1=kNO#F-8&poEu zej_6b2$kHwJS+Qa%1)^0L#TTOj;xNzXwyfbh4^OwujEw9yU0e2KzAbUJk<(2h&l-m z>`09s!vSA|)#?4V`GNoaJXzUQ+l|2b5%|t{o75h`H4xWIO&M@;(|GO>$k^b95!(H@4tdq?*5te;-acmcD#C>_rwm)P7PaRPGM7^ z@(VY8>7RUcuG)HvN!9{qmh;8Rv6^FpWA234E9;kjxR1p7ji8JNuqf~lFdw&b?w{I4 zU<7xviFGf&zq zR?R6fW)1(RKlbZ(-^ZqG_^L~6V6-0-iZiyfcaP0a`_uD%|J#q-zR*?qj|s$wf8tL0 zm3dC$zOFv`aeMUV{suCyf=v7i`=vUbIB~*eX8hZHsdE(H;+ewkbltS^BEZ(9*41cMZI+^a zX*h~@rdPSp*13x%cPu?=7vO=_(|C<;!GCZE9YKnw%*3U$O-XyknecPWVd91jyLYrt z#O*tHSmUa?MsyVRX(k`3>+vh?O6W8q)gkis#_?q&PFJVJILrzf<`g+(&g(!D| zT7Va^AXAN}?Pu!0(}SOr!FWxRE8_YW25e97&9=k#+U&kBTK{!lmx}5*R5>-`<2$R$ z?cs#2@}Q&~w`-2T;5^$NV~~&~w2gVSFNzdPk}+`C=X7X2Sq$yOIhnP1VjmvNU0}=j zT!UUk;d;9nOb31T|uf?~fk>Mfx`S-osF5bNZ_f`XV zjoDx6Yq)D!q)SWMkRV+$lT#h_8ZMSRJs>&Ar+V z^x}cwK%b4`b?Ra4_&cEcK74By3^T`1*;7wGR!L0cx*J|4fwYe+=NLJeMwL8a0$%0e zRN2f^p=V^BGb{0PQU6kzwh=~#+macTWl7M58!!UhiMaDrDeNZdLpT8HKo!3vOj9^8 z>8T=C3a#J=6I1^td|>>-tG&mIA*d54d?^z9b?T(P#Mjb)w)E@y@M6nbPhYeAZ1tZl zeN$ia{Kb~nl)v8j+3G)A`q0naQF#mG?DfxOeZL78Q$f-OPWpN>?g8zl+Vi>}!vz>0 zT;+Ejil>_Exp;2M(Ls3Epbmp>j6R+}Z1Rpm=a~=Eq{$GP?4G-9rf1Tg zz2J5D`0kj#4ac^?RUKS~Sqq~yPNijwqcSQgCT^sJ;@uXNQYT53(#C)+AZ-u`1s?1y zqz8h2%(LJx%7{vwY*c}+546`;kvWAwNp^)9muM~^fZX}RHuGRiyitt{mB6+9U)cX8 zTi7p>mIGP2s?D{rDD6IBbC2F@b16FrUYP%Lka?eWhfRHYdVaS4%Y(4<4E@gv%z95q zJG81V-InZ^?(mt zGV|6*&2duVKu_2yHO1>11tB>uD8d@#U1=LVGL+}V?J64nH0y=TvMu==^e!A?vz!`A zyrFSD|GnOO_1!e$+(nWTKu3qKu#>Z!Z1V8KwsOVmt#@z?ZG&-uItoQfT;bq(R90U3 zXc!+nxq;7NBxF)X`A7MZm;ur`Wn@5T2YJv6*ASg`w}=n<8&=QrwN&j8iwAZ6SQXca zu~h@*qfN_i>SdBrhjFYe0zHZ)gNIHji9=kvN0@Dl4-ttkVlsZ}so$}Y*(Ysi`wcd{ z<0eQ<0cY-i%59pntZ#9z*oo;=_KqKTzdiEAQ%w@Rg8I`x_WgFlHCNd9*oZb|7{{kA z2Jlu4C*(L*@P5nNggez*|Fyj?ybcWOn<=56mB&?^JM^$NWH9U|y>D^5eR26Uw%9*u z6XO#aGbbiD>3eUiSJe|H@IU+DC+pu2g}57TeidxRILqG)ns_e04HtB(JK$;MOIiOi zf$e}uWkBE&D{ovyIcd9T!$+Vy5qF-di2a5Gj9)Aq(i9DfC|JR3NsCKZwXoo8iw354 z+vw%5_YO$r=!gWxDMhrTQim5LH6c>F0d<}e;FvshQC{5I<%_94G!0Bqp}r;zY(4r$ zdE>NAT)TY@1G@g#4Uy4n3s`G}*9}**23pzcrgR4NuM0Nl@@7P$B@dZ|NSZnUPN;9j z;WgD0cmKY=8F%u|U$qIWjO)Sc6rzRkd6TG&m-?XCQmJ3yvn8e(c~O^h9paPm;DnR> zs01hgK6bEz@b|mF_S3ewfRF6~7a8-F2>Gf7Axbho;5UBnPvF&qSoI|O^$-8ckA0tg z-J8GK>z0yXpUj8M&=QbmUI$cn#ibY5!jYk|k31-I*GLaM%_Ks8nh%)tJgd%ut;|i^ z$$S0~xA=Yb^wdAF(eWv~axjY5YDTIRk~uL;i8+B)|0sYoO#ZoEQnaV6E7{Oq4573g zyn-a4yvWm;qxZD5=#y22jCuaMXE z|4i%u{H%zJO6-C!lL&Au@$saFG`(#c@qvH?5XL$DEj!E&8&3CfA`W95R)!iT;<7h; zzpnc?#|d!a$UN`)B_yLtY611o(21VRV9Fn;7w^p1#8prop4@w|=m_)k!{x1(gBWSqD5%oDivv z9{HY^8c9{-taF|U5)C}WJ!kz>l7v%SWuS8FLzWRD=@f@JT~~gaKQOe-f5e-!u$U6o zWbnY?k|sBvoWI#_UcJNS4%};_mwsjC*TA7c z0#Z-H_F(e*h}CfMeYoQTmtg{FzkW9&Ic%eN z<6snU2osr{faAAZJ#?(!3}b8kXE`-H?y$C`Gq$?CXr~|hq|G0?-xje7?(=iE+5Z0P zZ49r&4x#?W$0s$RHi{ik^EFewV%xJ~_ul>KHlY_?a)oW$veoYo@$iGyzTxV>&FV0c z&0WSiD91kqwG&@AZ$m|(I}vw2t0M*{b|MZ!N*J;D#Az89h$T$KEzV=xb|%o&gIP_V zYza}~#4#9?iGylhNW80tz@!9C^y8Y2%8;CuGIW|!D=3>#6y!l)+eayBktC#|fNeKQ z`QuQ~<(B3rzLq_;wlElmb^U)C)jwuwCU3zvP+ygd!=ZCH5bw6rDn2bBGuc@lN z^T0p9RrDW9Y~$V;oW#pX$NXr;>N=U{ZJeSsBgO1Gi?Yx;Zo1|&@cV?Adbf0-EOe2{ zD~U4nJ%;X3g|t0t_!na7EmZ zM!O`;e(KQ26MyxO6dFe%mGJo_+Cvk$b1LoOs+TRBu-f=|>-VC#bLv;!hLt&RihiXZ z&GAkJG$ja{<6qO>zOf5zVD1U)TRH&`u!}1AB4mQ;mB-S@Gxsy%n~Ic~R=N7Z49ND_ z6V|D~lQ8gLy?10hfM z-O?Y{(N|9DF=#B0XBdQTqq5KVQ2AVNWL}X~psv@AL^g?G3n${}_vxWaZ4CZ#1s6Ye zQa$jLQz1q?!Rzd@JzR2;b#bLCMR;8o{xT+E2Q?^s$V*FeQYC&4DW97p^`72m+xP#! z=#T^WF5FuI{l4w__%EZWJQC}Te+zh!`$IqRA^XG~?Gsldeb+m`&EE9Yuh#3RV|d6p zIx?&Yxk0Rk)9bykR`(Y$N^OC@HtQdbW;>vB!qfRQr~$ca^z;KCvz6Ir?a0z*`}ERl zZN7h#zV9~52{=rqPT=7yR}u5IR6^RbxQOr3V&&s~_k5-y(d_ygU#V{b_DB8uy>%Lr z5NsQu{?i0HA)IjwkJ{Q-)sljs3pQi~&i+@|hIHf0*r@E|3F!(u7J8I+_M3~ZZK0=W zFhj9A+jF{{-LfD~CiPtAy8sRFl%4t;VJIk1+4t&bkH1L;>-6a}s* z%-2GKIaB{q!4oUH>D5jrcU{Gvk&}VO4t#e&WSKoy1?t`sd2H$y?rz z$+$o84IJ>zxJm4$%GXr+#FMF-hzYaLIbfjF#~5HvJQZyYIqU+vja1^77DG8+noMYmRdZ5Tvk+vvTWjdYoQ&W_?R>*x`DRqFku&cHNH(=C-fAW($ zWha%@{-t@I%OgwUNP6+%oMz&j@|DL^qWGJ6{BLaW*(dDSz{NJd^;I^38^no;3EYtR z_u;srf*Uoo)IVfN4^==0ldFRcDGAjq4+7p$${gNUDv_3jc@A96&X9h4{ZI~XO7TNE z1bR7Hf)O+wFe&VHQ(d3v$gZ-a@A{wB(FiZ<&g;Kw zAQ87N=tkf`qb>eW#_?7;PiP&&Y%v8|9N9a2`8RpVdS zr*RE5{i7Gy;?XCpe`cR8;o-_MCWCAhuf4&CwiR3ooGaonLqk=oBSyBQ8ySE-@=~C=af$!1-_oV^i`0&V8}e4Dqe-YhPiG!O%ibj`&m}_ z=L{f59`8H)Fv&8oGN2FNFD_2l-nnb+@`d|s;mG5ypU$CtUJqpe51C3l^8!k7Baf%F zr!u&}OXEoA$^y2ZR`pQnL3>}C$A%LJ?YM2Tp8hdS;`r)Qtcv1_RLYbOl#i5nxRJ4{ zM8c+XQU6@I$G*h-m&MtWw)2U9ZM`e=Hh$e(v8ymX(wv4=F=@B8lm9gzT!T-`SOh{{ zgg^D6U$fu;=zo*j&BJ$o>(|>if8Cp~F~gWvz)g(vTWmu%gxBZ!S}Oaz?o%^fm4FCb zbzUIpLlw+qu6cYzhEL-9yJQb9UTY7oywXM{rfdQek>h}4n26&&{t&(m*N0d7hww^j z{v26cTD94k)Aq!pt)FzD1=rv7YQ2uk9jaOXs^?ODw!bnDd~GJ5qx#PdkbV7(`+ps} zBG6rQqebBP{_@&retg;4!~UT@!V4U&kBbd&@b$-4aae*}-I$O-y}p3SRKk*`#Dj)| zEtL_Lcx8Zo+|#J|BVXo`kL&@3qby;rSg9aM#T$?~ZA#d|zRC{jRCcpOP~#(E+VQ^^xtyA=^%5 zyB(+$Z%=uOo5iqDw|HJ@wGftN69bw*tVrP5lExAz1W5Vy4m#|Z&`XNfCyuwvu1P5DF|$ZEpGl^OY&CyhlORA{VmSmMbo8+ej-!XW*F zJ)Df=7`h7+am}=M|1;tdTa1CEA#XzAAV}DvOH??7eS*OIFcHTzy0A~Sr_DK*=e2+| znMb@KhFMjfP8x?YQo?aa7%oIcFs{Sk#qPau`As9%_#+X^2=#|U1&J^}pisxpwf|~^ zS`8WBE5tX{_&u$s`)`y<^LRbA3M_+O23{1M4s?_$f>HS8`Eiv01)haTPbeI65>lqb z2PDq2grdXZQhpx}6Hm|f?$(3QY}-`HO-UGg;$8FFJ`of`r6bmQZw&M=~Ul1G3xxU%&C6?AQO}A1c3j z`1&`$*53K{x9ZhTZqC5BC>(sn}qAv+-MSg<4N zU)kwL|C`M(_u602yw)DH+wqFDZ^ke>HjWi>m=wg0r^B)9=>R4L`R!=dDb3)UX7}N> zR6dAprWBmJBU% z5GTq4k}UA``aej!X#+=~yCUv9RgU^$bU2Mw*|(6#AE3O^&{Gg>wd@I<8J&*#36)R2 z?6}NO$3#YURthIBB>+J$NP6l^8OfhKJgg&P&uUVGM#FY3795v$Pjpvq%CP5 z34{lM=!1`}C`+GlTH&&T8EHEqQ_4#D=}Xstg(vI!6cSd^2*Yjo%kYNNk!qlO>eTXsjb?afv=;Ejg4s{jvwJ=dGkh0QgAcwHgJN9 z3AqucIMaEUpvf-LYeLKIqiop;2H}uFD1O-AQVunA32Gx1@hGzb&w9%!c9vhSu!H16 zF7?E9BYqsG?X|sK{}q#}Le8%Kv&y=#8-Y#{;4qNGNkY*zx;@92%wl_FcO~Tax%#kD zgOeYpdoQr@rN^<6*R1spjMVC4tnmy=VM^G|G0mXjuUaG^soW;AUOVTfhwrSQ*9~E%Yd?&I!T+l z6Ayl6uU*{6Ue{J=T}NKyveM2@6T|FI<}$BL>1#rkmH3P0KkJ-{-zVb(twG#F@%7Zj z(cO0V#16X<55bn_PGEHwc5Tk8P^@WCi$#)CABwWI=t!b6Ad(q81X;fnC>)t<=?}Il zK%iSZ_B1)|@i9!o_3M>XeiyEvuh8R9sj0sy%!%q;(laIWGmLX{ zHvQ1YYzeOrA6uTXzg>Kjt&Ch?o3J6n09L$BU?m(k*WhFv&pGkNaZ=+ZY~g|yI?YMU zFMjT>kXnOn+jrUaT^IRk$X;ybg|Ur6p&949{uM&hPa`Ci@mZA#Sy19tTc|j?$VQAn zcOve5)=D*S7Co)XgJHz0gdd*zk-!0hzuuxyJ+$Djx?y%lmX-8@xBTFEu5ucXvWTay z(62J$L6fr!Yt8iJh?0;ZlUI1F{Pfi!3A+AgOcbte{i{vXSE_8Q4z-)=&EJaGRR6Or z9Dc}7;x*NAY~av4if!s~!RE#KB}r1_lVa*aI9FDvr?r9BSe5A|IrYS!Wl5ao`y`Kl z6OKPL&`T<}a`J{TQnw5yfxy{}0p~E9dcZy&2`LpW6HMSE6383bkfb| zO>(-S6>u~$Cc5O4enq>&raYHj|L}FHT7Sl>Hi>qUDrf-M9_mxal}9QNQ%Cxgz7R@E z2+r~{k9gBw4?J}{KgD$YXFT}VP2C8r69IM`{;}V&^F;rv@a&fq!(u`v@nOWr56T|? z$i+2lrf&;6-{V*ncg%XHiitSqcwWaG2NF_9iKkFEhhsce#O-L`uwn0k!{pTX(TfS2 z!jlG`5(82&ohD>TJOZIzIk7Y6*y4W3=9?l|`HO!FCCDRuO*Vw3HU<4SbJOn$gnj1XWeCev$0}HZ9>5{OSAu1ow6)?5V}w_PM36vi@OwY6PDg8OH0U z+=yWa_y8v2IFZVGJ&sM)WFW+HoZvr>8~1+kQ-yJkUVHs3FyZE7E8DkvFDr7R)IZJ4 zX67bWN42-2n4!(Qil~ch_y}|-;?83g`8AmXJIA2tX?{{!#Zym2@{c)Q;N%!)f{t=H zs8;fVF6rB0EsI!td3NituX^L2lXpzauP5?&!EhVAoNl>2A`W^QoANT>;rjgHXxD!Q ztXu!=JI~$zH5qpcC*!a(?(hS4^3Gqe-q*iZ-;84r`QzrCCpBAd^SsdWR9VrS>wcmR zqZ=CC@I^l%S@KDWXH?ylH$bA#{;zZQs_JP{!g%?Z6RJhzMP-ijp3$*kOO8Lh`N@7x zJoQpnHDE~rJPVuXuD5Br=KQ)bA}&7K?=&nW)$G%swFUeLIN7(&PF?VoHZ?hEn>KCI zCJjSFL%3PPD+-cdj*DWJuAD>*byb1n$H3}3{fD0j!W{9?iZsuSa#NFr49dqL%7tW} z_%l9EAbT~@N2fs@cs0?Nfs&>O4OpF8EMTgXfgj^BV-PF&f&+xXK|t~p0A6u-{Z}y8 zuKzm5BJ3Wz5$F&Bb|e1fI7e8e60b57@v)H|Q?E|p2j^V>Hf`9jjE|KMY})07F1)4B zr-U_!QxTVoGx zU|O~Qu?ubJ*dDWmlaN1wNfZ3I4w`)KTV4ajR{7v^=tqXirMhpGS|TKkjAnhQ7(&*f zZr-^{LJ07=h^y7JPC>}P)b`|*~+n2losjNgVE#=R9cXy7CqUr*KR%dC{F zD+LQu2s@=_{VC8OjCR0JjhuYozu4;Xg55WNqdl^Ei;Yf9V&g4L$YV8}?}*9?L2k%^ z*J4qB>{n&q;^ZRldAKTWes<0tdf11&pdEo_JY={N?#ub8|1{EdS0Tyl z8p1?{izXvA`+tPe3%kb+8-eab+ zuF(U@Ej8TsJUH=uL*4oLoP}C0tm_$Sw`#OA4JLDq~Q-h&m_Hvd|h@kL|QUAD0 zunDY;o4(gq#+|^*xCyL`8^lFCpQ4o)aGL(&C&q}Y))*gOy{2n^sVHs&qWO9pBbdpj z3gi2zOf17+T{}2|3|^44Pl=8P3dN1Io=`j0@d&hg!uZt6y0y$G=X@{e6AVMiVjfFk;y`Y(9+ zu*m=VY+(>9(PlCB%^t7lBvm|=&cu!^O*+#!ld@Ip?Kkx}(fUW)lXhIR0~3Oc#4+sY z{ReFOR(wDnHt0IUak8={>`T+)v}tae{gA-dQx`E2H!{1&R;PAiyhYz@x=B5BP5QF9 z8s#c?BfL$kTpai*RwLOK2Prvr=Y~BFVF%6eXCN*Pa>$$dT}g4wItl{DJ0HT3 zie`vES=#KQ!gJ_<$mR;7eoR_%B~srYzR9|9g&Y`U*Cko+p_kl+1J?N!E0?4lfPqQn1I8#;70I9%Md1l+5U0@ zPCK<`<8qaVlYbMMW5omaf8OTjFmN`~^*7$8jUC9(4IWzLr11eW)(Ov5{jz}X6Ox*k}HhAH+wt|&$(|7)=EzZyChTd-kv(=mn z$oB0&4|r7Dj&4QWDpkb*V?Xpxp?sdIJTnQ&!%U_i&_$b)aUSem?nG(;%y<3HM!}UW z*;xv%Ge+IO(3B^XOx$6Mc-ib_&ct@Em};?$Q=x|4S;d0qd#&YoxeE7;vbHv(%!fW0w>unOiF zolvYfND{_)NXX5HxFU`ZjphfoNd{ln;z;R#rF8H6u2E7sAUHx;;!#}Fk`8~eBcO0( z2jnph<$s~f5c$S}Q|7i>jj%J2{vqF*a%^Ke8fiZ zS}OZKzYoV1aq_uj1fF?>C7xvIH=%G=4s1HT=Pqoz^$DBD?!2Fzd$S$tzZ&;-*o*;_ za1)c8Y!o}Aj^KN6+?atYGy5NR-Q1p>#|2?C*9t}5h&<2|4M+f-e_sg*ApxBvh^ z07*naR83oK*Tt84d*&o8Z733DF-c|zNP3o4K7w5`C{=G9TMhbQsjvDGbK-qDK6RH!Du1fg> z+hNj_kdk;#D5;VS8KEy_CQp+-d9$37!~^J-(U{sMFDvX_hrZ@@c|LhYEAWF;lZ>Q@ zMYq1|lQ|iO z-Bh_UZf@q3EiNurlPJ;PUSyDwWppE^5j)+ic83>0F9#F0uq;H2}R;{ zM%~h1K=D%`6i%oldSF4{ z_1|Nmy3e>LRMA@XuV{7;-3UB)1hQX}iv5xRe;O+S5@%UL_02597{--oT(#4K4I1Wc z6f5F55yyL_ykAO)EJ;sDO^FBE21**>w3jl%ZjmbwHp9R^u7ttgu5G@XX({fheFt>P zS6J(mc*w0HkmsaKni7w4Opt5C227TW&VC6iQI|BH>ET1zC0z<0FhcxEVnE`|PDq^2 zBrI{}DZhXu&pOGtNIvuv3NIkZ*Q2M77JDE|dg)X8l@PS0O(jlMN@XT+*%UDKBoygc z`cEESOVt)RF^q{2u1uO8xzgs)CKrx82Ayb!B&6L5fy<46!cmqmoOT}?N47WqktQK= zU55iY&WRZ6bV8TSyjoAGj}vi=HfXa0cr_K1ih4bfE21zVQ||l;X?2nSoQ6=wk~pF8 z0ma!NW&sWlQ2~r@!#e-(^?ewbb#^5p2vbj>)(Y8^r|J5Fc_fR`}yX0)+(* zBy?xsnVbTo?{qX^(j_F_>hdD)>Hf^-_I}n*u1wfpE_|&mj9-a+x=l8<`2ypl**GTQ zI4L)bHaCb#;C|LytQMB>)W<|fnL0s>L|pH?=QBDh57*yxTfM4|zOWGOC)h=*ykwHW zI3dKJQDacn7T2zasOr>=yqE(Rak z#hW)4#!XzCelTPn$rC4JYM9^kzlezf)ZvhIp`u0rU+nm%u7u~M{&C|lanm>3^nL#y zz8UwRox1D8Hu2gY)x=GQ@Vt0g?7WfEf1Y~3#xoH$VwA+VfDAmL^)Z5U&W)qwfR3+w zGm|W$^BV)+Mo1>0^N^b<=>4QRdL6r&dW*>L`f+Apn+v)>bu%{T6x?OTI_(?l|@B!Vho$lRi&rE!UZNZHpSHN)sZh$vu zyqU%gS~-4FTc@j2yk5RF(StbFV+`3 z&;{em9ZrLy97ot)g{zxF27h#7_teuiWyA9a@#7U^0DeIF2$N%Fs`kE}XC=(BF2@VE ztr)*tF~qtI9(`oQYqGe0@{eiP_6u##-h;yP@bvy?FiD0VzkF?*jA90NLJ~IQCvOEg zjyC9f`Y^7KV#nCi2QW!7Z>yLDO8XKv{g4Q%3&9!_A-ftI6Q{nCF6haab;K-}U)PVw zf|qzxpXh^6f*{&i15yXTJB_PxJ}kl}vyhPPpd$ym65Lny%+`~MUVRu9_2r3>xTOK-5Ap-CI#?!4M~YXo-id%xHfm8%y~Uu=U_ z$~uQm*_8FjeEcop72k(DeIM?|S626l>^nG-NVbwhUh4X%ZrK7z^ULu<0TY3n7X(fi ze5LP#rRLEE%HdDc+}##Vcq_DD|$TT(vDOS~zs>pue2t^c<6&kqiBW!%Yo zIT`nWow(zd&0hcGc)~lZ3t!7cUXAdD9?*!{P~!&-rE&5@DLYTX%tv9iZU4dK9My%k zq6Z!2;apt>keKpZL+GZ?`st-E_F@Pk@co}Z{lgTL8Dv1J?UZEjR5z_M5ap>1cP*rL zvDME!cRj?&e%$alT`)q9o~(wtx{TU!CNig8o;mJ2Y4!Bmp$os(CdN10=FOY&(cWRb znp&=gqqy}F#|S>f_o(^Aiu>t#;-`H?qCyfEjQq?KedEALxP1EHiE*fsP#IEXwIW#- zJPrD${M1F*DS2-A1cI*rFR=bQ1-YBojX*?z-IaeFS`)G;aYCjhBt1X-$OkR>=_vtR zi8kAdZ?LUkLJAWKeIrxV*i!X(c5n%oSl|>w<*tFnBycFyaW8H&RNt2bKU6y%;1C{o zT(k?{tNWaURfoL?p0P!&Sfiajsjy;w*lAPdqE_yxKa`mnu!a)-VPIym#B zEe>99Lwc1QlR_*SHzJK zW-8Q4FMggNsXQRhrK^yaaucpcpSD;2fKhps0O%kd_P5c?mbh;SvVwd~|1}}3$q-D^ zV1*Q~Wk>q2wkua2vW25h=sR&}qM2>N7+ z`$>5ruvr<++N2yAv4U?6J|z_=1}@X46MV&%t4;Ia9aqFLy_EIO7;uvlu412^owl8i z{hIYH&)C>iUxUwXY?c3cUq{={*8f$kf}{V-n3S5Cowxr1--P?({q?KFl*vTj-~Yqk zZMWZg16G|4*rv$|On?nwMcjxURu2wv5)2~^?qxkpS)r(BW~jk=y|ZM^`VaD@)3Gxu z#8TUq2nitEjA>0jz}6YDL~ZLYK?;SkQK=W`2Gvmn#eT zw#&m0-fgE(og_(1y6KkNHMysib$rE@_J>>&zLfPZePNyVmSajh;vXH(Z?(c-2wsYfbGL%AfN>S5pcL(ZZX9 zeRCy9gB6BA7k|si|Ho=z!wgWg9X5Ek-$@O9>O<#p20$ou=^ zKkH7ZbSUwJBq;I5X|w*>jxw)BcFJIVtyBMHeHHgc64xS=Cudpz>&ilmY^)CpTDo!j%3Lu+lx!6J{O>%v(ktZmu zR`X@UK;9~D&^c*HeiHfX5A!@cuiAvFsm}8?y zjd{v#fy_x;R8G5TKC_X^nDD&WPgu_Hz*Vs#QIS%23F|tiX0BeVCk7KIWNMZ(PRw=a zc0QNAl##kiyw2U>a zzJrHdJ_;z`br2#W1JJNYG?1V6w2hzOBkRD#E}K7auZ_(@_@(y~wH2fxQ5Td;ed#;TD>|M;$~>MXo{-fpJnbS;fxoE! z!`5DYAB>YC_&d?J%a&FKZQUF?NUtTAq@zaZDlR~tfvYV_0QKQS@--d z+y#4o2j4<`%7(VzXu~^ikjAz}1+=fQWj+5D4gC08hMsqQ;6wH|fA@ugyGGyhPv2p$ zz5Q0Lfb*4b{hE;DN;tj-%2+5KZ3%^SA$()ANx9o6(7!g2;x}B zR6$bUHyG)d<`?iW&ZEnA>@@Ht=6OfwK*i)T9rD~M9P+#^V;u&Lkum-{I|OE)GS=Z2 zO>szGLZ+0R{(5e4Tu*+Zym8v*Zd=AW4C~sv4u9c^&~@b%45Z}~>#@Ne+qtRNcHvL2 zkRZ=*cR<__*ltdxq|71JKN2#(6JF;j zqNEBDO^s=Zm*+95CBNeJI5+cZ*-@g34@m;h5dvG{4DzNmp40W;Q{dIO8h9m_V_#9u zv6#b)V91kQ)Q>-(jO(?Fcen4Hy6@l-eFVO*hpT%+h?X7ar~-i$OxsccP|u?>qAU-{ z4BCb~u83nAn|<7lVZwrw7ri|`fx&Y55Ri0b9?vzfm9T+JFUhDsQi;n<s(Vx`a*ob^R%15nY$QGPGdKJY=OE2}xDr=|3{n4*6PYKPE{AvGQnOXvB^# z?XvCI)pmLQ6n0Y`b9+)1>zgSdIgu$lfv3G~v0xj=KlXtI?3g-l zBepcM%SP}ZPapg*?)Op#^^|z}AD+A@^-nwVZwcRp8#?q!n>v1{^^I?{u`AzL`Oa#h zt5tP{-$}5R|N4$1c=^zXfB*9@{Hp!YpMMfL?dhFw|Ht;Wx4aH-7Yy0t1a>FIE2-o7 zF5CbfaB>AK>s^~#&j3J9h{>qdxjq#t(yhYVI|5WaxUEzy#d* z#FTH&fN#NRLXO{~Lp!5?c|Ts(Ns1>-8*);UkguiQ-C?7KYi@W&O!DCYE>}b1o|X^c znM(O3elF_&IrTAq@SA=mUHYhu#v#j+zERvg-JlWZPQ;z(ifMLi_BZAUdf#&kk)aQ8 zvr}Sl;ee<_LgFmvA2R_@%q-g{9$vQ3KaGKizn{A4Wfy_XBW7Q7ZJ)jB(pYkh2yjr> zg@x?~DVtbA&nwQ@w(@40h1vE9nb-_fuCuuDhFs<`fxo4_Hu*^twh)y(vS#xR*=x!p zZ?u2$lTq?gKnP164d|sk_y)%};&sPA*2*|;;4t;NAJc29oZMk5FUGFtY>(80zg+8{ z4pK~IDlWR381sv-F2K%3Do^2#P70hN{36VWH54EtZRV)Jo6j=wT%!eLFu4FBE{E+z z7Pqz$=`3Z1Z1GEy>o4@hsN^bIY^jyYSDH9=7)ko%MSsy+=yflmN;c=y!tV>A0-LH1r!^e*09!64x3xYh)@YInb z#y;dZT)F0T;{7Udw~G>zHu_zg#8!>1XVk*_uKjDub3Ju=*Z(uC|7HG=MC!-W^|Ea# zvwP@9pb>#)zibW^IsOqcx503WJ*Rv;GshJ+#raH|bp%%Sx+iyj?$X zQV6+#pQ-{MUCHr4_3H zF(LJT{OkY6KJwdt+{zed-}tp}u6;6xZ^!Z&_DXLGAVcGtoiZF*prjgDXw27O*+!iI5AS53k}C*O4N zgqnhwRGD~XQ^bnp`s4V3zX$GZygv_N*Wvqc@+l_YC|~U-h2xxP>hSV9kLOGo4&v+4 zXE+Fz8X2i8{h>Wf30?l#^#4kePzj1FNzVCuFZC&i2sPqFRWJ+P#*Bj-H z(>C&EhTMXz^t&m3UHagqA4&}a8mB>2gTEL4!=3qs1v_?f+72B(Vt??(Wqat*ioN4D zEHCf(W#v`<)vniA$Dp$~gW80RsPdF4;X3Wri5th{E#+k1dh%3>V9Z89zRYW=DNlJb zHgVH8;WbrE#&KmFCgZq)!;ANtDhyG>c1Oyay}WwsVhdXZL6$UGC9*gBdD1BpKT|+N zzMMc|1th=U#D=#eKNqTHeE5S$PdJ&yOuyK>Oy^}TeV`8MBR%POOd|=!&lbZQ-}p8V zvvC&cqfVTkC2p(JI%VX6pS$nzW4_b)o*%_OJ@G~x8k@onUSs;>%^N4;a)So)ub1f8 zz}5?G@$fTv6@;rQXsi21UgG%qD9CgA!d!>UNnBA;DRbq%7b`{}Q=)&09ib8nT&ACU zv{C{!yVelkMAgx$(+0OU^O7hv@x1Z7wpJZP$xF!8_5aN3e_4U;vB3+sp?bZp5-O10 zLpK6N1hO|L%<-XupsmKpl8>4A^GUECTkOZ{sJOpd!O9~hjh_&raR%oa%P`guDiN@) z#%T;kK1Puq-JU&P<2V_pPZ`O8EKI!dp&MoTPK7;o$ zFO5_Rv?pv44DQh6XFMtFE$IiQc40E*s10-F)#MgUrc{1M@B&Cy3*OX~g$YsK(tp#a zgyb)9#T~q*{>%$iE%lkK2`gDu2K6UrJIu0#r09fq$`fsftW+dnLdmO2YhR13v{`Wi zUq2>l_-!-3T`)axF=l-5e|o>!wyPtSeogVXuo?I{Rr2i$LRatkVPXYHqqi{7k`l5o z9_?X$8KEmaM=%8*R%mdf-jCJpuaqb8d0up0K$rcr%fzSlN}Z^Lm@hJjcK#|i&hrkh@=z-cEu^Vlw*eBxm{e(Xq_&o|!uYSljrc>?3!Y%`KW;b?GH`@s1tUAVxa&Qy&(I z0Ar6OMB-qV1B)`w^3ar@x?*fNOKI)o{E^6;KtfNg8d}df@xOfh-*MrEcE!aP*#nO~ zZucF)M@m1pVDETEFD`Hcs%O@gi4KC)DKPM34%cM0ZA`%@Me{GMic~HwzKC& z55Z|8`4g|~jXahyr5g!TceHUQDvpIbK7E|Hkt^f=q}srt=MC@0w)VagXBD%8{j5;g zLJq&NQ0NK)b>A!FSre$;D;#lSQV9S(S<+YdlzoNVH4cX0XB?$Z;+>!4fO;5Cy+X6E zDTAz30#@9xC~i2<8(oq(LG)1)J0CG7x1a5f?52k*B9ST+nDT)B}p^BZ;aN09abuYRra zyrT70BFZWc&3+?cB%mOs29(qBkVVCf2QK0%53HNQ-oQPtqzk;GJyoTFs3EWGzp7}x z`d?spaRWbk$)#$45$!JMMxZ4E9K-mhZXS>XIp%BZ#(9p5gkE0ru#ZXr#^0rZDfE9# z7IQ@$vw1*D=;Jiq7>YJyYSsc&ApMrc{dq|u9 z(@$PUm+{BRxRH@DtZ=&2mS_7h5qGGh3>9fOVW*UmJC%9euw<>eRQrKV*``3~#u653 z5wE2(o#?;RCUM`O*AinA?o8rUTA6j7kaA1BS%29U2=nzxZl18XFlQG$@#{9QbPV?; zUu}b%E(t%W&FxRW6H-swOTDH4qJeULmv&)c$^PUo{>I+-zkZlxZRwRaU27kB&-d8a z=&(&qj0=v9j@U3?*+n03z7;~-nNoLc5A(deJ4Hsy;mX2ko5uIxmQNkBgUj3Ov#W2$ z>$j6OF|`@1@P=&+xOPV6_uHu;+!el7ngndFX>!>jG^ePcV@Kli>f%JU8%2}jKc;F zC-3}Ko4^JRxir4AqY!aMsKUjATjzH2rqOMr3@7n&p2>C=RLS$I0P0;I6ebJNj0N+6 zrydu9Oyx1xX7YARqXNy z3#*ANn$483iOZXMBFqWXc!e1*MQkVra`<#!Zc>&6*YvIjM4?Y561ib+6awBeqc{5mUK;dyzQw#a7{o;&|59k767pTGCF(( zQ|56ZD%(W1iy&(-Cjz?Lg)G z4tY$!6Y5?DEO&hB3-+$}{|nVeP3_ndEhnfFwsAdyPs^ zf1UEQg^-EYQ{`m_@=8-w#yk=u(aX$JM&;|v$A8w{wV0Is^gZ|5Uq8BRH|<>1#TLIG z{Z*4jkeXHORm7VokDy!$A~%k!;AM>CkZAO{ouqC z{#QlCG8cu+K)&AuIgk6EN`h8)gnnv@igm52pSr*o zES>aAj0mVZb+{S%i^|zq=3@+H%1`Q~0HxYLsf_HDxIBH(PCxPqTkILO!xz29V@4CbhU6SoFzadrULvxgCT!tmdFM4$1 zsvV2Y$7+479UCO*M?0=ryS8t&`ycs|8yLr@_dO%K_>C$}bM(0`m*7M#e)E;ru#u&S zr~bTd$i7~zVw%`)6VrQbAHVmtx>*|rkdLpKr7t{B{}R#->J7I8t8;XXLTTE`F){N% z>^S3se@qO-sb`jwasAvQC z`f?jGkGV01VUkxd!0+Pw@BK_|OkmgDh-)_9bjekB>52cwtozTos{gc}=9DcreiAKXOKRNk zJaEvR*|lxEE)2S8VL+=cUTrg3D64)@L4I6BYI+(2#i5G%9Bi{Z2geU3Sx3nq zmQV{R1^pO&$VoXYuf`MQ*Y>O|&)Z3U7Y^S{`O?@MZD54o5b@P;`AxX9uLmmOB0S$A zYv|V7Z1FL?;)L%i_Tyv8{uNCS(7$5htHPOAB?U@nUQeu_j7dK2FX*d4gC+5R&hI=F zsF)V9k&@(b0*0=ZBZBgfwj~r(L6G?giz1?3|DUt|=g<$Au`;Re(#vgd$1Xj^&$@3! z@|>LR-o3j`Zl}C9qHe}{jX?8$FFG-7@^M35x;jG>M84AGYq%jda20Df{62-ep@an8E~@zgdtEk^1_4d@1FTy%}%Ms|7ddSu%I< zew%sxudy>~uiZ6+olys_#zfz6wGl%<=&HgL$@R0oGgb@Ao}{Ol4h!R`-d!n7i;vRy$a45G1xpR(L()IY%N zuQ~$4jFx^8I;hJdK!1tzm@>nT$fYiZh*3&DVyO$LQYDdgBS-Y)%@gLR4w#iXTrWwK z$P}6cLGSi^v5Q)@QxE-xKFWP~f z(n>gj3F?(MevQq1{LgF#=VRE3Pqiu=!%WJfPaNmMw`RjZD}yU)ZLU4>P7R9`%%`j< zPqKtb;=)KDSV9q0rM0Xz>?um1;XhnyHL zCx(_gcf*q?}RKbagG@p$M6c*Vjm_su+U{0 z-(Kn&z!vERAvZVBXrzk_i#S5%cueiUG1w3)5s*L409Xa@0_~# z05-wFDjbbxkjd3KyyvLLw^Zf|s4K@H+M|0y@a4TB?aDls_v7_ceE8NzW*)ZFQ@7d@ zCN_Fk;0+C8j7StIT+kUl;DR#Wh=m!6Cr8pIBuytg^UAoKnLO5kQwlWjT5esMdCW}za_lByY2CRiOm@n zu@dgB);EF;Sc=rNpMGUtlBKUaCr-JfIq=L;`=;-Gza2h$EJ-WKjTnCM{qM4ic5TC^ z45KzOgh@br8*XT50ITBgpfg@iEoGBAbrUi*?MeE~pV@<*QTKhpcSc=&lbsygfqS=2 zcqMg2tKf!5u=;KYyQAWjRQ6k~gbTTpU&okdo|<_)&o(a#*v35ebH<|6_u=^Z>y_8u zggH;Vip%4jmxpy7wYGH!gDoze$VA5bYh2h1EmbAYWNEa44G z+LL&B)50z9b5KZ}kg4e#`EUu6OqTPh5OotO0k2k`AR{&B&_$ZGjkx$JGx>p2s}D`} z0F{H%%h`W^ke2D>>=Gt!@ib01AH0X+_UPUg2t7uKcy+O59?Mypbbj;1bJZ~li&mgq zk3Q+WJ{*#xmPI^ki;$_=#?V|q=MKfIaFXhKf$%P8T*|5wIj93)Ww0;@8`?6F2RpW-G- z){EDhKTTw1m3=1rD=SjJ%YsQD@T_BLa&>MBk6|lm$S--Mk)9w+TR}jPq;00uA=^W5 z3Rio}Jn+vSe8A=p+-;}(F0j+vUvHDRX`7h9w)XfI7boBtd&d2YVjqUEUirt#ILaW- ziM`&dueG@&hi&BxpS4p1!#27!Z+*ClE89WkQ0Z7Fyi*?aXMVym4>( z#vezrG?#g0YR0L0YDGJc`X{z&716}(BX$T6qF1q58CMGOIzs!_O6h+>(v^6VuW1Kq zGq2<=Wt8~Y%4^cEcb=Xmbeq=ZO4|}TeaYR*o4yhzZRX`U+hrfd#Xh_?&HG7yd(9`} zPN_apSMxzQ>xy}TYoe<5z$AVB0f;=xzT4CD0vb<=g9_7xcPhe-%NxN(LYpic# zmrdc#9B$0OiMLUH76j)5{e#xSMFHr$Yt=#1hw?mIZbB{;VPcHAQNz7o_`8x^_?6e( zs8?e9aUIWp_T zOesEI7~C*fF1unV;DF#yAYEeOgqf08=OM?H$TiOnV#*_D4m2*Mv@qo+tYuJya$lzY zXGE6p8Z+(%aD(i1>G99VGD?V3)`L!>EN*}q?YtPV*h`t`$;%n*)X4lA=LL`QtBWpm z@d6k+lP_UPby&)(Wwz4Rl#Ye{VIgTt`)k_@c@6)yo9gx7fE}N(G7hh)p1SM9Hu2gY z)f2qRkEYt@xx{;vTz_Z|wH|_q3**BZPTAo+BuhT!ct&M@MW68%e;ol~Qo?XkcEws0 zd2CdO8zZIABLKy+Dfl5pHA}m+jEqD`S-)ATJjOlw=s+1`dKP6AwS@2foOlp&M_p)3Cqy?$6ojeoXY?W^xd( zszwx&y@GCU)E9CqpIiK@$lDN@pb!u;!35alkmt{PRQ{CB`C`As5zd| zM&(HcX??8WHGtomY9^NNJ$M-3D&B&zjaK2GRJ(r3Ltf=ObyKz%Ym6q&afT`N5i)Vo zZGfxdj_tvQ70*D&CedVh!sa+7=I1z2IyHtFvdQz@4zgA@^W-d3@+KsEiLWP)aU?xq z@@5{-+v61VmK7_mwGS`Ee~y2eAnx~-UEJl|V;RxJiR_M0<_briI$b=FZ>xf^ia5b#NAwa2XG^Bk63!D8{8_Ow9A|zI!@+AMS=%%EwOE z>vLWI+1C?#+#-NHStp&T|2h7xBd%M#K_bwdh}$3)v9=jL7_zIjbyo~JdO8BQyo7IY z#7)XFf7LNHM-|VlkIlQUy2LTcOB&|TfRss8#jt^SNVzLuwfZS(qdZTCwhY=g--1;I zOHttLI^x=3i^!iov22e#y=YIIEND9F|Nq&04`9oV^E|L$&hJgY1f(GdA_oE>ArK-B z0-^#bio5noROuCyz!*0Su9r7e+`6_%Nl2u1)* zAOHdwWXu4AnE^2I<+$JX{r#QO_uTvD%^=MKaQnS;{yyD*?mm6`_UY3ddW3xIlo?uJ z!J-gV+qPaOa=wdoAO$e#l?%TpAUZq!etKJt_-BLK4^m8kUP7~Z#MM2b~>Ru z3Vx6@@dKCV2!UUBxVgx@(()`u$JgW6q&MI7U{jxs^l$Q6kDhO0$FKZ83=n-JK2ya8 z4n6PuK@{McylC#$6kC?sIR`ryC&PA>yy61Ir*+gdvJ(yt0JRBKa zzJ(%c3+&TC#FUbWmHw4pt>3#9M9Z z7$IQA0;9wy@B0fa?7MI5@1&7!7wD5z&N>>yr+^$dpc$2e!%k>Ao1 zW^Y_|OIW5i3V8#hLsd%f}zq*`yaNo%Ctf<>sLy9QlFR zjA*v3#A9(*_&^LEf#taKjZ4@mSU2k%(x`h7H0f=tA$gYZCO`Ya zIM%~3^eybkww*2eY4s{9E7yTr#!L?mz{y6e2iJSkx2K66J2ex|vou+U*ba2++>5Y# z>U!WbJkY#Pn)eZB9PPGfKTSng#Ts$C&v7&Pe)oM_Lt+*;ziwzI)oHxjTSJ{! z(=rV|)^T6x$2)$|pSyFL7I$wh2>-|uYR_Q5H34yK>R?4WQxA_6s?mGn8 zS{m4LL0ZB<-01WJX$8AYa^ZChFDSci$JnM+LTdr*#EfZ3v@+f#YxWoTymML;_jIn? z?1t`6BYiwiYgnc!o^35{F5|>GRj{pw#4Y1ZTH4CM!5GeL>O&j2GBAlDC2Zh;fjH&o zBBmatEnen^X*^CXi9#}?488f0AeE3V)Yf>27MIJ|EtTPT{~ol}^(GCae{8EE@yj@` z%iyxMAJ!{>_Pc50(&9quKX`lE^89B~@94HPaoKl?o$Sy4DQ;rq71C)p%eP_w3a&XB zV`+IY{rCs|b^5|rzFDeR#ed-^e^JEMN9@11FQayU(57Tov*)}rG~IBd?qIuaW*#4Nbd|CDXH z6yvyYHmd)m;Q$;pJM#SDbl|DSnbZ=le)G3Mm)ND1x|Gjsd)ttNWxRzEE>5@nn{?}4 zzgYV#Kjn5HN@zW4-So}kfw=oNi#A<7I(Q(O0XBHi*t_ZD73Y?r5njXYJ6sgk4Kz_Q zzSjzg6!J1XqW<O?{gBHszQ3+Q8w;_iDBlXU3iQ z!tbQSCGT#{suM5hkL@&rC%lnT2v0=lLPvzahGRD)G5odjx(7H+v@PkjSzn(0VX^kOHx`kLt7~FDU;xhD8^ADwG zSKfmcCV0V!{!K3j{xV+igvwgAg8UTC&#aK=v{uZ{SJuyvSY^C3)%vL}YeVs~lh!Az za;!U{XrYbMDx@Ofywk-mNENlMF^h^<9H1RY3u!F%V|J9hQIuI}S4$;>jRjhZ7(Sof zM=~S=@x3_$uhk#Qkk_hX8S;9`191Zvrb!OkvM)gJ z>3eQ{8wMa_Sq{yF<0c6hJXK#0JjIWEGk@wO`I(vHo$~41lMj3hFHjfKy$jc-ht{se zVBZ*?l~~IcYr^s1;HP$}J;kTM=50vPWjw30E^s+=xu$P?_493bwDk&X)Zis~jBiLU zFy5&Q+n6E2c!tC&- ze`!APv2Gs0p!M8bGj7w`Zy(yX`I`RX1KZ_cTRpJJ2c^?mEsv{H+Mi{xj8#84n2KYy z2ivFfExRgH@aYuqlt%1k6=B&&T3sJYTGw6AnhgB9&al?TARn!*>w1k}rwO5PvMj?! zHn{Va;k^{W@e;pN+-W`V)i^p(o*8!nyQ%hIH`M`7xzBGEn=<3tl6?g`vl7)kL&wQ^ z^aAu`JO^DX4!`1`Q~Il1;e{o^3y7HyOCj)rL#_sqPku<|eWG&61B3izMC2sSA*k6L zL?Y!Y05SOEsuG4YC9*7?9C7lNrlc1u;)eW%=N?US`|eD0Jrn83Ik%+=3~)|iGX>6s z^I#ltdAyiFdUnD&80WRU_!gD6Enuxp!j)^@oaQgNEG-{DmeRBMq~+xC)PoIi^z!1x z2=CIqIy}H1Y4`+n4KpMY2I2a#*$oHZe6U_}@Yb_5IT-g+)Xi6ibGlldsR!8RgaJP+ai`B-85huT^y%(R{YLgEGn|0RT+`J1{ z=mk9#5j~ed=*j&$4#046_wzdp#O*(TPe$=)77~>&EXR;)hYnJwiaWnzQZd6bd~gg4 z+;2^jb5Ep$i*uNXg&EB3*JymDen!~F;M{t4o*{F~cq?{(b^(KBV%)NPUTOTMJnL>~ z9h-Q*HlE=|__npQpXEw={o$F)XX{e$2xe*?MVc|e*#%q_YWxvhkWD+H85g*SIn}*rYOJpJAat| z^M^kgXpQstzxTHEgYSD6X1}qI+J_l&BU&GB2rrR_FdL48Q(9M%yy>e@M_jB+gJoRi z0$4coV48XKv-~-c?wr3lJwJR+8ppjjfsGjk@r<0D*qVm87BA`rXTs42uFKacoT-1C zJox*DziHPnL-3xj-_guY7hQarX6IF#G{AN}x5Kf*U#~2gd8Yck-1toct-E1etD2bI z>&@hW?m*mGD-GT%Y|z;*1+*%yjbF5iZ1ddFGK}^+9!_e|*@%J(aT(&OhDviL^K;y= znO`$%)rfcuPBm28M*K-<0yLWQxLAsD!{9i(mL7U?UdNlS8&7XN2S3<{2hzJPUQB=W zD1IzcNDhqa+2I60+sq`#4V6|2ryjx0u$f;eTF?;3!MYeSy%VpNG!q*Zyp3lle#XnN zMHbRd*M1M?WsCLDG%rJn(8C`Ixo;qQ+?fd#EU~VaOs~y=xAi19*NyPmQ6h)n7udyP z1ev-&vH0%6jPh9;xhmMKmbWZK~!-7FU*CO_|cP1&l*E-@c~{QI1`ERqNdwl zgte8_fV70If&9atQFf*$vf@CeW+aU_(9|jX&K#CE`ZIwP%tB|011_?dNnY^}Xrquo zUaO0LR?VJP`LmsO(QxSP%H)lJ$@Sw}j&unDgK;^Gdo`mBC$QUl9+m z?c-mxqiz#(n+grswlWZ>gw1$l0t;=|au0S&MOdq5#IY?^Qk;r7xrXeVg$q7LjPE!h z1Jj*WB~UkHMB5+v{5+M%c^HUmhW*cA101|`jTgLR8Z^NVXDgKnvuIse55r1%8o?mt z5*E3?Fs;nplZK}Erj_9vq#GnwehlRL>^wq-D3ce5?=q{U)G_B7QfkNy9J|1_8_$qf zWjv_r^UGmR&7DOSzB!e@!XAFE9Bag}ZZ7qV;v2!kD0>V<`Llkc%idh5+SDzl+5S@Y zhH(>Rs1iloRfKgU(lBT}^PHBuINE!uK8@rKul$rYa4L#6k*8~a8zB;SpRn%pYo+Cd z*>vt>zn2DxpDcfcifnM z=10F@gXLqR(T^G###aX1F%&*vndx4F42jFI6SH>xdeZE^uco>EccsO(!SwlsTd>&` zW*d%AXuxfJd=fL@@HGS0_|>KikmVQ1_PAt17$Jk}x?!0ngvO~^K5U;j_)p*5_su&S z*eP9h<#if_<)C2HSB=`Y;W?qn-*x?T^)sGf6z7X<|E(P0>F8IWccYus1KokRvsf^0 zV-2ZFd@LC4uN$jMMI`tKpc{EM)ToN#TZYJ)Bdeo7RnJbdJC^KWM23 zAIeRADpm0e-~f>|Z|d{SyJyn*w@swosHGF*c$ktO6pHzL?V7de(~i=T(v;Vdc^VgO z@@O}7+{k{jZqhgPY393byg2(Ig@SZq#7AhQ$&xgq5Yke9F{KGMib)WaXC@nm>5dy; z`AE;?4VP8!qZ=LS8UnJMVMv@Z-dP@)nO{>M3(4?w?T^7Yf2R6CI*AP&dNDJu7rQ{U z=u`2h4mdPo6^VX0yS`Mg>XrlvvBL*JLH_#TpYOBC2W#GIyZ~g3an>OP$>udaOU@bM zA%cuUx)!2ZZo~@t?!ZwOlADf$a{1EKlq&IA8R^eXned<|Xiwh%S7{BOvK$z@DXop| zt_D0f=+XT2PkK}43kXeG5Abn-g?e!Sj)8-6%;VZ|O!FEqE?dJFphA~q6MMk*gU|zU z9GqocU>)&593kta2XcvT+D&X;SGBX`0XC0S6Nz|k3R}`PQq4RF*~YKp^ER!;wLC9U z^#PHtY@95Q*J^(==x{^-o=x(t&%@#Owp<$jWu-BSIhY zfUHjUBdrLo!Pjv@hE}OgsbS#d=PZ*Op-CV7Yu9d+-(Ytdm)51WhgR60CWDv5e#(-( zT=&?=S;~XNtC*?SI~1Q#hhFvwXfh$8>t|D6UOS)ln_nGQzjYu_&tc1L5SPAQ#%`$$ z$NMkDKpfVG!%R4?x5$2+b^*?`B~2uSU!!I~wh~X}-@p+cD9-xNfw=Aa{&O0^%+0|a zm#2~Qt`}19GS6K1kZ6ha388`#AR9axcu)P;kNi#gmH++!(slE6&kE3a13oemlpFQmjjGBtMl?CX4HQ z(FL2<1KokRvs^U3Q4G}}Qm5QFwy0MPA~0V_xaSzb;X4~w?#u8Y#m3J+8i_*e3KbEG zsS+F~WavataPUw4fWW6<(pUlVcN+O+{?SCIA4ayQ-Ncj$# zzb@@>LWZW$NNqir^o|=g___WGW-evCtqwNjH~4kb&$7wial=;4bj~!yukJ0NSWnU6nrIB-g|o=AOn|_L z0#W}{n#UW}r6W(IWBuo+#?)qd)V3Grh#1`vAFa^eop_+%b5L7F)?3C+aWtzs&X6un+>mNdhOON#d4#N&c< z+jmYqa0ml&db34JJ`3YP70-__(efhA&p(>e+IMa06z;)>4HG-H&K3va4lOOF0lYZz zXK$LR3PMVk&PCj|kZ((z3W0gm&O06g+sebd%_Bc~!P6;x{u!&5M!Dta1l zia@(fLo~-r`j+?$u%Ck~EM3iZ!Ynhki~KZ|VHJbKy+h-Qhc3}8Py$Xa7hacoIKKtv zGHAIskY*nAv-&BS6R*gXa0#EKGMpHELmI&>EiTgE6icc`_t6qI6e&>!)~R6+MI!(kq_$Gg?0pSg~EROmg(WkfV`$IOk2gK5GVJ39Lsh* zm7ZJKp1!nnYwDlak)}q+F!(m6%^7&_@pDuTcyjh3ug#-`FJ=8_*~#zdv)%KJJ39LX z192QA?B$NF=)bEUSlItkls~-`_dG3iJZh=S)9|IVpVFEkq&8E#*PG1)-CpF`ERks0 z+`?r|eCrT8ehMXhM;ii_KOz;-!07EsZfKf~j2c=rUW5$HcUmOqTs#dd#86Zj3ED~? zcvv0=*_>7(fwZPZbCn+~%n#gpjWoRhGa*b0Tqgvqc||@`WhWz>)PAa_0NYF0$^K6M z3!+R~T-a{I(*DX%u2QaEhUP;=rkS7#okz&f#NQ~t$;;_8lwVW7rpTr}P5uq~ z*uVjUaagF98#v6~`Fm;j+kRH}U{wy4jX@kD_<&XwMSTs(@`?~p^hm2yeK195hqg(hCiXGBb@6EBslY{n=T0!eQ^R3sxZN}deS z4X%HHRs4m71(Z)7#QGw)r$xL$e`@0Iq;v4rmIEIA?9T%rqA*5{Anr&-CQSGH;B^>T@Rd=2iP8}5iOtyU57GMyU7ML zxW*5rP3Ci^`60bNxAU{!l-YwBaqXX`&do2T z=Z~F8qxclAS{h>=eQ0p#9te&H3_BRNJ+)>S44=9n4Igf` z?5CC}pdL(F(+e#O5HqZCkT`qGJ~Pap&-uI3Z+ze%r*V9Ws&(Kn8*Tuf zqH;Yr?JUf`qV#p`D5>jso2|%NS(wHWA|FXB*dcY_;(6(-Ywt{b1F;#y@Gxe>jd00^ zAq~Lw_hL&v)(Lh_&8%s3<+$uX;US*!qdZT){;%nU>1w(c198o8!JdoL)Rt{Xj_13c z-`2~J`d9}ta6IQ5%~0sP$m*;78Lwtfkcl&Mw8XFSGL9G{o&}t#{i=*bL)qS5QlDKi)9a^5Q>O!YgvR#G=ZzRVm07JX2kD{}2FuJy zP|cIxD6r`~oj`fSwA{{{Hy}SN#D-6Lnt-&3`;tF?iBk_G4y0?iMjXOCc27N#aXV?( zKQ@rwaoKRXa4X*U;QbsX#3y`gmX01>OkdwSm!6mbG3}=PBZKLCuEmf2#ijI_Z!V;{ zQ`mpW6=UhDq2=@`;z^VGM7?eQ6~voJY43xx>Hd>p5pAW&ru}*V)NHC)yZu>TGC{Dc zxT?t3jg;kIP3M=xp&#oKjy2Ax#1R^28B8+QhETdnHnHj;IG$AmPbT8X`QdXi%p56S zL?Mx9i$JuBXY9Ii1bQG?2Jjo~z&h$!<>MxT`DEBpJ`yYWRxT%WvY(@1>||+X9M+7R zz2kS&+BNUT01iJ(z-QysK^_L=16sT?LmMqyP+v4K#(EQ-1yd4Dfe9up;!KMA#(_0t zAi+Lh={E0aUaQo}gCQ6w5_~AX(po4&l}sW#WZt5Ww6M3vaqS9}S{J@8KY8C@O5goc zx2CZzJJRIjqz2*q>7QrAS=vht?W1yFg+C-{0|#ITdAJA1!bR$gYfVkVOW$Zt{w!*zb5ek$@sL$qZ$ec1> zV@6Dm5bz^pkQ#_v4d+uRXQ1slcV{3r&QIchiC-pZn--3UGEbu8?bdlSl zFVk~A*NI!gK-}obd(!O09t=7R$v=Ln(!$$J3^($>g;d@u1e3A(7;ePRc!nF{%khv? zIX=ft--M-3j5k9YJ`zq<|74(TCNswg5lViFlOMW&J^82QYCTZYpsaq*ur9ft1=H3u z>Eei9Oq?ku`V#J6v{KbqD|5%Oj^bQ8(sL=+O2Y;XoDIc4*e$nGZ`MnNrk2V?&N}%{ z28_M|m$%^BZCLpF#G~olr~jh{{wB~r>>I@9Kj1_O48Dgd+mWGoMkqB=KLq`C_Cp?g z|A`39^B%*WBZo$zx>iqrCsN2NB?CYZNZGViSbc=iprNrdUm0I<6!lD zUBXg6)0bwR*B~615ZSwQdHPoRHmv2l6&sad2iS=%+8I^-w!vY2g4&P4KU(XmE8Z4p zNROPV{+(w1=kFU1d~%SI@VHr1`wPu}W!~sLPSfQrXOm`Z&IZlXR zGQ*ARF`nT@_@`-qrAMig6zrZ}4IbzY#GTDjD(~s)#>_XkZg|JDlDK4p%*Ip=v*PhX zk-JJ7J3a>>4e=VI7EG>MeqHdS;0Ia-3t2?4Tz3LXT{`76jW`aDa|}pGW_0VKc;;XZy0P2^-QaS z5ga?)Z$E@3EHztH+Rx;io}sY}Eu)6a>CCT%hA8R~vo1xP9TNq}l6{t4@yq$rmGKsi zLCdokB2SPe#NcYw!$^*L6C@#*b=UlXQ*&#J1#OfAGbx6uW^uX*6|#Opz&%hSkP1iS zx5>uBOT~r zOZWhO!C%-^*VWRuE%dy%$d1J94DnkdL~DAABhLtutm`a~_`sHaI*}J~g^4J2!Jt-^ z0?v^onbQw_GOf-ZOV12lo0iA-q{%!RjvdBw(1RRbi6Fm25C>iuWIKGrvhtyf?&)>m z0hT`x55&2Cye^FI8u`+CAezc(9~oCWNfB-k4a>9wkNBoBMAC8(b|G9{QWH)}XR)1) zX981mC`1sCm&R{G(Gw@r2%kJnqjcOKY$|b#V)o6}sfl#-_(>$x;mM~Crkme#wFo(g z(2Hk=4z0moe*};dtM9deNy* zXko9~^OwYa8UjiAw-yKC*0@9J8ZSK62MTY35-2jda)cM{iH zeEQJg^!G6ccLXy{n}Gvv|Lec_>9lLdHVwiJ<3%C|;|94qEI$wBy~n|)f{<~`DUua+ zI^oi>r}R1M8g_sE+Wd8CfB*G*Q8|P@8#iO{R}6g^=!}<|xpOIDI9Vu}wFq^FAw?JQ zQVC`;I4!PP&g`en&YVn-KXQLdIxnug`mK6t;ODv!T<;B~ALLgutV@OjZS|k{CJ+LU z*2B+PMmJsssa2+B)Odzv8slxjw1y>N8*(>clX##Uz1Sowy^hr2J;M9Nkf!s7);+{T z9yuUAW;h`g16XV~^q3JkP6aC3YZEn~(+x>v1(@cz6c7>{HU~ZQBr2yl?#ql@(wJ5- zv#$o;`g_y%QNB^d=LJAmhTg+Nc)NPRP`dhp7}z4+%)#mOsY5i1f~+HRdeht9z9rqb zHS3FM+V^7i+k=Oe(;OZ;7wjHRH)59DB%TSoE*eiiK02NL!&eqCy>21heRwpz18dJ+ zcG*C>{SLobGVkEG^G(ATgk$JQS6n=p?zo$4vGCspNBpwOu#+CqX6DnEk8)sfozTiY znpenkCoEjXwB*&iMeYSLEP0 zVO&Ed2iNeY0KSTwKd*_BuldmP%)Haa7mwgiKH^dFqxg^!{3so|>8)C*RJy>j3F{EH z@VcYV>7$HtCib(FGj`d#Q+ntV7>s*34ekAtbYiI=9VpZZ{s4hL!!p-uT$`JZP6QD0Vb8lKagiS7b&q*h?-Gp^aCh%5uLhG_|Ada)qJOC2Z zFaKE`bBOUw3J=IpvV8mUtI(COi9F!?V3nHJPVd652R8Ho8^mZ*Ye*tJ6WEqUiy3XL z5ru#g@%S?rx`ZbWrqN7 zU4+bUy?hEOLW943^zzzI-B=%^UqiUs6R*NoF-UB01Xk@5rxQ9}Sh8W1H0DWU#|ar& z{tU|}4+%~WT!1==fg-Ls%cTWSFIXQW+(e`~!}3=@8WOk;GY8|A=4aCGM?Z)iQjevP zJvXJH?QfENim=X1aGa2#nXfd|3zi-GkIx@Hp1%Jle>pvcd#V|@2Hd~;#ebYGyZA!A z#GJyqa9j&+d=#IJ4sktGzEHy8w1inAWLjvGA+D|8;$aMMKl)j0gwm6~Jok?DZ2u+L zHGB$uhqb%?0M^_a#Rd)idVv}Pab(9fq6(D5V5ZKZ0xdUs8$v6hu#eB(Qt!oX zsor$Ud4yM8ccW&*^2W+;$tbnfKD!FZG-J^F5%v z?rGC{pnh_1S`}Zris|0;4Voc;2(xVxEN^JsV9{RH9V9l;^}*q-V8WfFA~f#GW}~NU z&=4{xKSL!nP7QVv=QN@^Uhyl^3Rcya2xgpp@2y*loMv<_9eC_S`tY~%Og1?Hw&ezX z*PF(xLAaU2GwDz6n#E_PSth`-N0!rlkIbgqZrPgNJk^ticaEp`p1+v>dOy}!cxW!& zyb}+r?IY=mfw?3RE%x`O8!n8_1hpF5J(SK_oll1f{$c+>y5WLYi1hJ?XVN^4+rWPE zQy?CT%J_~Gb_xyqIUp95A`_5Lsb5hb=p!f)g=~B%Iin4Mi>W#6L1t57B}pFU(7;+a$6F;Qtj zZ~hfaWs{z?`N(lzDw^qe4Pzv$qE}kTR1_y$N`^s$j`1u$ad_y{n$59)@>YC0F@YHs zqdL4Emj}bhXEOws(Pq%4t3%fVT@SqaJfK!rC+Kd-R#9=aj%=RMe6yVo`y3}E8lN9& zH3mHRgK~uf9>`e5MppF?ZQ!|`&WtvCj`+F+-Jj%z_~mD>GKEhigO2&Z;l zxA*bL&gcD4AI_hQ@#P(D16Mwua$J#4r|Jby;D#i~%D7=q@MF@zmJ2oWX>|HqX$2cN zaQ0AN?(~E>Qxdy~2jeQK^tUl7)fCjmB+ADH`DZuN9W4lr%g84oGZe^xk&vJ=Sf~7j zC3$uIXp;wC`1DxhS_1Yu|8gRm49Vk`JU%YQT9|MGkHw5dJN9f&)N#ltqB*-(}3g5y$=h&c3+$xv}5 z2duHVt+fIhUPj#jv(?ccUB+ccg)K-qO2aXooK3!dn3114KTPveUc8yXTm;fNO_zad zJ`UWASvWI~Os7A)4>Q=zzlt!SKiztHybWGCK9m0Fj+wM% z*?C+_cEjGkx&8Sx_}%BEE3lE%8?PBlJNBJOhmK(8+{{?IdaN(q@TUHB?|n<5zwO^P zoZc84KT#7*z8g$$-QANuasXBo_HTdFXwovp=m_7j54KnbkjQ_)ZQ4&iFmR0u`2>)` zapw!oKFt^cnO^=@8AfU~sMlgY@=*T%G?bF5zYnrvywZ_z)Ct?DIArLdm1%94q}tvv zDQHiDFIot-JVDUX*TA!Qx2>->}kC~bimt)Piy`MlG zek?7aqqMlVs5L@*Q6GD_ehO^J#V%ze7o7tr9NwXshy$4x%X~qV5Al$w`?|6e*M9S# zy_w50tQCRy#{e2x6y?e|zcG^hTppRElYFTrCCJ@6AX?}Zt%Hk|y{U!%=g zN>5LI2R_+2N3-C#Ln#l=hO0VBq-6VamcrSTeU?n_itT!!>w%Za11;@nmQkiLDcaS5 zfx>5XV6iD=J{wsA#yP;C6np}xF*)TMqlUr>V&O-dO=R*$kl;!HA3If2Wb;|Uv@AeG zJmFb`j6QtbaQ^Q0om2N?_6`Rqs4-^@@exNp%@EJsnm>6)S|B@a`JqeAYhcTsG`IgN zX>#UU>DiTAQtwLm#j*?=W`>dULAFc%0mSnYRC+5-9+$OM8f_5248tb)CzltmOW{#wzYrA4L6lR6#C10+HQJ-F%{WO@r zX}Gb|)YBhMlPAA{8JfG(*u}R&gPJ!kPCHO9{h=>UzCZShzm@L%M*HWMWcUj|`NQd^ z>#x!bxG@gIabpI2lFA)Yd2qI>|D?b={wwb}{$7}Q?DJ{i(Ea$_do0~Pe`{JDJr@If z<7pT#O~=Q%mM_-X9>UJ3l)=GGRyI}!p7@&sf2vOzGKZpxVig=08Pk!WG$R1hbspJO zysZEHNvlCn`~X_vS5bGj55!%5<@IQzI13WxPsqM@F52s zAcnG`RRT}B0a7#%psa`A-WdwsYJkN&y#+&?T@x+ZLV@B|+}+(ZI23nx4_1P^TXAR9m0^da!DOQeqVrK4z*TT!hB{F+ zt!JjkWko@6mP>EiY=t`kGV@pN2mp$&K$milDO?i~#mqi`{qV?=E;Jmgt;9`WcM2y} zdB6#C4$R*o4r*d{x+xfWo0A#%t|>3yhs`vcb($Tu^g?W}bm%vZn1tQg*IHOJ9r#GV zo+628}lphDzo{U!0&{zK=~a#v#8EnE!2Qtft+tgH9OrLG^2m zMZ1n@#f;~aSASz8M^w9F6qk`S5GSrb5iQd6$!ilsV_7gkMe2o@|AGlWvKky|^&H|E zByYVb$DNcBK7DI4z5fFHWdAtLG}|j|*#7PQMikKS^1C4-uo)tJ>cGR31RF+k8(fr@ z^KE!596hgIB-_}CtCM;T=xttB@i>?uS}4l$K#u=?5bm4qph^|4RP^I56=VW{OZxp| zn-xECBCX#w;3<;Z@zi-Ps!T=<+ABG|J=6Ep8EEnB$S>#!)LnAJTCdu^`R{PyfwGZ@ z{%j5Fy$%$7_-{W-xEQYU`EPac5Q%>~)lMo#&-UBcbd8=@V<5aJ8~_C6^)&eq5H%qWqXiGCMr_d@$PK%ant zuhX#2Ivkl|LZwP2Ici|xN4wzw2QthlmO+k=ySzfxc2~A&A7N3an#!O@2{&{4I~1j& zve59ko}!k9$iUBjzZ+rKF5B9+72Z`8AJMPu(DQ=O9Dk!Z367N|kpJ`JHVOMu=vo)D zLry^8)JL#Ob&KS=HqUZxfFMCxjTHRr!@t|e6@>_?j(Agt?Frhk3FfF22qs+t$`>cq z-og=Kn0W5%Nc@|<%FQ|S20bWu9`1MnulTrx6pwLqA+P4i^_!_|Xu>X+9KaQ4uuO+| z5|GeH(s8%i}=X^!ScS3Sj=(tDggkMESgMll!u&~zI9 z!5|VYEcuF{|ha-BX5R!L$b|A&~1!CL^Sxwh{-!-d0s**NEZt1ZvIC4VCH`nuOiYu1; zLwzRud>p(e4<)EH2Kji0zm7FOr2z-&On^psJc=2bT5*i}@X>Rh@tig}hj!2hP{>cr zL~;VvdgKX$Z8ih58-rz8bI`(1nDbC}HCX55;D$WqaHX51C&UGRTH)bc$X^;Uzu?53 z^&OOv^pUzK@%z{jgH2F|-k{v){GW_|hqIVlFOmzGaI@z-_p=)tQ9Jg&Z{cLbnxp8W zr%A`dApzq{(CnijAcQnro*AMp`?l^leIc3Zs@Kw|`T? zWi+0!UmVwO+Eh81+XX*D;Pth(Az-!++fdq6^PJu9Wc{SCgn-Zox1V`<8g%8HB`<1N>hOc#!ZwXnvOnv_hf1hK<) zI8RCbYjdSA3w;+VU**3C>~U9w^Y0SbHPHejK3c_mXyM56-tH!sF&uf#n)p`Y10*+w z(Ql_OPOJ`>hdgsF;SeN>-bK;d(TC#o&wh$#2KA7l2@8FX{t@3RTpekhiemq}3kd1k zk&$1dO=BpD*GAi{|NB~#HS8*>R91^3;npWZOPgORW51M@Q&~vJ6ia#QY3Dw7iKo^i zy}kKYiZ+*|xfOL8JWe*SH^a2gXc6>yIrr&1qu8>W8t|gnUDs=^p{0^(CgXg##RHY^ zzNU0f->swJ5*>ey_{gY9_;y91L~JZI%)V(=hPwWaj68^5f}~~mqM0HDI4k+VTea`R zLn-Xak(}FKpt~e*308bVWhQ#;kiV~Xf7gp+US0hk!#r?p^3p5S&5HlU&CjKcB8|^g zIXFO0e5N*G7+7d(47VFf*q-tIAZz*R&V%bp!ODaFdD;cp7S(0emCE<{N3N365#CJF zy4Q{Ra_(7v%&vg1DII99IR&r1<7l`bhlM9NVzYg~1aM@8QStdcM z5Mm_={8Jjsp>PCKh~Z^zsXbUkI=e!u!yyzNkJZjp5Rx4;VEZw|OSo`1&Ak)Ubdwct znz6sxd?lMT=8*!vt2Oxe6IY)?qPQyQ%h&RWfGF&uR0ZGCUMQuXHjf$lEKZsipS>1cM8II}f2y0-Qyp%LrPD5C3}- z7ZTwX!|L9+qVL=Pbm5#i3Wnq1IE2{*_lUFN|6q&+18j7}0fVRtVzA;;93X0O1Tc;S zIWw2Z0j+Vrr-o~~jiHXzVStiC9k?$1YY@9jD;MFp_9@$9U zdgMOu6=+xsj~+z`$^K#UpHQLv&fJ4(ts4?#kE>0rxXy7j7L9Qy!Kj$E>cA#!-%JJp z0EEAT%SWJ(wH3ea0YB195f~XisqqGZyFx4kodj~|asq3j8EUB{y?PHmDv!1KOPpPY z-5$S5hfvG81Qp^wqa186J8@b5#Eh#*arnjI$k(uTr}jys+@CSXiud5CK#K9ksbCPx zKR?vh3e!Pdeu(w*-R_0l`%i}T%B76TR_(tP8?E;@ozloXwb8DjfX8qW*{&C9lF9zc zP49J2wdbECo+QFpMDj`LFhdh~)PvHTm=4x8B=uc2KM{DS9MPK+-P=t`&iZ}*6H-f1 zw;m=XkIIEE6e%!Q=Ke9Q%Aq!oE%Jaf zrNfB4y7P7BC~~ER zERx*1ZXAE})z#c*kcfVL+?Ee}4(yA*5y!r1KmP8x7T|HIKyxHL`25F6dqW0b`IifL z!*$u8geHxJ@Y?a9RKjN|A6wiMo5Wm1@X?F*-OkM&!EZ5_R0Ht9L+68;j$R9)YZ2>p zIZac#&;Ob%1)c-g&nN~N3=p0wxto^B^aACq+mMx_jOZ}0zWUTe;HMFY?_gYSg2{Gv zO-gx*hLi0Ic0vTVYq37A(2e#WTR2FDQi>9ve8DPXTTr4^OP>278Zgz9-lO=Q#R=`a z2REe6`)ts~2PBe{_o$!_XxNIbp)8*ue3t*r%CG#yzf*? z$MID?gkx*ebxUOS!`k2?^kELN9fG!|_*+qjti>@l*wE31Q#Ssyz)n|kkUPzg43ls# znz!i$@AH!wCA+dG`m}u`*_FH;E=ZUqPcl_pF_NLTG#&A}r8`b-YKc2kscTuQZ7U3U zmP$Q>7kSr}TU}E{AF|U&Jv*$A$8}oDB5a4dk>d3nJ~w~*G4gd{b2O)ZjA+1=xs6ae zo`Mj$slJQ_0%(073GQL_U>kVdV>YD7n-kS7hC}t#208x_JF$UKVtZPCSfj;F@!W$z z3QCY@J(T=Vu3a*O9vm}&7Pznx0)o0ql2RCP>m*5O)iF;Yd%dHA;2=(o;a@s~(ai&# z$=J`0`ElsB#VW*badI|M0tPQXhkBa-wDf%#)JgiK^bvul3L6Qq)KitG@VTj`@Ki2_ zhqAf1JqWN{AHj#h8GdgY5YeCd13&!6{vLrLiw|Z*&7!vjj-Q+kX8l&Zv+sr!Fm-(r z-UyHwleAR+g^Y1b6=6O=Bz1EDspoplHGafUR23E z=^H?qdevX6i@@XmGI5mJMJ0oyk9OQ$78K2cB5S5A;|n~c5g!A!EB>l*SUe@R_P23x zi!qXW&V$)R?mv)P&Bh}~Hext->kTlwe9{ z4JXIp-$L+AK_OHi6^o{)?cC?2tPd<83)&5bRb~HL9`L%IcoJ5j>(_@D1s7MM1Q1H> zL_DciFlgsmyI$;Zc~Q9z$3-PWy)A;jq`o2|!8zLz+@yWbbJ6tknR+lfF3qxg@hku+(#C=b=hpPR+JV`;k;1hvDJM^r3x zNy&jg8;*dn$JI`OS#}?17h@&eMK^=xFdavIOa~_?)nG?`g4owqj;&t*HRu0PwS$83 zE4ti0cB*TgZAYBZgUt@fOU&06h7Em}@GI)VCy%e)Diaan{*qI0e-?HV!8Fe z6EwM02~tXO!gutmhJ#V&k|S|&R(-cyaI6589U9QV`C{rlQG||6vFCMno(EY5c;*k& zptWwyZE?Xa#xfD-JQyTOxg_kD+j_s+2QLsK1oMTmNpgCy=53eLWFFFf zHw8^D$=kqBKvdT#sN7!?C6*| z!FBZ#ZSjGiXV7=g+igq}m~@A##x3bjJ4X^zp_#}< z2BCc6Ne{8RVKY{yU1xj6pKK9_=BlJ22Kd^-xl+vSbjWATg*8LJ_2>*&{+-&Tgv>oU z1F<;xv}S2)h*#flS^S5PbT|K|b+El%171|lB1bbEgBO!cFzUA&M4=6BP&OaXf+-{J zp09NztzWK3tY&}038((stgfEeh`)d6;EUN|2>m(2gd4Gg->Iht*hMlSRTL=5M?LDq zf7{|;&$sS$BqoF2)2XUT(M=V;OW7Xft9(Yqm~1rA*O&3W_+dnHK5Dxjc|BCpkd8d$ z+F<};cJG_E3VvZYv8|yy4e#o0o2GwRn!fw{?6&oRBVX}Xvs*iu8|W_x23^H1hCN)P zYJGe22pGh1FMsu zdEMAf%?J|t-Oah&FWzclEp+j`o$zg~b^Z42VllBeK#3@hFg+^6PP^i9ZNUr^=GxdW z%px18wH5%QPK8duW&Fms7mT`vbCIPn`FMVHTSZ_xsMqVZWz>@>@SF|RVq5Qgw2s+- zuApVemEq9II#>fdg5AWOy7Qc{*GW#)YJa>u4&-!R*^(ow~)|5sHkYA@%FW9P;gOvhTHIHWAq{e)c zBSZTnV~(aUU5~9?El>OuDDkD}M&6IoOoB@qDv17c3beGZC30;o4?^s^C=|Qk#Kt7j+eNhO@Ja6$(xiX&qfx{DoICl z@XiVsn%1UzN%6okJ>I@}_NJuFc9jdue}?%(i?seT);bQ9arFK{OPG5}Qt}9zXx=q1 zy7M7N5EG<;VsFy|_uLiUxPdXh4m_1v<;M7_{|om0eW(@6oG0CHQHYd`#xDkEAN5|- z=2}4EyCKSK89fca5X8sK1MZzDm zG|er3gG|&%idvzhek}DaCW`ZAMP?$F6>qDOrNCJgai|P;)sY754DmZJUViquczpIf zdLwyUc;v(aw{!PO;^byQ_E_3Upnnt`N+ai?a)ta^fn~*Ah+DFKmEg3C;+HbOMZzn& z!0iP$AA@?KH?^k6usZ4D;|4 zoTR9Ju)tE@C*UR=Z~u-RxA-8L8ZbnsGjA7K`qfi)Qv_r8(}EZNzemI)arZ$HyvCF7 z0LQgo9u*Vg#VmmTgv&Gsr3=2MrqJ*wG{p`R+viR}NA8Xb>*r;=@^%=Xph#r(+T{!7 z)}#DI7}s}kZI-+NNXl(W_$|8avfqqEt1`zq&MeM7bNGanF@VQqTC|O)0W6IAx=f>@ zZFNA@Cp$nV#oncgG zUXL-iH7R$o58{jVxcI8n!4?8l{n_W7JKxawnTYxhNn(WCEjWQ`4WG{j`~ z>o+$sn7$MF-*Xuzxar1`yI>c-l-euOnB>5NXJwl~wi7`gqYS4QmL1xtS&5Eg8!ttj z5FEFlM1j1MZ_%(>&`2zct4em1U_v7_EC+tL3qqOVEDg%A!i`vInG_*>=HHGIPC+W3 z88BIzl^eD6Jw&xBuyPjxjLFILc?i|)3DVAii}%l~pYyM^Yeu*&U-p7MG_QbzU5}{g zEY6uC8rZi#A=a?(b$q|nvst&dZqf=~`>f(AM4i-RiYgOhk%r(wxC?%%e?`2%pM{q~ zxZFy6wA!A2)Eb#<_S4;qCvnumqYkj_Sh>(qz4L{9_zO#)QU%*(wu~8O63`{PZ7FWY3f;Ya z2!lNRZS?cQqP_YbLrJ&B695CZzm?yOtr5LJG4Ddd%1xMDoXua|R2Q8cR0uLp63Cw|s&+)jrIKU((M z8(tC)=E5&B%K&Nv;tv#qg~ zA-Cgq6;Hf=?Wa81X+4*n%Y&pnD5(KXY*@nYlWmacN+RThtLpg)mb&`aR37}qnszD7 znu;6t5*s9K-H=F;86Gz)_ongTwlCpP1p?>0H1d7QkFKxTEW`!MLSAA3jo+GNf8n4~ z)+I^1QKB1a(~$&3uoix&RK(jz!%Or(gz>uM6uQEKBi4wg9d`3WME;!+AC%#e>vx#c zyr)lZs3$P4=-fwQLeUk$qmUE<$-THz(W5`?G4Tnek- z`ZjKF@|Tz*v|cVpP^f=&sKtK7&L>1=|Rw6i~S5AKPt zGj)u9yprF1R>wKrZ`7^aDhr+pLh1N)8BNmtYXE!-Wj>v8Y^5q>hDKw)Ig2T4b|4mb z6fNC(HxOQ`U2@XS8Ft__bvg$G?Y4791V(AxI@7Y8vQbhD1*1;EX!Y0Y7eNWUd~cVj zLTiZTUQN9UX1*raTcS;$2Kl~T${kYr5gabeAME`BjHVM4YUkv6)p}c3sR`-eiSEw0 zc*d42hf>-iPrCOsBcPR2<+9MinYi9ZZ1zrjrn7~0`u+=^e$#=+_M`3QD65P;=c24| ztY^ACNKM6_yQg_)Rgm9EdZuwJsg+y**lYj~mGHadCj)cBbOX3vOow39UD=9~M~*i^tNNX^d5zk30(}+M3oYlp}6x&G_W5(1B?=vYR7=$y78&5EZQik8%PpWURLH+xkQT)DE zhnt~%Oq=9ZHM)e zv`Mz?OjiN0TR3K;1)8Q~l7z6stjX+VkmPx?r`e5ftNGvPw?4zU6*3)K`yN#4&H4I! zQWELpD3H9KSj_=^FJc=I-)3%@mva3eiS|Hjf%L?OX*5{CaD)kv`K_soKbyE7jWXr_ z;^klM?JHMXgTTpl9_A5&+|?(B=^l{q+TFa9*9J_xJ!{Nsmb2IAluCPzboNT0thez7 z7O3mxy03V$FP;X+W4169JH4IdwQpZMuBH#=LXiiQ&29k(O>2#eccKoIh zRaG%J8mJRKZKLDmZPn8&BxFx{C*`569stEJW#`Z_eFLNk5z zK#JL{`J&;2N8@Cz6vLtIdtMM>9kxZ6Sd_haRWioT~SEr$n*mJNFH~ zZ95FZ7To6uJejl`@<5-QZf$GEr_*s|;q;M61SZFQ6dw*I;RhN1*eS8tXrr{og1O-w zl`{f|X=PVJrV7Ni|4Y>7!z~JLTi-v-GuE$J3X4PkR^TD3FZY?g7n;o^04)HfrP z7dmio9icXY9vfN{wo$F)4blmQEB&#_A%54_be8N>%>6$hXrIz#ltoBguKz^Ry*OIj z)B4%o7C%AGfS=lb{^%lDb`QxBPiDG#YB@|Td1q80>Vg(1aFE;4YKMc?z%4{rXmx}N z!qb6ubv!o*%;gGr3BJ97A%Abjw;q!l=grJLXx|1|AQH>Y>&)%Vr+vDPxYqvyiR9QY zV_&&yC^xo#YZzC4$>{xh%=yrtHuuW$PICM9!ED!n}>PC-QmiorfMbTOH&XN0#$_h=*jmD>BNK4E12;$u(8>-v8r_~YK ze~s?xVc(|ss{6GZad{A}<}a#zNw3wApX`qZr5gy1CXbyPsterqq$>Q?y+&3+5=(m6 zmp6|6@ruF=U!d3I|H}e!7pInfL1P^%2mP@w>t?xO6A+JN+SZYU;XX!s+B$->9~hS} zQ<~vRHjvTNp2R2rf!@5nnm00K_vOHHHvF7e3E(Q!DAS#M3c|b`Qah*yHGlq zYg_N6=y)?l3hE{+5O+gR=vgTak1KqCm;_etQSTsRF$F`r8hP1Jy}9Gu7ImPGcYiyE zvD-||>1nGOJAPqaOq@-?Jd&Fmo)0GH;N<9YJ*|%X+Yo?M(qM#3kb?6lbFo=+P5Pr) zrzBnim@sL(Z&fs}Ba*HB`Im)bSTXw;wbd%N*CmJ0UZm9rScvt|jsR>R$PSfanzHyN zXOVDH`IW!Qrd3o~uI5CNOcYNRLHJ@YlolW#bnnd7it5xqxjAlT$0$6b?4toV2!@-3 zo~RWtUT?%qWRlGM5+U=K({}@dWkpZXMmWvNG*jvT^!%y%8Fkq4q|k2lY(jFys_keO zEI8%8sBt9_KWBf`WC^V}`)Ip%)ovf<*cMDLhA@1F=jnaJJV!DL_{IR}Hp`xRk)#-n*F&Jh4}VMAm&uyfiJ;etHnNx~56$V;J-IzH z!=0u6 zbLqeM#b2C>3^N2WEit%5@cY?uGqJG1G^s{-5mk<*_EhS=(z(LY1N%rjAE8I~-jKO6 zcWZV9{U<}8{oxGLhBc%^ZB0iV3I*J17m2zvnhUH7c%GO)rT&LjeoR8QkAA;rF+2co z&r|Q(t;e%bvtPI^afd!ym`MJxyRBw61h_rgta^DoUR?ZHF_L=_UioutcaG3IjIh zonD?V2zG)(F_G}?;u)zIx}#`atHbG^{ZZKyxp^8IPZ*?(?lO(@C~=%Ja+^v6W^)HuZ6Z2E;Ckaj=v~X ze5#E0Ma~CW^`^q~R&Sr~s!&4rHshuPyy2D@bKa(`*zQ~RwGQe};v1d4T#l8n-M?1H zLqZ`oFT{01vx7=gYC^cp*1Z3U^@B<>@DxF|Ya4J_LtbjvJY$`?J2mUWKda*1ZC*B< zikO4WB*H*2W%Ub_rM?-~kd~iU9wXu;GycM7t_rx{DE}5dB7^2%I?sB<%JN3E`4%k-p&d3(lk>@WDILu`Ec2* zhk}vw)Dit8<%quyVyTPee57@ZI3WSo%f?88^l8}rTPVkV+-H=*y=fn-ZEIbfN}ZT^ z=-#*CmO(!%uU{^Y%yV%5_81uRYGXzO!v-rpTT$T)lLH{=Bg(X5a3<7Ag#GYZSEe~_ z0g5ygc(m*Ck|DvG{c-1N{TW8r@r8#ZpigOC-v?2A$#a|^t5v7EX<@fIdG1h5#RB*x zzR`ZB#l9B^j(t`kzDv=kHh+2g$pFHc(R!{mdaS&?dm%P;kRt`7zRt%L5>&OwL$AJm za}79YI6{nSC(d@6ybsWIDn)_dGG;jg)?#u)>AucZmZ@wA}qu%N6tx~ zt(N0OPr9$D*dGi0%K~j{FuIJF_E~=B6Z8G2iawtq8m)Uj5Y@E^g2FZkSUSa8H9A30 z)n;+NRx)jNNggYjPr@r_oQC_2`RyCon#cRid${?fhO;l=*H)7o?_UCF?%ZRga|*qK znMXkp$;hy?^ZQn~Xa0%|=e*qQNOqTZ#+$}SK|3Kx`0e8A@$j*F)N_-h)|;Omsm!5w zY=_FF!rR3)zF(0Nl0o0OWMX8}%;9f4&-jL$io@iqR@v|W**RGTbY-3|g|Fo*0^fC) zn*cwEMT;eCa;O?6B%ELO>Pf`tjYxHq0VLW86^!|j! z{$s|>WOqn7OVbjpG=*Z6q1kAhx;dZxFQmZ#QOAalVDdRb4ae<61^wWJs=uQqBLkxX zZIl!8|8XsA{@Jio#>~!mKiSm7?)a(QyVoCh|C0J67wY>pvoSJsx{z@oTfZ|JrvKEkbyl>m-5F_)NhJrkX_RgUk>&9}_+G!*qa12VvRN14FovaO*ba`-D^SY zpT}{!QrYx+rXYtKa!tN3$m#Gyf0kkAu*a#128-bZVY&2T#&QnNY^k~$z^kW-@*N++ z_h6YZ8k9n4If1zx+l>s^p=p-B)25OCv=f5w1&eek!pB41`gXmEFO~HEtO7CDsEC=N zVyL8N7NzgMc`gswI|W+~Y>kkv2A&efs=kXa&IduKXCoiP;E(dFo$QN+bk^~uW$$E7 z`(uur7g>Fx~W)* zGVbiSs(j?yxLnAt;8V|7tXD-G*1S#&<}6|ylC#e3w`eEyV7N+u7Yp)?{|6Gk^3GW_ z%E%~BdFF^9(Cnnjr`JfyAG>ED=bHyz?}&@W&*XPv=;c5cXE2~s)o*SIVsQ3`z0ziJ z?*5#fr%_b`r7{ms-S#(F$_?>hGNbm!tfj)b85VJEJuXqXehe+K$($E?k$UEtwV%#@Q+8KeVnHP~& z;>k7!?B%HG@dCYH%@VwGw&kkD91AZYHsUv!%-#VLrBAwG=eAelMSqc*j+ zdO+QnyKCE1>W?R|~$c+vto`V$!NPm4EN8 zS1oq87E`tm*9fqBXBDs78rvr1A-);yc2ZjkT#tWZw7D8?q(y;-*E*=pB7lP1=IxWS z%`w3z+`Fm14o3JE(IPSe?gD!Nn3$Ohx2ynZZh|*7r7OYOnCBh1AuCWVAm>r+kphY9 zZj&O9Gv2%Xz^=bolOcMtUC%!t7q>(IAn3 zwqZeQsCv!&HOGV=x|A8T%)ab$t4gaaklppTl9Z#qqq053Ijw&V3Oh+FGc! zvCDCHo3}>}x$s(dcv}l|bnjZNkaCo636ylFH))hqzmwc*^8#DPrIdw(Db?B&Tf>Vg z`j@mCNZ0IrLBvV7$2XrsuH*V+;keU+iC=Raktqo_16nz8I=^iP+;yG8@3isMw>&>k#(Ynnr=UPt|2Mh?`wb#ySsKp zYi+FLr;QhzPnS^-mS{Bi>ds3GdZsc!msDg%YM2Qt39Mr_Uw*9}h%-18pzJA4BT=)DV7H&nluGneVDR%mcu= zxiwY6t8+MACXH4flb-VnH?sxpo>g!WipRcO#ST~Evs6M;k1#ez#AVcw z@EN0&l89q|_x-hTow)?}_O1Bl&j8Lp=qZe88&Z31D#9&t#;G&=+zbhM_6ACw%acx{fcLJutsHyMDz@rlos@CyK9f`n$D327hEF(ove zAEQ$k3=q@UpRkK-kM0mKYLWlD5hRk%eyd{x!s);qm!6*h@b8 zh>CLOfxnY0`Aa#UDPKhfZ+k}IgmX0yVhg3Q?+e@JnAlhcAoPaOC`Ws=0QUhzRiAAK z;dxJvh4)LIv0Tk!mrbCLgZHC|{Q+%L!vjT^*AD33*?-5WP>rLg@L5ZKHb*aE z?s3N*jP`)g>gBmMx~#GPC0J<^w}-B#E+lX0gLYKxMdMx{8i-SP38 zY-_AG+#zS{v7$!`Dk@IKi|}|!fj$N{)jACiCEZq*Jg(7$Ok|@ z)8QWd5uaXV<%-iA2WjJNd$ZU2{kT>xU|Xse)+dOsv(}OJz6|uiyVyTDrU4Q(g-X?p z3U0889nOX|z(U$6M6tYbQjZFd1brcMDgye0PW8L#5lP~^&oE{$y}!4|1=lI68YHm` z>~l&gNSMrjW$$5T!-tFV_VayRaG%5i3h=Y7H2qjGPU5lNq(+|a18YVFDkV-~qArYH z1_{6fVm$Smd0DG)z^`T>lbBGGx#M8Zv3Cf>CRR55;_*+EK{K9C;wF)>XjNess{W&x zgJ6nv((6k|v>57Qyq(I~eh2jttjpNLW@oH|q8bLF{67--q1Wc!cA&NLPD`j;hs`39 zR#aw=O8;M8ik2Y0SlbEa8xGQ`*|Np_fyqjrtxdH{s#J@J-ex>*W@xHhM#>#a313fa zj}l~pT(5ndj?z6O?9NZ8Ggd*1YRfE{|7#dpILwlGpusrpTds~~o8r55UV5iyNOLRL#PEs#7 zq8i$bfFEFEtl#qdQ6dA^Y-4X~KfR*{$K;}Wzm8Zywb}v3;@`MYdYMeImln6CigV!J zqqwuv`!gNib_5D3_S&kCK5h<$2u=ff(c+zk=%pYe|J%H{;*DnjL`}*X=A!v(`PTX3 z9l{yO!q6z+PgAyYC=RkD1bi<_KE4^1Z>@SG3~76(NXPRK6l(YJY;Ah5^jU932Irb0 zpOx9CBlAS+^@$1=AQui|rO(nz={ouR1G2FEwy~XlK2ZFBNCPZH0s-QjpqoWYb!76a zHm~?#95Bl^RhqZiol_d*n+M|{vOtU zQ*l&VaH*f)6j)BSk~*ZibrhfK=zpXZL9u^B4JpMmPl4~xWdqW@YFUI_de0aGc?CH=U5+hkh`X|!Z zTiWnbqHD&nn6N0TRHg5HfI@a5K^ByZjx{RU+}a!=xt}KIBLcO=QyfPvkf_<)$2}lM zj?lBDZLFg`ipwpd^q2o%;qtovM4&`W@P&KIp%|J=ci;Llq1T5@(kDf2Ii>%A_q{-Y zdBu)Q9kpXpHYEAEW$b!!0t%)N^94saz4^E?2C~9@T)WqvGVjkVN6o(L!&>aK z+4c&-EQ>ssbXV$RkD&;GIJs4iT(1~M_gJ*}*W^Qn2%S=F{%B;GMBDOWu=Kl}h61@a z3@>Z}MZAg>V{kt+o`)vUy_ROmxbo(81qOk@5|PgB73#~kWDzsr=|@n{B)wHy*g9AOBewFS?9&MzzVK`&xpn{Ok;oi0y#iWW4@>T~TLl zN`vzkfzINDRx_xJ8Pt0djScJeV8pKr*YH3mLuXR?9Ma8`EtG6aVVMaD!qI0!Mx9(j zuL%roH&xcr%AYKrt*B5NcQdXH`BbP*F!2bMN_z1hIdMfhxFLOtYPs+W^;lV`#$gNp zD)qQ99S{i69lL*WbPfkYD>4Zb$>uA4BWzUWgxjh^(D22kejHvC%kOL-3JO1N^OM}v z?av5Ca!R}cV$LxZnU;PVf%{>Y-lbB|9L)S``(&OzC%N*$%bY1WzjP(;dk@Xs9IU1` z8xk*s7W`<}`wd42l;io&;QSx7*F)vZwVJUxfX!Ak!PL>03gU3;Tv6F}N`(`5=s&^! z$K;-)dV}#<5qm%TxkWKLF>%4dk=O*u$??j=E-WB0Sm@xKQ3cEYe)x)000178Zdoy( zq%N!^3g&=<9eI9EVGK49wd23 z-dSZ%f!}&s;WyWaqX*fxGHG{QjsNbd{}(E)#p~-D&~z_W#*aZpc4gdWh^G$hi#Zvs z*Q5B$3^(Eg54VR6sm0ZZ*<%4;8XR@-$S|L3BMf2?*~$`;Cy!Ma>G9a8b!_zgsP7>Dnk( zQmaT%jgkiKiU{4cn&u3sm;W0`8hIrAYP`C5gpHr-+Z4RiBocmFPjo3C30vK-I5XIN zgn!|-BoS@b%LH6IaSz_(A!96YI|j}Z=TyJnQ@pAJ-u_${8QHuKca4kvs{Vh}S>tE~ zPlSW+sz}&;XiTTBCPhe4`-I1#Xn=ON_4QLyR?aUWGn(0jSjHkM`7y&Yyf2?O3geYf z_H#%WGbq?^6bPYy%dHJ4QXmd-comgDl?DER=vcy=(_mC!_+K$F(eUQ7lxOI$v5V~F zL5H-t0+X!7#1UAwtF>Ngqp-$orY`HXPg78DF5yo;lA7+I8Xvzc+*U#fphQcV%!W^E)DN1RX?By^J4oGowmMDM$l(|}`&!0#p?Qc>5@K{3sY~({ zwjWsSGgf+wOgLArhCtmuvG%=Cd+aN|iPr7tSST)E2G3ZqjYb{0r4B}Q`(>_U+I|h% zB@OzS!VAq0xdO~uAPnI9T{JIhuS&JU|AufrOh`o2MuaAPAywi%% zPVsM{mq0mpZltkZMPpimi``CWX-VUw2qyIl2AFAQ5UTNC?52L5q2ksIOMLCijo)8P zhIIB~8NYio={RsB-1$?VEX9Zq~anrg@U^Kv=miCIRt=BkV3pU(;F#88ZjaO=1JQzo`rbUxOd5dMA`%es1Na0u3%c#zm|NkMo`Muv_?3-2-obJT?k|0MMCH65vcENd7^egcR+uMR1Q8!hu@=+d_{B+m5j%=|Z2b5V zm$>_^{RLmAjg40i|`GS><2}6hftw{tRB|9qQUAT zdOMknpK01h+J!j$Sqh0A$_ojWrF$ptx>I+K_!MAWJ-!|{_{iP~do@oEk#!%hV2hGs zlH1q;f=#>=PC3_MxA~?+v5*l+c9kSNc9mt%vZb3zfL}W5g8ky+W7VQ8M1Gk_F~<`H zx*r_W_8*Ih7}rB%)z6oyLg9=ajm2e`Nl&1qiPEcW#c&19f{KVXFR z1F9WJ#89V{ zFf1hxHJcM9mo&7TkI+Q00zAEMAa2aqF%_C5bO+q{l7iXk{;0b&fQY6m9gz|iDg<0K zc8NxIV-ZN)=Q9`njU$;B8#28mOuHQ?(iye&NIB##jr6M1@bj|~ycgEajA)ezEPD9u zHk=2{%l_5a$yHB@ou^0Et=QV-HlrxB0`i)&Qc5|eGVs@I_d*fu;_FppSZaLinQ6#X zMi0UcwEknkn>xqutQ$Fw?bg}w;)+U|vZnn>>L-Cvc#{;pX&6ec8C13KXxMC%df*%7 z^B^%UkB>mKu`e&)_PkaR)gMV>$*u=ie3i&-5IXj2B&#(vdo09|Q7I>!%;9k~%FS{Y z{Q$j{h5f6)DwE8bFUb%XQ z>hA@VAXgEqL)T<|{$221?`{q6-?07r?}^KogWT#Q116fUO(kdQ5g++%{ zts^JyPV-;V<<0b!(}qYgkRH#b^_xwz?%zqcIo~q>)?~Xq_9$udSl+6zj$(I(|9?Ea zWmsE*(zRWn#UZ%6ySoH;iWezfin}|(-HR4?XwgD(2<{HW-5r8UfRA&|`+R@$KUcE% z%&c|KtQ0!9p1*YLIhvwE?!0@>8wEReqM&R1vyqewJ^bQOY>v*CnaLO2m0AZ9at~*w zlc~VlUV?><$zCvrM~_#$Z;v|Siwdmj{83^@#!kr+if+6j^=hTR_rA06{C)AQ8Qv;| zO7%qqOf`OZGw>8i4w#K5H$5tpWLGU>IsRdvdL(;ti$29R%&+y|ZJtcm*IvR5cK=%x zHrcHS&V!Hyk)!(8&4o1fu!rA1`t|j6*LnEp{&x7JT*gKQ{FkrBUj)DGR?Uc;N49%_cJ=BBqvIvdAM27A8#Q*n-k4M5gw)pyS%Rc`&>#3=K!S=c|fmmu`3aDAZ8$ zQ+p~(vG9T9HBeIbaRBRbqC1r=pk|tRdhZjCaX-?LcG3Nf?irQny&S0?V}EMTU;0&j z(6(%ILBd!69S7pRE^`+btPrA8epqTvu}N_*Z{I8@#zKc* ziJg@WRHUx*H6syd=pv7dMGVb>=iB-AbIE0*{5;lB6;*tzOX^#`vY|**ZA$t&Z+9+{BwXd zK}@0J>L1e?D=+3&sX-M}YcdNY37dG=N2#9!;VNuAScw8MAo`ywro484HvY{znsL`G zLms_tdl)2CG${QwP#Z8{u0lN`y9A_5udWWM0&IUV@#o`@viF2C21K#LRG%=tCHvKt z{mI7~l9PVTxWa->opGE@z>%zw4)_a|%ZXW1_)H?t!P92!c00`qL@%hW0PjY>GJKLW z`qUuq7}iaPJ#PK&Jpk`W1jc!W(QuN>qCovqdr-WMZg6(C#IIgLHu4762+;Twp30U} zLJptXq+A_8iFKy@1OR|2sXK=NWMb*wJ-Fk`HW4>lh0I!-hYGvj!J6r>Qdo5QA0NE^ zI$Q2UieNAa`=lbj_vZ|TVArZ|Us-?%GgIDz!!_y8M}L`*S0S3})}gj+6HC`97k`U7 z5^Mx*ND=U2V^vI6g+l9?fUz3%n-!iaLP7G|M@P+p_E8_?BykF39$1&Id;mA)4!a=k z6S2|p+qO^_yTuq%RA*sNQIlTkBBCNsGi{O>W&4OfY2Pg=3^`c<@|Yjg*O|9t7}{iv z9KXE&=*sl_4SvhCO0`6@-4qH?sNOPIjVi1oHOUGLr&xEl2qDGoUCjTjeh@-p`k;xC zi%m?RUE59HnLE{WwKx@U(&D#-&U=AC9T1fXLx&(z0YSP#?v&!~q9rrZfj^5rs|MSg zJU6=)19Y@P%2lh}qg5+uF^OJns|4c@Zf8=7UpEg6nAd9319+h)!wHk2O0ojk@k0Ey zz2O}7G4*ZRg+WoW%^633HGo19nncF&AzO)VBrm?r~f(1$>ugf z+`MGQf@Ai|mp;t2Xy5>QnBpwL}SKTrIZeuR8Jk+t;d4Cx_V#vQ+PM#c(!9C+`bxR51~CM> zBV#%*!$(119+OSRKAOt~Q^MuFUxXJ88dPg$5nIRu%_JXubDa)9FpUg>6s4gHT>2y> zQ;^*xOV19f(2a5!F$>GWOLlqKO)`5^PWtys9@_RG&bueyV(Ce2?3;)Ece zW(V$^9XbbC@^5S0aSA7cI-JxA0jxJQv|=Z3V6eeE#q^w$vn-{m4)27YY_p~1x;_2V zdPzdQJLyCs`Bc81DNj%T1A8(07Ir)KF?fOhoZR7VB67hoGo8V;HbElwN~l zTJfAkqE_OJu%CAuyeB=_cA%IrR_Ti+{aF? zODWMur^mw>nPWE=Zp+8nL1_Ht!OwUn)ZQWck69e6hpBhRxLL?6Ey-?9Sqg#HCTVGq z`S16#rM7^pzu`XOfojmV(D*YB)f9m2XMa+p&Uhx;-CESz7c|OiQ(*8m@?D)j#F!9F zVq9vB_McDo>xB;)k1_F#kM-Sn1c?V!_n|A#Wyh=c-H1B+Z;4gCW7X~_^VTNV2;8BR zY!9ik#*gnNXh$)*(Wx?oT~8M;1@mgybV|DWX17>eCL?>N0?tCMkh0?|;%3gN(w5-UZS;h5S7C3<>%foIv%g)NL2)$;LrFl^}f%{W0 zf!xT*7ou_=Cs*Ffqy)dd+<>Peyi&gdrY1h#F1nxT%PSulE0!-m3M-s{7G5@-3T1xc zEK}~D&coe-&6`^Bj_`{oFPiSGcZ}i~@T3^(mCh66`fS`8QOA&0Iu9`etU{TtlPofA2ij&8(7pPjb7`O4!ClBI3?V7 zB(pmc26dI-pOHfVL4KtBWhu+v|aN&u#k zN=t;0peQ`|F*=T!vL&RGl|*S6tUOxEUjG?=@z%9Pl5U$T%s)0Ax6bG4TWh51MhR&u z^S7~Hr;%@fqG_W1WHtQK%umMKK)o882cPCc8h7TBP%7l<$^O-NfszGbvF4~9co;29 z?}*&rJcZHnF=t2L_z=o%$2dq4uYs8kc(|LVu9MT=; ztK^jSUCe^n`Ij&Gao91!F=A8B=9t0;A|Jf)va@G~gZv&Pjl(K&m4J ztm`N`aM5RG@J`|qiNIFTmS6?gz5t?0lKB)f3i@<(ljRcjj}oR~~Mns?KHG!Q*^v zX&f{?Z~Y4#yOnW{O_xnejrH3}baFhdh`uu;%7oB$d2!{^qlbWM^ zYAIRj6F4M=eyIvjzOO6aM^dv0v;FA1At284U1RIUs1NPbM2$iW616Ww1zj?lb0Y=o zB7gMk*z@jq(34v&fTjW+*tqi!C0!!eXl*0fy3knS2r*>j4Tjy<84)W?k zG!t~;o}u$Za&|Y{+V*1`heZG(-74TrEq>WxwD1ls2!#FQBG=fv3*J7erQT2>$u^-QF&h7Uca8N z!`+%kxi9~~6G8pmbv{e==4SIhRn!volo7^grMe@Ba;QRHC=Sk`GU=-pQ}Ic^Bcqm+ zn|png?J_gF2%IhptLjxy1ZRB`_jWR7yjBoyzq{qZ#%~az{2by3*5YVx$gZv^mpe*R zRG#~n`{i9gB`%S@0q|VB#fwV&`5BsolZ$+Nm#O@NVX~F(b$Cg(xc-v~f$27aEG2hd zuc|zLbLhghc9Aiw*dC8|FZKE`WAM?R7^^+p5Los=^ej^wX$p{EKkl5!(#*4J)pEQ{}p zz9T;Ux+8HdJ<0tzF(Kh_+PQ|#oaP=An$~-XQM2yi?c!7S$0u3|j5gIjLUZ!Dx#L@& za(Sbpc|Rkvtan4~)OpG#>xFF7l8R<~Oz%Wpd_6CAL57oX896D!DE@GXtr078Gu*iZ zo7ZON`<{v@r<(f(+|#kue9r+yh|UEHU5yy%u`9?uVWd;;=H{y2x+0NLMxQA*Gjewn(hRlk|}uV8}4mFWWmm*-n?n)5j^Zpx$-Y?yI7g{^H_Y z?E%PyYuuJi`)KM#+I|AZSO4~&zdmeiFLBx}fd#j+VRp+&V3m5Yna-8>uWu@5@P{Lg z4##)hLsMt#>DLLd7x(8gu?IitrY`cQbH~5n%R0d3cP+BQDaWE-f%g-Ge_&!K1?g(K z5!0hd-TE6k?5OOlUy);-0)&oTT~*%!U2MLOaH|yB9kHgW0`n)`L6qG@I`n}}2VLeJ zdV7;&3X1b(L)Pkcp=+m4s)nvLL<@&Bfp!*EPsk zxYOKo8jPbCAU&~v*UNMN8L2<-_1r1iYuog-&q@*q@jiBhO%e+{w(k@T*{E?XaU#Oz z`H+UE?*R!+@G!-StrzP#I;R*0t!J}mR$i$>6mGRj< zAx-}7XU-OUKPB;uRaxzJ&BwmQ~5=BtJ zmIT0TmyI!$crC(MXQhqR{l^zGTNW&jMbxe~L}ama!l zyOgt9h@k#%(NoT{2zYbx$3SmQ?NEl;NjF7eb!bZpRSNl2Z&Gzwq@QTuC`Bnpx#97z zm8jtaYwlHn_F&nO?bvP;o$mil4q8N)cY3Uh%Zq_?N)3C{5noHREy~Vop(`$}F%tI8nI!u?j8>7 zDrP%RbxGZ;%`$?+WcdDUq|hYtQsU}O$6nQjOy!dQ3cGOO&0EauOZ~5(6+9`hBxLuv zHaR7A(Qo9u<%vSgHPk2MV@?SoYNxVxKZbPa`1QaIIc=ys{>a@U;ex39C8PRji`>c* z&bp%9AgtlW>HBT$T>rb8bW1nID1aH+XJ3>y%TpmeJ0^C?OkqnBxyR{tmzu~>(0LHV zDkq!up^tm*k`1sdV~JujkL$iEwK~~w%KP^uSIgo}zS~z|{$eIeZE8PdY1^tBG|tmY zUP}NE-t`k5PTUrS*RFH-%A2|?=f zAxflmMj%~0#$B6RV+P}1D7WZ8<$cECGU#;PRe!r|E?h~ltub6G<+(>1=6Yr$yHg^) z@37*ZI%g9DaoOM9T}twBD8h~AMEZ*#w!(IKj)Z!*EakRx;47-PA0*4tGj)^FX1ZO= zCJMszqkuayxlZkU-+ut$pXlrb1*JtCl zIl|lN4-uB*p`)=*U7_p$thg@^8mvO~yOf(1#sXsWoJ2@@ z(?IIs#6OV+-F^V~!sc>i(qO_}F^ndU3@_=!eX9uuCCCwqu0Fw3S`;2Qa!NBA@gQ+e zGk<{@~?n&>q?r08TNey<&=m% zWfx7^k+J4o`eNn-u%#WbRGr_Ixo|?q=d8n7`L@`)KLR)pYJhI~>#~P;A2pzy`-Muh zxoln6WzFi5{!U#v2U+sSOIrH3&i##&PhsV^w5u{d{5=3D+HqLSOMN9fhsWnpaJRv_ zdrS7PI;<4~53+!w1XUa_zzAFPJYM4C0io_m~`DRcRh2XE?mj>?p*r&kQJ zBQVeJ>v)IEea+7$RyMA2(i(ee`31hc2oSnEA@1mm#m+YJ9L?#fOeKCD`cMtc{eDz( z)lq##FE*v%$UIGbXE6`L#pB}P_V2#~-~PJNY8Sm3^1W0i(C51h;b+?jIKcb&wvq@+ zrNznp5_6}RHCM1qbB2Gb`M6LD&m^^a#)1#h(Ll-msDSp9Xz0?C>yJ)v|K$yxEWIQS zk#RFgQjU~omGS#UXPPhY0-^mCz-}rt{mVq-_7!CG-4vrVk0EU;#OXhW+Rm=~N%xY= zIWsZwjwLWUnNXiS=GpW=v!xd&d&b(vrK|dhAa{N>5xsiuG|u=OOsG4-&f(%wgfcS2 z+M@L@)Y;2o*THO{d2(ZMZTpJ3w6w3z=_KZPVhcvyjOqnRWXo{lJ8VD_X`0BJ;dpf>d9h_QJGdn+fe zj_e^&t2g<}BO)!@dNwU`w$=pqbr@UFg%2!O&yAV5kEGsCMC4Qk(`>q$GKFV-1o*qs ztmUcli;-=$@7zEgjXMs$CE9>OMRA-bPg$kkeGahsMOsMeFatQdg#g5OM%reDRH38BXw+I^?ZolN-wdU z*Ks22L~ax97}fE>AKvN9UiaoHfAMg9sQq40Ga|Lk5(i}qg<+a1WQVn7lQgmW6X+ z6xzgS$2VxTLMb?SjJ6^cZ}7iI2_eKr4h5nrB7LHfnBLH~&Y+)>Uhg6IW1uhO1ya@1 z)adBK??PL8B8Q;m?ALY7J4}0~@+A z-d&Ky4~F_nVJx8nNgh6v zZy{sGTo`$bQy0?8T>xq7b`ezPS3=XTS66tF(Z;Bb7or%noV3=y%naJ5=eqk544shb zTZqV>l(&j^3-&cQxii(D-Nt*=onh2dLG|Cx@=VuzuxU#pmpi?J<}u3LB8M`m;xR=E z`GdaDAFoG6)WzaK9scULL2V?bY}UcO3O`l_f<))7_g7MXI$3=@%Xc#=e6B&^vmw_CXRK4dZ14%2d&P8vEYjflRag~A92^i|-3f6D-LzZn8(@a|@g4VL4Z!!5gM2}wBo4d&WHQIIgR+mpY9bJo=VYEs!_@uJV7V0|FmbfOsU10 zI%53p+x@pZ6u|I~TQO;RhHhH*A$5%3d91F?TJ^%YkjR0Ty)Z*Dqx{KMUgq|xgQ&=) zsA-e`wrR&6aw8snH8Ttd^X92t?M(nRFgsb00PGmv_1JH+Btq_}aD@npsEV+uv&=G0 z!@YjCZ_U4V!Ipk)tSUK;>Zj(7%?)Ve4AX<`ZJcZm5fhRd>UaegE+R*;IwB*Vl)b4? z5w~dFLSrqsQ*K$1aACNn+2ZD0lqM0wHgWS8m$@Z)+=Opr<49fzu|TffP{z7BqZ|{@Xhj zRYx}B{!pa2`wUNkky4T3!4zQLd)VC2_8g;dbM2bZ`74FRtaElWgelgCXW-Up4Ec5W zK2b=wJ9V@^wp2taoT2DHU*IN)*~s2JUm3H)j7Ld|5Z}!xi4G~)vv^df_aQy+t?&WY zk(l4U-u;B1o>Uwx1LrG=x*nQiP>g+)1{sCyVm5DkqFk%KE>I=xmOFT+{HJW#vsGJh z!~Ib&G-l@)(R1zgh|QxA?mV;_!iJkUs##%d`_XOgEeLN{MM$Ur&4)Ylx!z5GdIfU? zb@{S8>W@q>{ws|mVaFKT_tD*>jJFio%q3otnx96Qa(ek(A$(Apl?8#fUkH^b_CDyJ zh1?+15V{us+U{@YBGS^|)0!j=!qa;~Q9x5pfhAlC-SAX}sKS@C90&~R*S!v@D`>;Q z5DNcGmB$+i4T*q44cRVu6}g=Cw*#)8Gve3srNs`upqEWA4mdRD>^&Jc1A^^k_jDvE zpU46Kg)#&_tOL|X2EJr>hXvF?OY2jRMN{aXCt(73obbKgUCRT6m(%m3$Gzdr%LZ7Fy+rcvQC z?OAiA{@#jH6`Pg!+=N;V$gBXw(7w8@E(eV_Z(xx^aAJs^V#1lWC|NFxz<|Z%_(dOZ zU&GPzt|TH5X6qc39R&8j_VG2Jl-L?P8;Yvu2oB@K^zM$}E5$9uXr;~*IoR?%{Du&B z`NJTId{}2hOm!w(_~{FAmj6E$e(;jbk({U|gw~F6&j9gbs5ng2kLMy5N%pZws0`UB zkQdy3dv=*ito(utf+)RY943VVaX?|5mqHdje7M5hCbmdqhCxyz3%Po^4Ux0}byWQp zpYa2=I=WpzN*$Bx7A^PIsp79hpuY}0|6B@zwl5f2>*i^_=(Yp0I6WZB`z zw$-4Q6U^ zHvx`a1{w$KdKgDUm%2}xDk<12tt#)XM(y+3A~nEF;}mlc&_^1Voa7YjsrV}Oteyt5 za+B(Y4>4+G`gPnG&Z|ba?V!Kq)!TzlN5mbS%*$t&zoZ`Occhna6M^E`Wu=WBPxKQH zND(_9A+l)+`az5H+Lv&Kt5@Q|JV>43Bm;^v3Pon7Fdqw+U}iC~_-1Y@L99kd;ct@i z$!d@En6dHZq7SM>-0y>B!%#sF&t-aug%lZ*V&lLFx}r>Q+G3ZaBb&S_)3c{S8h{B= z4VJ)5?3MvfMK@p)sx8XEWIOL5TlTbII!ty&R0_6_hSvc|(407ltGzbG*OLTaKDQ_( z?WXWehb1J3kFLOkT0BPS!c$_r)gje5^?l$^jud+#8QO)LTFsk9L3p4#$RQJnK@LDq zEW3M6V&%6uvwG9Ckjknj4Ed+r+^)=I{68C7d-lW1zSRa?G95J3EoUGEbHJ+*Q|Is$ zWh%W{YBVPoli1Cw5X zfI5nVR+s)z6qA#R>XKB{O=GNnkj7)kag!tT@%+5kW$slyD`Ug4jL0?I_)PK+==4cw zmwsf3Q&)DYXoDP9sF#G&3h79)vhC~yFv~(>w zkn--)dyl53e^k8T`b3{8qYAkc%{)aX5ww1LQGNaMd3>y~&In;=h5cnUVfhKd8Jl_h zHF)G@6I#ndQkbt-oy@naS{XhGbRplMbm3YRvhp0fvtU zfJsmaqJiB_$?ob35fs5@I;bS9dIDVZmjbTpfs3y@Ej&M(MVy=*0_N}9@BAce?tzF> zB!3mDEt~&6VAmw(*v{$%QCyL#9jymLxyjEnyyi4J;f%;p6dL#FF||$5Z2#*&^ho>& zYKpkf5s$_S=y=a4QF0j=)y4RVdYNsc1(IZ0se=nksDadh~oKDRA~i&pjj8k`Fbv|7V2zVRM>6!ioFJx)1L%(VMS8az1O zHG}h`(dX=#YgU#vskcK@yF+S2z>e!9!tV&kIh&Of`dt&2;BDJEy)j$Ub#HH@r-0z? zVukCXj;(rwlJ5NBQnei%xvZ+r#10u_d#&H0U)8jYd;Aht>6C1;c#Q< zJ)ZWI+cQ>8OUNm|?;YcUSf0`M*HG2oXP9FR`w97Q>B<=_Kn=#kD-I%t$Q%mDEV-m| zR(FsLi@Bex#QHM(&NivZ3 zOm;w2vVoQxFZ==16S-iT=xuv1r>@uOjaSZV9_w@LwpIu!9ENBXGrbY0|50)cAt|VOv5joTWe(0b1v9>^h%I5mwwHruI=@d-%)1}8CaaC! zpyj9RJL|un5U*vA{d2wT#!R{-0$=h1=(FTy{OJ+s>LkI(-l;}h8QDTGv&|JMIill=5+@PCr$Y8{cu z^=J&6$#h=NHx^k2ISkSd?RK61U@iDZ-mJ@q0``>*Un#o@f1}(4C2kZ zxW=Kj)Si9nL_~It_SvXosJTIqZyp{Jqc;L==15$#PVW?WOyud#P&;Tl$n4aS3b5xj z?1}bRvvZ?0!9|o4&_59m*5|hSxi}Ek#w9$GU~teQ-HDIZRA9E z|GD)x)`9c*-|W?&A6ATe<)hJqacXQ()Xh|+y=iCqj_W{@hfx6xaN_*~A%+;P7F*Zk z=*WrYTtRG_2-X6uXG#Cntf{Abj2k|?aoxpBl<}W;{3)u`Yv) zfVj;8V>6qzX3~pn_za3uugI^M-V^{z?@Er%h9#QCx98O!xjL#K`hS3x57kCv;XD80 z(7Hq&DS|gg7FgLpH77DTe|W0Fua@TGEfvyhh}li)uE=tCbK3dqtltPPl_o(!IIIqS z-#YA9+u`cBnk_kFb4bikk}c;xJS3@T)XkaX&U@~%3*zEXD@Sc*!q zy!Dk=L8*gS`bX;F|J#+A6O+E3u2tiUfahxH_KK_f8_ar%6tj)2qiJ=r9@*dQq?%+)zKKc^$08t41-c>EZ+#0-SMjAmGZY6+ ziE1FS3g49DiRmUPSr{-rmrzfpmur$;^XM>wuyFTKKwTPsLwJePGN1w5PftUhU9`uj z-ns_aO*xjYC`(IS@%O(@GkGnH5;3pdqm_eV+~Byxn^DzFByl{pWX+GMf-;!6jvbY9 zq`#@F_HpUbBd4=|q>`OTaQyhm`7o484ysk)?CRsN{(3d(H)6zk<|{u>BK8_W{ziV9 zgy7Nr5FP@q?FoMFhPY`FZ5=N*#ojKNm&6u-vynOY!(!IBHMcG{5blQdJkvAFx6-yY zz=6$%>g@j#G7tAEk{wiSzxtJ7bdx;sLFVl?4LmDLO53CzFqhsir0+6g`jF{7gm0em z$esCuvz?Fo5$S}4{aooCz7;5F+C2w7ARV?;69z96v&9`y*T5)f-_-O&+x!%Q3I+qm z#7ln%gEe&pYl^E@fBx6{8t=!7vl0*RFUVVidFz3Q?v?~0`AsJ>qIyAckqf({EeShA z^H7a%P&PPw$gL|}mFgEW8V7}U>-o)!$IUgJS_MUC!N{O}&i)^99N5AnzPI^{(6dw1=LF!_>~pWRfln zN93U5(tD7KSfR>I$7n0v`Sgq-5u0^nQLX@WytRz4j5m|c|EVhPQ|0n_piSA*L_H)- z^BEPq{9Qy&m<-a#`26p=;PZWF&!d3w!)%er^w#$yoB`AIFVgtbM#oqhdE2oXpkR)X z!M&*C!OQkg&gJb?Za&|0j>cdCi?|qkd)RuN?pC_A#v$#lJZ!-4C{VgUB zwNKvp9iLKk6Yt|*?O7~6OM`jO7LgR)tx^cXDEm>=V`Qy(%%5G?6kqWYxu05UdpFuY z%0z+ejH=+jCQ^;5yY>eMf2J{Hh}iZF?Re~cYH|Oyi64C)C6oL(Sgr^2M1GHIITi()c<{8A9?fPd+bWyYM zrI@>AsFX+J8z%8Fd5@QS-%~ADK@F%2K(o1p%E2eI4kwG8R3!2l;N7-oT-i%0XqsgF zP`^=I&Q6Kn3oGU=J^HY`9qZ0Jp;zS$vp{>|V92%xV6kEX~n zI{P{UjtMna7db(m*?aNb?pf*fEzo(%$2Ls#5JJSky>hfuB>r4<^T-fFtH7OH(xw19 zgI{sKYQvh)Cy{5y|E9H5Fd#g+&Kc%;T=<#QZk-__SkT;SxT0RH?j3YRR$?WIU%8Uh zIqjp_X0JG=+Y+CiUbfkD9mTd5!o)vq=TpM7c7ucePmZr`gJ&zJTY4 z1tT{K&99Q-q}!FAer4{rD$DzE#(cRcU)>H`n^vZJH!OC{db|1r;Kcd_L*fS_`j@U7 z1>EP;YDgkX0r*I^0!YpnvVtr@m2sQfsI6?L%U7;&(nXcf+f;6g@c$jl24t7;PfgeihIPGAyBI=k^pSg4g$3?dMSH@>c=ZNTLCo?+slXACmK8DtfTqoSR z{A;H2Qq@4hLz2hz2BY1c*Fb2@2>B$gLCk+C(kiWo;P(>|6f1|Ts?&v&7#=*EtXzYW z*IrwTJyUsKf6kcUrA~z~QRw|J+ZT}EG9#)~3q72KEep)CTKjbY*#bRZNekterh>?O zq#09?+iwodkv`MiE6B)xk8>tH80)`lx3+SS1sI|Ddtc)9T}=h0alM zG~_g#3r&j(-LL4intMgT$=feEiWk6@52%}d6gxc9C_-Ug;@a_qK!OT1%#MR$!9*Cp z{A+S-!ivP7uJ{E_?2F%YYI45bI}?+U)Nztbb!CQROUO&DE{s_nebf1WXmh*NJI^mX z-~l6QD?>eDlX(k-$)1uMn!Gbo(wwc#t9D6NC+b&*j6$1SVQXiF(0b#0F!R;CJ^hgQpwl4b_{2h{DY^zGh0JW z>_Gal!_te}?61hJ2%mphff-@AsGl_~{w^_y6CMi?UU#8feV9<2T0kSgQ!pwGN|Kvq zFaEtc*Hi`sKFAQ{_13foSzg6+Y$<`q{3*pgfy$txh=W`QcFmdBjVD83c$wMocNKgj zM~qU`ad%@WZ#$I+n#2>*%RZtNFW6_0FLIN_9c7g? z7`r?|aV|NhP;_=o{c_^g`{!f=y@Qd5AE!45|q!$Keq>$|?iUj+R^ zp9ph>+Bt_}$cYL-J}C{df6Bj|AZr8~+%6SqJ7%-u{k^Otemo)`aJfOVf(v&-`>NyL z)mrD)jxykBWWY56K=v2d42uY9zbV=9Snl!vp6{T__L3WS;lI?{IMJ#xsqH{EQ0uz zSaTKoT<_g|MfH;}D?&F5l!h}H3vKJ*Zeyrqr{ApPi^>H*85NHZ2ru8rCps;XS@DrD zn$dmODaV$Jzz0+mv7z(fd@i*X3=uld@1M!8XnI8iH{hLZZbnoQj|NLx7SxO&rsEjA zGX=KdaF!Fj0C9OeMjhzjq~bZXGoh&~^AyqOndn?xsMahi!s|Xx*w;Q|>RSLyMr@s6cHu z#6YUQv;GiYV@rG9Oi8kf+CIfuW#9TzaJ#w;s(`=Oo-DTApSJ_U=tJv0)Ni=UOK~ly z@sFKFX5Hs>ql>|1;;9Mf;6A2;1JWTSn8NQXuj51 zy3r#Apww54Fk?+Bwj2>RG!>B^goAy(b$p3`83jG(-57NGpTb?t+cg^z_)mu4*Fx3cVrt#fEalM6= zjK&Xe>sF6H%G~1hccmnzaP!*USMA?UVNP<^9sRixOuU3cOV+umVu<5xc_%FToBy4YvJ@ z0y4QK&TAYc29KwOdsxl0+;%P1pP47tCD9-Ii^Jo+R4p1c&bgOnY2XQ!=?szYB9lNI zs<4<;k1|t{*hY9wI4chmyc(8Kt5=B@#Gl#Mc-bz6723Cj)4@M4w@PNo2p&cjE72BP z&U5CiSPsqa*)faTWp$K8X_Ogygz6%j<-hSlRYc}B7H{_td>{)NM#94`9gp6`59y^A z9BLfI#C;1wsEUh7(sGFlCn`OArpjp!UXOonJy>fau17=ag+q>DpWw8coak05`Fq}( zh5bvmtS%OTSAo6$OvCI?gE4>lb!4VNpB<`o7vzWtT+58P`lxzF?j zic7DD^%9SmbP8;4inY@Yila*(Mj{MS1ZMV5E@E}MBiHx-LSsED7R6@1|`KEo4J2-SWliY);0c(y6>Zbd_y7#p18kh>FQfI!aZ)>A4xmpo6gnrkcpmE*e7%@W zSs#cnzuBR?;~8Q#5cs&W%Be4M{gdsCmcOyrhtA1|3e~$dT%HD!7CVs?Bxzpq|ZJTC>dG_S0ZWPi8}F8EqCGS`WO~hu2$4Lx6!DM0ETsI%tZ~B zF=hY(^z_JZf9ZVWDluUK52@TC>8XLScB^BUm`zCkT`JYvi|Pv|^(Z3JG#z zFdP+oCq7xMQ+fIu#Pe-`LGaMmR#HhAs`dQF^&E%9F*kwbcuVK4#dph#>w;bR>dRFr6>38lRfttp&N{-a(pdj z&GnyszZ0Apm;2`0CIX4`a6U1XTwD5JWJm2^;gsgW_?CHIE+zOaJf09EAi?u z^}7xK6&gFJAGeXaU4_>2@3F<`wTZ9E{QPwF^)ywq(V=+-csZ33w|P_FRA2E%1^;yF zMxOE9Ue)wNkL&0*(mC|lKlGEvG~(d9?mu|m58w16WEB{J>g1s_c8VOLF%@u@@43nH zSqOco!I=1ENR3ZdV_(Bl7388aj^NYr9DC-sZcW2?N=~4FPZm)e7Z4ect0qr~2ch=E9F-!bH`+~c z9iuyG3bMcWyMBAG$d^8d#&Bci>ApG~qa@R2cd~z*Yp@k?v%6%M$(-Ki!S1R`FNFO0 zfI}foVZuzwQ;(f6?t`;iQ3NatbSnbit)|zhaE_f=C13;{ z=YWBjMF=)EU#ojaoI_AIf|9tN=liQu>aeul&$^GMg{G`;XE^)w^A0=EQS27*_9X_o z9kn8Lf8GF2z}7&3_i?9?0+GMtMB5b11Y51n3~(pFoS5AJApWeTNCN_^Qbhe=Zkf&L z-k#%B5UPK9y?+K~*{nJzyjdp?HF}~~!Oo>+1&wHNU#2o*VSCDG-i0@Qb$%IHF-nTR z{dvai=d=FXwCTh782>d9cgvMEVY|OkhUM5^3tsO689OA?F229b75w#;l>dihLdw5% znGmeo&2j&Tv+X@bIo8#@cl=5D<6&Ag{3T_JFZP{(Hqg{{(5SW!C9f$dW^y?UEw6S| zzY}|BAE>*@^;V|dY5V!5}7q&$~CT68(n;hQ5#8N9ERXF6mt7aIpxngXo`miQLe`bGj{L`sIkETYQ`Q26$V1+?tzc9C)TQaQx z9t1g|rLX$`(LAR(qR7r#k#$XdBWOWbqoKXmF*DD)I)PM??e~xiuP4kQdvvE(lL6-M zF3Hk^;Tv+QWSb)Gla$s#Jaa!+OzwG;-JeB4n^&fxCV6x$5~Pbyp=ln`a;^P~$qDE8 z<|aISXI*5@=KR9^j+T>#1C@E0>M1)l2Q9x%Qf%4SOiEWN?L0sjqdIj>t1LjHJg@Fn z=PR4ZKErwLKe)0jMs)|t%WMQ#1N$3)c2ERz)L&Qvilx%4pXStMm&zR-7O=mttW{CC z^dfUulZ4cLADLVC1~}LId&^F5a;bw|@ttcCu(tk*3&kWxft^R<8DncUOhPzv z=QxBN_s3+idU2~!;4`MHPVBL=ZXNv)N3rE{5N9oDSC58Pr_B7}dzH4EYJCK>PRKyVpVgD+bdY=g@G!HE(DZrI0-W?W8+ zVubBFLBs>oy?lW&0|M0?N8La_zw|DlRI?LbsK>~Lj&?ZD$IHz|6a_j-B-^Wp=_e=^`-cZDP@z;e=)HWd^0@AZS>wL(B^hd&Z%C z_07Ma+9u<#9{Gv`%EA5UX6O0M8-2*?(f`NPTLwfG?oq!W3MgGNGzdy}N(~Kyk|H48 zAl+Tk-6b&~EgjM@bT>$M56u8W4KtT>&imebzwP~b$McW1e(MzYBbw-3(UXbsD1{T_ z_1+=#OD^>&OF&2EX(1O`@ggkif{4l#Jbxb0CZjp@{l6pS*rMO)DC`mvkv}o~bB@cl z{tb{O=I3)C(`fhM?f7qQVQK#Q=z3~WUWVe$M^Y5Fzf7i2zaLxdQE+X}?H~Z=K|-(o zdnc%y%LAUb#C;t$IBBA>WWVyWI$4xB2%VDLnGayX=uR?-o3?Mf6Ua3$KZM&7qBlS5Ua7dc-j^8@?~ZYdxI9wsmYjr| zi!KE+?7qo}e*--!?haZvfV}L)-q<)hye6B-mz-nR)3FW?0WLGW8FQF@6pz-FyK?v4 z4W19*6D1-d%A|{{C-++O?^x?3ELX4kA9!PYDsWj(V2EnK2Rk|}7o@T%Eb{s8S`|Po5eCsyRI`zM zOg_1s1La_8qwC|g2E7Z7)}$(HN?xy9Y_YTeU>R*rvidKs~EW5+lRQkoMg0sjm_Pyfkqbc^Ji8 zfbulpC*CK9fFMvh=WGP=mKM4m_4L~dF$V?eV5XAp7%uogp4vziWn-m0QiVcK=m-B< znaA_&H7i1{IlJjGSg`lmOu(=IVlwk8{dBh+<7DhuTTWC=KFD0}*JAVQoq^YsXWVfe zPu_O81I=X%2GS?zSd4A?FE&Eo-RRC(mB35dHxnT~E=Yn$1PDQ_yx%uckhBuG znUTt;GF9nqwaJH97jiTN7cZq1e`hBsF9ZLP$ z!+v~F`EjW0+M-XbCES2){YS6RXVk~m{@iFf;AM*E2Nksfug{J+`Y-)h8~HF^w^L!R z!`btsrBs2P*Duog)1NaUPG%}(CggTs|D8!Cyl{VR&<}i!LqH;kr2KcwjZer`4rSlI z3A=mo;~?(b(0_>9j|^re7p!R1``#8*c#?YCA-i|)6z*QLOP8c!!QnyHLUMC}aE3T_ zC;y46UCs*|ALLa?G6RWrG&vJY$VRQRS|oG3ns&M)@b^*9kXGtb^xHUqM!oZNujc#4tv zDT0~1Z|&_G?cQR$Y(n$q6A|<$jj}jqZ=G0lJAA%U9rIwjaWBTr%sS7Rh3q)bg?8r% z1O=&$?LLpPU3)lmP*sqrmg&c*Pp90IG)kD%K;bc zc#O-*?_TLQ?B5wz(m<>;X&><8%}aXxX>Y(=DwpdJ6u}2fPqKc5=hp7%yeL@fiCA#& zd;`if<(sCmUCifPCiVp{Mttn5hm1AZl-`}1W-;pQaF~}aGw%w=TKE2yegt^}q;2EU z%aW5Zf(Y;^Xsq&4Z@gucXNWx;7%{6Azf4$7#5Hq0ThnY?Fk8KJpNNqVK>o@vv(kkQ zc%Z-}zxn#!MSxYzxl1&JnCdWxr455ss+GqFk9}RMK*2ez{aq}Vc!f-SMoPd$6{XmX zwXVm5M~!WB{;VU0M?2{TzO@xgYl1NCPTt3hmY;jY;UK^92@`b6cFt^^xSKzQPiJvpZAV!NU!bPD1Tp6JaVUGV(Ip27ieEz-bdYy-E4?hqNyD} zzRcI5VYUg>#vSJV0R4&FEyW6m$P1J?+%^0q&bWNF_MD?DP3CC5y6j}}?8zylAG{dk zl~Tta?IF)N6~Hm2z;^KCYQ5<-)Ma72?2Bp3O^p2onMW(g!gk<7ad(!zzjBR+93Z-I zGL#Oo=|ySpTYv`ya{Bu_E$JJ!M3ET$3P`@5%NNh-9#KsRH||D{ZIy9o`o(zu-5)r7 z=g}=;i)H#i^YLz~DU##VCI{d1>Ia0nge5NC6Q(M_A?ACif%ZMSHo;8ll8@D8Cwb`T zR<;6ZcC;u9^84(6Er#!V77FeZ_p0%H?+%zhF4N;0Ue>R`l&_vc=|;fo`i1A|HTjtFi>oN%KcXcVkq5%F zv~Mu;yWwl^>>rGG+4$owJb4s4V`{QR9qhLS6Rp8b@j*MQA;tVtx^tbZLsPP z!iv`a&=~&9qjLKZXxGv7_>m#nY7I>|?iMFkCNZiZCfNArDUbtJ{YxdxKV!T@n3(b8 zBpv}6k{JwC=M6dpWwqEw4}xv=T}WM9x2D!;dOB5-dx*|Avy7BBLbW=cua1zY4tu2wT13RD@2R?XW9H5 zlwC=Gr5iCKMc}pdCq#D4pYkK}RhHlNe@=?PpMxaN#$T07o$#yt^OkS~Ml zE7TaN^a_+qmG65Sk|~I|2SPv5V^YTYQZ(8%-Ecw2CnuIOP32Fn6MNk^T{xE$Ee)Ga z&sXepsS70IfFAj%30`7FloiNP_kFnRV=fKST;yl4%t)M1*Hfa^ptp-11(;*z>hJHB#W5(oq#u0*+wu0MFb|%b!(h7g>cWq)^nlIg6V|84H&o0mE=RS@6AFYFkCYtz! z4b8c%|DrPKiVe~{da;5gc4=!Jbf{dAz3eOgUcOp5^qLYex4`uk9cA-U-rw}7ft||h zttC5~_BlhE&nX=Jx-F;!Ge&6&i>;uY$b_KoEyi7X4#i)Eqj2nGX&FB&^@XlR#!Jcb7-a+fX2(T= zNxxSbzdKFFi8=(tprzw6oh$o;)V;>mL_-)4bA2*OFaMH(F+9JI$qI3Hq!u<$L(_Bf zFm5gtlEg&LarMLW(h``%?WgEECgD5cj%zL@w%4nmmBo8tKXXQUe&2H(7_O}!b{Oqk zkgCzAjWNE0h7L@Naw-Ir!ksvb^Ndp{hF!67-LE>fa?qISH_`8x)6M8_-~RaWE1AtG z&1|ITi}>cqUC2Yvtx*dFzGyuT6jH(v=$8@HveWr!cct&q4e;nwrV=-M4$}RZ<9`~s zh8#byt*sPN4ZOkJVH=s)E9K>_hoJ==gO(|@)4scBUk(J5QMUw?+!#ZcZ-b8eKR2&8 z5EHW2Xiz8dRQ>iXmM7(!lQLc0-rIpq^#s(Lj-SgJ)DVUwwjUC{8_nMP@MN5(JGuI) zd?#h1gJ>ztsW6iPV70l!KOlW1?T1wzU`6c~Kj8ow# zYRu=VlyoWQ{=(sc!=D~qDU~r)z4T*{F2KF{67W|2v2IRy#6)nU-?9%?Gx9O#dOJPu zE{JNSu>16U`UDJ-Hf*pPM8xLw)~zb7bmD;oh1wFsJmx>7^T1`R&f+~)n6-j5RLMOm z)nD38aORsYvUQ01&1=@a3u&9a1-{n^;xGO$fZ|O^bt?EbVx*j&d8lVV<=jX(NPZ=d zR$O>f!i~C2(oD1plV17mHDu(H`qzfmpu?U7BpOHl?kcSKx!avRg6<}{!72wF{@(E{ z{JSiNAN+l@94!1}u{~W$!_+~D+|ARAs#vc2tmmRPS)-*I^Y~#3jiw1A7GM>-Gil6# zWPrJJ8NZ(;jVV!jo_z+B{pxMAeSs!^_UifClVrA7&H7a6P%@|Nm|pnnaLF{rhDTpF zCsXxf!N6-R(TKE*Y}8>H(sP26i4Y-hmG~=+jJnV*iHTFS{WE`@Uvi%NOQG zFh-$>ZObXh2Vbfl; zcSh}f?3z;;4l6HBKMyEOAxtPP36%>{X?b(lBKe)}EHMUlq>ho+&;E2Fr8s zJ2}N;|LIA>l=$avNV**Bl_6!VXCVhNIiq&IVXIO}l;XX1PV*h?RdoVmW` ztohCjXI}r7*74}@@A14&bwW>yxv{m<`oAaMSJ4yN`vXqp_FP}JE3T*kx%WMHCqzF) zpJ5M|BQ4;Yf43!+jGhFotfsro6Y-vx-n=2P!o9zpICqE`@mO6fTPfXTAGYH2wRJlG zU2A+EebPi+?XQn^Dj8#C z?qZ|EPu1<}$EyOsu@Vk?Y2=Kf2FGjzN!tHR|F5=HPZrHKxPS^*;rfDTrcz+tNwo1w z>KpK#G$o{i0v~kv1{1-OY23;>Xn5&L&*Zr8Ih#2>J#UuSFF<}3Z?PbaMu z@EEXE8mFFD@9$oE5(;t^a0;p=<}LPSJRc~AiEyW}#^f;JjlM{z8p;Z*N5^JhG+c7P z5P$sbL?d{H+H{)xXa>0CY5)F-E^fN@jepVCR_^cf!h@E}Y#kZ@4*Yo(orQnpHiK5G z!rfQG*jG-vD4zM12OD)*gA)F zM!*;+KW*{l_`R`g0t_*_k4V4$zI+-;j3lxA!}ItQZS?YMtSRiPLwcrDs+F^XmDfZEBAYx^hYo4 z-iu&R4H@Nk0q=XA<8_~P<@Kwkiq|{lI4xHjoRsDA%3_m$9J+Cth2&LqAarcIOr0AH zEz6Gq?3Z+Y66x>LIf+=Wb~AKT_jFvynHpMclp8&oC44nIG$!btJBGbC5ev&ZVynHL zoSk83@dH6l)Jd(DiEp3Gy$^CWmQp?{b9a>U@mXR2L#ay+(g416SbrO4%iQ^;lLe_} z;l?h)kdz^6zvDn&{=?1+ud7%8-5WkC_&aBdO7_a zlw3*eBNhj)WyF%(J3S>q5>f{Plm1)2nAEuK%RVeq{ahTpD?-L{Gi*FU^vjL{T(SX= zM3jJN-UGY0O5POuPwr;B))e8>JQjE}DDr5|RMA{tK*MazK(pqsR(3lEN>O7HzJY z+muc+S4^%RJnDFtalzXgkpRtt1`lRu4Nf;tFCg=tr||eYgxSTBOU|N{KL|!W+G>Gz z8+)@B>G^Fxbq79`D~a{apHMegRMI%2B&0v4zj~~#s8iV*UF+zATv^`LI+4Ql(7!i* zQ1Ou6INMXKY~C3wtkXDB2Xx!ZMJ`=U8 z1y7mZ>SYkaFewHiGfZ_W#Oq##(H4UA3|gyz5GIJO_fL>19|vUhG2E~_X)^d*Z?|@v zI~XgR@or3B-oun^X(pkYOVFw&Vs)sy4#xVAgb`bfhbdEH(rQNdSK%k2Esy85ifa&m zM8}ak_`uKIYLdczzt71B0)*E-fi0C6kx!6Ga|p56#^WhX9r8@K6P3p8NfrH>Lzdr- z5oT_O7b0psTzAx-PO!JzM(Z_P!GKxIeYi7^N! zQrz&Rmmxo^-y`|wp7I}o7Q7f>t6oyr@;n-{M|t(h#R6@|B!!QRO|!9|$6|%}CUd*Y zS8~@ag>EifdcDJ**tO8p#8a^ka)S41h(aowiC)d5orxCCuNJEUuxo#NBpb z!(YB2QQzAbJ(*w&Ac5noTSZ(Eea#`TA*$~7$sO<`$SNQOXOioB9x(!m!z z#Yjuyt-*tMGXS;eXVnRnLK67mf+9dlXxqayOnc$t!=ZKZ=uaOPvy+QO(j|@x3#DsUhi#7dQUPhY3-!k;L9`@pii1zPkfUPx6Aj-1 zCNHa${L9H$ckF8?WrM0AC6-nmJJDqR>eE}Gm0!ZWA>sgO3%*Y!rEh)Zzul{{6z@OD zXlsVacy99gh-nkWq8Dc~0ezzVC~P_$1M(>#Y3WtEXaaBg=u>{%=b^9Q6v)rbiO}Dh zPu&$2bsF9klgvNw-eAcv6X3o6b7EhEh?c*2f2^Ryl}ou^0Z*n#Hp}XV9brv&EGMzh z+;!*hZa;9rEL*uhq?ImYBJBa4;qHD@7OKVW_F%j9TUK}Y8MqRUxYLs7V9-ma@yS0l z`;7~Us`F58x*|nqP2^tDVMd>GiR*<7faCU6jwuN~%1SP#MDw@(^A7#k$v?A=?tuzb1@&RIFnxVGXeYDE9WXz(>H&S?7hd2ru04yFWQ^%UR5l z-Vi1RCM6$NKr_Kiky9{`E0=|YaQf1ofuvk>(-J65z=->LBlyL}#;0O|;4jTtF6k#< z&!ra+EYfdGxyZssMWj7)_l^wy=d=(}L$m$;1RZ!dy}d1~olNPvH2y^pPE$24W;Xo_ z?{T)M|CO%jLyX&aZf{awB~K3o4fjGleMes@h@<_oKo8hVpx{sNfl^dM_`F6a=@Mbx zyDRjNMNQ@yl5Q{h@n$W4l~H_`q<$$FY{ORQ+z8iy{k26wPoNd3620qdJ58`XDwtfL^E^2fgt-6*@QhX3Zc4-V}?fGgl%jf$!);YZ{z$jD7JK z{N$A-?6oe(xUeMFMkD#45MK9obngc`ZLGC65dMz+iI3UCnFVjFP?%+1fy2Kqq0tJC zi5TP5+f7!=w_d$Z?s)vh*VBQF3_oG?Kji%Jfil)!Z+S$=X?1WamM4P%BVaBxkeLpZ z;gG8+M@&N51NA=TF__dijmsj!z~<>PX6aMR=@@4j=W-`unzIsO|ks) z=eOKZ?dZf|CZ4R&5|A^#|9uhc>G5nMw9&>2rTS+&Mr?sBwRdwaoIa zVdLbkBLJ`dD@H2D2&%QZ^()tFXZr{pT$j&byaP5Om~p|B|Ca^Odx!36l|(b?i}m}I zLAlLE4-YRuB5WUVLSk*H9$G>7-}hJ(;7PNjd=Yx3HET%SSxP;iDkOj+_4%@lZk$3W zOax=g5+QRQwA8V<$X&AvcvNcV0sn*47bY>1=M2#VUxihSAae{6dxYRb%k$EqAUAxi zKLf!Y!10kE8Vs8eVEPlF3|oXDgD8c1ey~^ zBdPfR8()8!MagpFKmCC$5W8&@JtoRnX4w%tD3}NuO2X4U$C`;qz9Ef(R-uWZiekmS zml3o@Op#XDAb$Rm9>p+n*gvR?ZkD(KmsP@Kv;4+bb$k;I)JVGVsx>6H?<#VbO6)|a#U)p1i1{rgYASa%H4k~s2G282E~l4!PrG}Ot;R?Yrrun5pl+6DaBHH z7zP89y_g{yolUyo>cXf}vzL0^f6M+;lu(R@a(=p(y(axTMRt|58=bwrdqK2cwYd-- zlW`(Dv0=9rjdwu=79DG8`TUu;MZo_ww%mhHna*`2o0}HYgEqTost6tdr}N)Pcrd1c zUxJqezk0guZc=E<4G4+O0v6ySNc&iaJ00Dgf#GRx!YUocX{h?XNM0U;^X~K>C&oiP zn6UeJr(5*O^uqgk6oN-R^4HdOe8y||oGvw~LL3|g}o3}Uh)hina5HhiHT3f>@d~opO9Dbrg zj%N;d*3Ke&(FFQb)c(q6gJM_^I!k4%_t|D;#pR+JZ!Mm)9lBok?llhq8p1~moh_hJ zclc{p;|EQd0sA?7Qj4}Xi_wAd+(525)ulutr-U=q6OchIDN1k_rL&3*`BZ~P_j5t*(n23I+m7zRhu;B8|>!W725 zi+>hoF-eHZd42dR=8K$U;6V@u8k~{V;g66gcvp4j&Pmusto$`GPQLy|uKK%1%t=+r zjy=fpYe^~|57qWyAk&blBOXwU8TC%s@}S@SWO6;>qc+2Y6n>hhFgj_%a5b%p*9au_ z`?d-t)H9ci*%2>YJ4MjPl|>~m$2|uL>t5wgK2w|jdF|6Q3S?2tB6dL1nU=70{n^x1 z6m>COJz7BM+W4JmReC@jFI&1+RPOZS%6z?w;$ZKpQw(XQI_&E!ouoFC`Qmq2jC4PJ zp-+gQxi%{FR9)WIojk73h%NvPtb1cb&8oG$iF;-lCiOHD+adMmCB{4FB`Slyqiv*< zZ~6`%h@SKoT!pbLb^yD48ShdOXm|#qgaCjUPl-;rYrGt0u5fc)FqwB+trv3F39u!; zBoGdD`ViSu%cvM(igw^cnSJkW9ol0zO9?p@EY2E?_r)Lv&d4;_L1`{upq~R&AT<7F5NPAoMu_0*ANF6|E-HU?Vt1VPLi)a{rO@ZWhq9>YHtIkUC z^RM49Fvx0XnjbtF+0W?qjhVET46l3%^t|Zpt_$qPH5LMy49%a&*mP5$XVsTH98^s2 zlne12DsF749aLm(X(J~MS}c%D$O~ZiCfWxQ!Ql6r0avCpjV+`TE6cTL0-lb`az-J^ zbhxa1)-kNu!nRnSkZQQ71r-~tdz0<~U>MCk?j7yDC8dGjTkq3RQaPh``C)s{QHpN( z_1#z@?2tkd%pLNC@u-HYmslJuHRVvTZ33=sZm_Mta&b5HU$4@)UA!V4JQ>m+756h) z5=RLZZ>4O}YfNZ*^Dmc{NNYFF*x~5DKEU?z+^_$;VB8YC8>AXAA~)#Rh$DQExWDRY z(PpGe`&G#4`rP~xbG26%hVlf?*ca?LD64&Mc%=Dp+GQwHzU;E2Bxo7GN!Pqb8JG0& zI*_<%xu$%(ET3L|X=)TTvq4-r!J`kmkM3T+L?+6r0hhkB7ZWdcyHp@f*MveY$#(3f zQ?_q29(ehQYvw~rKn_wK4A)G;^VpG*2c^FyNxwl)*8&mtc2FK`TKEk)=`uV_ddSzE=w(Je0y0Kb8QQ^N=1 z*}ADuk1ILn_d}cYT{mVCD}C0#$g#Kv3kd9SK+sAP)2I2r=i5eM$IsCOTmus-b|1yR z9faAdY(<+{b=^aWBwPD6>w#W2bDdsx#3eN_tDS2NdA!9fG+qG#9>PRnqnK&E)NxX~ z)?~`=3!sR{WYv1RxuC;FEzq-L191(Hcsd-Gee9byAH##c&+uZvTR;uHLalqSX1?E_ zxS?j%h2jr-Bn8k|>C?@1#t-Y3obP4L{2mkEN)O-F)W<9!IDX}H!1!IACyp|2%JT!& zW##Oxr~igEqBo*nSSEPzCY^J6{CRxO^=%g9z3=z^QyGa=B8M5KKG%D4YGPc#eF^tD ze4y0N^fXX^v9BqF`Ao=D`Af{s)g%G(B{90*+2DZR!F3BujMW!kLZ#r*o|)g{>sj^n ztWWOlrXW88RxZ#{&foZ(wVF(Ez+xV{e5XXtlrLgmV6rFN90) z+n*v&_YFl81JanWe!P{v40niIQfgX6)V{%a0J@?0VJNRtkze=96xNSa;)!}%Q3NP_ zqp9CG;R`X*@8B0H@7!5l(|OEE$2f_0_*4i zw%s=v_$IV~EMmu{{QLKbrtimR0Giq=o6EG~!&mme@;EcDJA#>74|8~akyC}NFPa8`$D;R3-#IC3!hf6wJoWeGrm`u)J|4z0(_f+&B!FEZVQNtkg zR&8}d&=f&c`b&kcxCG}(t9cFLzoHfpTEt{hGl$iJagH~)%LB}mCXuZ-PlwgoG%-$U z=XFdzFWrVUSk*-0Qt^K%B_fA-!Q4{B5JB@_rxR4Y>M9b|#Nl2g4#(@7x-GCIexHHU zqeVtrLt)KEZ;4)Nwa@Hw|2Z)JqOY%z8_~o?oBE(eieLD~e=hwK26vO5^-D<;_guxJNUY<*v&jT!1uGz2^i z2#IeU{WbO-@qiwxiG-wEk+UDRp5E?RuOr1HiP*iw{Go;Vo$hNK`fXvZ4<{?ycDJ)C z<4cjB&WQG{F}YYFi7d?U3~clKlTRP^vhKYi;Ni+c>pB`hU+47v8B+|gmu!cUY!Ad` z?>G*5dL_DnN$rS31v7aIlx~(5fP5AU?wN+22vM_*b{-gdF=>Me1`} z6l?9{{8Z)Fc>d?httnEnqSKUWOR-r5>k4*-d_r0+oULoZ;R17j8bd@TsIg#HLmM@G zQq9=Npxtkj!|z7c2UhJT)w;(R=k*lNlgKtyy!^R13fs?h6u7KZgzz%xP=2mR9lD}! zL}8)unR{*IOX=@{8zhX?R`nXu~r6XPU2Qmru~#avM7@9+2@? zGQK5$#o9lkx=GgXAV4J*$hco&v&KPkfoo6McgIQs)P<0BtLRZIkVNzMVK9eZ&sATQ zmWLDwbp<(@nrHi+rB?dUm zy6eD61awYvMSl&Y7QBIR6zLicB6I?6xVoJ7hM{Y$7EJ~N-M@VE;cUwD zFR2a^xvO60?x%5|V-Kk@eY5JfEFG=<^7vH016#`~;29Bh-_D={`Zfbp-r2`kb5A(> z2W+@Euo++-c{QAs^OVVCK z)b1y)Nbh?0L2luXZ>lP)mTUs9-6&zL_KYC!H;=_zGT(Olj|;$AA$nVbmSjPaO%2nQ zla*BUTG%eayO&+pd?Umq`WEHyH{zy4%k!=7_WCtdNvrf~>#@XO`vZkfQ_CceR+IcG zA~3>%lIVbALI2tHcK=|LzaRaN?rsqeV24R+U~y{p4i}6{+-0EZHwT;H^MPm}>`)^9 zQ^nDEYCfGjY8+ZTN~c6G29_TImsT;=TOmi>ck6NNK>>zK&BiX;l8+*s=k-`F{&)S4 z6WKqx)7+&0jU;!i9oOEHQp1e=Fo5QM+jG{}O7bT+R|luDOKq^M4L}XZ8(P(Yl+ZZG z*e$2Q`8{?)d5=3Jydo%U+B+z2;(o8|e$fN0!(v1#!7sR?h734s+=-SY?g!1pcY67L zpMNOy*}OF!!WYd(Ju^!cMjeh4-Q_7skum6WI;`zHRl}JJYhGgk@a4AmMs{m)9$Hz4 zY&{Fk9%KofUPEN!zX7fbae$%aiQe+Xn9~s*e}AKD9ZL4e=@(YvqEnX@OdpG1SZ$6; z56K{sND`GYSzjyWI`aUWR-nlTK+pF!%rc?(yM#&niRb`qf}(! z!N4Z(e|^MxG5X@cnOSE^r;c>sElgbCLKZ7`xPNg_ew$)Htoe(r)udR55N?G4>6pE* z{LToIc@-6vQ)g0tR5GAuFh@q%S`IMEg?i7WY!&p^Q5}{GVCef0JgLj8uoO; z>mm@yU9A;etU%W8qW|h_tNqbdYgc3(6ifoFs9hM|{cSa=p32tH!Tt-kD_(bQ4W=5U|Y&9Li)G>GecLzX3$QOP zO}-EB{xU=uLD_el4SU?$n#F$vG+zR6cdd-7tJQp<*Vd8)Dw=^zuA4l6)>*Od$6aZE zKdxF77ux}!VclK_aWNc*HdIYlSwye`o9bmw#60rHMu%f9y@2@>C!zz?fR`up=0VCT(mRm9Yvd*6H=W+E9PW}s@wCNckT3ChxU~r^!8S)p7L<1A@ZeJawePE z+_zU^Oq7o)R*tBd<5r~^pNIL+7yBwXd$I_qSKavL6ng_4<8I?GFCg3?bNO%E*FKK7 z7f?~{UwEMPg4BTdQ5v#V+a0bqH@Osl_iv1CjHPD#cI9TX~w-*{DJpyvzH&tl?`*^InSkmtJ+;B7u}Zf8WW)~dj~ml0@>RscW2qQ|`F`VqwnymZ6+<*Nu(lrDI4XVTswuwjb4?kB7G z0ck40z*{!uGU3AGgAXTiUjlkxO!mlT^`LR?1Y-foiZGoKKgp?+0i*sGPv=eP#xXvS zv?ZvA&neI!SrKbg?@xa0>4gvlnN6c9soh=PpY8%(d+w5Cfq`uaV~Q7MrSYS<_t@B^ z*2z5tl2;j1s9vEo%tX{~1r%UC>XKVh9~qUp_9u9K1vAao4E(qL{goEKp^z`UaOC6O zyS;5;O6H(EzeuD6R1o@uT!yqM~9%@HFfdOfks8pIjRmwWf)M#<%XNew1oS*{f(5m3Z0+luR2( zo}7w50u9#ViJW~-o#w&|O7XUHZ*a~nHk|Ooz}#a&hQg1(!~(peK(k0U-~Dx?GcFqW z8#ga?gSNeZ-QB?&!&RE4#4NClL!4jNynOsS+;`jIC{@l%nlJL!YW7Jewq!{hvu~D( zim#*B0Bl(>TITd4QR^(I!@0HON5XSa$5q@#1a=f|O& zZre@NP4#;Hz%?1gYJi+^q%OtT7}>UI`K-_xr|WKuas6lqEWVb|KL6i2DEfCUw;TCk zlvC=9srN&Sxy>2ASb8N~Nfq>>aC{W`V0^CwV3!lz8y3lU!Vl10?_rEYZio881jU7H zg8h8m(B+vq`4p(*1`$dVS|_vPSIg{3h6tx$3CcV^iim6;QXSGS&rNz1Dg0w~NnyzE zFG5e(YytZ^@tKCATDxDa3CGG3QLMAOgZDwL7TNRPM>+gYNmjZ_%H&mY20CvtGWX2O zh^2m1(^n*k0(K__kY>X z0Y$0iyRdBbl;BBWalt$aQ8+5RH4;arfPk=v+50E7boF;{^h7P;YPSlG=nTG-&l`f2 zq}#Zct|zLyf2jIl1o-STv~NT`pM4|KmtTA6KNt&Y9nhemxx$80fr6(v(`gP9&5#PI|AkD7KTlZB%VG2^PADJcsu* z+vH`@PWqh)zV7=#l{qcOzAWvy+$nLob0Z~7AE#{t(}5Sdy(MLor1Rf}s2sN%Z^l7Q z>)lQi;)vi}izte#Kmb;MiJd*|7I8SmFBb~@+?a=W!;Cb7Grn(csJed$#TiZ{)on^n z$n(d6&wMMO*y)`FBM}b_Y}}Zl;zXwH3c>ID_{{im zrX+>{B0Uv%B%#%ZBTBo$;z9DlpQYEaPZX*ZM23lWT4*6#H2%J9o~erO)ZWnkf~sF! zZKibeaZaO(J}2d2MCpQ3IoA3ig_fKvw6Ns0a5N!+1h3A>-<9qQ2_FE}@i2M}$Ht=Q zGg1qGp90R1P-5sTiR$$n`do*}bdxxsLxP?W&b{dT>MPoSwEqZP1AJiv3wYY!s2{Bk zSn)r3ntV34Nz>Fz0@hq{u2=9c#CPwlOPd`a1%h-t$^ZirV~X8|5#zW8IG2JGFpfoXu? z`eMx_Nat&dOaBLR7IV6N=Z}0gCnCn>JMy+B2y=?er{c;9-yVVPHM=|RZ2}qNe(u)G+f@S7VRcktl*b~|RbOrisByKXT z`TVb2xjI|k$K?$6{V*l)@$0(5jny?h3K?1^|0O?sw(jK_{^FeW!^!qs+^6~XF$PpP zZG`#IMM>}%Xx^=)urPT#19XbnKU|c`S$=g6&5m9|6T#9Jcg;e;67Pn{hpB#aBNH+f z^w@xEtr=07QC%(u$@_gOs-R0I?kxc_XM@Tn{2*8Y51EDy#wTP=j~GI#>U#Z3+QkW? z4)74^`+n8v&n4QqUh5D7HKI;eNLFKQozX%#Fhny@WsyM{P(TpJYT>E=ql3Kat6}Uv z@>2fikHe=Db%NgW9*3%#I*IriPm7G}W;?XT^D;`z`J>$7&rmZv7ZeP>F2}|OvtSp} z8)c!ND(XHLNleFz&@4bFIkX=?AWY&p{qxvZiBy*6hkGCx9 z#?P*8HyqPmuKffS!Gb0s)PvL-dM!6IcJ-Vf!$umh*}jryt}o81ir5K+Z{BzZ!rQWQ z(f>*5hkC!@yq4d_ypD)c%S*~sVRrcairUHQ{telueMQ$H%NAnN=FjgUj_Y4IlAKL_ z8m7q2@7?@@$=E`cJuEGXm=~=5MTo(#cMm%*7040@9zQcTqZR9S6hKLk7{{e^$hi#w31z}C%2yu%3l2Sb!K~#EEFt+p4Fn)Mzuq)qz)cRoUAA3);x&6tF znwBgNJLsnP9DCS~BmUml9(GP4W8e^}-wPF0otL@S+6`ak(OQQrSvYU8sbl2yh0GR3 zG!dbqBwoxPuD(ZPT~tD8{4kEE-r&03+nu~#(Cej+FHt^b*zK6|aCB(4r1qg4Xx2I% zP5*FmVKtj8)*Qz9dC4OmcZl{t_&g1nNJP`k_-EntVm!dSI^JKxui{&8Ki&=tUmqJw zy9@IheS}wYN`*Exe`t|#USk-_vtvNjjqk3WP{LgkUCr#1j{>x^+h9Jhx@2{furu9U zW2J|kmVjRZS8ox7MnxX5hdp=})Qw7HE#oA9%>e9<$w+L~yH*#8Gb&`|krHsPQ&9CK zQ$V~^H17IDVOxm4ww&4h7b&ED_zW?XRovvMfquc081z|GDPa5^7f zf`}*&MVm@WZ2IoTstWbI@O}s&_;@ z9HXZko+RxW4ZKD>zKkICJBlrWA!!iCMK>B2#Q5v>UH-#?Uk7v4m$th{0@7})Mp?2D z5$O&jbz)}(GN=Xqeh>O0FXCCt-@!izD^Im@?l6_pjZCHxW9{a*e_qQ_lr??(z2Egk zvk&+CF3fbI9V(nS9e>j$zrzEEh&Os*=YG)c6u2&JYQEe^E{>GNxl zl3YG#N6x%|>M&*JpGiAjQmGxGyNl6L*)t+8!|?Vz>%*d#ng=lf`!UMeRay4=mfUV%ZBrDWLmc2DhCi!UBxM+mZJHA0W_9HCDaE&C zMwwGIRn9vG(CtO5U(F#l%BMv#S@{m!GyTk=k(3rPNc$h(O?*2{OH^BdnCcs%DzXi# z0gyh=JZ$c{K(5ly1p6nCa9~jNTND^6u@dw6U#t6w0<_<&V5$4fvEC@ApYR{OJ{3cy z6@6!<@}QNCEj~!hJAUJt?lnugD`T17tEWGrZUYwZ5SR2ZQ+^66%Q19Th|7BGl-{3= zd+N1bf7Tm}TADwa+|gQ-J&WUrC8R&cBCOm>PmONV2)l9q@5yV0(f&m8bz?FL-l3a} zwRib02~(L>14U`4sTY9hFhF7iXN;_)b5W;3go4O>nL%*NLBbCajiLP3XVH@x@$_+0 z(3dU>SRJDG)6vXs{pGs1D4NM!F}GK}*C?B~dfv$cbEvkPCfj=yg?|3I4j=v{O1doLyRTWL=SbF0XU$b;6hGQA@#fuc+)M22mUpsXmQ_#2ua zf;$YIxE3vvgkPvMDn5bTHw8kJ_*joAq}*Qj%vXjxjj=epb}t-ID8{Nk%u``t3%c6O z$9)A$#q#Gi_3D$x_C{m>xjW%xSf!?c+>jX2eq+Hd?eG04u7T39!-W19CDUf8Mo44= zK1-UUc()5ZlnVvueKCI4U-$5o*7xYktE3C`(^g|VO|=OY14JQz^%iYB%TY|pPK-D} z9pP^LZqfYpaa`8g7>(i9+mL2DpbWk>p!eDM@RAcyWZaFp_FVX@hGi?BtX~}8 zP!Wqi5mIR0O}4>723rUp?L~||<4KLe)RT-T!aE3^v%9eVhq%%uE>0~2C6gG%{=O;1 zsB=a=IKz=*llu7UkaGwb9gWBZR64P4-?fr4LJ?UD zB{)7bUTa8hDO4CDm%g)|brpNhr?l;v*?lE=;4#cP0P7&#a;V?Q|5{8@xCY;t;+iaN zDCMbBYXhlY2@+<#B4ZszgGi9~{0sbbhz@7}lmOEV}*<=dw6FLHU&fvCDs zQ-=yLjakRvCDf5j9s5Xsjmg=fGB^Dl;EOtqzwBV!IC)By*t22A- z7k@9pduh8w$f~fDv}h<0OK5psfH3ln4a^=zaXtIF?}*-?4O5$r2*uQuG=@K)Dwzr~ z4z~EcAd-tz1asJgxN!uKhyOe065i=PB;WV>n8BkvqL+Zl_CczjdZRz3fnFqgtkpn) zjga~>|D)1&g61ZUVJLyuc-E#g1KAP6JJSPFwwWR05S)Dv#n~{#wmg54tT6(fFBeMUY>kh0)TXC)%S>My2;nW(FS<7^^6+PFJoCBia1?i}bdr zWcx%Sy}o_IM>tAO)AwIY7rOYmC>_y}+Q_ujos?G*X;~Uk_5pqPm|JkEyV`31p+OIy&Epsd1Q~v(Ajyg=4 z2;GLwW*dwW8{XOV_>p(>N`kfEGj5N6PABIgX@!A>}FqKQGJk zK3Y@T+|)F7q$kZgcIB;*b`LZe2}|1a9qMD;3B*stF!L($A1@uY1Ce^|2!$k*;NX?w za25S_hq;CTqTA?b$ZJ=}Y!LF=AzlG`(AI#S2zA1HicO$IM#k7lz)pvqD43 zebrPNoxEiBz9IH2tb?R_x0Gh1IPMys#f7wsm0vxjHolKKXJ~^RTOJt3uSt&qh>OO) zHUAEJceQoE4oS@!Xpo2=neV+ryUdiMAS8}8hD*OecVRWN?d6dAwr z0NlCVK5(+RD!$hcm|8GuClrH;fWLgS8Nqj7nkz+;$|#Y0w?=H5K9j6wX24mIhRhNY zk3b?@_WL30&Sq~SD8@umC`LoyR;RhsCW~zRUzf&%`iw*ZxDL}(HGftIbqWF)dCE| z*M~rYqaQE;Z%$KkteQ=$8X3LZuPP#2XO`X{)V@>)Q@m_RyAX~5XIxOkHG4m?CbRFN z41JR?SB8>Wl_mI`sno~ue@GK?q=lR+UH$Ykt&s{GXz<%H#jxkr>yoJ`QT4z zqM@Yp+L6B9W$CtqjAWFH2Db z0Z6x$wfkOqz9*nEbje4-lq;G}BKo7okuJ0{eElVf3+Lcu;h6&KJl1BT%e&_RnMu!# zNi}#T3$^3haEj*^Q^G#(i4MPs4Sd^ zqKE#&{MHfPe)Hg2JJJP5$J+*s#~RlP)oJ3fy`Q7E*&U`FG_UyGc%NdlQ9P;c`*l7U zX5c4o@Aye|lcy`91aSn6eO%eBPydFL^>QkUAN;Sf$%zk+WjGA zgSW5)#5nY5bt=2pF#6)LSH+IT7?nA8#yd_FQ*0G#P14shBZLZms6=doM}#l1JHLC7 z0(~qAjY^WXdT8!O!N_?%J#buNekUN>=%jc;r{Sd?xpw=P+09HBEvfutBKEmqxMa$F zLb~8$ZCo$yph9tIK()13M56jfMc_l48Fk=5rX`iW2V!8OcTjhzvrM)l9FwpeY-Ieq zJwI*#N$^d)n3Z)tWlJ0XG%0j(XK%eC^)NE%Abl%J1Wsj33HGssR5j2F>z zA#W?V1f?TNn?W{ZWSHfpNqd0KN)aEMvxw_%aRF!J4@;`&Lk4@1?OojXdt_n%k~a6F z_u}d`w4a3VunOO5OnM780wh~1vpr4oP5AIJJ>jT7zMM!g2nM5DUA+s$m5&Gf^ORqM z!;Ne}cGj0DuZE{bGhf$vwH(?-<)|(a@-8+0&5V(inVsk;I3j;W;`O6Z?)CERfAUuz zs>cNmt8Vw0w_OIaZO{5XKZZa=#hu%ERgCra{2ks(Vpf57=GvcIP-*^>9OIi_#`*Fe zTx@T~w<2+BWb`K+Lg2A;MG+I!M8E^4u#v$TGuX6${XBajC4E=>{#_l^0mn}crhUd) z(TM#N34nqfS&_$LsrPK#yYYiW%))Wppke~{e{c2Dt1$W8)^swk!>)MRJ%YVhmyDvI z{2WeuVlYt)5~n}kDp0q*vCAj;U-vT&O=;ZPTq+98J$;YCI5Oh(75jr<GjA--z!hpX_sZq>}Ctab-=`Yn*n8k4yto~8$`7j6 z$kKU4gVO%C3+9X?G>wd-W1j&;gFoh<^bQgzp)vmgTQ0cU&m2yfs#(U>dSsM{^oz6o z;_CaYiSKy&O*x2UOae_4CmaPAi_img!_JWK0aJ$ON7B7xSGz2kCfB0XQkJ_H*kmlt zWQd*y*8=aYN=aeICevErO>~YQ0!v&V#)9<#PTfX`_JAGc*z>FubfLv=4;J!dC+_aA z2tW>u>PL3heAyKj`n8!Km9l!p1K>IhCuGA&re}TOnFtoOFx!+9_se`V*+3bZ`Kj@J zL`klryN46GBvTDu* z^c!?xIvX(i7LNmD=m%W+4im_pk5mDpMn16as*8eF!*N&NDLy&+`Hh{k^p!>U#Y_3L zywO{)zDub2_SXSleszXX-rJ92OZRltAo8i}b}&#rizyz(Ln^5~yDt24JFH%nG7bq@ z9F&q?j&gutv{sUHTa4=qTkzkunL2niq~3`tiwK) z0mp0UxXB5Tg11s7R7J<->^=lr<*BZIppfAW4l$dek5RrmPFY=l@0BeeRmfe1y8b_G-uUTmS_Dh1g32L)tzhVc>wK- z7#$_+v7n(%KYEb8XAo}YzCFPk$=Ej|i zZF)qAHCuOy{=#Ognsh3`5qU^R$Vm5I&eEh1I7&=WoK z_=O#@5TH{O;%i7>&h>XX5lUh1NUX!2!I${Xs>-UlvY467!dBpm=FF(&28K@=s&sRO zYTbLD#>cNXW-C|s+FLp0tv9>Q8eRZR;yxSw_9Hs*0SdN z&l&empIKK&o1{;7qzR`e3w@SZt;-Y5i}ao-m)yq|2<=zyp++kmfs-T3XTFM6ju4XL zUqLGo!egGT6U!yU3r7`}iPZL}WCBC01!8#TW|^}b4Q}3Q|4INyLNSoF%<`v;0{m)+ z28B_n3VBC3P@kEnhgHO}sSN&*I&9pw@hGQ;SDKAT zmE^SN$MAC7PWp)#5gh~;@IJsmMXyRDSA*VY!?$X(NAQ`rG{wthXZd7Vps6M|a- zNVYBFE0tu&33PV;yU>@INxmw@R&wM^0V!+6b)%7Va}q#wB;x=#N2QvWdC9E5?I9Y( zPrsVF5A!QdZGO-P~PeN{vP~B{H!Uz$4}{20pYH*@rD& zOvA(R92l22G{W(+5>V6nYF%u_M$BSSl`i*%=sH6I_H1N z>m29tj%bfWAHR~=dNIydy48*P4gxfV3_~OA*GU_(Cf5euLdIqg4vy@iqg3teUt=PF zF0?}lc{=aY+?-GG(!-nZ<7%*H%og^=29X(D6IQg#{ZG|Qq z&(%m(<=rJCT8XdO5lUYN!7^fjp(Ed-5lQ1k`Eqqjnw0Ay)7?~&9eeM$EE|=tf9!rt zoD*e}BlY+Crf9;cc#MKi>CwMT*J8V;qH6FBB1Z;KS9;;GFcx>SNUFR_ zE0bkMgI>o|_u9W5YLDE-_M&q_Ny7Tp+Jv97*k&&1WA0?+T>jtt=&l3N#0RB0M{H(E zS;6UYZ1%z86R~w}Uqrp-vni7Ar$}~58c9MBd%Sz_Wn1%RDcOuN)s7q}df3jAV&`4Lc1hwS49W+<{s1yjGe{Kt({ZRP_Je=D%Bf5|Yx10qS6x+SyY!mzke|I{ z-@c}o<1sT$lDrd@UiT&In;pdwj`gq!4e4jl!WDWP^(4C53to&W9EHwXS;tZ66Hup`Cs>Y_BL~26sRkG*dpx^>4#?$LkM*()8P4QkL!;THdi%`;Rruckzf&f34 z0*ba69HdXQDIk7ZQ0jReK8Gjw`9H>LL-1#Oz66GNc}0}&<1$5xFh#X z7w1JpJcZ0K+(wEsOC*fIr0#E3$KOJe%9_}J%vRhjf4jn%BfcK}FejKB!$o2x!hs@~ z&-wRJQB+2LX<#4@CkP$M;6Bo)^vGq)z}xqNOPO@L^UaN*6p0}fgBDXo1e;y-Yn%Yf zCZZXc8g+V3>Su?>K67LN{Q6HJq$2ZJEeccW{E1NGczP(&p9oSx1|>7>pX~`TNU6Nr za3k#G+l(DZ`-D!2w6kP2f>MSjP;c1wv=RUNcUqu~n&>A;jn3qO-#iIkxO&%yXJ4w; zN|{xrSSdfbt3r$p0lx-#t}G24wffz9xEjNsC~sQ$`_9fzs|NtS#z(2Sth8$3&;M}F z^+lM$al5OI*avwtKLS(n>k~4dh5J?y>P}@^N?qrctb*~#XW=!NZa%Fi{a+(dl_!UT zNpI1l>+_QS8T%+nE3>kAKe=Ss*A-%hFml%%cK-&ZU3;RJmMRSwlm0z_%QH_7c6F+j z_cHU_6pGexM2kXmGb=DgqHkUqu9dSB2dZma8OyJwbsun!60;&oH_J?yG!W{N=G(H6?Q%7@WCQ=*t(`bQuLmN=RJ z+)v|)D@7JzLgVNBtx)e{L&~g$kv>t9B3i2vY>UYCjjtyVFz4b-3Rq};s_(t0%uOmd zMOsFlL7IW?rAn}A+TH?aFk^U@S6^D^gp0>;`a8m6j+X^gwV%g)beQ`6!1kLJ`Voc& zx3LlBD5pG{hmIkby5Vjl&okjXePhZ1e+}~(h_O)tEF-M^#ogA(C6oHdv7>S(5Edq7 zW7lcclvda3=7BK}XBMs0^kgEN?R04NW$wm&0mxxSI^MR>F|q2>xt8SYb^DmxLprBT|Nd2GX7<>%+}&(yh9RyEp`;g{K*k`eVjY<7z>(Fxf! zQ|OBH+j52&*M43RmR+`BXG7Q^a-b&2^GJpYNPkpryd#@nMo~TkRFFr_bSOcGth;R= z5RAJg&|PKgFXhZBuFhJ3!J|Cm1AW}M_fq<1yfVR=^=pK1KW4j%O3$^MW&%k$ETjw>un}?URX1xV~{MTrmFj z)lTUN%acz3|5*UqdossA(R7AcL1I(i#N)RUOJA_f+y;zFNQb+{h`DGZ>z$nUKZ?^4 z`w^)6x{Q#7e_INx~10 zj|E%eqr`Y>X87OlkOw&AgtO)g|9PJprE#NsM2qRSLOFIE&90QFYhAREK(xyG=uQ^ z5Ao>A^O4|68As6189>WD0)>+>ypy zgkSkO!tDCqj|Ik48d2X~&Jpb~f)tuq-xDM#c5!yZnBRG3`L8mDCD%e3_9yJ`9>@)Z+l&&+$pQoJtwvIf8r|5nwJsvF5 zuFP&XXd_w;WcB>P_(8}9TH$lI6V9f{CI#MEPZ&1zew`Lx1%`c*o)w|itYURRkzz+w zSJo7*AP1fp-bX5dMpX`q_)+8XV;(yUph4zhK}sSEu3J@CE@RSZkCdM0M1pO*|ENHR zkwMMCH&E19o0t#@5V9tBQeYRaR-gJb85*6K*G{8U-oB7^)#$T_pm##t5QUfbN7klD zFq5S^BnnS;gO8Y!DcE)Q|GN*H;}8pLFqd2f$F`GLEDq1|qD>^pGz+fnxN>lrdD%QR z<4o;E5E0Hz&0Ih2I!FLJ)tv`sg|Y!7l&(Z^FSfwLMy%h_hb*FBupik6k-M1RO{Xz6KzsP1$*He2F)2dY!=ZQs zU}EaK#Mw|@TQQQa7PY1xM*eL^teXAT?C4suNEB=n*ITpP&qQ>7fu`n+Gt`Da4@Th! z>tO$6SzHq9AMsyK>%xGo-6%&E_xUyAk8CWiBH0hNHKbUoFW-9HF*<3grptnos59cG zYH($EAHjt^d$}0HX?a$1JV!ZByX+kLa7lAlMP7J*#5$3QIF@+g(@Z$@TE% z9Hm5aqK9X76E!spvyihHarHaysWm}4oj-}Ua%ZztQWi2I4ZuY}3d48M`|Td@gsMNW?hH62@a9+IhXajPjN#IG;rc;m%^Utf2wY z_4c$zGTDDk3;pM(l?+NP0v-Iy`dl90-v+);e;7r|$+ti`BFcphDGf&VUEElIetITc zI19VzIc~UleiDH%ztvx>1$r2d3p=$BzO4LSVm0eWB37jRsdiC~4!VO4+xolpyPD2w zVU?r%gc82;hHL1j-TNB?!P&28v)SiB_osSo+OMn1YOB?1pa!j3czfi7XETAL8_AiHCEDlnml>Ju1FXk4z3$d z(y&iSNlBnSVh{*)jRIo^AuP33t3mnKUyEhAS!wiKkt5{Y_*q&>_@T)6SS`=$s*>_H z3=zP)I_wko3_OCc^AGZ=M#LFzSiUtvqPg5#f7qS@Uc7? z&Zs24FG2+o{VIe*bCNYGF9r`k;}i0}d1YB{KmO?WL0!XQUFewKCa(?I`9S0-BhpfY zHbJnNUPJWwKn5@dd=~&6EwcP=1VT6b1cGkyPm95drtX%PNd<9!%@+M96o#OyI)CQY znT;cALk_YLna$RjV-_N^_ZzKLKlgU!YvjVA4Y7{wdg^@l$=I%YmruVOkt;_W07E}* zTneb{%v`%)hIx%@b5kUb^q5gp!kZyVQlCYuqgQI@|GXF0%Wmo4Qf2dxh1fntI@OpgKQq@?D8W^Yh8op|o+CWQOHD#|oB#-s=~{ z(=raJzkPgKH=f9oJG;$y>G;$p2!H9)IV&&XqW^s|5fG4=On28y4I_@w4X}kudz<#{ zOrjTIFf4D?@W_Y_b~#p?bv=!Gp?EYvTc;of3E$NRwiuqN3H%CHgWf3!8 zEn3W=yq6$DhA8g&o4L+^KiEIrp0~a|lb3gUnP`26(vO$a@hc)EE(XZt#Vg9mC{sW_ zv9XGb16e+?`D!+FYtMcr@&*hxF~8PQH;PGg-urf;Qn#;m_? z;J`fk)B*me*PZKVn)hsWk|&(1*5#XRCHrTCdl%IF$x}0+ziTja(Xb&Vg|Ch0GMRgj zVwO=6O2Dt#6R9VfS2+KldH@aM)9262>uv6<5V@|Evo9D9HgTR`mM6s*4*T%t98d+r z1Sl7Mx?RKxVw8n7?b4MtczG9m&V#{{SEXD4B#Ns7A~)0I1Rx>}Wam%gwD~i9hh0Rl zg|K3aFg*R~v3SDJlH`?-%0zOFo#mj9G*spw<4(H1*Lmm|povBBL-raWw20@zN4`zLf#* zX7L)`;d)e`)8@1Ma_8Ulz3Y;ObZPZx!A^7epsG@VF{4TaoE;-|R|H{Tq+yCqNDoX1RWQ`i)EDJ3$a_n)VkYvvvC5zf9jqMg-Q@w!n{P$W z^@EeplqBsv0kH6Z(rY+QA_lNy!!~M%V z{by8-8%@!yKH~$J@4H-x(^rzpfI|Wvu99LxD_V7uPC>affjtSC)Zz&O+p`A~5|S{r z>~5hZy<#J}P5vq6ocf@;BnSwVze>AeU7#iq^p(S*q_x0FoI}O#Q^Doc9J6rMH`<61 zP|zAeuq~ESUkDmaBE|9Q>Wqj_rK{-ebNw) zyzqxgOsAOT;|Reafsfy*hI33LygUzv^DnU&-bB!S!Akq|nMloh?eJLlFgC4;ojK4v zVL$Y)_Uh%mlf|=*&K5i1?KQs@5>V;$9TPKtu)0C%UQ`gj*OwQyw24TTX|AoCVWE@S zgL=7|9`x&=F#^X#I=5>lEaRLehNytTn#6VKKAf5WMmvg+eHm{GSRif2n>-`$;p54M zzrOrMMpx4TwvwZe-=R)B18MJ&x2fxJrdFrviT0U;=SG~4se^Xjpc6M4kaYWi+TJLv ziQbdE@;~q7o29-J5zuduE=oyROUgSgDucH};1C7vj%94@Lh_tS=Iy|rAfe9PL_zfA6IJ!1KM>%lcI1q_eg||J+sE18Fh$Y_%_M@!{ zGg7POZ$t9T@)L4#Z|_-V8b9T41itDRjaJH2sCSTx^*+n5uC6-h6gQnTijXsL;Gs~p zSBe7PY3Q;yF66wK^cB2Z_izie!BeSRG5s*JdiqMNRE=xY_wHGr_Mk>0M;Nk|aPnk0 z|BnWtIC1M;;r%?eN4tI-0YyxD(ib8+t^i zx82m-gvI)#N57$0Vxk?pwN5i5fiB}bzc)UE2Ul{GHQKQn({lWT;KWpw`Xu7AX#P@Z z3id>$`Ds?GQuz<+yR?7{fG;vhJc^meuInysw4(zTXncoro`P(X4dAJr7dAt5+nCMK zmu{yJ^+?hVNYHtcKiZ%z*2~l6HT0D4@IbaL2e))H?zpk&7@!WBdGLb;3C zdjN3NS}JAp!?jpy_jekNrA0?)v(juZ<^_4}cYe*io2l`ki(BdGZrMhhxQ7epk8EB2 z0>pWL>Bq%alif-Bcl<4ULf&h^mjv5a*3fz&MrM7xbMe`umh_H!&M?4oqqn*Fbz}w{cOyi?Fp>jh??0Q3KZ1hsz}Vq zM1rjlr$T~2V5uOLlo#EQgVg9fA>jE?EVUnKg6Q)2se{Qkk)EILOmJ*yE_m%u`o+>= z707M7X!o}vs!7+)1hM49A=9x|GK~hVhe}Cx*>*KQq1P>PIeG_XnanW#KzwUp8i_rt z6Ji7pj*T?sd~q3dq^7YHdb#KT9CBhc>H%`Czo{-lA@mVexDmmUg69=C`r{aAS#%qk%^SO-2Q)eNGVshu%0QJ z7c>B$&7s#}-658ntkP={95j};n=7nG0r+Mu|Gd!VqNfrp-aS#r6~k-ka_dk^C%m%S zu=mHfBYvMFL_gD>087LKyToWQ@V!2oal{VQ0Jwn<_dCnZBD_A=7NMiK(w0%+=j?5n zJ8&u1ivRWLMy($Sk%w;iyW#x`o%+D{R}vm?85KrDe2Z>BDR%ofc~=icSz@r)cJmO` z9z#`vq`DNnND%6R5VbBp2t8cl%O4Pevr|DTkb>wQqw7mG-3a+ixKgsjw+t#v_T0S2 z7UR#^sjV|s%Oht9#jA0B<>_G_2tW}C^+nw^h)7EQy?ye>o1x}8gMv}37J@kyhz91a zYOB@ZEp)WG_k)t*1(|yeJ9P6h;LEnJtQ7swNDaDKH132suZ-;CQiT6knJ6yawFR7hNS%CU;yJS*J75`aQtv}YOSxWX3)QW8(WBfre9LBodY=+Po)L#l6%18sidV@ns)F6 z31Vrm1Mqtqmogq9oQAxDcjEQ7u}&c;7L+*3eL}zf31b=^z>8t{%5{s)eDPr!Rw~K) z5PqNTHe9bStiwI4^Mf0*cL;cn+`ZhnPy8}gG5>TToAg34sc?JPKtwnNpc*cCsxJ7e zn}(GU;wy5Ax-`+HFxpoMaW5hCfor^~1iku=yt3WC|JnAt!FaGa4wkNTuEzZ`1T`#f zC&a~&p6po1MQ>KiwAO8El8P_6J;K{UXJ-Do;#yW=c>c5Svs2UOE?Wqv)3)i?w}H?7 zJx~K}y)ib^-M!paV?h`xSDOK7A@Bb?taI~ge8x{bwC$;LE|VLM^E-XcIEzyLEi@f4Hi z2Vve;&!A@63uu8ib;Mfe=npq%417oQ#Yz^;X!R|MHc;RjakAe897V3Y|2v8YB?|KH z`5q4!1%OAg()pmb&dUiKgX%)@{*TWrytcP(gf`BO^WSeT2nkWp?6`(hl!?i3{EeZt zZ;mI7T&^)!R~B{RWFTHzdNLl%Wzp{nRT+X7Tp@enE-N8xH77+vKfTW78glf2x)!ct~i@m~fo}xmW zQaW-B*_X@FCoUGO16J!wBK#~O~#JES14Wpj{91NEM z21Cq-$uvq-4q|Rhrxs)o9bXLo*SUiaLU|*hRJFT%VB*sC&~F?54+lkocGWFmkvw!1 z&=jJbT0i38!loH*jncUf<>E=vgvBy1hI!pD*-*?@M|?QO$rC+NVWpeCxcE% z+RW*jtzvcQ-Y+XNvmOj~Bp(wACYz1u%uoIW`p+oz=Brbg7IEz?75zXeazj0kkWbdl zUk_rd);~TuY0}G*K8xtVZgLs*qQ8e%;kzxNQla^jAwfjti9I5d(o6UBK4pPP(ct2Q z^iriI+lH~~TiD=QVp#umpDHctz8Bv&>S0N2%DP5*D9Roa>%j~>x==y*=Eosqf)|{> zx;OKmW@F6A)OiXY#@RGk6R@-}Jl*uWxHwe!iL-`q35#*w&9G}laEXeig*o@cGv5m! zm~$5=2KnRmI#;{kawE&-GUi7}@W~Ss2Um`5iJs~cHo#D=mr~Wj*M_fNt!5nW;ilqPLg+%0qIb*bhq5j& z{`9!n+Ri_|X@TE`%?nqt%>UqFO2!O$U?smhH>kS_^vg;oNx`wRnZs~|hfkL-^}r;m z88X=3Zl^BVM5l$OYbMj5lxHXJarr35DMlPbWR>mph1wZ{idlDNB%kAK_zkAqw~*dE zG|2OGIFa$pZ?AXWKbcOhuIr%G1KXbO-}2AGag4y}Fa~8bA0(NxLVNGzHUbkHBVRra z0a>VYSiLPAidY=#zDRm|Lo6*CICw@E?^ZB+^9(5PUEDmJKgKI<1v zZg#`(`}L2ef=XA;H2%fInTUC?xfACu=L#945Lvi-4;vDFY*Jq&i(3K0fgtcW2G`%y z#qwVyEgp%oJp$O_)4aGd?BkctSS2VcbF%6J1T#r^1^V+ea6&B-$-sB#wH#s0I^TaE z{*-acgUVn}f0JDg7lu3ehiI?ks9)=BVRUt3*E1VNE4*4|cmTpB>mXo`+Rf-(F zqjF7jtf?EBr-erUqq#?kn}IZ?63j>+UJQ4h-MO;R7bNGYw3~=^Ra@c_c}41+L{brv8(}?!g-Pj zUuBQZmlg;PV*`s3TM|}Q$D;$W6ov_!q20k?nbDY?C))v#H=(Tqj2{12=xI1?Xx=qJ zLZFIpC$w2kO#D)Y#f+0;_3Q4haXc)8=N|rM=f84&sWsVE#(SCr- zu6%a-OyTzvTGHlglz2RvZq7*!IS`*#-Bk57I*AGw{=(zC4 zq`JQKm^o~S^B7M1d*sn@=#NNO)Hu&Lh9!z*O&NpwulH?l*#asFu4WeC_t6ezl<}f@ zF=9!{L3#EZubn2*49QWIlwQOkj%h{=T;GtT^~v-1>Suk3&BT@?f3Lh{*Fjf*XOf1? zlMJq0RBhybY8No7)o< zZ$?ULE$HMsNFrQkYpelw*$|pT`Og2#LH(s1WkKz@*ZA;`eFuqi!vrDMtE5I zFs9{txf?}=j22}%p}(k@Xvq`?eN>gZ#$wY2qCZNq2@_Ds4{{zu1}a;q2OF`sB4C!i_`}yc5j8zMBGuaVV$5V${q&ijkK%c_#OW92Cl2s41yu8`%y zjyO=n{DF8MkVn}sxkg4k9%R1x7F?z0QDSqlgI@w!5? zq*1|F1aI^HBASeJN_<%}%NEMW)uqTkQumlEo4Cs;L5YEr@`8f(>7(M98X3D<_2=Ct zNuDSrQyfC)7}7d~X1%5i8{02ag0-z+H6A$fFST|(0mDjaQ3N)OX)3MJ zgcHt`N|-|As@@E(L4is`v?cI%3W zf1hv8h26xG)GLE#Iz*mEzg(eUps3;9&j6Gov8@mX$oMR}U}zMBHiYUjfSSSHbCuE8 z+NQ?xUsqma{_tK52Z`>z#;%FL&OjJRJZ=9n6(~Te%VNhSM5y*}atyl(Zf^mnhO~%` z7Rs&Y90n`;K1^xso6)iPRd*i>u;5fQMt0#KsySp z{0L5p56ji!FqcbJqgozZI2bn&+nWea}*_G;c)!8s=)Fp)TKJ6RLXpOO>&A=5<_zIsICKCGM19Uf^!fhtU7C&*Y< zhej7YuvmaKSmUSnEThw?WK%2NniBA)6Bcs80Ac1Qx-O zoU04>*qTW_A6(4|Zr=GpZ90dDaTlVq;JVk9V$bmvfF`axQxq?3zJrleRf!594aI|U z`F*8w>Wzwx@^hRj*7ZP(6SuZOEXR{joY-sIn>+|3{sE9FY7;GeFTwG%MRx+>;5wLh#4VGe=A};7ny%yz86#6$3gP15? zddQHBu6R%B#AueUiLWEe#)rdrH1Ms-h|{spPVzoO))MJHD? z+ytXh?!~o-87K9L-m|wKpJ%-$mmn&Wofa51urW^8AKlv}qcc}hX#CBHo?|NdKvk{+ z-*E~zGuXG52Z%Ayn@tlOiqj8<&l_HR&Nhk=E4!4-sVvbNId(1FO(iPg$|gM*7<#<~ zQ{wZ*I64N1S4A`5yGW*oHz<&|8g)kdZYgU`#`ukdfl6Ef@LD~NO3h1z)i0m^R8m_{ z{-;6^DKdbrX@J49>x_&v~G-9m6B~VP4&Q*mvOg%+~8z(~Ndod6{dG4sF9Uqlt`=+;VQm5r@vCY^Biu$9lm1@1SM^sn6z44)Q`xP!w@uHOE=s?(3e>?^pU^?PhXVtqH(T1#&dJQS0sc%p1A2@1mh- zVuLLOqVHDMD|9gbQn`HWbRy+KNAp2@tY}}IF+XxO%NQO#lLc&O8T4DpKn!!q>T-)y zF`PnatFXZnJ$&?5WPWV@#zfW~ZkvDk+L|A9er`ECIDDGtsP%G) zFI|{a=q_9B#wrolC7>72))e`uqo(@c!e4}KL-<@U+QoL;@X;wRT<+2H*}TfC^QcLK zqzVYt>(fu+3Xbs;SZ=m*&z${#OuYqH6j0j*s-(1lbc{#|(uj072#5$s4Ik)?X8?FZi9~uTrY^P@2*~Dw4BQm7RlfIJEeK>#U{S&t_x9M z-qiB0W3Y0$;roxotOGZXkn#Pmbjq*Y3=EVXS3c6@3cc#aa}96a&hDjl+iK+aKpx5w z8)Z{sK)Rq3KtV; zRJd_Wla&(2jLy(*{ynWPqX_A7FUNtvi19bkzgiQ~7UxB9N|7j2<8cIQhHV85!u zpzuD5Yd&X-SLd@IpKF(e^tU_ZHfhQzJr&b$CcPpNXAdDd*GcF)hq>up3L2iD92~T_*nJ+ixBmSNUAKDC^US6VcFel zPzU%om)*dNg6ak5a#WA`h4Rs}**U3vQ%?R*H69aKe+%d>v^rSq^FOx5=f;z~2C5i} z18!`&i0dv3UZ`dF?BmbSnHUo0qg#;%?vFMRQGKqapuy-L5dl$i=ts7ZpxRu5khiVB zB(^j}sqIo$MsCtn&)cGfkLbxn9mliYyAr9U zt#aEvI*ox(ZY@tsA4hNBGypsx;f#hS+u3PGv`ceS7N|>t2b2A%GJ!4xEGGYqv0XRG ze6l2vqP?`iSyFq>`S3R*%vJ$|+NT6Va{-l-(87;Vc8@X4pfY25Q}Z!%S8yM|gW( zN4j2D;3UN?Hin!k$i<^J9WSK&R0_S)209XF(%X@dDHcUW3L)kA}HT` zO?JFr^U`H<5!YqcYNRnu!HbFt{bC_?`-Pm3T!LWLWb^HIUE8NV8FyDdlene>4u+fS z@MhWRBvm>l0Zorv=?rQiLCVmrKdcjOk3XlP!FH?o`N`2S7X`FuHEY|8EV>~gnGCjK zb63kKZs(J6>-d*|it>wc*tb68t-CLGDYcFoG^0OW$-Yn9RwwZ{f}|)2NzjU){~}Cz z9sM&lJI8zZ;$5W-=aN6>0qO>8#>Y6<=slmS2?JVJz+>M~4G+FQXUBE>vp47GM6=&T z!1)LTnz4C*EGhgF!fA=qG2?0tgMNxOB(V&HvI=DNE1~Ll5OG{-{qF>Pk!8wFeS2c( zhLJZfQ>_3$?h8NuBg9o1d_BcT^W*hY+4Jx(gy&uU*sqS-^F$ik)!L*L`{nZ~0ztZ= zT!mP%6UCkYh(4Blz)Q85e#;8?JMX-UBCyv^Ut{}6^S*p!2HBm%bEvTP&PnG$W^m>2 zrzwpOcM$Ezx8Gxhq&jLR>Yzvs#=R4LeCpK1kYgZ@Q{T4fm~a z?T?xlX)TWi6kBPd&YyUMlE{BLA#@uyn@Ki27Bfjz^gg&sUh zKTkYwS)p4a-DIfVVSc@-%+;l_FWX$6FgcoZ4I(LCzQKk53g*s89PvbdXnna71=8CI za(ZK$0%-wQI;Di5)7I$imhVh_`bOEE!zUM zu36vcP8?EExj1lbeiaaee=p(pg@2X=q0GS$Kqs19f5-SVlYe@58=Zjmyh#mnac&_f)dum?;%q2zyowg zqix>@_(-&YD6b+^`-JskgI;V`ysS!1seGXfaWVc!nNwV03|NX`OicE)U z0yXc-Ad&wm9)K_D*>>!21>ljWomab;LfuB&Grk)LIRcBB{pcje_h(UajC!5$@Ji!5 znaCBE=~CieW|m*a{*v&d@nIPM1`q{9igGK`xIDS!22 za=-L43Z?31jqJ%KZ0kI49&6|{C3fTZ;k#B`K{ohaP{n~7Ppn7SWNRc-j4g)J=KMvd zkMP4&-OcVqGL5{CVdu$uQ-cg{F6xdd>axuTp@C^ ziUhi$GMKkwt$UuWS7;<9C8cWaeamdu(TqqrS(^{GZBV&tc-bOy=Ka=%)SsYu;GXr4 zfg{VQN;ayYtlYhbkz>-4PyC{uM{OAL$!N0NP~DGgF5)V=9a7H#+BMe8J{#?`p6_V) zrUx%*&vJ7-HQU{se`L}aNp>Jm`-ZBfZV)z0^qSV=17jqFne_`q>t5h%Pn7%qB-EQ^ zjmgFHirFRalewjlG_BFeQk549A3=4q`KnzTbUlczLE8ME!;%WdVoruwNOik?ww1?o zX0)y9sKyf2C@N?=Sm&nspPwc;X7qN?vgTx8W~~$MCG7C1qIYQJQD%}ozfIY)>Ycaq zdOF&j{yE00jeQNRHLJsGtOL?4e;(fVdMKt4%_KCm(=x`1B2KrrnWMYhRNHa-%^*m+ zYqY+5(yizE=vW&|v0^~_D+4Zu@6_Xja{z&ye%D@_GrWI`0=bVs)y--(;&sL;Gz?YT zjKC_N4L> zl}#BoN5bTRPY0#UAOg0<3izza=r5KBfh{>ltdn-q{f}0bK=vY9~`s zH_$&yOet;|7+Z=?Lw~N1dcKGy6Gg9^6u?R%^?l>-0^}C3hOa}UyZiq=bTu>KUo|@r z^K7brG!j)!^ZllV9u}c>%#BeoH2k3+=gRYWef(BNc3pwnPP#6TepJL?-6{3eYMjV+ zf%>iBt8Q85$qT0@`NxWlyFju9(|Te@_x9As+WWnv4Nw(AsgK?p7l@9K2nM>G2yFH$ zN(~s-;Czn}rfp7C*>^Nx3Vd0t`4;48@I&;}eVdU}qF5u~997kiFofEY(S-W1d1TgT z&c~dgpoGbbxB*Lvx^0qu_wb1gWCpv~m7dwMvk4OU)X`9tdHsn7(l>f#gG}J>mtGJ{ z^Ib1W!Eb{^X?*4G-HD&g_)+kRSYWDK29+4^^T+e@azAp>&Fqx>U;J5TmKiL*erj02 ze_K1a`*K8vVughuHg#K$rI;$w0=hEuN}Mw+ZByzFiGhbVV<=D=mAQA&fc3&YuYZPR z9)GfGImbHj6JRzitQpidGWn4SVruI5!Omp-R$QxAStSH?xkG#tiocz+0&@W=-Hx{- ze`Y<5M1q;YTov`8{kVUMaG6W+iFJsZHuX?oez9b4_z4(4>GAgj(3Uv`s+uX=5^j5dh$1~iRdtJ)8GP>pDpn22+aJ($L}QXEQN7#|wn+hQN}j5;BuAd3&k;mun9c*r-zE8ZOHzKdDBO0^wcO&KY0 zyY1H(5*up~Mixw??YLQ!L6+vd)kc}BV1Es9u5#2Hs7G}n+Zjq%X=-7_? z$KrtpP8HpH;|lUToO_qT90?v{A6sh5{@sLJxV8@VL0_D+*S{>g8A(d+K-AF)wY7 zObiQ5pG?fxiqVgh19?D%-88{rzD$Vr^`n~AWO-SD`?t({VgKd)bKdc%EQGJ>9a*&a z2fOb@2vqBbrKcYq)sNcqnezpteXQ*zkMje>^9#y-g@gI?LgxiEx;l>b%_1yGhTm`Q zM}*#hiP9~8|}O&{N4&~mLu=7`H4ZB%SpM=Hl!IO9O2W9i9wpeyCJC*Fl{fIgNn=IkG31) zKod)DTje?r@N*3YHZ~nKYl-dP9d`&_dmrSo6R;l&cq8N+XQyb*@Gd^+oCiYj0uE}F zE=l-4ET3Gv2*@tjXfhX8v9}Dp7f%lEG5zj5=kqt&7C$#K#kDzM832c)Ei-tmCzJ;M zBd$Qi>{fs&p18L@ZL=@hxE$y1GP}Coh>1qHG|MZG_Fq$vxc)kBrp@4>kX2KkI7dwE zE#;byc)FH+MZ#98gnmh{^xd+Idb)02{MFaT7)&IFAHqWTSG!eJ!J1Ci;z~3&eayS( zY+I_{IbvB7dDcIF$9ha%e;KdI!WKTmP*A)F$5+wz3?k z$6!~}+~7}`%z_x}!y6tt+v;T?kjO)qvs*97f`;BB5^^aSM2uWuea@U`$bLYOR~FI8 z_(v{YdHlu93 zi6sIBX2E=(SqyQ?bj&eK!Pmh1qIBav9OH)~B<=0qOz1QFYNs1;Ce<@(@d?*F#s6}% zqrgN?>b4BzWJN(U=s#l^e1A+K4-y7Eb{82U4J(n08&gD@K65igRq13=Jzj2}0~|C6 zTk*t~sQOv)BWTsrgonQa{{y|>|G^+c~?_lkLsDaHIQ~wj_Xb+;Q?CffnS~Yj%B1IONn36rAif!gc9# zY1>u{?lwud*wH%fNbbQiZ7tb#;u83pVlGFiQ#PtzqCr=trK|7%+B;abU(u9i)4aV* z&60*a`{YK*wO@5b`f|@kaPv&ma|odKvR+k=l6Kt}idR zMlqIMg_vNiRND|EajX^u~WfNo`TD3gI6xmOLZl&@m zIxMABd)g0qB{CF@<@^8qI!6j`aOUD3{1Mac@}m97jz(N5A@uL-nQqx0Q%HElZAJpu zmB&Ipg68ZUHzrP@?eVctIqwj`Vg@enamXNwKECA()HT^+j(?BXjlOfu3cB{hRW~?c z(pfGc+akgOB1NT);|4VGJmO8p6L>yWzO6`hYvsMDQt8e}vEh@J6wA>s9yzb8uc3zU zx&Zj;aV74z^ZR4zOs%A zj=6WiMwuYL3cg%9f?r?v{_*KR90;$#>zoZJM2e1o<2rP)aJFbhrN{$x|2+MC&V6Z0 z`Pgh_cDH=Th-`#sd2Kxhx}*aJf8mhqr6`d^i?qL`r#2eN@-8qEzjq{u4et2fU!e2% zF3m@Gyv?UI4ox9)@!VM^p%#DP`PeMt#l%j+FZqL7+={}-Tn`lsp z*PaX=R-7vuf!9k!oH2}Rc1-t|?lG$lQxzL75@$*{WGduxOTP11e4?SpJpSgHoBno$ zz*mtV6^hTw_a^h+Y1>c$UU6inH`rY0_S9*RE_O^?CcL+vfIH)Ab?s*BV>v)dMz%&i zR6Dy$h7Z&j-}Xge{vV8nVRVvHhPLQ{wKVZSvfWcUfA=-P_r{-RO2(Lwo>(R^dBN~- zVaF40Ct%}v$J;89x4n5A!l2V{(&iqdig z?s8=6Rro~Xo13NgeE|pLt|(dbDQPO{t9M4s$j|7vf}@LuR{xCFOxbm40ofmzP7p5K zmS1v;8BM)ifVm>NK(emO&MplNS}^%{uCyP_7ue1yNR1roR~#l#62(5(O2D>cjB$&_yj_^N(sQFH*@9fY}GIJ|o4WqfVYoYwjD3lAu6 zttV8U%U!pRR2q-o+fMqiq%!f7}_ZqnvCnhYf39s)chEHMnZ!h+O z^$7C}zCWjkI+N|$0E-Kq3^(}%^1Ys?#+Pufw3T(p_<)wPk-jpiEF(S+?Y>wct`Yz? z%)O6GW}NL-#l^yKnWG{2{3iePar4h-@csyvn3;lxsE{Kj-kj0be}+?Fv&=Ra)QUV@ ziV3&EqK}VagFwROG5l(XI#jhyCT2`!jOE|vTUjmcab2Lh0<`_bl{Ir4GtfXs{8F{E zkniW4ah<*SrK@Vg^-J;np_zB<;Zo1fuJjK6hwT2-?|7ea9h?4GHfuM>5#j65F^YRg zqe2^%_sJ1u(iYp*(%b!gxVS+Vm@LDwm8Qhi;@h>ZoWRbqdbV_|#7a@4Ow~+aPq_^f zKLLIaRbYo5g#Mu4Y7fq6=17nEJ^7x-ef1;97W!^!ZfzMbIhmk~y` z6fF22e*@t)z4bic!tnFso$-YGH91tvcEz=Azs)*j-gne&{`7LE#J$%jviuA$MMQ1$ ze|~*jQ{80psJLq029FStT^Frl;fUgm^KX}Dz0~sDi_LP(!DJ#-ObCoL_xYi@A%^)CI3CnJ@lk%n z2}>O3p&x|u>QG-klf_L6QhRTg%t+0{liSuQa+`jn+*l6ktc*(L94k=kPMIkD7W@F= zGBFB0Dp{SXt6m=WL)siwaatPISbtj4FEycrN_eBuyroOYoxgc;8HvzVDRRs#b}2+Z zEX>%5j&%5U@C?on*B{9Cr+vW~NtHQyamVO({c(?$#YJ}Uae&@sfOxy%R1??vG(O^6 zEu(LCA=rxAOi!4>z=daQD&DSlLQ32*Q(_?3Q1eD~7|kKUtYGzO1yspXYJGOYIuTZ9 zm$6e|V9LQ^H33y!ptUfpwL4PrYPw0xqS3xOer2qmtVRx=r6@ZD%ztHEK=sX5kiPz%2s7PPnt!8-y#|VT&i0l~tZx(<+oX|k8Y14_`zfC+h ztKIU(D!_}~@5!taCL&wzv>`yc*Dg6-cTLzEPlHyxV*yoT2EdYyY zGle?Qcm*2mT?oVyxMPU|N5W_aMgB5fH80@cS*#;8I5mp-KE6q_wYxrU71RFI+PL&J z=-R%#cKvGtk#b*yWYT#-0*ES)dNw4w%r>b^Tk_CmYK%;q}AAm zbC@VN&VDyKYyAFW4lqJ_=gqCtbM4_}K{s-HmZ%AXLbezyS7ukU-iFS7v1I*i(9 zbo8TL(3V;#-xbIDV{N*6_d0JhjcZ5sd$LF?M>j+xB{v=*MDE|~K-v-3^Y-!EKVu28 zz(7aH##9$J2bg`-oQ?KA@AM;f#gHaS1{V)T1>%p%$Ro9PV3x?K$2*+Q?SauRy2?UZ zUyLa4U+89nfC;|gL~cZrzdrjtFQQ7^y7_zz#ep{4@tFO*b{9SN*2D)@>{hWGzuSgB z5qJ;a*}M1fHB_t7j&BCIm71aaZ&9mf`HQUaL9B_Df4EDO@_EwqWqY8*ma(=Up*h>n z*DTONg7-=*dz@c>8mf!TnVEaA*esULs|$OX>LKHRmnW~C93mW@1mhnDRnhs0Ux zx_R$WagY^%_K1NJZu}f8#t(qf)+`X{t$I`25d*mMot6Iii8fPMj;sPGu=pq66p-PQ?t0Y1fyb(jj8)cCQz(m&_3JZu!%PZF9j) zs%@^3;qoETE?S}IOy2~_aM86|Hm4Rsh*@A@DnbF zor2USLI(<(Ev#Y^K3xOe?{@Xd`GkaY!Hya%ZN_myHEW zvrmdK=9uN_+bpHXSg$x#YFfLDS|q~0Oa-Bf@2R8L;PC3NMfw6#a{UtP5=i4WU2BMX z?YN3kr!O_6L3N2gJ9k;bN&ncpnII#Yf$B7SDaHA02qElp09Oxicqif1YBD8Y?uV zC=ny9qL%f4ou1u62>czxcJ96-Ey~^7_aaWiDkA~C_Vew-)EJpU>=@)jrUhUBI?V+P z##!&!`H6pZoBm+>R#T=l=V_S{IOKbx)OdZF`jEZRW+p7Fy3bEa1#g=G)D+|I=ejx! zJdURb!*4bwtFn!a)JIj%TzQ=~fcI0uB2(esd_KW2w-c{wyQP%Q%Ag{|&erYMjd8oQ z4w%nq=lxs}jrWf!{7%O?j`?k01R;%VrO*l)|M9?l9(Zpta_z~X4`J?7FHQJEDZRr7 z`iBbxDAr^yBXMU>kj90S>9Dv&*uWktX(7FSg`~bOj$47zP zZR`K(`7}3RbUs%Pl-Y{6bJG&h`B>olNpcN5hfobF#^ucOSQTp5lAv=1Ah{3h~10#wEC(|e?Q-A!GwLVH_5z7 z{6ut@SsdSnp4N6qVddg-W}6-RFk-#)~_nb`hv<%${~r)fq0)=IM(*G5$tFTD<|x9_y?hkxsS z0$46KL{3z6c*ZOP>l@BUks*@8Nc%xQwC_yG@8UlZeO=FkUfgYD!M1F3OiR}0`3O&! zUOb)0gwpm4EP~~CLWHz`0h7>{L2x2Bb)x<%8PLJiHXHB zk(=a{1|C^jN0kQlU};eyC50AZE81$tO4}1Ww~nr#fU(06*Y?;3n(jyh$)&qUj6t^I zPr&a7F*+&Fb`Bf%tQ(bHumY=P$$JYw;esyTFvY{nLcg+=H65D=A&0(`+6_PxiLZy( zPQZ)b7{R|*tDmPGeg1q?5HA^wsrSgE8Dst^PywBVhH0zOkBebrg$1O<-U^5KnVIfz@imAGTKry0VI@ppvV|7nl`82g+UuVzh47SocW6zQ^a? zcJ4vP3S54O0v!@QO(LC2V{!vyW;knr3sJYA(j3i4eWrmA_e5Prog<#9y#yep804Ix z8huvwNsM7*rUvKo>1=#Q-D6W_mwO7;23PkV>LfMX^{O{X>Hoi`P#y+6{Qz&zqwkIu zN<=RettQ5i%x%GtMINak*0?p+Lo1JWG)7aCT)DUcFpk_`fgb>@aTET?Hlr0o9d|_v z1Yd@DBr)G*7*cro?_mSzoQArrhu|N!C5{9GdU@n}7)O zuy5TMD@?;yYRf@ga-fHCY}CbdXKhH#&{lZzX*TPR_@6mB;dVF!=eidguZc%SOp$8n zGzTRsKTYFeTUvuG%+DU|hyu1JVkF6xduSqSpg|1kb=VyBOL|q`@J?!%o`9c;N2S2_Pu#5) zhiJ&Q^~aRcKMSv8t_YsqY@hposK-_@8JJv5B6L!fTNe~T0~taNcD(w6SfNxxffdwa z?z;7AjT-2@)xDginW5qDxD@|Nw_}6}X9jll!>ey2lA}BiYgd15#F7L<0|>(d*aVti zPAI8z4Jt&NpFGqR9}hp`@LXAEpG1o=nUePjk0TSsnMa)IvnjBA{md~j3;#rRRD*F3 z2;`B>#~SJ)F$dfxI(Pg#mwkYP{MUHwikVw2^d3<3 z{QAr{@wR`7!`wxVZjeP7k9lUEJdQa}Q<%4?^HBc3A=B>Qcl=mx%@|Cyj!kubF3!#g zcVqMlC3ekD(79-Wuj}vYd`{efwIluPH+xtYFG4Y;zW%GHnxd*M^-=Nef=NGO%o9~F z_R2|5bl%9P9H09-EJj=ZdRpe6Kfd+_&2hZq_hng|U0Oug&YCp}HMVd@sfSzW>VcBF z83<38;afun$Q6Qr%f^P&j=ROz5g}g*EL6(IePLl6<;8pwi^-Ah%J3Q{?+H;6WBX33 zELB5ap82XqPMc-5`U+S5d8%Q8g9)Oq^Y=*ufEvo?~s{ zi)G-|4gB7)akp3UCRk4=Oq!BwkXt>Kd~1->*h%nz*hX(czU~+K6Sv;>)g_cji7WNS z?Rkvii;K{fp5yv^ua61YhTo?nP9(fyY3mKVCX1>b`p0c2mekwKDlgBhX6~3X!l)@` zE@>2gkpfw9=TvNqf89*{Pz=Eu*1*n}bw@o}-Z}-WARnq1gAOo3E~V|zqHF5{;Ac!- zx>&mJa>=+9G0QjGeJDDD^7QL-oW-IiEIj+~%gt}D*Xu*f)fRj2Gb;b4n*y`gtd&(V zNmpuQ!HYHXRdplW*TXQ zerTK-G#*t()36B@op4#;nPj|UGkP@#(hiQ1t2V!Ryag<@M|pHpR>o5uuvi4#=YTXCN?2upJVc;zZY!;_Q>hf3`z&980$J z?F-g9)j#tU+?5Xq-2v?mwwKXcg)y!^VU^}Y&2v0Og*D7=pf(aN7r zBPwh`HT)oPQ^sx!o>+v5l~c&`DEjzM_mXfg$hjMVl!PLu0d%EQ}B zUy+SAva7nzNQOJCc6vT8?q}}P4B{F=#bH##<4Qb#Do?Af&lD^kp14^PFFgy}rSdu- z3eZ63>0%|0d&n|R-6c1^g5_pj&v__&PkcaCn>WlNjF$3w5E6j2**@)Kro%VD<~nt~ z9!P>NA#>lehTH3;mqtd40rVC)X#$~gg5!RPM&!_@k(}|o0Jrt!`b`V-rZCceYuYgNlgi6Dd&1{l%wX zr6Top&K#AzkjMavhIyl8m8%@>Gh%fP<afauxD1@t*PwUKyo_bVWCp=BAIj-aVdjYKf$O)bsOERd23V8h`dVn?W z*!YS#ttfk?ac**1F5ghH&Gaa9?#DkeR;rx8otFb#%CB>9ne7*cw6`f{XO*`GZmv7q ztm?|$w597N=&>KZ35nyY7oUX`HU$91frQDXw<9?Q^~a-`3&&D>^upW^6(?8PdFUviUy-r6U~QSOILTHA{*O-d4QfR%n8&7wX1gz#KLbm1%Mf3 z4Lt}V&Z&43`iwKnU`JUQ7*sR*z|Q-4UCl@BgTAQ<#9c2o%kqt)vAUFr97jsWB-#h* zX%z)e(X4vE)D+9C*I^A7ld6?rWXSTNMw(BuVn@JMt`|ing*J zx4qld6m-z9H`^Cd;_#1QX4GYyFnc>oisLe{hS#*{rO8a^6D$&Jne=IY0q|C zLwf0^H5i@QikcV#TYju$-}fiiYp)*PhtI&OGrR;H)#v#5xPKbESo{rr`H!nGMRixY zj@O`)viT=w>qeb(hu6u!tAs{lSQac>CWyC?_)YPS&Q8`l)-&hKg0BgEJ98_|$zng< z)8lXSg83*mqi=l8FwLv1C>A3{AoXM!xC}U+aX?ZN#ijGaZ<;+gj-=}ZLIiZiNgXlw zbNAM`;+D*Ta-z{dcaMoXr@?YE#D#a#1l+}E@aRMPL;Fsq*@JbYk7n_^SqNw@xjej#FgH@Af6@0*_T z4Z!!+%W$xv_RHjtqwhuN^G6eX;!-HNntYatCG8$|7zw#QR`1~WcpE(53F@nu+g-+Nk*1eu-^uOp=B^D%U@eVqM9&%Rn!2Y7pU%V$(Yb)Z$ zAztRkGd%(D;eRP6?V36i8`rbys0T|B48ecIWn*`LDTCJr&UiiAOEvDC>G5vb+h|U~ zCPP=p-!s?ZBsE8m8Un#OEMXuca%qGs=Un&8Z`ah&($`#lgA+U%d*i@iK|L zzfaKOIz}Ma5=~40WNeJ)Wi)f&*M=iUsoG;YeVe_Na1`bUtF!3hH;|fBK0irS%XhIe zF^__5tFsYIknx58C?C?7L#fa%@iamL$FYOi?S$+s#4cnRq#jfmva)r>koDl?7nrjI z3^F3_l*xc;O5f1q0kzd%{JrJ`!LR-l>rVtNB4hm@3A%iTJ7L-DjPlPmw%w-r<2hf) zfvUgpZWmY4Cn8Nha1bQ$>QTK@-@89Pk|3SIyO^{rAG5h_z2eCxHBMfgoX*%MHN%Zs zdmrdeq+F?QTjn2hTO1#4A`@@HAxf3!z%|V?%l&IfXaE^w3a(p_^xM?#Qr=8yVk@IL9 z;cnFNR7#b}@(?Ke`c~x8yRSW;|;Hi2uB zGi#<-60c&&j>|v zJrx9_DEgA_!+B!A2@Mw}15h(hR2U(rz<}a}V)m3Vw{my%)WvGp_U={NVGRnB>mwkX9iS{!8x1z)g)9 z_TG^{nZ$Flv-3y{o{XmPmCU>;&EiW=HiM4GTe_nz@Koc9=$v4U$3gp+#sY7d6A)Zf zj_~5yHad68*Df}_w14Qd&Ajz9@ZD~*-Mi(L`Zd@{Wwo}MS2$O@OXt8SCo>5O-3ATs z#}6*E;P(UQt-)j)prxx2? z{oaeH%vYWlX=G~I;SRdMsAic;)dWCg#w;*DN)Pv@Yy}Ch)OvrqD?bvu#(Sz9Wo}@&ZqGbewaxyRH52nfD91=Lq_Ram1HnE) zi#3mB;ihh9>v`BQhp?t+wl?jCOiO58hjB z4L6`EbJ)M=gY%Drt)32X`&bBafB$_Q(smt!zue{J>vA2Xp?#c`8E?34Z;5q?tR(UJ zMzF94aA`b--c+q%cq(OvjVqs>E?ycM)=Lvq=~ADr*NTJX(--C9=uV@qj$rp92y9Ue zm8+PqMoeVnPG?B~(QhI`1aHWn-lWxo80)mjALvb4El9|rJu|%jg)=9e&h@PiIJv&; zF>Mebgs)Q}P`bd7dyz!Gj zwSjLFIHUjEZwOmYpazX3pGp~>?0a+>nWcC}ZaZ4r{j?4xyWFlq?z^Ywf0`)+-Hz&S zj~37F3%;`w;J-QpBB=t>r_a|5`%%!COHA9n%2~+mRR|B+JIa z@&`@BM;RZQ<4@rsYlW4Ko71LC$a6E!xDpSsQj+_F;mnABTt5jX z9U*X7ng2SK{m5(G@qpO~I{v`(SJb+r!AnM-HukfsCOYV)CR5oc}8y$Zp(3wXYs)8eb^F&xXHa>fH z$w-R=%`fV?md77*gY_6q(+FgN8S?0KwAdnVQBw^&Ls773IVXwX$dTS{({>34Ka69u zO^BgEG=zD)kbq;EfVOL*vqQ|(?I5MazQb+MCugRl>!~>9B=u=KH1g*Ryy1+x&TGBP zAY6=A>*Uki~%CZE3@oj2G>`RO{Z|R^q z6$KdQX9X3m6_=^Qn-8@+{KIKZUpzY{FJl#uhlzE@4&`^{AA^9u`Ww}bJsB8*8X;fnv&B>q{pxGC!+sJ6|44?9`nMc!5qJ>`)y3JBJ#WJ@k+R&NqHhwC0+!~L`+)t8cP zd)BxnZKtg~zPxN6F@sIk)&{c?E)>mCMXUyVbpI^q1Uy>j^~AYIG1e3*k9xR}a_XbP z5Eu{?=}yT#jKzT8zl%z-SkmeDfn|F$e>T9Q10h}d7+E#YO*cM25_;1yMdhS{lHd+f z;bi_VoJI-_gY0gG+>Q%gHo9aEQW=DQNlkfsy1H3FcOL32^l zZ|mv_bYau@&D7@(0D2Xej}cTY7vZu*^#0a1{Q(;rE+vPW9M(i5UeP*{_wDnpWLol% z@YMqr??X7ht1+j!QjX3GUDY%!>`1nF%VhU`vM4zl`Adk$*a|DlSZ^__ODDS)w2x8- z=w8A?K9$dW-xD~{yxO~>Hz!7osJaMrmd9sx%hdMCHAvnWQ~Soz;dLKiv5lru`+XwSgGp`aUN^&UJcp=Lc~uZ7c0qt3vW( z82cI?X+ee?22-}K`1jTB;_oTCvS$3ajcg)Jk45Oek3>VV%&`H}_o?(eQ$ukK%lRC> zhR9yW<&V`WoGzW#yb*4erlV!zdg;3HBcEQ}ZxVXpNkIqcvO#{3s49!@sq&7G`WwAt zUZ1%94Yv|GSuZLa@26U6Okp5Qs~$f*cHC2jmde zdP#Caf)}akEz5Y9&S7~eLKnkme5_Hgd3Ii$QUqQ6b@N1omp`1-NeO~qQ2?~uojrpZ zIS;-#{ylD(!Mh8APz`rE?+)ocD<=L!`VvQrZl3#$22k98cQJdH@ZSd5e~mTPrGDg= z#`+iB#nZj}FED61pXntWW~c-~{=V2RqY9(9s6XAi1$KarQRu>T$mTV3@jnM}5Y4tj zw<=ICLEZMPZ-TPtl=m2E!?zmBlZ+bcmN@k)YSaJ%bbni%$+`qq4UhkLCzYKANLa6kx#a zx85i`#eHkN>hw=~Wy}ho$S$gXTE41KeV?>9K6~80EIGJjbsEGiQ;A1_O)+u59rcI> zC0F?li{K|l4qh0au8y=fTtp^t+v!oA*CEW-Et3r^H4{HwXlQB@C`;??FV6vuL@)Lq zbWOQ3|JCzXUL2^|7zMdpv!w7^%yfC@gD*-@Rn}fJYl*6^ z&1f8J$g{rzg@t-_-!|s9mK4`jJsi@0$AMPk@?DAJ@=eP7xS-%Y+p5A(r#8lxiM?;h z6^eYUl}|=q5({}p&xqa)WH;s6P^n5L-)r!AtCg}B@6ARdKHqg(4l9PrY81IsU=qax z8_#GTmB6XlCud)F=!g*^x`9R~*?yhjzhYc`rYp9$#e%I2AHrgpVMa_THjs=^cFu{-oYxlTUFF{wcz=m?yS$?fe(iV;hsFWc)l(12azI{~_uu1ELDM zu8krfQUW4ff*{@9At@l;-Q8UxNSD;m-6>rIBOo<&cQbSjFfhQt$Me?r|NJ}WzV|+R zUu!Mh{kEX&@}upi?TF}*vo3&lUNe*Xy(_s2|0&lXdIgscUJP&G$Dc)lKkwOp8SK8R zF85Do(%833Zy*!{%doA6Qb2wBr0l=wgNZ7V&tLH7|CDVzYNzV^(0V_jXiBFM%uVU< zm14BuI712|%y?n`)t!nDyFL37{BW13<8N%=ky1!eCXp-JO!Ioy&wNF_o15A3K_hoJeRuq8vZ1 z#sUQbEI%3lj+l+P6E_B?)Tc$jCN12RixWo$xn?xfu$BIjjUFuol6Wr=U~xEN(67{2)cBo5p!1^)^gQS%?ioZUz5}-47<5y9j^cf>DRCRwhaX>3PPCWK zj(NUQZVvtbW3%Ie5Fg*zcFFOFL}0AZ+}+MjSa>%XTL5q-KW*oB3`PY7@RP^0S3z__ zpMJ8`R3BQ*?4MkY>kDFR6^@aM?mcklIK3q6<}`-hI~@OVzr1sQ+CFFbjqGomMn*{7 zXEro;Ni3zrtm1f3(pn`z{xIF#ES(BCNE6=i@wH+TNTV~WzM?ew`j4TUA};#LY&?d` z$cacVb9pmRE0+$)e}$K0dpSMEPtxH^{*gl6cRUa3Zuj1g;t210)?UO&s>Ry zM}^p7GU=t@JbP|^*yu#0_LU*Ar5};y@_iGpC19%C_WqiOu>S%UgLc@~UfhVA=4p18 z3G~BW9{s7@#-diTiryu987X0(+4TzI_>b~b>N(5aHM24(Uh8?;M`NH4U$0{!tzKG^ zrQ*UPW3hr*)5x*vtqg_(1G9N5e$J_#N>3UiRgFlfs%oGV9=2fR3tR12G`013fZ#8z=UC$`A z>97ggaEsO=*7MDN)c|Dx+Jp_QB)C^|0ACRG^}HnF=n9-?X^XM!{=-$%H8v*>bpj)2 zv~x1XI&7OsSZJ3e0+21HViCLNi>HFQi_zc57EE+gX^rN1?r|CU&pu7KWD8|K_cxJt zXc4aOXM02@iPuF~GU1+oO@t@owR;PrN?R!`T;IUV69ZKsHh?-t;nNn=|IVSpN26r~$XId5i3r}NTa!4Tkj%Hxtt}Q`7v{uBz6|WQdzcyN>GFutjVqq75 zc8w~WG(eFN*f@32i_1<;S6L?Z=oD_*x2E@g<-SNExOI~s>mvy>S5JJ4Lh~*w$Me-5 z_I^vq@sn&wsf4zaq5Eyx(swR3Nr)%T8ZtDn}?OL}*zDw%gvq%M^dCKm5Fnk+%r@tt$ktSJtf2iS(%#6FH#s=d z!oPPTk#8}If9gT3Xt#oX+1dKT7o;^no8h?1Xk67#xjaqPKaq*k0G>&;u&@!t<}@km zgL~Wwt>FJn#_bDV?fHm~+KLnBoZC`;Vi~L>1K`du zO~c*FUD;F6inGTxua<{In6v-Ho?lisF2c!wd3}Z70%wHjEYh%P49ED|veHL`06N8} zs>$QCr}IT$7QIscl7aYHrzLU$pKWk-t8Kt$CY2-3vie^mkO;C7;bD}%wJM^eL3iuy zR(?!O>|Q#8Ck6JGU=u3dKXm!PNFtf^0yqBitjzpG)03g5koKa<^30@xFl1U1lu^|F z7KwEvcr1`s`hriiE!m-uovobUd_gA!n_^{GY#H@We7c1 zo4lF0aBdF{`pCS5Aky&Z0u+pXL1j893Xkupy9GokilW#a)(Da^;U@fR zB%gDRe}#lw)>VQZd`}2PEZ+2y!tl@6KF44F3fc;kv1Qf{rwb=cE9qGOD)!A=0#HRB z$loSi*$8vZAq)6U3Lja)J7<8g_EAO9(?xK7uR{v~^2vjF<3_tSLdRR0OOD3}1dfN( zlSAAV{A0Y@8=V&XM!L@z!=bR}d2x3}RIx5{ShLF_Yv=P9X$LraBq+SPWF zbfRr1{S_CfL-n(bu%HSwm0<#9QHf7rtPy0?1K09Ix^!yi=@1I5_!tfu>K*Z){!>s0Dp4#>JNVG4bsx?6QtI>$`f2`hIva=>kSa<&rG?5W2a9ty>(ddBIK1hV;PiMnyIZc!y;P%Y!S`Qe>Y@jC+pmmPT5ILE|4;Hz7lKNpbIq`$CQ0eUEo0S? zkkC=NCx85igC67*}KK@qjL8*el#jH;o15g{VGz#i$99=`5KLFz^ z06A~}w*BN#kYlw@au2c;jv}6UAMNT*kO{6EoT!;&E36Wk7XeIP*22Ye$sN zUUBBSR68Uodx@J@?1>NX3g>s6p?I#vIjjAXbJXXwFgzohmv3b9s_A5IJ8|*OC!jQ& z1c1;x=+m!{zg4|HDu?yEOQd{SK3Le-Arfh>iLOfuA>^QU&n0&>sQ2GvEvGbSxas+N z>~t7$(I|Lea+p6x&Tr*jv$yK;$d6>-foN!)x zTymWIBo%p-@&;;V4zI|ck;1-@jAiGHHM!p@n=i$TynZVT+00@6`KkxvBBK-kPVper z`Tp&?6I2Ed3)A%)7gMO+A=Od3oHP4Ma(~%!+^gJJK*UkI(f>J`g+j-pZw#CSd!HPk z?y1H|p;0KGx64H#nJY)oX{2qOo1bH!r?Y=#>+J3J&;Scu7PZN1!d}UMQFks1l$W&o zSZ_hl9BJPNZr`{4&i#4tsr!4h@If&^YgHYHfn`yypR?3*lVR`W0?Q=NeF0N=YM0z@ zfu4Nh+SYN(Bj!%1p~pXgRDk)BX)Su=nVoAAX3oJ`C&I4RU!IIo>$ygUqk)T?0*Bb(su4B6I~8OqTdiZRfHAY4XNOUKL0w zO+^&h?F-o}S`r-G<4Mx_>l&i8vPSQ+afFdXk(z z>}&L3JJt@{nQ)+cBd1!0TrX+;S#oYt2F@E+uG~r0ayF>}j5i5rhTGO;a&4MH+x%`l zb!Dg8-A+Cue)x1b6UfJwFV7tokB;jUJfft4@VKmm)*}vNkx~1m>twEfqYO|f)d^_v zP(4sdEfX2VGl!^%y<02vR~dP24*7IWcik04z<8|?Gl%qM7_)bn$z>z4T;4f0F9o^n zYS^B$x67$J|RZ*TU@tM^xJNkmeH6KiaLBdO~XTLE+nUgB?)~PU+pi-$~(X@Jb5Cq z_t9go{)}575wVTp(!bJuj2`_YbSN61EaN7xa!{_ow3M4?t7>+gwVf-L`M6Z~@0<3z z6|?#AsZrxud>SQq#c_x|;`O(clJRK6qnEIq^$x?|Wh>1+)b?4x^jT#hjghfCf$gVeQ zDD`{a99G*=HYPyGTUm-yTrw-|jZ!;rv+jDnqXzp7ZI$eifT&A)o*JT!2x|*X?uq8$Rjb` z%$ZUo0m1xRW}DQJt4hdvB1ThPB{|h&4V1%tw zxoG}P?>qTH*g$s01i@TcIpGe)y-5Wfx=3@0(3ixr4%VB56_gUhH%XY6hh6o@Rf^1Z ziwJAh#Wi=pX>AVy%jY!u-P8ob@CpH93d{7Yo{Nt1ozvNY$OvU;>QcbQIn&5%^b4gG zO}`Nzi6Ot&*jS0o<^v62Jg;{O4X( z-m1FMvbs|+eyuwKzjM;CHMsFX+tK!hmF27991dZX;9segQJa_yUE=Nczx*5UI8jiO z0FRnCQ_hWa@2@M2&w8HWdC#+;GAu^Xp`0)wP2}X({*^V(r`hSX!$f}#@Az#vp|?}O zphyBW4V`)XJ{P{J5T+?nBszq`n@GTXW9O|o4tklYjj#N^V!fW;{MR7N+*jnmC-&Ju z_YCfyF`u}goHxVp)vIfcpXLC3cuDcBd{>THllX)1H9Gu4+rxpDc%i*D1CUR4ikndbFBc6>$5AMDyeNR%mb9_rjl zzsgikpRFw0y!Q&BFc9^P%(ix>ncFuqil^aM^(^22$G-5f+E4>bm_n2A0=Q?jWrKPa z7IgmrN5u!4E<2GNizQ)bIq5qh;{w?spZIoF{h%1M0=S}e11$w2r#U}mS5oO(cGP={~;b6F$XTr^{~f9`C<%&P0Um?8xCC>j+d@X zxk@Y36M(ekX0me3V5!-+M#0t=UL6nk6CV3DU=uGl1W#)0%T5;6DI1C}38(af8l}XY zf_uO=D`@viV4y-EKp1BZr$&#L30a$H^I}-NlX+Oj=l16qILp~c(c$thG!RUSEYqrU zq##mwiolTZh#Q?fHi!s{YCI?PGef7@^7HOlU-p|^cNC#*Zw{Nn8QZ)-!X}=oox@AA zjI-4kgB6P`vL%X)70S?;-F{*IzPzvct^g~9i*u+a#y@?VSR2!Q6+~3n5vutht(Yn1 zv@u5KQ#h4sdR7_hMtGV#@U^owcxg|_dUZ#U^!R`shnUJTB}-Ruw@Of_@Z?C_2rNc(09C6l(8e2f(lv5`{Mo**>_l51a3yHgUI#U zg6twIadX{dnu<}ECjZ{@%_VNdwaft5Uos{~40hIJ9O(*|=zo?oevy9VQQ$|OyZ1=% zzKIsTf5=p;q17ub(&J$oST1uUv#BPU`|w`w+tCo1?n3+Pe+Fo$b_{2vqO0QQig@U7 zfH%RdmM9Y5ULV!gD);^?+s!G}$)LtPEK(K5w=QL99;3!57`fRulBl9b$l4d8{}yA! z{@s1c{@dyAETK-l^A{lr7W}<&Yasp=%Q5j7$4;XZCvdocUS|1DZku<#5Gt*B07>0X zd>858>B14~Pl7&q88n0fu-v@Fv^->cz=g~6ZPi>bsN>A}q#DxxEBS;YT5-|DDDmxc z()WMoFk2Q`5%t?Jp)(nNY^%R7T6+ylH`~~%1+gfV|3yU}Gwpx4P7I0$erA@p38%#& z`9ri5@Dl7It5t1smmNOhc4~h3ej|4e-iq%aH&xA`Y=!Ce$7SqLO%dTkYS#3^r|UVkv3TE=g6m_ ziZmw&rgf+84FIvR#PmlW z1{lzWZk9;BhQ-w{x1Vd6OJ9ouZ__6FPPypq`@ZktXF37dra;N243^U352t@q2AC6V z-B6*N7Ed7-zV5T53$AG)J-i3M{Yb?1Q6uc{DEh5)`I zvpi8^k}?lgaliQb)@9(H$ltVca{38=->g%z^wtc;&iS@T2`6xzStBRW*l;;F)B38g zOmM+DBjzfJY;5~xq>d+US0EXN@2_-4*(mc#yOG7E-^vd13G||nr-a)XB7HAv05jlJXX$wCDdtWs)vvxU)5O2S=){8i~$)P_{ z_bpQz(f>nT_qTL864w7?X0qEXuGE8mo&AIg6Lkqps=V-6Vx5PtNiU7I+9)kt>s9D{ z-(Y<>(Y2fMFx6!ofjyMoI@6fi^O;3Olu|L}0njA7 zWy{sfW2$!xXOF&BE}p8NnR6Cn{0AlbRk7*{3UdTL;)rMUFc*VSEkdci_uSF5pY z^5gCb0vD-^uIH_|D_K3I=)E z?{QPAyW8F4@%jG{W}VBjeBTjty0jx1I^0gQ02@2lJVL!D-tM5zh{T+%f9!v&`V}d>PdIcU7M?fd&Qy+)e%(ky9JM z=&iXU1`JkwTq+E&KObefSkQe6QFi%MJ^CtR^XtQf2E9?Uo^921J@A`IxAh+uWJB!v zolwfdjJ4k4AqMH1$%RYiv zbS~z$1Cx=n0lk%>@6AJ(;rZxF0-Ms3)JT?BUCYLXAl!OB(RvmrN)%S9pkOJ=%_M5U zU?z|z=sX_GzIzu<_vG^F{TtsndY1ePk3;ul=J#)yLM?m13PKTF-zntN`o62cY9S+;UG}i!MS9xcT(ui9~Q#~90 z$9OjR6j#*9vJdwjGzKAWQA8>xLA!v|f7naTNYe z!(Zslv1Iu8kSbJ6{+M1?I3v$2x8xpp3vubG!OOdXArk&BXDK3`*YV`jLAp7|d3sly z$n*SniQt&vw(pug|4dKxzi%8$>OXEG{iSdz_KsIx+4JiY+0E4U)Us7H(yv5QX(Qg6 zI1sUrG17O6P5z>wJoQieAO0r>&}q1D9RIX!Ma7RRR>bYstRLOWBhUdMerI(iqVG-B zEfuiD)f1EmLYwU3sQM4Kv)H|^jz<`8u_OmLdRth5`-(ml_eE<(uuM!BLVsE51Wnu~ z?>;R4Rd=OW=cYzXH=@mSO0#q+@wD4s5ubX4*4Z5%#@cVoda`yt?-bRnA>@QI{BF_S z*(vzz34zAEHmJb=^#TaOM(AR?a)PJUx!j@o_{!d;JK=)Uyvq@eMgEKH^VD9Uak=E# z-DbDrIWyfM&~#D*nC0B*2xT<-p){K4sA-D6V6_yLRw0O>E39F2=BwM=#tW^8TCoyK zjzMo>*%@t{j$wa8|0ioNW#((wH9J5jGQ}x)Ixz>6)Ft zGP=3!1|+6L0w{LBG*J(S-E-yHpCx}=;WI{=GM^stIhxnpMS(}#Wg8is!E3>%!x^-z z@+3n>Ui6pO_Wc-qxYS!dt6VqzZ6ox~W`o*GZK@LNo!+i}>l=dGpgkq~N#Y(B?oTWg zXEl73A51WhF&>$ZJ^i2Vq)=YmEE&ZvzwkrJp-I=ihY5*O{YLNs_!ROOJW6ozIMfK7 zbeXmFTnRq+|7G(xBhP82fNnrfN$q`lSw|IWx**9+!F!?)(2%oxM#Q8%f9jXx57gg4 zvG2k!NvGKm`Jd?YroQJUoxY-tZEasrubPPIb*~5}+dzo`kOXUH9xNQY(`2U|ohY*` zp$Vp&{@3D3`!=i(1G|U}l5U{X`#)}t`!yl08$m1U0as(c?YbW!Y$}+pG$id;tO1W7 z^wGEeP$bmLMa|t6zrEzlzrWK8ANn$qj)GPDA;7_$ESm5pJoZ&F3v<6)6{R731je@) zWUQ$1;`QsFmpr&yn0!8z98T%(?PYz^mm?rVpHRU@K1tFVLwQG`bEQ`)*y+|fz`uuP zq^`+!gD>Z$g{ZvH*kOT_ZF9Be8*r-hn63_*8ETscM_OL8>F)9%cmpNxmUEFSA_KozyuWnYXZz zFCszZHCpZIi9zj-GA}lFN~GXPiGjaHCBc;A{3}rWZcop&{L&Y|>QwITl6v0QsNk41 zzjM+#fG1nK6T1H)oE*5n49z#oU0+XK(6e9g>sVLKy0sXE7k8OZItx48P6U`tuz^0~ z3@6VgMXUmWJN%2Uk{46v=NmYJ^atLh`FUA}*>mfZRwk=kp4hc^&gUL4*G_vS*hC#` z4er%8JDhdE{v3n9?0a$fMn$#SUT#t9t|a#J{4lsCpW9<-%5l5r#Jfai6ZBKr^dQvS zx9jt>w?n?1oh`ud>2^)pOM(T5YQui*SZFB!a6Cf1J#Q5#QIGKN6=2b(u<&d+O*=LF zF+=?}EWzu%LAxcdp-Rv`y~RlH1IuZj9ND?LaQOiSR!!1Ha@ZkGp~~uQPO^BD^Yx64 zJ-xvL6b4ZK@dl4yCD~si+qcLytW5bcHal6idU~7P!&v+wx-9<)Xn`$iBQm zUkuZ`{@m~gaol0)8|hy08tBg9oI)*j6H(KM%uV7%kh`?wwaTkRI0@itR_|g5q`L39 zKoo56aJSS5CIYq9XjdCV>%O(!C}6xaCGP(%-)h+!h|a1Sf6Yg7DD1Lrn$5?0t2Kb^ zCxFU<^(C9k<6C%J$sr55Z!Oww6RLo3za_B?{8t!#gcRF1%Spz7x2tx#TQ*NbUB%>* zK3Fuyx|*gNl!?*%p?F%Hac;zvdl;{l;fcgmY?ev~+<1h3pjxOmPeB?lrJ-<@gypQs zF0#I%?6>t<%J2aV)OvAb1MJ(&vYH>gyX_)G%3DVlJ;c9?Z1VoC^3O?hB2^>y`pv7^U(9 zPvU%KyNW_gw-MQQ2|e2L6BsVBdzRIf77H=EE>AKY=?pSo6FQZ{i#)$Gd-Q0QgU_S< zy?Wk_`n_{c&y98P>4Xg8sLO@_T0K)*q8DQFs%^FdczX2opWzXG!vnxOCDMn>}NFpvLL! zB--upQeJqz-7A9qJJE-YZm(GFueY=ScR$99G9*;+WKC;SR6tsXCY^1b`G*r^=YjVI z`Vd_hu&7%)r>68s-*_5NtH+QykWYx0WoLD?7T7B2Gi8f?LtLgBIA-b}XM4;ts;4rd zg2r^RUqUHeVtwnCJcZ9T@imA2cle9STFzY8k+tS4AJMph_mSPfhaAl7BG2RAhxxS2 z-|M$g!ox;uh-kyyj)gt#kB*wBph#1FN8v%}Ov48P9DY9V$DL|_i4NJ+WeH18ixpQa z91mHlkozg?rYa4!pq2h)c4Yq>+=Ga)jvj(gG6aQ3e%=9N4p-UU(;!Ae*&GMiwj`;y zAm{jD@mhQuv-361vi_Rv$<1jz@WSil>`^+TO4P0$L{@e>el&?j6)xDD zA&$DkIQukXBoB^C61bisF5*Lp!#J|jsIU~w^nN^keF}lAdU>j z1a<80UmQxZ@Y`+EMCOMuMZ(FgEs?5;nHc(${;{%MJy3y?CLpyU!a@`o8m}ZLc$q}$ zG}Xa8XQ5T--{tvHxVA27Po1?tBiuHK9Z!Ag_yeC_~+)02uAxVBve7zcO5a!G#~^|3cn4d!#bNVQZ;T32is?c762)$A&q8DwQ-t%LsyC7h-CD z8!wlYs{D%a6WlJ@T~zY$9mX63t zki2X1_cIj^Y;SHBWFu_Hq4$Bxvl4WQhna&9aI?l*bN!Dko55sm{3nU&Qxrn@vVA{!Pa zGid=vDk>CSGt3O0IR%ub!*n@(>?vUJ2>R7ns6dLT(ZigpB$JWL0Xo_8uVe#H`K{ki zzsgLG!Lraa!3u_&dpYm^ARokt*EmIDjl3AJ(%2-w0I|ROO16`)DVT!7wf+r(RJF~{ zyWtOcY5M5XF>`B#_ujvZ|MiOoy(}n*Ja4PMjC^^_2A(v9-R`2u8GInD zsX#ykX&y#~&6Ud<##v3$6^nL^R}l$DKACqSbVRTE70@_2&2>(PB4Zz&&x9T7PDJe+ zuc})9B3R8C6=MS#T-(prlAFJfhzs=cr@$2Tct1s|vfb&nB;v6;DrQi%iFFmfdLh5M zt}D2Fe{S^NjSYWub8bYX3vXMQL+X&&rW?+3dz4v^sISk8#(Ub-4l@qgyTOq3ax=+wNLOl zZ*GAe0^?*{td4p%8BBwt$kg0Im14Rn&wnppt&P;94a&5=Esr0;X;PXF`k;ag3LgXI zTk{RRo@WOr=HY3;y}EC2*Iup*<$=d6L~3yLDh^WqqM2;@^JjB_pHAmV@$Sz7E${!b zAuqTm1s3~4Cp>^n=3ll7Bs+%lhOpJOSK#_Ht=P`w#S95gI2$vwJy|kZY+8lq@|9i^ zj79&*Vaa@ka(S~itHtAy*tmOe;K7jvVs2bh8%wj$>K)L&b+}fV(>2@RPuOyLu-$Sq z0jjd}KOYe~W6jbR*voWI$i%6 z5qq~e$B%FE-T~Z;CDenDg`BL&M5;mM~?zf8# z=r2JjYtU~PKzHcoUGY&m&alnrKFn9d0z^M3`vi25a4N&o^cyrhy}k7nzj+73kH*K7 zx>f}o&JCXzyW8(7$rS*CKvC6VQpX2ewJAa&@BU#;kJ>Ik ze9A5ib$Wv}9kQ;a%)Q^fyn*e^c4G@eT|MMGg^ZP*N|8Hq7WI*9l@O-o^X?OR;0(V+ zAt@&Y{dhGPSDlm-^dqV7Kqjk?NOQa{T8iKELnRw+vT+`t*awuy5pr++JQs2}Ja-y^+ zK910HTirN(LG04%!R+33*ugJNADcBm%#F9yax*hkH9l~UecZH~o*(C+4?+&eKAwi5 zsL`oK;;N}oJ8xQ(o|SCOa22>i%Gos+S3LA5lX5ph(UgF%I!Svr7?-GL5o^XPAz*l|scZAuZEj z&Gx@Jkr)@6qdvoA@4PFu>Bjcd$JFSz7QQ1RcUs>!iL48oR%P;!j&b__eXAWLU*vy!sJ>&nw2v2G`FqxnZLBSs zb(SkGiFhJl`e;k*`l#)n^Dj!|DVK{f>C(8FW%#uxUvEAA>Tq+gEPj+HAMoW$MJ8hp zW*Cv5W-Jc)jy%S?$>vPQQ|xF;E(9TD=|Ii5&^jqkYh? zx2UBFHP-$f-un2*!A1DQW}+VJGQ7vJu}JFm?pPyD9R=TFBprp?BhpfOQW*f4^YJ=# zmvqg#>3j}XU&pAq{}%xZhryq(rx^Vs{1lv?&iuS{gKr1Z&%>4abzAn_kr3q!__5H> zHcE@xl(d++zto*6@febAJ4RccN}9y3C&ds*azj4czZS$;yt#zOE&Q1yC~OYPcKgI- zti>g_4|c&Y*9XKH4Qz3A1@;)7Ruyz#n&uK@hT;7z)71L*kU-j!io6bBEBymihH)81 zYSn=4=8 zE~SR|k-z970jS~w4(=*iex--^fVrpH@M^LZYAorTZBrReSaXx7vY*7DkF!~%`T;*`iDP;$Sm6PrB0Z zCULTqBnFzIKU07cpoIw2y8b$3{n~Xo(*8vPB$a*AgpNG&8$;9;7VKaw3K`Hur>A*| z7;1uRKGP#IQ+X3`9@)oOw8 z?g~|p$&*fPvfFFJWzrq-2U)L*MhgI11jMX1P#i2TEVY~`mNtv+VRxCUIP?J0+HN_=&%F z_r~ja(T5ZF%m@)FuRN(BDt}~B9UYy!kGwy+uzB-E!r`gbiXb=l-Z?tZo@tXNjfD53 zWtkHDnSYR2{6!dWTT1ARj-NQ5S1CTE$kmEeOKq=aLXt&nBr?W$Bc0ne&BTv5v42~QM!nuyE(TITOtqL9P4 z?A>!5wQ<3Y#g>mpUGuj2b~Jid5j#7yx8Q>KY!;q~R^ItN1yuof@g!OU(}VRbhg_u&<*QdhC`KD$_Sy@_U2^ zEfohkRs@lbD#nT$++p3=;y%H&g+?%dcC+)xU{0F@ndUU0l7%Q>29LQbz>Wu-Y3;gj zBD&c&e(Nb0)5+VjW8aW?8rNw4RXPS|&<|!fdJJEy=LCHPv zD!BpZsbwT^uOP5?*9V=A%lf71!sm4(pCc-;HrYc!wCE#0q3Qg1fto$)yq!S8O(V&- zZ~J;Mun#z6K5kem`8@O$E<#CMq|_|iqCMcJPE~{Lqs>p_(O#CnS3+BfYj*pJA zetwX3KJdiGvjKd@{#nO}xem}-hj3i-Il8h$Fa!=%1Y7r#h4#yHwbXl)7Lpxb$zpp@ z#d&=tgT9}zp87E+61?>NwfC?gmdjisA*#p=!?3SGsMUpwEf(k8B(_WigrY?_p0Y8%CqK8kbkLSsddZ?D) z?mYpm$dQzEc-FUS;=#YWMA+od7INo#Z~^Bf|adn?SUx&+wFH7!5rHQA8>zfAXi0pkJ?}aBON($|J8}i(E8xgxMJnTI zwsPV`u5`~Y@^`cSkxbHoCTF}EXKX^z!Z80CSAdZq_pK|2K`<)T(Zntdj7zn1aoKDF zAJa9{#*T9=j9b~w42T|ShoRVHhE$OAVnE%J6pC9e0f=;;5LaKCT^F1FpnYyNW4&G* z;98}gPLk{>LU+o@s^_XtpDmWy_;a{DK)=kB69nRv)v2?fq;%%FkUah_6*iCbCL;EM1aj+3j zWNanRN3;6jsY5lgY?m*6V{zF6#_^!-{>;@@Dhqxqg{OGWxx>pU6GO!xX9-ef; zL##LTE}w>Y%Y#$gH}LQZvOf}BI-#1!vfz9YMjcOhb>F@vGpavtQH=EciDM5f99r~< zdhL5p#ORUs#H#o#uuqw$4JqmbmYA_52+uV~e6#NxZ^}GQoFMSDvsy!Q zDtpZ94F0~(x_sqiukNUY%8$uC_LexDgxIldAIIT!)jY+fF$bCO5Z1xBzCp;9Pp;_C za_@{(%9am2mG)Zq5@7ta-o(d|1O4c2ZcQ>7-lLdoB4du}8groY0AFk}(eY7KwO{u& zoKZ$#dFb-88QODL3tN0}fd5wZ=ASAc6H=s1DbrqOQmN@?qFH-@RLtOj?>C#w<|GIB zYq%W-9xi5nBy2Sbco2m@81Rf3CE5wKnIjf2mY=M_D)HSK>-=zyxgzW$iG&+kjR4|T zb2v~2uYhQ<1Y<9?XG(hO_5e^63tcJ@HJyhV`nhhDuh@C4c;sV-A3zsweFz!^_Xx=R z4R2%8QWniW+z%DRtMaVAzPeY$m%ZmOoV19Nmlc>{(;;2HPU(SW#=4MPr2iqS!tZBC zbM`y2n;pU?9SFNT_VgfBH75iYIXv33&d5OHd?dw;^gOzdgLXdbc+V)1C1j;I4^i6# zpb2FO8*x{APD#)FW191nOX(?_+t**3(J>n@ZA-MgT`h!U+B==cu4JAPU=BE>C4 zFa$THZgNw^$=&bEeVXZN9t|eli=nw^Z*?aU)yQ%qKNGm*%&fz8jGV z7q(qc5A|L`2DEx@RoWn9`t|o%$Yf00ceJVK#^irw?_{iuUk>7b)l*@r*8&rYV*CDU zK0-u!pNw1$JHUq#5&cPN=8U$sa$wEXDnHZDPU=-4euhgq^Oe$|7qICrVMNHgWdyW= zh;X~u!vRBYtVhCO&w5jjA*l)h4SvOeAZ&{;bElrl($2@tDiute7wN&Uyy09^ou-F3 zj^+0y^SNZBgVfeqyDbsp0l@*YqkN{SLfW{1xGDv4r)e5y;^TK;;GF9=$8~l#$q*EH z-+MwC-h(Dx(60G9>5Fnk>^?@IUer<~;JBor{v{g94AKf#tP0k3I`|WnqrZHmS+%t& zZ@s^TUPp96a+AGx6ff%{!zg6MuP>#4Aso_QfD~CBG!n}e8S;Jyeq)ODk=`#$R zu`WvDR0R|V0=;+J$$9t#j{wB<>+bIHN(!9wN42WWpi0D>+h&|qKX?QVNYzLL5q(r& z8`#@~MwZ7o5M+A)K{K3G3%NIoK;5%B1(R3rm+ULUj1LP;&XK{C`n?qq*YpP!I+jGb zQVY=-Adce*eT{uAxT^9%T<=_4c81;W=Cww=kA9f(Lm{T|njx- zrFE8;v*QoB2S51jsdYR&5UsBZtTH&0uQ7tA%Nm@?#Oh%E1tirTGw)0SoyLb-4n78| z)!9ScgT>h8cSJwOu~)vP^}3tGeGy1$-~TqEKh$s8lSohu{@Kll{4+nsjesIGK~F7> zintRKZYf|c-JK!y#v!tkGu5U%11dg;mKEQv{Ow4Jjrzu5pi!RUb%wTSn#JJd!8>

CcGa9j&(&}$1L!5aBysQp)&J2hnZX| z{owo+Ty+nz`u%Cs|J769V@5N+_xHRA!{DO3RkZ>s3ahD2hEiQ|{<7KF z_gFq-Y4rwz$e3+Eua5JL#FdGX4z%jGI;H01HB?zx@36-Dnd; zY%A&7BQ`twq$+5AYai~xG>{Qr*&JbTIsubkQ-@MG8d+8o7u^o13jfpdV8ld<*v4ZP zw8GyRfG`Fc&pRQ!90ouQJx_fvm6Fo`25&qDesi34GlDV3_>MC8 zA`KSpr^^IY27__~y8>>nRhsaW(G-XHoBXeAyV8%5VpFT3|CO~F05+9dfk>0NQ1Yf^_9z__lK6_oAkCfd+ZOVFY%pG>H zrNphU$(2=eR`c|7k0@ogRsNh<%vZ*4JjuZP5mrglXLAN{{A^j)RStRc5f`voE)o{n z`g)KOs0!owYdax<$+rQzor6vvo>x_mxzWIa%|YyeL5wu3;jz3o(i-Hje8FCzC|6U| z03V~8<+s)8GNTH2iy@aAsmgi`Vf6_4JlE9kmZ(>HURoSqUt%|&%(t}kBECQvSdU*H zgYA=oFXltTCft3!N)bErM_0gZBha7TO&P(eEoF}jEiX6kNi`d}?WMTkrHp6%b-e$VXn$*c z%k$W|XfnJaYvXgl6+V7psR+;V;J!sI?VdSxCpa0;J%$#qj96cG?mh+in;0*7k=q^4 z?6A3sfWwKZCw0GvtEeAXuG*!5<$76b+5N@b-6wNpezd#;25J23XqEfOGHivS7zjX7 z3p6+JUbkb6EkW3O&t-7D85P0O*S zO?eZLX}qncUJGs{**eN~@%+5HM+eGwJXlaYMVrR9GWjeq&jY^de@paVz3X3;_a>N8 z_3EK2_ZAbs*${H-!a>D22dUF;bJtWyhDo<@IFqxH8b>L+@g_>@19DERUp^x2`Too} zjg9j0#g!9=<5_s5xV&-*usnsuSUNr&HWd2qHt=p)pVvVMy1tFLG=Le>N!$e%UxiXU z)yH}PUCh(XfuG6p$V(Sh*|fMd2?tuNZ+;|d8dRPKV%pJ^~Fu#r3Z|4R` zA8;@BwHi->vu-ud^x#?k!5Zs24~8{iQ;hV)24>B%FN@|1>W@MK4RfIEm@*5^ZXovM z*o!64RRJ}EUF0S=XfkFAelPVR9QxL+3!&z_CvJCrVn90$Q^^7}3EJtRnZ7p`!y*VW zb1C-v!Le7X^%-#^`&OHk-R^3i*?+>pAOVo+`>VmyQe%=OWNaEOy$%d!j^5MQ7*_iFS^hl^hk}rfcl`<1A*Mp~K)AoGF@GY$Xn2)hB|qL;J$lYf(Wx|2R~mv7HKO za2TXR#117zFVQnnUdD`cLgB7KjaN7$WLR(dQral#TJJ*%oV&l> zQ$h?uc#)wu5DCQno`+J2m)-vzgA;V}Xnf=J+!;xTwY0Mj^x%*Ol8X=D)u?*o*x>xV~3uV;m4N z{yh7z*ok#q6)L*CXFB8tM~A6-=&Z{+o$L>k*_@vxu~ySC4LQ`O9E0lvUG^MTe0RK! z=O*k6BnDisU4l=)_}Y3ceA_D3ps~P$nRCgwwi&FnM_JelA5!b&PB;B513yz4236dw z7op%_y@0z_O}?1?Y?-0i98MUb+Dnf@4t4UCjwM|(&W5*~=6OzOw$~(iSf%UBVqe z1%yM#r9{66EiP%#@+j}M_(vRu?4EhjOiv;vtlKl>6>w9`v_N1iXRa4i>P?7p0!xL! z69uUJL1DABL1BXu8DF3&lm8D|FJ%C1tUY(G`!isR&&I41VZh-~RaaZ2nEGA&$QlsPaiA_( zr2q2sXeuBFc%-Z?lA#S~pj;nR{s%AA)S%uHmQ z%Xv-_YMb|PmWpobenl4c01FunqxP-NPHIMl~I#3CrqaH_JJT{Yl{JliBbWap{S z`nscG66e{VimBv;Rn_R3Flcp>(Fe_OgbM+F-lPmwJX6XGiMhx5OG9;|dSJoe#u*#x zx<=kzM3FJn`HLp{?PJwlWysBzF5T+4laR6Y6u&>Ah0$0}H~+AZXy%T|9x@-Nh&=08 z8Om!3HMjBsnujc$DUMyar5elXW9tF0bMm=sBQFZPut}!Up-UR!Geh;>tSs(?NFe!Q zSZ;`2w`_r=Dt6-$W1yoXhJKp2V#}MQ8)_b_6yZ8w7d$L*H4Uwj$RLpT+hEt^o*)Tr zI3jbavS%IEcNR-dmzD~Ch$dZk-|!d%^it&O=%ti8-kPrITaB@^KKix0>l_%{OF~UG zUmUdsCgHU4>Hcf=bkJGeA9zrk@-hBO3eegF_R}e_J$kq}#^tj(d@F8?S$TxC(b>cs z2Q;*ZJ8zosaEXlhZhX(XASh!$uBvXsg(g#wfFM|5Hap|`o=HQ@ z4T0y?$FtZZpwt-WV~ME;j#_;zR_6S z$Ct3x+nA6`w(~_iCm|&ZJLq6}BHwZoTH&CPI49grZ~ww#>*0i35U0DLxJ-0=3tnP!C;jgv(Wn= zsqkZuf=l<0PpazlPAiGQ!$!xIE}tfIU=6l#YN%*Me`Us~HiE>?Ys;`IgOSDSnRE@K zfrTcy@a;`?N8nQMg$#sWu}+-M7>)KZC~H+W!vZiZBDw=>BFWGy-qTfcl;i7U(gv|BSnuO*?BPz>6+9&VLRgaT6KyMFFo zn_&>!Fp&^}e{`@cQWTrbY2L@Qn)|u$)n+l34Tu>gF{oCXGLqj@m8L3UDJ`=3DOtg4 zuoWPVngH%CeIpb@CN2S4)Nqw0{e6dU|0^@65;YV*uK6St>Vgt!-GDz%4;(KvQ&_fi zQ$FZ6RwxyZR{`*zx=V$lzMW0D41)}0&^_sm;ryEtZe%%zH?}vde>n+XZR;&1bNE7+ z&>%qwhG?WxDF+@nEDpSc=X!(1hRm#=+_Dc1$^PP#Oy{S!izY|ilRk36`tu)=7m-d9 zT1`NkFjvtn&#}k256z8@91!ODA55_cx-3KW59C>4qQBEczXdUn zttt5IDn?2^PDj3DVRe(!kM>^|u}J;-6^DLRmd?5R#k9m<0l`L=BH59vAZsQJ0-b7M zezxZKmg!gw*+)JfE-f+oRy}2J>@|Og^vAAzgDHC*dYca!ljLSAF zdw74Kp&=K^XL@+aL1li4t50@!O`XuqAo6zn`=!Zdn+{hslXyRExX>1B=QY^PVr{*E zk)SHL?YpxMysh6+j%YHQ4U&kI$2U*su}<}( zwr0V5J+cqEsMUoM)YPY`)&-ZO08NE*35TURnA7u}PQfpV!U?!}Ba%8|^TIh67bQ*R zry@@Xd}f4`FQDF@+Xg*t4JNHgA~|L9F?>8|M7y*Gwu^76y)(ZcUJsr!1lemkW^`~= z?E5~X${u78)Z}aRm@oE=k;-Gm!FRb|uK-=o${5)qrOO5(TpG7_R#`~c7E7NiVD8I+ z>h8SVFv>=5c+u;ta4ED2RBE)XOy$AQ4{|5It3DCpo#u3O zlgoXs^)dc-89dGO^a+;X4POgbR?G1B%)-v=Oh4nRm3l|7Q3nY&+{c!w$_BkJOmZ#T z#T%Ua4+33@C7Lzg&_Z8euICHP(>wo(zCOJ?CH8^g`l`GajRC`LDyfHE>7O;|Jd{+H z1MMFHHdsGOW|-n!S0omb>`%+v9twpU8Pl_U>YERV<0eIR3@4WxLyCjp>h_XT2NzZ}{P}vb#mL%$5 zIbf$1{pQ(7tK;>i&!)JsCOPcgEjf2XjO>tkhOJ{{m0vGE% zbs7Yde*jN(WO`>3wpMTnJE+-VZNIbiND$k{K>D`oB z@!kq7wXqVohfPS?Ae=no@cY;^vs3bdM`A4%;atnfEDRNpBY5h zi0miAc3{X!>*#%9Bcc8M%=&Bpjdt?JP$xhs3OvPkePD6TqMhXqU6iXE#K2g14%6r^K+Xk|DdAO3q!k%y!D1dzDRXj*- zEZjvL!%pJu>6$hAB+m0I(tK$89#IKpG#P=v=EqF#Wh7?g()R9-)UT}ehTEt?ehCGv z)B(9?jN}BjG|#3iiXY>ZehjY|v2E>T6sI(fI^moW<6dqD5KdJw^RH{BT+i31(p3)6&WRFWK z;lci(MEA#qWto1szYf7_9n{C(3pB{|*qkfT9$CKLFOC=n4>6tve7XdbY4HBaLH_3J zW@P9CF;r%Hcfsw@sgD{Ug)zGbatFY0zd_$<-5k4zIn`^~UZvHQm?ZZde<@aLG*6JV z8B}&hCnApI#*&b#kqyNqc2(szxyeg&uL-SzuIjJlgVOa0T7Z{;fK~O6^Z6*r-3KH3 zdmDq%c+uwsv9Q9nkY;q+j-prm`71unqH!ZKAYyy=4V0~#8W>+@PFf717Z%XY)zUZzO0SU_|G=c(WQP9q?c>Nht_$e zbrMYkyW~Bq!RFms@$gJc`xD#dZ+el7YWX+YAU2PS(qFnFf6zl^OB~uRgdb4n(4M`g zvmIvuVkmh6_T}vY_a@nX<}IiNt`%JLoh*d4-Q)1A(!cm68oj`{0LGj9^3PpQHh?CJ zJ;s+73#5gg=dS?zq;v55(MjRwr=0RbT0pe<39hYMQ{IVmih*THY&)~m-kR8GnGba& z3M3e`R&Xrck9E=2*5r2g)tRu+(&%= zcr761NE>2^^Dxi!-#pxGYg{938uC6@8Kd>_d(MT)0|^t1GFf&YFz?>Sz3=hG*nHCs z9V@v*4E#e^yvHB9=hArHprTIL!ESMtJ8LuOP z%=i?IINP_LA9HzHr`l6;K3o}Rq|T*en$}~!7?JpFO_YeW5Isv`rMX?p zHJ`bVQf|k=|M3on9w|CCk>|)iCGSh!J2K}@I~o&Ld|&TsoA&70tRkjtmASbSK?}hl z#i^Bx)03?V?n8}&KD*H&r|PIU^NaY(3pbWOXzP#n${H53FZ5--(G@Y2=GZ`My!|=1 z^8^lPBVW3r$S)crsN3aM-q~qEB|JtOdZjZRtDFXNV}5>GEJh_voX%G~iYr_;wSg#L zm~GpxjZsNmPY^u}vzvef&piPIC%Ltjk)(Rn6#(bOpJ9nj`^a+&QVcVDn8~Eqso(|< z!{ECXD-7eW0qG-}3JYJ!*@5UFMI?pLSF!8iA=II!%u8ZpyBGm){8QJlL{ z0mBY;>}B;l_9G&8-viRj+e-Qo&&D?Ro(brDo6uXN+gCtpcWnON zXxYiyHc%o@tdoK2W%*6##{t0`pA9yes}fIseZ@#jf;_V8e)#L;@jM{y+bI{K}&0Y)lzWb)G(Anm*fpu}w;^yWfs&7`ojC4CRi|2t!5^dEL zp~FE3xmB5cAJOWpS8LcV^L{z_hOZ=N3EzrzB!sTIZyd$pg+Jtpdcc0mM^yViOvCzpOCD$oTi6Tx=>)Xp((NYAxO8n46oajGWPxh8V01Lr zrmTy$k?WrSU}?lI0sthP1w9yZkh7pmXEpobn=EQix%tQ6$D-Ts55X?Aub$Hbo+ZQ! zIkS~4nuu#Yx4Divk}GUtr$+XCG32J}+DO1gJgHBy)Ut=)sHjl4_p)IC!n0mH`2x9q zU1&+KH2xa*h0YrfBL~&1Q37qnfevKShB6OEcaGi$E+!qWrAm4x(eVsJ&xLm%YyteR z6}NO;d0?Vlj4WGbw;wO2p}F%>c@6D$?B}psp#dZp$gD@UK7~93b;Wbc|Wa~Jc2lNlA zu4NkGsL7$e@8i|&wPZ&)W-wk0g98+bloZ>6IM9Jp@A6muHQJnbTCwNTcoV`D%YqSQ zO%qNl9o9=E0@PXuaejSV6~Xt3u{Kt9{@~$!p9S8%Ht6iwXeASj-6#*3>u0z)x)#t7 zw!;2Zqo@^DYenP0LPL14DS5DZ8=Mo!q6L2l-rkIm_=FKKIaw@~+jviPtN*dit1JUCaHc!HR{qm+e<%67+VK+d$-3^h*whaQq1i;K1 z7LApe5w?D3GR`-AZcS=hQ2H%Ncl2H61w#nWwwjy$=c2So9n!3Ky0W1g3BI3>36xue z&s%dxWkFrJKtj&y-cb5)#F~Ksl(&pWgac|@L2$NN%&J9+Eq#>ykpupA!Rwjg4lq97*>UPKyR7E?t=rLO%F?b6OO+ z_S>$CUdM0WAPpJzOv?VRVfn7{irkY?jdxp;ux9T{bQ3Br7z6?{OHY` z(nNFFC>SA7`f=`PlU5st^#HhKt5XTpuCoHR2n!%Wh!?U(Y9cr5c6N-3 z7q`zaif4eUnG$q$NJhL5qkeOI9t$)=emFv>Dq8Bo1*SyxhYy|(wYJumbMFrSTzF*<>7E6Ai0HPe^9p+~T2Cdfdh{k|ph8XhO6*}5LTM5x zfre?o6qWHjs#14#XJrZH@!qK^XT}&aNJwI;Hi>HioKi8|kEkK)IM*%C@%WgOUj1Kv zD}(SkVTE4`<(u0NsLiWOwwOGRNry#q>U}``lY=iKaxki+Pir@xk!PwxXIa?b2ZqiD zKE17NA&alf18y6VMiZ_aTu3f%PW#k?b~mRK_D7>k%GwQ7t?e-;Gb{Op;)l_)!{{{{ z!9=Fb`E1Q(pBrMxHDCLD7{X zAc1Bpyzz4lk!WJaq)7u3|yl1BTm%F(PlP%5;Oob+757e7XD& zG&!q-Tcv(4yySQL5fl@S4n?=_Bs?Qjx?7*@RMgk&WkT@YwZup+bCWDpNTlaU-|FQ7 zsW->{bF;JNa1bHQTHM5x zGv7sV4-EXZr*nB)5RBF-8Uu356ym8C=+2rCfvp&tZGNuvc`?pide+wV=&vZPzK?EZ z90d<^8g+tRtk6U6q8{hQS!5=SfO)UtW18Yr?AD83t4`i&=z_#r+t)2t=|ck>YK16W$uMY&w+qpZ#F`mh_A2sokV_)e{!O-4&2MGawCk`@R@V_f1? z?(g(X|3N8#QRT2AaDQYjuxTHPl8tp1)#S4YI(*K2M_VU02f#+Xy_k@}}1k-v33qXky-wBB-;rB&0_@W{{hhyTA0PV zZBurw6%>c%^qZ2ZF)x|ewOmzg`UkNQOow=RIdyBsr)hFIge(Od8!1P2>~~*AyzvRX zUm5$(@(m}w%C{Fhngodlpk>j;@`g|BzHMAn5f#fyHs`*}LxGQ5bNlfr%s#%ioIgu#%Qujn+L;HY zAA#ELTwu9y7vXir^k}&AVsTI3W%JNuwb)uPzFIARW}4@>HZ9^OUq_D%gpp`>QpU#4 z{rK?|#2xxJa9ZH(F)YTE;a-QYzo$gWVQ_2veU+DU-{&Wjs5xTI`46u>mF*`V^E| zVJ|m_D00m80*uejS~8P1h7i4Q*RI{B#7G_p;zZk zxrwA;{1Phs3?nlmU`}f;^^(8fSC&D3bw%3?h#~2C>>cFVA**OeU|RTfSYRSUQM@O7 z;cgbyXwDCFHM!QZo{64>s#Dqa0Yn6|sl*ODHf@Fa2ek_ysve&)(7Kt8RZ9_iUR_&J zHxl$disVj=<V0!EeIB3-Jk)clH* zn`7H|lz0MMlW$QN@QH{}{b=pOE67b6Y}+RFlW&pDrNhpfKf&hKV8eG>rMD)~tnu8L zIy}I!Nui~(EDl<7WMPuSYUG&Jc>G9=&60Ii#b<;eT>gsEbk!Ut9dLV z$?nIWsj#O~zDrB|d`pPQST8yg+|(z1&!{w;r6nk;B~JeHIZNjAlX)>P%PJGkB*1z+ zeH}iikur9CYQ17ii1U{3yQWqjDgC+O!Q2=4t??;ZadgRMFUWU|k!1l)56@|;JKU)V zEZHUP5=1B_h9tS-d8;oC|IN_9xSYDDm2tTIun;3Yw=sC~;=;@GqDw$)-i5=?aV<@3 zC03}E%BJzWzb93>o4QY5seNGyahw-z_Ol1VPrLGDu(aU@@$v3ttS-a zHV&65NUf%6L>Z0%*TT+AA@l4*yvdK7Bad=M2&gEkSC=lCvM#-{{y9MAu;pCDT{d!L zpVK&`t)f0E>$^5igf7tUCUMZ%o@&*4?>w*dJG2c<)?l_~x3Pv*N?!aYg(gNzp|~Cx zCoZ<8#qE=!DlUFC#axuQWnU(=7@ykO~C>)bS|HF`m6Xa z9L$SCocbes+uJE~ZX{HecRBKU<}EBioB0Pc)FS51D!W426#Z#kQLz_U&2EP`>^vLd z2P{#dRv%u)hbiE5^$2GjivQH-ZU~Xm}J`B)VL=I-?p& zztcz~FivseUQ8U}^-S6sIvg|jWO8DnZ9lnaUNA0B_>JV>q7~l)3bDc!MZTabMu>w< zOypX?CmFPiJ#&HEr|lt^j9=cBtfa++D9Y1Xoam203)aI!Nb}p<9>?c!%vUmOnxBt! zyIDL`4BXD8-IWK)Sg>h(mx~sA>7`uD8~aI-mzF0=X@lV>Ex9h?jgxhvnbi#Xbh5!9 zuUq;7Xmr%Ns`y?U$PbIj!l+o^HCplU%X9UXTs{1xx4- z5kUOAC8Q-OJGrFgyFMK~z+3)R`+s6Ps`E=XThhA^@7acXId5Q`GX~W@GM_E@53B_^ zDeYmv0|B_ErIUM8N}r_KGoCVWkGA%YeY%oCvm^4;~4lZfTi-2UJr|8%>|bon-Ef*Osl-(@KL*0xfnr_7uQ#Tjnigi$OkkRb|b) zu|)4~$=4J+vtM2e6>l$Ks9tDS%NON;9BO$wWP_lAiI+JJ`yxtdpD`6fjr~e}r_9%Rt4t>A-xSJ<8t<~?~vI9WF3>z5mq%a0^atOkE z;|k_ZrrxtM%0E9Jky0aHz5F|8rb~w-U-<4bCq&qSH~B>7*7Ni1@6&Z#EnB$_$o*RA zjX@Q{TC+hw;8}qy0zJ2RmUA6T){<#xZtz&|E5j?;5`p~=faKk0J+P&l# zxyVT<@<|Gs*foFPPgvLJ>E+ni*cZD)L<_0DNG(`gklGD`ep`ZSi;25ZR~vBR+<%n1 z_mkyct>}B5V)^WPfWsl} zFStH;#U;`RaoEAu>8b2$K>yL}h_f&*h5COA9@3-lvo89|EHy^CMI7)`yTNq54@3V@ znd#^J=(~6D2`dH}H z7A+Ukg5w`D{&r!BkppX%CBPEP(oNNxnVA`PcXxUr=Crk+*L;kpT!~d-#8P8E+K{^l zE0=kWE6VIkbW3`9_xuX_U1FRf>Un4O#l+M#`|h5cYxDvRi0AVDrfI+OgJ9`sKDw9YiYqdl|8la`&38*Iq7USntO6Aj714@&Ru`Sj z|JApQB%PeIpNOUJy2`AJ5j7u0f4jGrgO*=X@sK7+a#G;Mq?hMzMJ(KewJ57_{gUDc zVx8X?T^vC`xnu1Z3Fa4Nb|Gt?e3Efxj-KfM>^`DF+doTX79C1Te zN($dnA4)Fbu-8B3sA3ndMzo+V_P&n)uSNg<@)tk^YiawfGupoNU;K;g{HGZG-|O<{ z%blCV>mi&yai{Vh=Kbq|{@2R?leZ~~=C87m{+A;H$5Ed=uO XKHVvv0%pGp#E-g)_QO&o%QycIJr}FN diff --git a/docs/img/graph_analytics_pipeline.png b/docs/img/graph_analytics_pipeline.png deleted file mode 100644 index 6d606e01894aee1a0c02e636911e782ecba539a0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 427220 zcmd?R^;4Vuw=Rskl;RG>in|uq;@aZw?ph$l9g0(+xE6=tZlSn46nA$`a(MR4_kH#| zbN2oR&JUS;Cii?~UCXYuRwhwuDsq@;Bxo=&Fqrb6r8Qt+5K3TR;PFt9-o6QMXD5Jx zfn~Rql2VhGlA=;`bGEW|u!Mp6h;MFU!sX)O>Sp=g#N^L7Gb5Utw?<@StcFQ#*fYKmaO^7u`+wXb|nLNWs7}3e#TCJaJx|4z0gYckVCeTqnW&F9JkKbg1O5> zr>x{glEf3QpG_~oN1NtIg#@>R6as^aa2R)Qn4tS@G6yU|8<6|soxS)stI0~;h6TVv z=1-BAt1r?-J;Br_C`54K-~aCqmmr)Bm!Q>#eGFKso_8Dw|HDJJhp%Mp-T(0NzZS|O z!mehJ;4f9F|8M)OW^j@IkB7W1B*I5s@VJxq8q# z3Ge?T(A!zHaf}WB%OMy+>Jo2K%OZN&&;8H9Nc{s0(*JY75Sy336C_iKJ9b3s>phKT70*|4`eURD_4c5Q?8Ys*mz;{(q_n!V%vbyOOH)o^)Ee z*;hY2D@L-*vl_w6;PhrQsN3YyX>q}tTU=4_K*-}y3-8Gg@2QrbT{rP5x~grcixHOz z%_rMFdwpGZIiZ=vx1L|Zuone0KAstOlM*bcNV#^f*+z3;K4q2k(XhgtczN`cI+b@9 zO>^x1Yf@DFa1ln2Xu^=z$-W(ZmuWRxo+=$gO7NeWo`}pqEP}aJ`Y$*wl}qnh%HGx5 z&vX9L81+i)fw#X^}c9@mk2U4E<|PY`O@~2^XQ0-~;_FP6KAze4MY9 z9i-K|Jn(I8E1SDn$R^ICX(r@MbIc2(=m7AG_p0VHs!tjh0-GJeX(rElK)4(U_9xPI zloTkR`i-B@*3SqxrAy;bZn^s&e#kvE44I-3vy#B)aw3dl$AogNx#iU?*6@#VeJ_m~ zCielFr*~JgH1g!w$A7UgA)|KLePS@(GQ?&e;jIa;N)SP~Xt#8Y_xuRcIa7Z!B=#Ku zxt8@gWn3O!X*v4x*x{%|N0dfA-v!0@sj$OgF?Qg`8-*#YBVvOiyy88Ie4LVr9dc3b zIW{zSyXMXTIgFt%N11UghUoo}acKYQ!XVx_05~n8i*K#l_s#`nt`Ds0C819$%IaCc z>NLjkuXBmNTY}z+DLE=st=E%2=R7&PsOBg${{FIB3**%K68N~okQ-Fp6j19sj;8ap z>ixZnW5oK)`b#$)ciJ|ty>VctzfxQ>H3u02+l>7T&P{#{Uc#0~=X|RG&PON+Z(Hs3 zsTA=7OzU8{{xQU)FqK>s`J2BXXRCp z;S9~^-h}WK5-=KFEf#QRN1E0H_G!2sR$bq+)^DRlnCKCET}tR7YwQATAuKxcI-3G@fWI?wI0L&J2uNW)OYQXJhLj}Vd@z2K#@QH9aVU^#!VEKn&>a&&P!0@ zSAoS<^iG{_(`kH{ox9Pz{LS$ejXhai^&t`Vi=f4X#{QIWMwc|tPLk?wC)7tZ6qTn0 z>-dWZ!hhOGC$8Yt_NDNn!WCZ2yZ7*COz^7*#&BM`0a zWsLan_#z_kbrj$3#9AsqLjTW~(d&P$GtUDdzCRuM=sIM@`?>{IRIt{K?5|z4s%JYL zN>vo8Hnv%w+6Y7meC!+TP${_6?Z=OEIjKkd*@#(XiQRcEsyG}->fV677&F7bF@Ru!X!j(MuZgZ*p!WTSXu5ARwNmq5AZ}XY4 z2CcYCqFCk=30%Jv)dO$&kp0q*cwga^XM(}WJ4Z&*6yO&-rkIF$?EX$sxu}uv`NzDk z_J(^VR~a%?Oqw>j-5(_BXl(Q-J@NLINva$r^$H^J)fLk#rs_oO(5~-A+S&SQ)CKhj z`Kx1_&g~T&XWGo&fd!m-F(LW)6=?ql#6@gBcni8T}UI*ttO-)OKJ=or2i5i)2`b1iqIWz^RnNl?r!vQb^6;^6UWHb z^K9(j@H!IOvsHDaSLGX^_%mbQMDuus#fWf(#PVUEw{_9FB`3fux4EmgaV^s);IvA_ zhrCUsZ%>6U#hjINbd@Y@v`Q(2(@_)HS;*R#%RN%1pOI?~)|w&Vp~YN@v)6#D zpL6~0gV%Y!tX_|;&K)K>>@y&mYtVpw|J=bs^xKWj;DiZ1W{3|?jqS>;(3!2CGfbt+ z*Rwuxx+75TgXAJsv-;dv7Um0IO*BJ0{hC0MGV?DO*K3bJlLd#_OI8}O`LVB$q zY$L9-7Dcmi=016~jAQOaxtePN&E|jG+#P>ej*BkgRiEhmWjeC%?V}Im%4o^bHe0o? zua~-6u`$kBFK z*~t;e6!tP=(_CE?u8^P~M4DObyl{qM@xWo+?BwadFttQhrPn~O7t&VB$D{2y(V0Kj zB%-(XAff3FbhE6NFBkq=AX25*%B9z8r3k@P=o|QN{4*8>&T(}SbOR1w_a%^_Ycfb{ z)a{V9jb5mfcM|2}tJ&o>dX}}*1jgN$xG0lsunc+*qZtfU8SObew|OeutT^hPjnt*Z z?U4_Fw8U5IZIyv=i*gk%XG$j%K2`B2jKlq}!Y8Fk3wgSEgFU{5D$D)XS zmw-PdiBfucY{bFMEk}<@&E#lszGiKT{d%v71&=v=(6zG??-r7RzTDix3Q6OPvA+IX zI3VJoQCgp`Ql6eA(}HwSy8Ix+tlyZ}ak=d;tLSTKVOs&Y-(tC~;aoHkMc_g^x{RvE znToTCM`M$;y*4eWDl`8#)ysjiEjW|$@UlIZHa1Un`_ToP)aEBiE=sHywta4&R+_bU zi41|#O{phMvQ@n#6&0S|z(7m$4wTV(uZ>>Ta?FfIeZ4o*Z0LwwSe3;m!|K>3MT?da zN?yOzDNm@RJiSkF97lWWX_9qf%en9oosETrNu+uPsg5)C_Kajsq}X(^*QWnBLU-fp z442u3?8)}031hqf}uS#bx`vUx{@!|R=jgm%P zyc%@RxEVah>qVj#z2!Q_*7Ai&{`;@3q^@xleA39BcP7Tnf5mSW^0UB_8h`EWm+Y4_ zhNk#SW*)r zdpgzjP1)4us=OjLrv{nmBFinXHt|j2F{!|mqXvz-ewq2N54)=)%CHnhiiRzW zj4s%zIT^IidjK;97lUq*H4hEyj`7lc%yQ-S1xLSwM$Q#C4j$R^ZYg8+urJeHYqTpr z2LDBsBN%0XP)EEbF||G4t!iAYdakV&b&;s;QfV~1l5;K7Wv^sS)w0b{vw5=fEyM_G_W5grE8s>T`bxoK@N9c zmr}r(i4YEmp;PRKZyge?dBkrOWncAVvrXIN+_hgf>(P12kTS90E4lP+p?|e6YKhvG zh1mDWWh^=^y+6aE#MIEy2+svHh!2D1Q2Ftfc@OQT8EP-Tsq3%Hs;wE_P!R}3bS^G* zjI+L{-2d*hI_5+%6T0XMmN8?L6O}RH2!r_-SGI*J6o50%*t|OSWPi>7>L*j++hm z-*}1v6+#kvB}k%jHJm=ByM)Qo5me=O($_di(DIbw7vTKGZHFh~aCM<&QggG}Ky&h8 z$T77x?as#_GOm1PF;5Vto<*vM`bR9Tk0IZH{mgY7e1GdXHDZ!|Zg5t+)Zv-N<)+p4 z)mER;fyQbe0OL;28|5AG5s?Jd#fACi#Py}*tl#0|U#YzA27Y1sM}ygL_Wpnv^ce45 zYxYm~aU9rK{Sa^~Kt3*TFAr@WzDo#M|I+#8L=F~UEI_B{dCamn4EMNRy3lLxEgBrA z+UxBF_SW%IlnQs@aJv9E13d1&_Ch<&bi?KN3enD?W7hds{P9AN} z)zawTbiOyr8i1)MsvA2iAZtSOW}2kPvd3+up%4-z3R8Q+YoYWNwmm)ml=X=rQ8-*# z$pQaWsSxK1Vv6TGGg7{@+7o>N33rgyG2 z8~XMc19iab>`*T+*f(-*sdo8ft-i8K9mL}L?z?yFFn3jfY$x<&Hb@UBM{-ffb zmK8PJ!Nq#1Jprxf>q?1;mEY6GnJ5@{nZWFwo=C%ol&MsyL7Mfsu+E9%rU*b)R@&MG z8x{gmJ1ZBh<49l1{sEkK{9)j^ioh^i-o};X~GX-wmQr}kPtN9xjHt2`z6%J@gX@^ui%+x=C5F9XXBP5m+)vX1T}DB-N_!?VQQ2ad{TCRdAaQUiKRo1^bPD)7=6&DCkhZ52 zff=ltR%Bq`z;eKu&YjVC8i{6D!h0AwVly^`4ky)4&w3-kVYevq^v&7EXLr-Rx7xk@ zOrzS<{g0EbIYm&?nL7{foMT!01ypP^U!U!?q<8ISa(Ip3x>ETkR~wd@-6c;ab0-^v z&^CvqdA1-`kteP-X!}NRobeJuf@oJFsn79rLXT!b_4g`dD9O3*>DePjYM{UfX8puA zHiWG*c<5?Fw&iXyy}yec68M@)S!V;-W4^5L7GYY;6vOP+*`1$};Ip^*Z=swFfy5~^ zVH>mW;IfNsUCR&AnThowji}$0c#&~*g=ODMomN3_G%e3`c_EMPkI5Izplp10H_NU~03H zS?M=&c()r1Y*BJjQw#6;`?hYe|2p(Yf7Ik^S%pioIL{enXfTwD~Axg+jd;JUduhf)8R~7H`rQ$%j_Lp z?BwY)tf)SX=|WL{M|k>W-W=_UcHiZixoYo(MBs<~Z&EIvdemdfi6eX~nd1eLs5CB@FV1ez4?I9q3rfa5VNAlYdGMtGk(1`?cp&e zWQZd09fi-Og#UGn7Zs@~epixD?TP%Kt6wjsjn=9E)^8?{C_EhxV*P%)kY4wMzo@x@ zt;frszPzIid^be#(-sqrEL}N;qw0Q|r8;UWW*)7haxR#e?GmHB2uN`~kqPjH{_ET} zR0>oql;On&Cuw=#zmW6(BRb>viVdheoWZF1<>71%#`}S-!22^4;j7L58QEE(mgXtd zQoKZqrw9r!%ALO`-46pdGG*m0?(k_shq{h$dd%nWcI5 z7Y_d<%51w(+xbCGN~TWD4h(hv)7N|NO=PE8?98t{nVzJZ%j7S4g%_5OF&o{KulzWp zQ+aj&O|_;pQ!v=z`GrS0Wr|jJtkQZ+qlOez47wl z^{P}9)_bEK;rMFa3jM!1H|SlD=Un&s(mV9K!%oGmb5$|Smr1;C98l`>>Q?H72qtB> zEfY)jjl55osG!;#IcAx&p$%_Ot7?N2t(8Q8taSoVUv7GQ4!;4=nnf!wuy((ybyFhZ z-k@tflkj-2rrD#U#ELJ2g0_qwFhCIs3+}YUDVS2<-=Lf9xaQH)(&|xue7IsjH?&Wn zB?5i>j3RA`#(JVHz2t(Rd{P*{b$z;mv(*vlpIJyAVmXuVSwa!#IU5lQ-H2|QXRSGSi-Ux@us+3x(ZZzEH9Ne@xB&r$Yawj8rY zaeFw6`Pxvea*iLgrmpQ#YiSQ0u_yiNG-9&a=-jY*?o*}HF1WIowvc~zd2VwO&O4o{ z%*s?ZP;5fjD0A|;34}l0Ha;o)qPJ>h@Vs&_L;96>(9Q9$+%-mOj`V3H9}l|diC2p# z?r@>a#|*-}^}6bxdk%W-hJyMI|4t_eVc`ssbK@?zPJKEhPJpTQ{4$^J$>W8dw^OOu z4ulF8`jHFCC!L}es*HXXAlpPYon$sB**Emos3&=<+_aG+;#}D1vb#l`ochFt2&B}e z#XaHX#8nhndpCowJv_-`o2t=WM?3fDJ@K!Wrs)Hw7Uv1h0=?XUmM>cw*8f!Yrsj-N zRML67_A~Xd|1d-xWu!By{B7R}MGy^zx!a5Xgw+~6n>==*3T4$P_G_ih1aQ5>O3_MF!SZmaldf-)wTM73E>2lpQ`4M@{a}C8rB2Fr zfxmcFY3p=Xw@!k;KU_g|#q^higVG?n0wQg!>lMu-4?~QqocrT4FIFL_%y;kD23*2A z-9*kheZKxu5b$y@gN)t0O^}cH?~sNl3kA4UGW>hakHIK`$7<2dmAG1yM7uKT()I|b zx2Kb})yA>WMzgL{&u7Op1k$P{7;v_Ve&K*=E1=?88LZfkVX&pM7N zDcO~^H21-ES+bwviT1KWc|=J$$OO*W0|%M|ps2IMo*D(cKm0$Ir}2N`qA2wLr5KHQ zpx103u{t*7TDSC=Nhr+Rl81s}XKrU=LdA67Y+tx0Z)?y}Fp2f7*5^l@?H)3_TVKO!bMY74#U z#;c6~ga|f0ggHullPab>jOD9I8gFDULc2`hG$!s`(;fKg=PmOHVptjcZKECD#J_Uy zQ*SdPsyY-meaz4#?QD5WV5|KC+7|Y*ek>m1=g7W>aPt}hPybx*SU%4L0 zl^*w9&=jiS7pdj)^WRypxu}~(+%I{*X!VPIG}$1oqOh|w>>0@ae4hJ1s^ zd5^&&VD(knW5rl|;3WV3VtxEUkRz#kjXWu3WY?us(v^4+FaFUAX(!OtGrrNs{MKi} z^N?dPPZTMQ&15v})cG_eY)6vOTvE4Mez`<}968+Q7vAcXT0ohLSX!){FerB0lg*TS zz<&KNy#99Fli7|vfynbFQVn9g&_wplLaZxrA$C2b>M7xA$HdPDsniq80$Fe}c|AdOmJs}+Cnn;8ry%gyDFPgku+>&eQb(x- z-l~>x2RiB!lEZDasGW1>Yl$n6B8U5~b6Zd{eHl%xGA0SnaiB9i@$+Z#?ZrU>3*%8W zau$ccoI7y-%CRT=+I2YD4#3FG@I&@TJr5yV>4M{*4Gaxagb&(*Vo~ZWAwa`d*q}=X zo0EVG8kd*)QB#H5EU){=z{f>)DE4D2^qN0*bO>U_APmmpHzFbWnf9M-*Mssw{AAP& zeICal(7(2+@$~UY%kHi^c%xSpbm0X`sFAFS+bGY!_2(MOq19XjNvY(bc5X1_ym~D(?UMk) zvuclei9-;fHk1u`0OjoHz2at)+JvJKH}8Xi+->L~!fVfo&r7#cwf7jBj1$8VBEdfN z+@2mS_Bk#>zLt#pm@9Z!9NCut{cE%ok)_cIhUc)iGc1{D8SWX1v4rHnBr_<7`jH|J zDK8Re?iw@-Q+ONhuyWwObm?S#a%M}`n%9y7Qys=!HMspJbZ%-$iswci_G&{`h1#~J zaymKgc2UcIYGCu;-KD?B+naCfCT03*j=wE{X=gt&`nQD`jwbB8)8}b^%je15i1VN8 z9QjPaD|)2ZvS{u*_9(G@a0f!FJyDu3d8MVuc6N3>06V6?86T*9zxTwI-v|gyl@{1) z9_M3kC?S5I)KgUQc$6bG!aG2_6(G2p8LIDl=By#xAf^h`OE*Kqf5fJ|-d(b!l{AqJ z7j?*Gri;kvvWE$bT^0+ny=3mb#j^3JxyDZhtQ}rlD*s5FE6#(SHxXY#^V`}JPQzX@ z#V~G!-*G*AH}+6gf$25J3<6mz9Ri0QYoUrQ(8h#Lr-WoZSx)4>s2MEw8>lN6Bnv#m zcfl|QL9a0$gGO9HJ4*WNZpRDHF8QG^mPC1t^x>l$V564fbL@c=VZDnz4A@6v{XIX( zvDT~;3wFkFo5t#BCGV&bW$kFCqm-}Yvs$MPG)LRyJ{KaA#2%I<`UKdrX2k1Olm`5& zkovxzx*~7^jnM(TT5w6jKLYWLzIokw^fxU+7y>E1->-R&_7mTf#+sn~nZj%GpJPmu?jCT)zP}E{eG(nFyp^N=n^AxW@A!FosABThr=ctnrCx(FYGLFF0C-NZXf!~HJy8yC&9R{2^+q6U6X1&y_EjD^mW__Ho{BL;uYxbq!3CtJVSb% z`wKs~3OWw4>@G$PO2VV z)4-14Y=!+zswla8+k^X|l)EZT*WN5@U)@l-^hrG`)PCqH@?c zF!(z%rZqs1^hPZIYBoIyf~sNbz@7@}*10Qnji?rRaUEaxtxh&RQ1s}_rG00!;K;!$hU-`~&?(qJ4+&ii^xMzl$|j_zsL7C3{0KSGd7A zx_y(+xyn&z6m>02XSp5Ge8sli!vMYL*AI=HGVNb^1g0wdxq@bJ7y^N)u0|jki~(*u zFD;8S0Xx5X>l!PhYSC39;;ifnDSM@->;*Tj&s9=iKIWnr2o^Jc5K=Qu+7}kKCa+bC z$dS%P0el-rq!0*|2=tR1y{;9{cD*{T87cgmo&ia{0pN6XInwxVSL0uT7juFmL~m1q z6B(SsG1&uQ&$FYj$C7`>qQ7W=Vef<-P)iSCA5STcL6iZRZ~lS^+W@!$P~a?)$-051 z?CBt%5>(|k2M?dR0vAiFXFtZLyRoJ2T3EQBv~`dQjs?-5_9E^Tx>oB2{w(3eS`4oL zD$b^YP@qX3*<8X{B5o1Yw4BWn(04(WNQ3+|#Nevp_En(R&n7!&g?$zXDl7-E=`yNX zj%uGDFjs-z_5gD!+dDkB$C(+aQkw-y;Hud5hcXmL-M+#&DQy|Ei%R5w8Kn7v7)Lyy zfZkG&Mh__vlt|XWaW7*V)YHDXT-yV2CfZOG_doxPgWWCmRjP3m^wsrZiQMb=pK#xE zC0E&*Y)<(x-76za-|t-0LEk-7^v4_n466Q=ZO+;*3$wf)t}A1K@K5 z-^@ic&I#cruhCsDBp!M5XA^dx{RV)bDbOC>AG)FElKguxAI=4gsqFG1^3d}l_DF5h z1{TVQ1lNG8>$YVO34ei&x4|{!zv?a&2j0{v(eo@8z?ttZ@%IX2NQe~b0HHj8jyO-6 z5lY_vEBZu@qnB`$sce3&V)W3eGvT`j0(w-RjQE^|(`63r30PN%xPwg`7_jDYbbt*f zkFj<|AtUsjZorF$#N$r}`qKMFBWc=345}JAf21*hhH(Lb$*p0-mEBYkp6t|Me9FPl z31w(ZKji6)=vw9Z(OIEUVKsAY4wgS$Ks+1P`~g?MxxjDI>$SaWAbFM4!2tSC`LR+S z*{Sm6(b0YjrJ=h+Jg7~fWSLCD53$kXM;vr9n`M(bVkVK%2uy;c$=hZ~!Wr?IhxBQ# z!#m$u{t_FJJcvwJ52WAIv_Y? zB~>$;(PAb2v_lRrJp}Xzb~15!AZx2e1-l1Vg)nt-@h6j;=-26MrI+|_KR-@zbZUbG z%PGpq1p?MyYtd~@HWq)!MPLi$0DvFJ&d0%teWrS)z=yyUQ3=+CaFmieBQEpS| zlqhh6m*NPL;NqV7D;6`-=;$O}vx)Tp_XF)p{b#y>3})v(yzFd}hQArs&*;YsP&4#U zc*I7!#^$LmsTcIo`33Vphu|nE%CT+MqA(83WcGN4Fwa$)u<45()PRsRx7+r7g_ih` z{Qy@}L!yG3bXqKmJUKLeV0T|Y@z2r3m-iaB-rFvzLZT9xn-Ufn)Qx51R`8M`#^vlETD1g70Ecvx6J& zx~<)VQzcJ{D;G7zDC>mSREmsMACgTTiPvmys^zvD+C5`NMPeSch$*7XUkYWJ_CQmXj^J z{BeK9hnm&bM`Ao^-%LTVb##n0YW?u_SB#RSc3@D)TKsWFMORwXYxJ+@wj;_-TX2GG zXcL#heNCWhoiOj#DsYKID8s@|$vSYQM)K%1n0*-CHfn9pR_X4kV1+f~Ghe5#RM9Ea zDLc>o8URo-!7dTf`b_;p&i5qjlkJdahJgIg-52e+s0r0}KUX~95HOPvmk!lY&U|*` z+1N3q(xT{8tWNw7cpydfhjL+S(%|n03)BI5|9|0V}cwN1shcL%-c1H%fd^(P^2oc>lb&GWhfdgWJMK;ybtl^;_PJ-Y-_?? zl``v9`Q1Pv{7WD}i#1LeaAbc1uy(c~vcolVJN~9IvHzt2+;Eo+>!v?ZE9fT695}uZ za1syI^$)|SH`nP?<%b1!)^>gDcL@80E_Z#!NKxjvkio44IEo>1KE^y21EElBlvP4; zzY!z89DuIG0}2C*eajh2K62BVNWS9Z>y2&8!2*qh7PvguJQz7spC>q%9X=EB;;~pw zjEHiOIccwo|CKHYLFSc_SwZ*}aa(uFMAVG!I zSUAlk!-RjPZZCK?$AQvt-2iljas2rp`QSYDfqrKJRpKYA{0QpUn5A5;>v(EF zO=L1tb{rHAdHf|ZEDm5{Lo@|hLYT*jS)xod1nyAYePR?xp4X;wA*%J#h<_pz7HZw4 z8FgK{+KkYeczS|Em~D$MrtC3QbSKr<4g3(k1G;O$B$(kf&I_nBvz032ts-MwRQ&|b zzTSrKV$XoYrbl!;Gb}(Sa^_3e{9YKKR4;^LTHVD4!}1>*;qTuGt0_?fQ7O=Br!Y~s zXUjPyj_FB1{TXQH2c_woLWE09j1>41L>|^drCrcm6n$Ju=iZkx62Q`4 ztIFSpiaO$!h`~F7FG|RpHtKAk6P4~VKjX>`q)$t=-c8nQ+qohMGWv!HRg^4CmE}~F zxpMkBF+0UhXiztV#1&sTc|~(Q$d?rUP(sRGyRG|M_caRAEo#48)H*LmkF^u?G1|d{P)nrRc5&3HB=*)IVjGWCRQ3 z3}j0nYZK&ti&6rA2e8EKNL1O74#*O&3+t$bk5pg(pvZ_HEytR+MLE=U9+%2AyS5xD z$AVa{Ox*_yXl~pla9WoE**}8vmPUTwbuR<_gt?${lP>0Ov&DpOhoAT$bTn>B1+a8+ z+Og!Oe+Gle<7V+B0i&@SS`mklIw2JmrGeSy&O6<4(5`Nz`|>8bg^b3bsT{d^==?da zSGX$Q#cF4~av(+Gj7*xr+aP>^d%VzXP~7}?{v@TEU{1*|&K?clt3k5F4>Z;9xd#x$ z0*Z-kAq52EVsjbEgtH^6?sW7Jl|b$MM;)_geQ3=14@^b^vmFmM3G@1nItM7c19TgC z`$PqHrb*%`qB$Jz32XY2H@3t8tShfOL9bZ!rgS_1cMK)n-*;U;E{LsMkZA4&gH6HC_s$T?goeRa#UYV6 z$JgYieh}1R>x)YV`9Mnghq@=#&Dy5~*N=RnM;$esU!u$*Z!6Kj?7lgFQ zyh9tfHbK-DD`zBA0>FMe!aV^rqBbe<2?!4$9n9?XK^kjxXLV%SAQqlCaoq?(&-hOT zitb94Q^=li1KFu*W*DJ0V)SbINkfDFoA-BGL%0-VWg-?ujskXb2$~v-{EZghLO;t) z`uJOD6nS3T=#%_oqz>piW03h@Zi8>unSHbOo9sYvh!(jZkL~C+c3tzvEiEE`#NZWv z!AdaHe98bc?W^W09Hr<*?TpM>yc5-7UpH7GSGsGLN#D3zUCd2|XteBQw zBdDl9SO^k6y+(*i-ZWI>>rxB<@G77-{E$@Qp>#0aYQRy#Zt`nYoik0*5vqNNCbA`6 zs}T^MexjNA$o+|tR@Wfhs3$rlV?1fbvP^5!EuG=|2Ns1`fjgImZ+MP-WX%!#BMLz< zQb`>pg6bL72#o^j0yGrwNhg*lM&>70Ys^6ac=Z;n;2P=Ca{%b&yaJurN ztOTi?mZr0TeCA3j)!N)n888i&Z8E zZYw;0eu^sk&E1%1yGU`38t{Q2-Zu*}b5ITC^0CjwX(FnGvd4naR|FH@6i`d@smq;h zJIM=uq=)SOt1F%$8u_&5l(5*Z5zSIMxV=3x{?XiV4BzPdCt3k4RFgqv$PX+7 z>HS4su%#Lu86N2otZ3sKOHuV-xFO$&a4XjIJx;dJ*oQ``;!qI3)L$jY+K@sWAv6zp zKS&g5nXPl)eX`4fB4Y9XHTrRfZ8kw@3Gq?YP)naO=C~i-D=;`l{pr(b1UwRw775a& zqFL;ZFP|$a3@u8@>Q{?+{ZBr^=VF$_f?Q_Xa5qdWD>}Vd$}4F# zg9sgHOi&p(kvDogrZW!xg`u6nQ-|)(LS(W9f#72tC^YIr(4ec{2sdBoSy>h*Z;~%P zLl|u=7Mg1&q6K6m`M6>2DJx-l;Iu40QkD^`T+}d|*3k-xaNyk22B_XpHoUs{1x3oI3u?680lv8a- zk-tO(ZsX?9wViogZvod|t&*{X%H<~%7mI8uheL)FsnPT1t@FT#0z_{_6~)Hh^1(~7 ze#IW;O^^?{Qv9cWeGkx5BC2_J0n^RN3n{EzOQ3BaJWQGIsSu7U*J&!c?L+5|K9&__ zYAg#h(>=N`!dO=S_Ndxi&!b$2;Hd9!?C`@P#qAoto1Ld>Z$L_rv0Iv6$(8zQN<;o2 zb?BCG3I!R>fhEHWMniQQ_2FYW_j3}%!BOFz*a~SOosZpf-?A8S*OIU|-9zq)peGs| zp^$c;XIU;Io-1mR%fz0UWd-Ne7R-`pN_pDowwaG%zA(-aB-g z9!Q;yhkk^80+2??|HDly)PcvJzsDVvkYqjrkKDEUa5?Oy^2y(LGp;F#6XPvAu_o%qeD;Id_P?%(W^1h?A5G7?MWF5EEN#o)hOd_ zdb4GIsGRfEl@vLh2*#1iKx^H_)X(^SJ_sk@Ajp1D#hg;oDi4v34KxV?G&5rIK@3}c3#v$;)a@1$gZnIr-4*&itP1Up*DX7#EA zs5m{5;go?3P>f+6AmEt94f%OC(ArcmLSrq6E$#hqGS3qgVq8>mQIt7ft^AMi7PEU( zVfkb06TAtWVHx;Tn(@g4@)e+_K}CaOMeI!V5&Pz6F@}tmlshr_#HEatze`Pkqt3OV zU{XyBq^?;Isd~mmsX4ZH&iUf-Jj?tS5~j(PUlT%XSVvNmm?sAX*1q%K0u~L!7vB`=SmmvRcu@*oSA0;H^B5rK~YWphbk1E#y0CHHX4xZ8h zH!yN6Y9gr9 zbYa)#c5RpSb2i3P*mc>uJ=shan8u8-9GdK?HkF}zxCY+gGQy#!OG@&y^CUR{l4FzR z5VtC8!_NZm@|*Al_2TQO3*7M5DoYr>h!qA(HGY8zS*^~;Lfj0h1mbb^HgQp7TB@^| z@&~Gk(=;oh$MlGgl%OquD(JKjs1}@TJgt)l&Bm|b{8oC5Z1MN#L@psRd2pXH9G;Q4 zMB(V_^QxODAuxR}w?q{QcgwZzUR0#TM#; z$_#jR4><-W<>1D0nI7YJ*^i%@TQI-kU`0TNTk3M$wv5bNe zZ=wol2^aERJskRmGWLk2^*7?qR2Y%G2qhcj3#Vvv{P3SU@m)bb`Tdt9;_N3RIAgm9 z9HJYoy_8Gcq4Qs#kJ1lqn z(d2C^K2L<^{65MQUn#sJIRKr}Hl_WM#*;YFD4*ig`V1T(Q4x(z30<-g$`vL?Rdb%D z{5JT!p_{MQ*1%q0{>IcCZnrU6>V$G;>9vb?Qb1ugZmEgoWx49DCG!V;LM!ngjofPN zP_9bjAY-m{VRhNrr^gu823n)pryw;!o_nUjVM)spz|28x8>#5^U4mJ8-JRH$w#rMw zIG+mMTE>UPylxljkCLsZi8`2|nNB0^^jQIJE-EoTxNXItV>?ohk)QHPmZF^{g7kV- zCy)5E47cyl$&1`NTL@;rhy-3_Yz4-Zq;Xgc%kPG1fkQf)*Pvj}l z=(q!+Y8!*tzaXeY<*g&6;c1dOFAyI-7l{RNF{IWALcZ{0sRxR{C#!%5pDB{!Vq*}FB45XO)8=BS$8q4aJ$*uD+QOd3F-j^ z)gxJ@_Vc3mVpZURIy^se;rM5<-<6@!sQ6PGo>&tP4}IbQu^iR!zoR^fv?a9+k*A*m z%i@vV{{a1fY?H2qluNnheop+f-XV$=eP>GxO-JWd{oKeli1(+32YWKR%2vtph*RQl z=gwM9R4X7{;SB;NpMSc*s?XY-I2pj2WVaK*V3Gg*#ZL=*T^IA{^63|ku|*yFuPdC% z@ifSD!qvDwNx$*Dxw)lJ|J4KdfWw0t%?3FmdY8?F{_R2bE$D>N?d)pg^?Jy73Hv zn$!Ko4>xAiL1pt}b|Y!EZgnc5Gby3HG1|1QhAZ|aIZ|l0*e6BxRoDBXE|M39EW+h% zZkFg*V7jvW4PanoOWen0Wzb{d@xuaVSEgJT9-a*$^BhSQ)VSp;VF&}kXb9_wN@OM( zX`A&*1~v%6 zIba`0A@V^q1oBTof7qOQTx138PX5 zAM(D@i%MB#1`1fDQ*C%%bOrn-xa>zMAU9|!qR3$6;~A)8DQz}`0dnx5dx+cnTJd2w z(ibvF?=d#gLbaIKRMb|&t1?>ZXJep5Sv>nzZl}gbDZc3D5*WS@6im6VR|3@Gp8)C@ z>PZ{R2bAU(80BV34w|4vPJCb9^u83WUc#SKMvO+njhVwho?epBIJF#a|K59Rv$@w_ z-FEQe(!ehcf z25OecO9tI5cF^p9HN(by&MC>} zR3hf8?!g2}1OJD|n0mq#B@dyZnNgEHIXIm{pLj59WedR`sFH`9*N9?UtCEE^TL)m47`ro%=|e;_cmiOhMMNC53+JJ8 zY(E=TcuI6wMHuw8+V_GtsVwEeKSbLFGaSC-GVlGW=^Wrs3g=aIn}f=nPYpM`mX8k{ zM87o!qD^PuoNn9S(Zvd?ho;QNWEENGFa%90KFeT3_ZrK^FC$Z zil)9gkIy54Kz7EGcPqvvWVSpBM@QeO(-G{Hfw`?xE5?vFM?p(iO?O(^yipsM;(&T>PC1D_t}y& z&aLM!iC`3_3cI`ls}EX>b03t+=O(igRE6)hnEmD-SB&uI7A7N(f1|RncTNzR{>qHP zRO9!gm}9RVU?@b&d$>gvn7v2JUTY-(>j$xSc^tgxi|fAC=Kvpj?<9XYFoj!a-vp~V z;|r0}XKA%FRN|6;0lX*m`Xz}cO)y)0)VkARfTfo{!NXcNrZ^XVV!^T~oDh_*^;dF| zEkr)`7at6*>XTv?=eHDn{?SmzX-3s~InVo(-)joYXUxWN^4*oMdfJb4XByx7CT|Ia z-FFxwhdJZkhIvb;Q)e4PJipjlI0uVtfGdV1nt^5mKJRZa+KKnenTy&JTM2|+0?baz z%2)x^3TLh_Sg6V`Tu(Q+X0RwZcBV=?P-unk-Ld(`Y5KIOKP^S0<)lKk$+kgFbfdA zhlCa%%%`d=JcmF0-Pn`=DeYn}BCaEkue^FbPm ze`JvuyPu>2Y5)TY6ni#j1c!9F$nU>y8O)zW`33CbwsXsq#U!N?;wczWOd;I3N_12H zVf$(Q!qN~Q(?7vj@V5R}8x-3qW>2AokzHyXD$_K@gte3>V*=%I5g)Ye(EbyOY^wOi z6k2&&j2v!6QiwNj3o4VwRK`LR4F=}t_|iGiX&Zd*v{FU0iUo$IZ9D$N!w=eaUVcdI zw;$``%j;-}ZzwPKg392^m5yT5-}+x|kj()Pv?(gykhLeZQ;p63!-?JeUu}ZyDIBjj z&pW8?<+Uebfq(vZ9gk^VSw4G>X?5IoZTtLG1}fsV%>K>)Jv;I3PwiA=wRMUqP`>7I zhR(gNob$Yc+Fo9MNb9#B`jeJP>)G+z>>oW{`v9?2zt@<`v=1oFIOggrEXZQ5>raP1 zG3Bqb5MZWy_&q-jz_da4Q`8*MGvNv|8aEEB*+d%4&cJze}a{ zM&ACpL&e}(MrZgOQO}dyIjh8Ksnb zUj{9qIFyGxn#Up6M-G8_+!Q{VtFrm>O4{^S+zI6*9pX)8n#Yt#sy0ODSsSO^Bv}88 zt(V*W6NIy=;u}*~hdULWQaV?d#6F3q(s453XPc1vN<-!P7`;BR#Cd*28HvOriQZXG zs{LeCV3SPTINP2Z*%E%#7U|=)>8HJHPUdXiBU0V8CYpB;NZYY;cxv< zMx5BI*}wR!=kxwpKvNRgk4{Zg=KGuZzxW&D;IOjs&&Gd$%=Oc5NRA_$|70xo#~dFo z9cyg<*AX~29ZsaiV;q|qJ8vwHsj)zD)tJisbg(qQQf*UWVk`Y2T(udEnO};bYrwH# zI!q_@=_i)Lz9u@ANm_LL+=(`&F=<1aO|e$`G#JNmZc68*ZqgRDDM1qwi#&26LFKB4 z#(o~xS5X`rro(hXZ~Dsyfb_y}`k%i4R5`I4I*uNIGjF^MUB}*nx>@s}V*WKS_qwZK zpbvTDrcot66;(+w+D8?>vgeN{u6trz!#ydsk*bN4*`oMHXJ5Z`DH@0NUl?Y|7b6=l91`e=09{H z!bITUqnFmXn_>zAm(BkkL@DUfqtWQDf4I-k*0t}r1MxsIa+jkL8f%#!^HLKtH1^9V zt#zGg#BLSSGl3(d3SvAG(>VnXNU=5I>uinXlkrT5)3RjS4{nl>rYwx`6MFw?h(rhF z@5ic##(o*4v+XB&Ov?n0kTOhv2N0tphcfp6KxHLJp)FM6H=?2%5rMmd?Tb=kA0kk4)P9d+@f@finO$8fqi&|4X{8qfp>iR>mjq$@ zlAnWHADjU@|NgVEYxAGO%^&+?d}|P&)WZ5BZ~r8?*uTn5a#PTAeY(ZpND>M?3J)=6 z(wQXJ4K<=se|4aU(K8K|Qn}RCa?PVS8w8VN3=@pCja>93ldn%WCQu#HK?3QWrqA_+ z6n=VaU;xhI6LmUP<2x zhL=kJU+nrP@@G>8H>RkY=_7hHLTJn^)N3fFkoBY_4~v-!5009vBo}c?;KxMiypyN{ z{+W#+;;mzg(mT27G(*n=m1~0tf}px`Ol79O^}lCN@vLF?FP;p2+@%@dcw0@?)^p{*f6OL(o!nq)}1qZm|uJLH~e?NXhs=woH z*#CnqP*c$llNMYHi*L9aDyt{qx25oDs0sL76q|BdztQObYIKMR^OwO*5BvfA_zRzb zgU|m1EV<_)IDZB|yzb_d1W(=YaRo*FrLg@E;5?-+Eq%@2LkH)#)Pv3@xGfS1>rkEw z$N6cmPW+fz1&6ec7xT%4m-w)bClU+m1Rpw2^OrKkLCQ5xYNwAO|6KZ$>Kz?t;H#Uz z32(k}7xcu@3Fm>t_xZ=voVAiY#m^o_(OvESm=#@_C zSkdQV)c;cN|M8)J+V1_cf*Vsz$UcT%pfP1?TI^c*rPA(%EM9|`)T}rrFV)dWGt-`~ zxh_^_+KwBP__jenLZ_}+$1!b6#S4l>KdPtAuzbXaC^1mJ=5dD0i_3*PGjbU zw923I{j}1>vGUb+On>ea>;Kq7V%w#hz`)15{maDe7BA>W1Lx19dC*Xf3oyKcR7iiS z8{tPr2hI~{l*SnU(=XldKdd}~zUM&s>V|}IDy@F$WqL^>PX*?M1QcuV)0!8@%FM-V z_n0>Oj}c1s2f<2EId}{8KQjp6jr&^rgn3#2r!ZE`2tWeZk2x(_;+sTL!FPPcztC6F zFX^MbaZs$|3~~?gM?a=wYK|7$6qA2*Nbw>f{ImWSXb=m6SG$Q|@!*jYqErl-7|*p& zn0UDMPnGqL8gYSyz(2JQjj7a63lZniL4wMq=c$+&Q;?XxA5G13%=rO-8bgWHA4?q_ zM_MNAgiiB_fYUCfa{}1|TXc~NZ7OD+l$F;ZT|s3US5yCnn&MvQroAojQWoL^MN}}`&whl^wX|_ zROA{=ps5_O^gGuh(j4|rf=s|Q($s}F!O~y+P1yJCzlRyu-H9JxUkX(buATfq?9@jn zU(@O@T#otL>|pF)?HB9B4+9Wy7Z@K*YfK#|X~GiE*u`*lCa0*5r1c4_Km^8JHUPQ0 z!`Jgk%uZu2bb6$9pr|Pgk}Q&^N-L$tLam@=Hy>38rBBDIcN`)KhF^i3Q}`U-06s<4+Yjw0E1;{h4*L2!;n>lsFl~ARG&D@(r>7cbOoytf zDwsUEj%||WPhV$0otB2sA6@+~_5L3(`iFMqpBLYlA|piqBNyZwMHp*Z+)|uaoy}(+ zr@4ZwNSSFrMTST~gOeLsvgGLaSf-8{*4MTvX7N19(B0L|tm-CBmc@2M z15B)~OyM9hv~Iq(it!KYhxD0p zB;Y(E)ZnEoQy<+^s`XPO$90@p&2sL==kX}jh#bJB-=r>JdO!RJa!_ht^fQS?0tP796kIROrKT*zkKg9m^QTzzkgo^r#m{KtE(3d z9(o<#I@ARR4xE6#{t3{3z6!2chIG@W!F%5OPM9%c8lEr35AN6CoSDcsmi14tP^^pN z8TXrZ2Qtw-;YYko{|jyZNnd7DgV>m&KEqB^Yo$}yj~wGUFZ{4$b>T-&>hBS950IFkmBWXbLDZ77HF(S>G6vhW5V>e{x|y>1oh>cTrohjBBi56++O#|zb3n2eiI(`U?tnh6upl~Hgv)|@+c4$hrD3vV7e zfD?H)ynW;_-hDcS@9m$3{=TzFdmjIKke+_{rWPN;ufzBC>tOD@`7pJ98eF;bDtuIW zD$JX=5Z{~{z>8~nS|jK9W#gZX|CjLm*M%ENU0rzR_U*Uu4lep8Zcx=tnuMs;;8DzWB50 z_%Yvk3l`#C;K^|HHP;~jI+!My*D=#KrSe{EZm4(W} z_QLvMc`km1^0Y0Di}5Q~UQE8QJeO=L3zdcKh4sVoT>OX){@UG%y)VGwA3lR0G`|(* zEWaDg7&pmj?4jTuUk_UIao3N=kJpF385bAFnM z94)2w(jbvH$xGAvGWuzu(qopN$`!%>eTQ&E>L9efeGqCY&%g(M;d+=gdkQRBItQPa zs=;+CJ|)%F2fg?q{41Btflf>xKYAKE&h)~WGd<9aH-iQSPQVZU{a^8BZ7s$x!WCC7 z#?7j!aNP}8<0jQasH>ak@K%Wsql~yoMpoYP>wl^D|D~iqSzXbMDRx@4JdLSRW}pZM zL$X=TBYN6EqW`*)6qFNBKWHaVOd(f$a)BweG|e$)6|GM^rK%iRR?Ly+SXWBxK*jv< ztdKm+(AAIAt`XB8fuR4^|6*@s7i@FB5hm%P_Di-&GljD^O<(Hvf9^~NKIwD{eztcH zbavuTF?V*t`~{2nuVKb4d<74GRvB?4*yxvXI&mY5Hlud#*beRO?XdlaKf(*(RycXQ z4KH$g_%2d)buV{J6jkF!*F?;#gC$olW1e^aLK7@pbQ#Q@yFfb6^@sUC!a-P&ol|7v zKVxwzkAL*!;i*$6Vc*_e_{38u`zQI${Dqg{H!%|56pH~ki@tpH?IY0glb!H3Zf3po z;!bF9Ys2qp9Ki*3KW-4yANeNE()jh@fq_c&={mS}*^T%ujX7}7y}t;vXU}2Zq>seX zD}FjYvQ2736tk^*<0%;~=Gpu&4qVRl4`E9EpxDOUKuyt~jvEwH@be_6@KHpTdW9*a zv{-wkmgkZ|Wu!7qYh0|oT>Shz<)g7*mMdMXy;93_wdu>IX^o4umy4gDr+hT_%W|cQ zwKs_JsCput*z<4j_6y&HIXArz=3I}PIe2Nsdr`UM@bfgSak2Jt>EP!nAC3L8TyA7q^xuW7v-hqMqy z*UN#B_;Cm1MAx(wQ_!^6xD>c!y1h_ol~ZGe#H}-lL0CJR!!?jn+E+dr6OZ%f2cQ>k zw!Zf2>+tM%w!xH1r*VF(fnWNioAJq~DfrYeUdmHLgcGrG*Ukf2udllg+D@H?#X-8{S-PGlzuH4KF1j(ryUXNcXhktNns%1OCP#VB^umPMT-}D1u4S2PJOuX)dz34o`y3W?eO)ly@C_?beOmB2Dtg=TjAzg zae|+}kiR9tW11Tuc_7g-icU1GW9N`pT-4uJAC77KSe*Qg>P-W32wRVPI$*V zZpJ4a7w{95Hvf;_`iE?n)>8B@73kFO7`=+|^QeL@-13#DnC3VM>*pXTn5ANC9P$h0 zEyyfKVKIKi%5$(xWrg)qg@ZUzy)_Q`U4r^Y1HEwa)on22hWEnErSF1D+?4XC{z0Ic zkQBqSSb4%JcPgx(gNioP7X93Z;iq5zf8gYy*I?GXWeJt`yT&2E;q<@q(z;1&E>*Y@ zygb7|*C$$0)5=)N<5FI~Vhz_xR8Ow&l%lj)=5c>M*R;x$j!6vtE*7;c@<=R_M0msW zIDGfp&*R3_4!k+@6ZoA6-v`T<5O5y;HN9<@J3Lj zG8|Ow>rdBA=kdF1ot-^6$J1+}IB(+z*hwb&5IfBWBqznfK9~NY6wy&2La9S%toAtX z|3qifc)aLOy5tT;H>UVvoE;er!C4N&f@x<*$XQboXEt)ewuz5gAC)0%eLS2`gFEaz zXdqEMkhpZbkXIX}7=sKl`RN@Qn>N#vQ4>$6M@D|rAG-@tv;G(R!eeCiAA9{+@qhZZ zR9{~&oTn!w@aZUe5@Xsl`tD3E^x~!zy&81n@Eh><;kRJ-u9xw(pkw%?)Cs)My&5lm z>tO!8dbs?ui73A6Dq|Iw>^{pg$Y z-8T9e>uJ+x;6iyCejIobbmP;FJ-t2n^x|9a#-RhSXLk$wDgMMd`sOv)EQTqQr=dSi z;XB0FTwRNcalk+FPuw)dr35W_yLrH0DlxwrH^64YqJ;~g zeky+W6&Km`)7gTl;7#+s&Hoo>{X=6SnR#YxRQ{#XhwYQBs;i&YyrH(|>rQFaTjQ|(q4F=4K5XCD*-wul{oD4x0$pv#V9{OI zD*P*M8Nc=`j8J zcSFbCmtf8n?|=bZKjjReyoKZ;2D#{!vDVe|I96pPmgQ5Sjf}XWODJPeQa*}~;}9=8 zh(yy;9ttVflbA~l6E%`)9%sY@R_^*jqdLf$eBo8!`5@BwMv0T)G!ef&MK@pT>nHKpJ7?B3{9a=>Or0_j z&fycx$6CAKtv8M%eKl@Y&BE9DYH{PJ2Io%vg}+JoeN|jT(*~71d7NK=8P_ZLZ3X(~ z8h_&zH^RE{Taxq&Kb2Qj%F|f2_{v}n-jpJ~6Y)EfG%wS-nSzf6=^)|K>wjNw#bLbJ zzw6voRCHrX7I3o3BTEWrG-xAf<4O36(*_$uXk1#K%!|eZF*7`I4zDp8uks+SdeNfe zh%6+jzHp$0Jm>MwE7iadGf(JZ`jEeyon!e>9+iPuQNH59=QNNYf774P$Oi2@iYYLD z2w03Sr|DVGPT-ua+&t30_$@sIm^A}tWr%$!Rp;uer zYyb2QaNuYA;P6}f@B+0CpTM{lmM(cGzWQ@HEV~xJMK!S>?+)Rwo8UxWSvi1zG^U`9 z@&VgMndS{l;k!RK-!ca`q|U=fKXMMa68tA)maITn9XwFX!R$YUgO0rvQ?T)$Kqzki@vDLaT5MdDloO3J_!4sr@%T(O z>wHl2OP&9E@COL!iN*K){QGf%Tn7s;TMTDTpMw4P+U-AVdKwNL{2BY^bD6jLZxU+FY1kzV^X_N{O&8uF)e3K|bpeyN=thgx6t);N?e;Vdbo z(>S2N)b+=mZa8)5$Iyot;kC2o;V+`$#XF5hdMy&*75DoE=>Gv6AggNd6|lCmxPeiN z<0W>Y_M^t3e6Ig1@U^koH~u^v`Q8TnZf6HnSKwNPLO@@({wLlflj1_>c>;~~L@coo zaaK&Pj8w^VOfaOMP(m$Sp5w;+RhV!ii?~sH%2if(L@hbWi((2)i|Ith07N14j#N=- z==HsCJo8<=k-7{1_s}L3UB*wr{ir$=YBK6VF>dfw z;3g>s;)HMXxr98mM4LfVr%i_C%j@9ks~6x+qw{d^U@M$H-38me|2ll@>n-s1u}--4 zwp(G*r87t*ypy+En2Lic11KzwWwv=;}F*Pk5Y# zgFkx}U-LW8Pftz8yOne1&SSl;|3|L>d8~1B2=!JtU)cDMc1v^XD9nGnh$7diq(}E_ z`PIE@^wUoKfq?@*+m9b|-NU}w({lzEE}X%>dBY7$;hL)#!c|MqH|MpYPxnC;??lle zoL1O5d=vAkt158Qi=I?OFUHNQY169lb%!aq*glDOoX^7BZ|%WFbwB$10%({yi@!m| zcYx8ibq*u@;XqBep!1r>Hvg9}|6|{9(Ab^Ek>ai~(fQr2>3rqHC?9VvQyz4#=|cR7 zPD4P?_KkTQ6OE>oUl@n_C|`|>(NnqdpgNlNb@tQBSL2X=F#NPG#R2`Lu7CHrc4&R= z1(iejMUYWqu%hDQFzh2lyB2{|elVIP>P4aPH(AFk@k(4F7}V zSAoOq#7VeOu=Gagcxx}rSagGk6tq8L{VyoQP^j3HA)}AP9$Ao70E29*Pt3fSIO*n- zlmtq0X2MU__1^U@BBOaeUv0chWY{1K8ieEmMcHS}(rn`gq}D=)_% z{F%%@Fd{mW2KWp2Sf9RwML!;N;`njsICTJ@gzkX4nqx2lWv#0{16?(JxB)hR8=dsS zL{(5-dm86!eCn%m3Vwd$P1N-iidliY>#z;_g? z(E7zfI1)#U7iwsfdU3$b(53|Dsb#U#Q9Wm9WYA2{OocBSxhoeQe3YcCNOdT1eRPJz zF)%Y%UCm=zs3IwW5#?i2pTMBX!pre@va=3U!O?3Qq_@gI#YAWNyFO>UTsrPIhTHnz z*(2KyViVgh=5yMW6NGAO0;d$1{R&(aL4V}!Hw51fRBCbKD_HQ6*s6B;KeK6$(hiBcQ}tA%$@j4Uxc;}bnel2 z9x~{le?%kQGYx;Hw|;6hZbH?=Ew`S4mtHysfBUyz!k^}y#E(f|an&{O;a~lz++m_R zu76p6>wmLmObB>0Cy= z$00quYOV~lROZG`b{R@*S8Ljp7hRhXXRS-=Flaj(hvjr$s7!fl9M-40RHi&MJzV;L zpXo2YocG#<`Xe9KRHpr+G3A9p%kXKb0el*y()}`T-ZrpqR8@n` z^y5dVJE3OMG>Gsu=0fxZ?S*7f8^UC||M4lVDKp^QTQ5V~ft~mv^PdC!x#Ik7k)BRB zQiIkjH4AQlE_}*7!n<%4_$$X89F3d=N5xj-pqTl4^2YZWh>r48_2h&$qNqS(A`Hm5 zlF&%0rbSTF!wu8 ztHGy~W}u#RxF0neW89?TjjCFvBP?lrAHag?==7x4##zQ65zaMfk|aFc5Q?*C8& zOs=cMHTQIss}|SkmH6$*Zd?NZoH|nr6RIZT-xT;?|M1WF&DXQ={(Jr-KKZ-|ZhPl) zyjd*U6PH^5OMiIw;%sNU*}vLOuDIyNly10a%uEzB=M=ig&^XsdO?qTNF3N+F=uBFN z_{$Kai$%^z(JKwb9F&)nCmg3j{&5{wUE8LZLP#H$$FiILFnL@Xp(>$6x7sH(YfUJ_3E^LVkB%o~$5$$Df|( z1s82ZDJU`y2=IQIwDbZlAm|>{#F`4+=$e8T(tsarK8sIIorl-=d>_Aga{w>M@ne#= zy%QSnn>sc49z6$QNliJ{8N99k5gr@=WwXTDiDxflLvb0MXqj)mnn%UK_)1l{4kySp zalCOfTollUTca?sw8vQcDzX4pq;-+&VFgcC?M-n?nqNE1&O@e@Y^YY6lL*1ka z_zFQMyzvHp@b$ot@D<;;@oC3u+@QK1Ul&{=uiZL6ibYO+h`(a%@K!pFQ~EPK5{}^b z7s|tGsEj0T(4Qm{*Wm<-HiG@nZROD4wH-$!uga`(T<9gJjDp70^&Y)GnaVK}l683Y zp|pAGyDIskuzpBctUQo6mVL1DD4oUuKhqxpq5t|S9rn*i_5V41GWz&W{sn4h;t#A` zu^gY6!cL)n(LM=ngU+$6ulqC{etr|2J8=}Q{@{OtnyHs3n?72X;#@)dUE_d0)&C?8 zt&6U>4Nm^}8*m)Q(^=QPhlf-AjZ)zr)K4{uuU|*FnbD6Qcb`COB%fB>PgTGZ%2&Fu zjLx-;VhT#HY0i{G-tjc9BtppehOBU$qY#c~`ZQFs5rh_@PbVOKI6mr7O)&-HM}kUB zIhvOCC?_4#O6ppMW8~F`-$Ffo`V6$Zyqn{W(`WHU3cZ3?fluC_$0wC~;AHD@e46+t zc!RSYKKTCGxEWOgORvTs=9yN-pOZjgut~ptFU|9156+#Z5$<#NdMQ0cg&R^6@Q^SZN>*bu|yvaEM7yty3NPq+g5FkKOA|+8;t)x$~WXaa9)@#fA$4Z}VefIa+ z*2=P0$zIu#*Ir4KEHTKWC=vjXh@8U&OwQrW%kjQ=-`(G@s_UF{&%JNnfJB%9ZqK~i zeL7TkS9NvwsZ(9u)meu{lt$LGa{m03{OjL*5kjhV$;?ziXtfpwoE!D_F^>Jx_%HGQ zhh_fbSm!a@I@olg&9xK9kH9Y5Y55;t|0}uq=G}7FCvPgko4`(vM-#xCuo-7O78aMw z;!PA`G4d#Rv#So;TKvkKmE*_8<#&GXH92+am`qOL*bQvMtX;EKFH5N8+1%jRZg(Db zzqJ0R_iMlur^J9V;wf)@|sx5T+*fU7ZfPSkStxKK2+p=(ww1x+Au_v`VS;HP8bZ~D~$eSFbsX>(XUEBUZl4? z@Uu9s1f$X*Ek}Ny`xV)C-`|5aN;BjH5cqrB=ofL-O>I&SLBh=Vpwu;YB1gJFx`Qbo>12*? zrzx{)SOa`8@O*@eNzT0?W0H-G!=BH5=DSq>7>>cPeJ9IkC0dWivDS3{{J4nHY;Qd7wk;K*!jt0C-E_77+SwM z`MED$1Hn*(tlLnJ<9ql9tGC;s?J)}S!@>_ai&)!iOyy)4={U}pTtaxM13rXO z6>VrB1y&VUYWbPf90gO2u=m*0(~5DrDBHL9fPERy4f=p$v}hA+cggZohqv zjE$GczxlU+t2gs@;r3k)l@{D|;_EIN8Cpj#4m;W0XPSlMJjwiV*wUTW?*;v>1G)+# zXd}bg!?3p}|f9HAfjE5t>(MaPvFpqLh@X1SQ z{!a7%lD@o(4D?^Xi{}yfz2EsQ9OHXcw!==%0}p;uKJ%GRKx3(efUx*3=!*k z;ih2fG<=4&Hu!Ymh1ebvzdnh1RQ$~Dv@}=aOH;k{bT`QAzFJ)Gdjwy`j>)snyaeG@ zC%#r@g|?@xkE{<)J^avD&%(^ZsFnJkPPal1mmmox_W5Y_hde%4dRMg4kj8?=oPo5I zh>nabs0|%Jtkaw&JRFtNNEX#UZySAWN@@OA{E^OzDd5d%P7Hb2(+z2cOU;6a)t>BFgJ|;p`ffqn&d_+eBXh~6R;JGE4fc=Qmr%uXm|6l)Fo`3Es7&c!m z4?ge-@aJ99-iG5W=vy}8OY|HxLiOL8T=(1$i^8iwEpNP8JEms@r@0*p$|f; zo}LC6T&-HgMsVo%g(|iag zU=$5U>7KVpdFElX`wSxNZD72Imu(Nq7!VBc*kzJA;pz&~~e@f@V<#)=$`Fnxjpr90WeR zy{=sCHbw)44UL@jBYM;Tu|WH0)I`P+z}y4`S0e}yX;?51OO9G`uw4Zq6@_a#*rejk zdUTDj`_$dlB0I5eZ`{y^+o!AWk?DlI_6p2peC0c`fA5R3W79C!^>VppcN6SALMIPJ zy}SY;y>vj+#?-6~j;s|&~Nt8vEw zj*=P>K)EzMLbk|b4_MZE5{F}WHgetz`dcSZvsTU@8^=C|t=&^?h}bRGa2bQmiZXQG{R;aO*dx_N%Px- zj#cERuMT2i4ZAun^6I1z%6M3P7D%NV4E@y~-LNx{SK#3QPsPw2DPCU`jyf0vW5}c3 zFZJIDuFPa~Cu&Rdw_#Hn|L-3EQ;@!Z1M7>6^D=_#<#|--*zvN@GeEg#)o zPzQ#fX?|f=sxSwX<62t5<`N?mfAq=7{PPLOeMDW-CG**`)(pW^O-+St-`0jnb{y55 zmNTc1pw0oDt-ni;5w>=8sBw1J(BQO0e(~vv^N@zae8gI);Z+vwSCr;ILs)74595#P z8ON@#XB@K}&cVH3>)$-CiRKFjH3LRRhG7)-qVi@YGl`q`ZkFCY7}xE=7hCY*Lf}p8 zM!}oTi{CGAD#v*D16%W`Z;BOkd~%K>FVi0o*Mjd zYm`7@3@wgUo&}(YH(76K{k!D;&&R0y1$YHgb>1`FmxZwTmN{9JW1G*uxASnECm$a9 zB>55$I0tb?JL4gQBLE(@bdPqQElP%UV6o{wnKO~$kf~^YtXttXzr98C6Mk@|{=fV5 zU&0aOau|r7pBR$yq0=&c_$4{@;=@?qMYewWr}5$p4G`P}s?&wi8pOE-{p)cX*|yLB z6WI5e#px=1Fr(e4h^Fs6{kd%@=ctD*pT1jvOctw~W#F~%%E-azrSrPGqG}+QoUV05z-=h7D&AH%H93Rh zYKLTG=nQfUt>lK6q?zxBQN?_~(49E-qCz^pMQ9s^7cKwBUU zl|E^?&vGi1iYOCcMH$kKBOI?~x%#8m62K0Gw(6s{EWcR+%#fEg>mUg*_?(89&fqO8 z^`F=xzDo2jjsL*oIv#XbC?9F)UE=>)9NXLX=Bx7J^UuQY_zN-#dr}MY8GL0L#tkCP zvS#%>zBIjwv{U%PQy~qoGt=3LmrfWTt**d^P_UuInR}sONKOd3DfrIa+Vb%1VfUHG zxj)2($3<0-$JO;~8=$3Cjjuq}^6axiauEycfBb|06KCi9zVfkHn*WR_h97%@iBMgd|07hq8|&ZPEO>M8 zt8xZfQs4N-*Kk$zn9O8GlsCIOtCcsWCf<;jUOWXYrV7}VGTyAgalh(vXcU%HVATk= zq$U&i^9SKg_7p=Q8+da~U%fOzNY&9%BL@yl;BQdA{@1?+AzCAhul~5K->^yUz3)Ng z&1m1^OlF{T`#{Z~1-w-93tt6aUTI_wj#-m_Y5$Yh|8cOx@Zn4_#GWal5doh`^x+EjL(~Q$CW2tgGU;#89|c7?QL2_s#IPRE z#gMgyJ{{Y&2ppa{bxx1Lz3}|Q($}*jzy7P&$m+HAIL3!#pjm@Kw6mY2!OYAOd}xtKZNh?WMl|KLuyHn z%@bX?WQ5Y6#ae_L@z)@^3LM{reX0fAoCi@D)@7auXvf;wDqFX%#j(VUy!g@)n5`I) zr=EUI4jk;4C!g$>|LMop>;AW^%e3qkDY$h9PikcC z=8wtsbHf!<@X+N6rW}jGw+SFI5s(pe5{NM8@XhzMFi|KIdmM6jS@XS!SIP_) z=YC=R4kMJ!&#nMC6*-`Si3;#7E)--=uai+84t`<%rT#k&V+C!}#~X8UM5kGc+`SMG*!hUw;iYolao$G$|eJ3s~UFR2{gz zy&lHS>#%@kVaRbBi+6=C-W3(F>i`vDsx|986wbX9M@5}R19wDU+ zJzi9y`N13e#_(0Bt7}$$9c214%;4l%3ZhAh!+MNbAg?Unb}d;xR`*=kAv6? zz(D$`otQ^Dq^+H{I06Lt>>oFZ7tWu-i^Ht;mBMA-2ul8JZfON?@=LQ9LfNpsnNL^% z;BlJFyfObXM}B$6zy*+3_4xOGC-DH4XTXC#M8pqtFMyxMwUtU% zpA1J8EsJZXlhx-Cho4R;H z`u9I0j+^e14L3h14c(hCjbJW8nYc-pM}G>Wtk8Q*f8eX8 zyb=c2p&5GpgVMR>X6eVxf&(zLeeRXVrDN+J>AmSeY3bQ0+02a8G{83C&?1yrYcNk* zFKupmXU0DbG8-=tw&O%eAMit#FdkMvBqb+(d+B9NXcIh!0t*2u;3|!*{r-|1JWb@TH}SZC*Dehwd98!8dcW2V;fberR6 z!$JJadQE6#T2xHuGaCW(7U*l4-loSRe$|MBFctgfN_-@(tgMBWQ>AR%+J`-C6*Q=Z zaZ}(pw4f$s?}2d$rYhxz>u@>%YdiPfB%^}m4~SD)Ig2Cov<=$?gXfb-pO-c3$Iw^U z(^}sm=Wt8ksdHm8hW*6AU>i(fw4#iQEC|e;(T<=kj&x_}N}|8FmA;QPZ?DHid3@*v zQhgA{f7G``|C4d)uv0l5A3Zw*y3Yg;)2AaVII{Wd;TZ9j&xQ;0dmiS7OA%rv_y)%d zrJq835)+Ye1o9`^%W%ZPXda>IXjaeCy#48kXr=a*`k#-#2(badMmgpQ&LRR4NG7C( z$)7pW@vYM|5vvH-ai5odb&dZlv=p9w>Ip1_r{zEYKmS4YT(<^> zl()#;pWG;Yt16@eA9<>(pnhJ7TSu^&q9TKSG1RfdmtHKmIAbsTLIhQ2T#Agpq~?-O z2*|=?MR?5N1$JeJajSLV<`HZ*8laWY4x{LsHn%{~l$GoE49UTRqw?Rs@-5uDaz@@d za8Ul|pZn*bGn0$L_pK?)b#r(OSf|^0-A7$A-W8 zH($ZcLddoW9Y-V{^_2PXb5vr zOe$c6YZ7qmzG5s9$7dvKQTlWQ<8gepov8VI)+tak>61qUpXK8^nZsc@eB@v>ocUHC z+iI~z^K08IgTY}PmY=`Lu+?wvu(VSDohHPGhv^%B7#HKv!x24{vq*L)kOE;ua11cb zA{;(n^{Ko(J$hP3UU*hUj=U-tp&f;@N>bb0Dr>I6akq8XNb|Z~(g>lB+FZh7f3?z| zc7$lFsIq0Xthw=X(u3Rj#s`ke%(=aC>XH8MW!j(-w{ zbTAr6gbL3_DWwxL!a4NScxijm$PmXg8zq)U_fr~|7&C<7gDwdI zVK3`F;pHDRH}T}-&&tW;Z^;)w--8P`YW0@AEHt7PaA8g{$J^}qw|aFQa9<`ncVLCY zTDP#UB*#x=@L_gA-a0giwWeHF_15B8U^R}^;iE_pPQ}ej_SIYEH71`!7+O?%BQEi= zbO9MT&g-D015GMuGB1_4t6yHx41rU%-1w0#a^0Sda_7f)(Tlp)002M$Nkl1-ISt{Rmc+nQnEy;mN4{AHPZeN1-k zJcRvwH~brLoUa3Gdsg1se*l-2T$Bet_bD88tW+worQRzKdruhean?)E>Hd35|HAQP z^?Q4*jq$IKbst~a7vRnFr~Bu9T`oPw)Roo$xW|Q|6$Mi);s|m-jVR`Eq*&OV^aY3! zMO-q!5yBQ^&+2vgbH4AT-A|J$KX*Enr93V_XYnU*i~CwWF;G~_TVi2*D)m2!m3-VL zQ%dybq*EII&R-lU$uOT)Qvt%EBZuFTk&z*J=sVxS?ImM!%PrgF_S^d8mYe%z^QJjy zN8yZqW)^4pvyz#mjhiKXndA*2RaoTtg{le*cWo^(&M!io36dKFKAWRvR{sF6tjAYI zl%*CFzYrC=xQwT9eCrrTQZ?*ZRbgYXX)}yM;%t8Z#c9~u8IrwkJT3cf_=wEndf*Kk zd1XB{xde;wNeHC&!I1mQ_#$>vK5_$(Ax%Pq>UC^xnxwO9t$gl>e+-K>jy&mG4(n%a zh3z1`bgJjn_A$c>&S~MkmM_hJM&E$DBrbj4cfS~Vllh6)tC|1Fn>_Au;lg=n8XcEM zANdY6k4`CXu3Oh6pSrtGdV3ad1YkjCr)MMso}8MQ&g0EW@FaP&8hlm@tw$;6t0G6P z^CrXb_;%i;G1yuNp{uYtZ)&0-EA8SkojB?xg|=n2&Pk_Fosbi_VBy%&mtZsNq+GX~+s;K98{Q|czar3%9F@E8 zxnFu=6q(NWQ-gUwA0+x~KNG$4aqoU2NyoWiKZya$VN)4+fl1&1o-62`TUw@XXj;XbOL+Qi})Pu;$4l?u}&m%)+4Y5C^2`}Go_O6+^r z_0?hj%-jDA>JscvHa09E9VZH)4OL!I4I8mDNFSBn)n}y#NB(#0=$1G3j>*X2ECg{= zY6QIw7nOL+6p}YJH2+GYzx6ipn`*Ci#^Vy%Jm`IOpYh3XM5?Q;|K7nOezCW#ylllb z7t9cFpm|LqNM;V`qL)OphrLiLotrn?Fn9(9oA|MfyOSeT993D}-qv(`R!UC~jFsC{ z>VJ|_X?&;g8I1)mlp^5D_%6}k#x@aWTtHOX6UX8QfBdtv*(|;gz4fM?I(b4Kd-OXH zG>QDizxV~Y?ba52k)g51eyBCi!x*D|O&T1T!B-mG_JSATdKhn{V5y>_2|_6>qWIFo zRbGWt6i_7{kjo7Kipj+bQ!3bMgu%xSYzTM^$cJ&VNII-@AqkY&c=X{muzJ*;#TT!U z;bZc|6Z`Nr=vCBJCVi{d=*utRoIZIm^ zcH!msJ_zCJbn`)jrOtya?_q5No>v+lI*<|rYGr;b&40-;kj8f!pE_VT7QB#$=`;E& zU;mb1el%ma--YhpM3ZUybcFki(+{EAZwbc&WLi2E0%StwI=bRg7NxNBM%dPw-|p zG#i^y{_an%mVNt2U>EBp>4qWnRehTvwA!E=P=rd}{N58!DsRru56H(pyInFfi}L8> z$7JuDBXa2QX}R{g>mktUhR`c$VLESl<5;)HF?f~J|Lu%_<~5-Zkx;}rj@)OKWLRJF z5l5dM{t`P+;F9o((~us*Av@x&D1CD_JI0l00VH{O6BI>)EG=(wEN^w0&*wr~rr}xJ zmTix>%Y6mPpWw7YgaZJ!vebW$Dsyj#{<+E;nVmc*7hZT=&cE_qnI1fk8}sUA{S9|W z`?i~;p>Ky&)#BwF8;#0xd^7^iDW9=-6{}Clp<}}92Q`>48emr*{b4m*${-YglmU&V z$F3h!z#o4#0cY`cW(R>W*Ev0vN@4U1f#fO0i^*F^5FP@ zC{`d9$9kAOsxKXpPr@j#S5YLp!t(SPd?+0n8kO0pe&BsTc5L2;wF>({tbKWHPPNO+ z>)fLcz&rBr1VASQTYTVNat@kEW23kP2=I6r5N(lGW3O7F^rT>lb{Yx5`GjFVE{~w5 z(N8we|HFc|CiV)cX?irdOm}+gEgH{!UTd@1u-+u;u$fAFzrMq(z_5;1Lckcz6 znwY?zc@%wuwzJ-#Ywc240H!n`Pxn^B%^8J}?S;UQxLdznIsXzz=+zgUOfjs7Q^P!L>EFvSua&i-;!009x6Mqgcl z$P)1$#|(}E_Zg)<*U|;`6~I@lvv6_jJ6QJ#H9u@m-rgMArgenPjJ6_Cpw#~o{e2AT z*hu|&nvbVwtViPsgQ+zBV=i>wa-S*S|AqN^d=(joM$|CQoDV_k;tY(JkK;PobI>Zv zU||}<@wWp|@tu~R_|mV5ogDEPwayqjL1VRDXY^LgJD}3P zWpj`6W@96@7SGFTul36dFPw$I=eTa5KXQGyY}?i<8#lIqk84%1)Y(~!BYx#H|FpbA z+TkN(I4U@%?6;@MMv>QPA?=lJK95%+J?X$k!BV=En>xn{Pt<2F4s zm~Dmy3urQtq|`P>zB1mx2Vo>kR+Lxr(x1><8nj=cpo!_YMK62=N{)5rb=;6vkyA_S zV<=DCP~(uRG$Onng4<^OyY<#>&>mhbqob3st2H3M^ZPGD*i<8Xc0=2%8{-apVfKG% zX%63>7O}tXz)64#Xd|CPiF^tW)EesJ4 zaZJ?8%MK*VzSHziH^S?3-+3Spzlq*>-{==}gyVUNrC&~*w}pA%tNvfAT`LNvl0q3D zqRF^oz#U03B0c1Ou~Kmy>+w1mTYRww83Q&gFNtID3(6zjEa0(_JS|KX*aqut^>s902@E zyxbqdRl?_?Mn43NjV@WU2DhW29AB3Edg~#)nuk$L7+oG6gys~^{X^>`2U}A%omm44 z#t$0caoT=nE6slqTbB6W*BBe!I&Lx7&g1K=9-+l;PP~0?WN1hR2QTWM@#ct}K7A5L z6eeVFpdSL|5eS!OF=lUXhM!18&}@snBB6e6@uBEenC~6PxTd2 zMFZ&Au-dAZEr0fSjSD_=ohM4%=eYFs-caTHfat}fAcGdS`HexOJAV^`&y&#@vqY;T z%AekAM9HB+nvviv!P~%R57Xx({;W=Y3~fokV~)^g<|y@_Sa|p7zl1k-YDpb_=)cOq z{^zBsdzGxd{Xy7Sx?gXkTY@ojh`6D>Vf6Fi!B(kL`XUtQTR_mOfc|CBBIlr6oH!>L zXxlW=&^eBkRba2=a}E2K?23)c&~F{khMym}|5^z#OL)!;e6|?+&Ogc~A>QMdo_2(E zsq}RI^s|>;uD*iw&}?>vPSX=znV!yI-59~%6PJC!2)G$AH^GdRc>BLzm)3`+kAg17 zk$x&OQjnoR6izvAX{;YYM?KzGX5+waC*-Y*jHj@Ow>PpZ0s2=QJkkZE@D07Z3_>Mp z`jjuAEcOazEf@~Gowfl&C}=;|*T}WkZbX?1Idx_pA9@$?F?B&YySb-EIow3Z{WESgTeP zOz9>GgWWd-1mfXna{$uWCdw$|Vc%F4t2jTd5S2~@&F2s9lT?W>kRI*6PpA4OZ|^%Zuqo~TZ#VBh%1eM<(kXq|3%z7$CuqV-*^Q#T%DC? zpLs?uLg;kl$N?1|)ncJ-?`X#nnkooomhfUYfwaT&r7t}Ib>VgLr61jmTkU9D<{)e_ z4a)K3yzOoph9sBdwp&)IQ0hm%xJCMK6>lfbCMh8$ZfZ+!wYAP)bPcZb563D&mUG?j|TaiV{kc7@q-WzRxUy%d05Vzo04aq zfok$p7Fu95iaCN~PlNK$fBjb>Oe=#Oo;lgQqd`9Rz;5(8D<@7&;A>Kg8d6`swn4UU zZw1VZy!!IevsRVnKVk|*bNb;a5&Y24|Fr!uH#?)e zIS7I2_nvqRg3vK&E550`SOej1XIC3|vl^Rh2(aeHAapt^AN}aHuqSnk{OFJDlJy&K zb0m0kbYx0id8Hp3PUDi9TvXm{g4W>A{L~J;CcC2(yot>^ZxdRHH)-^hd^t8Y8}VjK zOLgE)k~G1bPEh4d{HRaQ^B8Yt@zr(?hG`EU9+M0G)AH~mX8=1ZQyJVYIErJui_`MY ze*GUq@Kp)BQM0fqRWF}^;5ywrA3Hj3yxCkWYy0YDD-5Y`**q;r4()?B*}Ux7c`XEQ zb%E(F<*`j+OonXE%lhaK?lbRIy8q#x(DsE97Y7f)1rtt;9!DpdqcL@%2<27jd8QQ7-L+^#(=UMQ>}cd|G);& z@lMm9@G2VWr2o~2?X=O83q?a6?_M+JGb-4F4vSiHV*FIU$38HPRnv zr67ER22JBCnH@YK*~!x~TUL(`UF}k?h_|tr=ye=?*ZLo-NL1}rF^XYN^6=%R$Bhr6 z6ab@nJmw1WuMpwwa6jGN!9g7HK6G5Vn&$OV9M87`*vn5wg-c2|D?)9lxDF($U8zNk z?F{5hxb%nGaOF+#Yq00-!XE@mQi+oY3lRD&%H+gFnV+A+Fqy*s6xvWr5Io_J!X$p! z<;NR-K;rRv8qjZOg59aIDU>|`8m-l~EDCIvL3QiCd#;tWtJ~$}mrtrWiv5R1Wdn}) z(_niRf-#!0VAI=M=75_eX>4po9k~1ldvAVluAptI)jKr*SHJ!W_7nW*b2mOV*CM~I zt!aF-?;d9uN&TiWyxa6oRhMcE9+4rjoJV1eL$ddUK3y8#O6gWA@tXqhzV|=TG!L#Q znBqoG7mDC}!DyY=2=caK7I(PMI7?{NIy;j^(LZMV81WU@tsy@vb!>4(+E)KJeym_ms ziK@|pg%b8}V6ZXD5+*OE(LRpH^8*n7iaBbC@CA5$ybM|;1Qr}Zxj z?Zwg2VQ4^2;u!5&@aCHk0u6&V2e3iSfR}K!bz?1V{;7veh;HT0b7%X(o0Z_zHe5ci zT6XVRhhwL>`K}&DOu?ITFzip>Bu}-%xOp8kqsUw2O$wDLl)7BrByI?YS5K1<$eYxL zjE1!%7WpW|^7u>gCXi0voR};_-;Fmj8CcO6n}YCo4!pSt-W*3eE2XoYya?U|AL1h~ z=!e@koi{PZuwQMs8Hq<8%gTqq+Y8E@u~0;*t8~8->CgVqrN{3(IsOUT2;+lYtF$=K zLiq|wF@K>|x({@2@%+Iz<^f29$KS*-i1^B2ShgguY2Om{8Lik3ug}8)e#jfD(|jhJ zD4t)to7W%!AgHjnjlu>V@Q@bjupXBB|AW$h5&T_IQzhqLz)hNueoeY|TrYjMd{*jO z*GUED6YLBO)j<4*iT)gdjqU4UZ3suXE*_SNssU+VcRkDqO}~fZp97zBHGJjeKni$F zp>y_eDlsBCC3rZGrBKexE0Dww^D)iyYf>V!2i;HCuVqol(-WXISdgC?q4L9ZM)!>A z0`JMQw-&f&%W1z+C%VY7P`KZN5L+-q|XbF4W`T04l)i8oi^A&(gcg546ZN#5bfNf5?^mt@Gk7GVB z8=(vV@3W?Bfe?p?@Oatv5k@kzx6}P}{VT#xw823;3_f3uTTw8@%}RQs689oBJ=IbU z$n;_zqnFJxs1}FcfN}S=KEg<1E3Y@9fRc;}sRrcbwb~fPTItgL@p{}(_PJR8QvU

gFWUHbGHM8#*hcm2%XGmNQULc3V z?t?F_3*$uw#aQKBd}eFL&5B6WMeEb(JoVDzmfcFuV(a5W6$fj+lque7d?$dwCcK4F z(X*X~(3+V*0s?=~7!;Xj$YAR!gBKlgHn3Vr%qM zQM0F1c&=M4ZD3E=U)?lYfw|8opor^=cq+gRzibwQPLrp~8dd=( z9AEvWuUe1pC$ou-SX4}VpxD{{2JuevdXx0t4c1P4^WJ4{jfP{`jc7D9^6kf7I}zyH zIhSD-{UG=AbwmC0qKTlV+mSQU-){D|GmW*b`&WD<6UsW{_O+RAEZMp&_Zlket~2W< zVr|~}f=e@>+ow9ree5ksKMn=uT=2n?H)QFy$_X?g%rBa?*4%p+{s8+ z?*jU0!?wQEN}!e2?Q#6zlgBC7w<@x;wVIv;`TP4K0f530nyWfexCWQKmCmv~SXlkR z0=g#>gP2XX2}PK|q5(jVk?(j4trX9&FwcIpc~3Jkqao0wvYvQ#k}=QU@0pAv<(sZ0 zFSmJQ`JDc4z}xCVf(sC%z(yH;kKlFd5TEOKpMCx6Y{X)FRmS#)KAUb^%BY=8?2_e| zAg|+G{DJW?-+gBWNW$0oD4@nq&djfs&+pkPWJmAg#tR4eL)62{Uy_X*sfD7={?&)H zqV*Gcz3+W65%w718jJeIiYfI8LvD^iSclyE!|^fSjoW10Mm#!rG>uU^3UT0C&g|OW zaz>4&aj2cO?Mqc}x7;Zmc?w@WV{MNa!Zi@9HJS!v5qAg(+|Px`VAlu;5s0G0EoE!6|SP)aQ)`(Dd1})CgyTe zLC??qRHy1g+mx*T3`>&L!*xrIy*hBSuvW6FcM!2?oi^oYfu}Isc|uaSfF}PJ3nb~Lw@0l00{ZWeqy6GG zVzPRg4K{6sTLIjJY^db$0Zq}JLcXdcQZczyvirwGozGbfn&ffP#|ZOiLTotH*vU2s zNK%dqlt8|@9>tc+$4?6*fyR*6H(e>ALj8cw9K?tdk(s0X(x~c8oh!`;Pc<~!SFo$G^Ep?3^&HG zPW)_Hb@kf&ASuoKP{r2lR69nPYj;WzJ2@C-gCu@GMJwojlU&%z1x<)m-jWhwQx*1F zF6@iD@dpg+mGmtl|2jC{+>6cTsRN>xfKFp6BQxXHP<#s3*##hzVf%CJg#{Vb0T$}! zOFHEh$1U=b$>?ckQ?!T|U!A7?5#fM+;!+8G#%ym)aQ~pH_R|J()>EWi3s`EYpUn3M zdM4;Fszbm+9y3w9Y@p8+=1_|IT6t@@-TkQeo+w4mY)cNX0%{MV1)(BXyoKXy=mpR? z%6AFt#e%|$wf7tv|9Y({h5xbaT6)f#qc7jlhHnB1%M(zEP~;Keo^GT9bG|X@)Jkjo z(gR2N^Cso5AkR@zFJxGn`- z5KgBIttcOkZ0icFh~?N!WyjUF0i-um74Lmdh0l`=0awA*Tgf+}mXni2D5GKs zo*1uJ4|B`=fgoXRPH{K6c|O zhN>nHv<@JI)4m(g3xy;Rt6_JpGUC0N)?+H82CM)5jR5g7q#x-_yNh*@eHX(d~?^?tHr{@XK)=tUU6 z64ZvbKucTIor3vyAlBcE;sc=wHD1z!9vV76e~OUWjG3kdXt0RXc6=cLOjv{<@`?Of zxKqGzeEVHENW$WJ9GN_J0S6YS`lc0GX3W!e8RBvA_Q<&H|aE!hj1GP z%!KD!HkQa_PL$D^yP8jzX;P<BBWY!cMLrob&v#kdf9f><6C0#%nRYk8T*cTE8ZEIy^3}q4|XNkGTIK zMhk)F*kZIRT2%k>C9=I&VY7R+3YVaS8VRn6wuN zA;`!#GR3f<&HH@F9~QI^Up`h}c`kEHaO}!wK|Z7M7G5CWkKA=QXZV1pftwDKf8q1r z?}PnEhy9IM4e`z2Hj)h&n=f0m!tZ9iRl@HP3?1HE#=rqHXbX9zQ}UaP`a}E&FH1FM z`#UznV@5nG+DkMD1^?mXzyD;5M!%-`XN^%Ngm#b;l%fH6^=RiBBa&fiNST&%?!O-p zboi~)1kC-zRFkQq4){R6#wZGzfd1t){?`uK7I3VdKvzweviIbFG?c64M!%8)B`<7l zKljzeT-#HD7~5w7Trw2QGb|3 z2X}S7uy&DB!oelfq=coAxQfg<{%vg~t%Re47DB`IS!;mr-`s?$6TE;fVQ(A#yJJg8i3y8O)>=65&h8p9K# z7;4^o__Nm1?}pK<@xf!ul-xt>6|q*edggsvhp(1v==6W0LPE-DKZ%)}T3GEqbaS~= zt<^>B$fLmP2HCfuq&KR#SX`hV^)LAvy~gI-O{YZ@2`z1X;`LJqxjo~)xsjOJBy#dP z(4W*L#k^!Gx?;Vj(r$t*9vbUu{c{rjK_Hts%F0vQZ;;mg?(xIp_+*pvy@C3pKnI_w z#p-5`+FeF$&+dlvQ zLY5!2qq;DH@Y>R^o^y;FE=aIP7ot&u)I=yGH&a zjv#EUMpx^4cLqb0Ol0A}(Ie^z(1GXSE+H#G^R)LpFVY`7`#&6KJH~DVi~jlno%U#| z-uR3)R(Q0;Lu`Gu($UJow*u*pD(;E;a_~oE8X-|uepv$9*_X!_|n-1iRHDh-p@nO&xP>7a3n3aLUF);p_< zTi6MC3-%1of(QPr>-y2?aGATw-pZ`kVR9p6OphKOmiXJ&VEN%)?=Zr$jhOtSZ3{Rk z6XV?~bP}dCPpnQmw3S$zXNd+Zeou@m{~rW*Qc`>oTc>JPt6%SsIeQ*g0QFP=_3s)M zu9-=-C}8EC#dBl9r0GvJOgdGnUQY#QedkW5;e*KkKrMFnIQrL5|H6XdAAt)*mWSih z0WX&-z4BK|c?&)7_{3dt|h=3Nb~`q>g&njv;{OoRg`~gMW#1TE6ccQ z1G5#Vsi62aa^3_WD0f7|JLC!Y#&}qdaN;0=$~` z*6oIR4yozi$`Kx$&!#7AX&CGk+Z`_s3t0Z=;QftQBlma8esVcdP?Q#D@FTq&7i>fn z{R+F*9~sP$2jnXGdh)zB)b3eW_?LV4k7NX##?(;@M<|O}5kpD%;C>g>nfs$NTsVO<&F+mV^q6$brZ`l4&b{HcD5L<2o zvaNA)gA19MUj2L9e^@{h*?9(jvh|IQqAjrT?ES~;cH)Q|^=QY8QN=_?A8of?W){Da zX8SLmbuxW0(sVLKJO80mYtonCO`s&Nz`w(8^e>jhlRe$1e&2mrI*TV(5eMwTA_~SBVIJZpTSG9;`U>{{hiYjAcb@S}C9h}P z=x^jsrbpj8$%6j)Lu_E2zYLNE)g30r(8d>^3|9Ep|NT5$56)thqv7ofQNLP9t<(;< z*leqR_TPXB!oF|1%G9Z;7y2?sWJ3GD_1Yi4S>VC-P~`@a^PBMB$m|b4XOf*M(nrAr zhvxB$;E!b){jhPMM$R&_K$?NER@LwrA^YnU&HsH53EY^dfar5;AqhJyWoAv6Hlq41MDKz-MBmgt|G;qIfPlitKsoS?FeAL&nYzjO|kRvbu!I%Xm zoaOTt=S~QtNG3-AOFLsIsP!NG%#a3T&KT0|KcC^|7mj#?_U?|XFE2!vTEHKcZiAq4 z3hRxZ-0jlN)`ETiwQ6TCPUn;9rIcmT4D53TQYW@Pm6Q(hPBel4BS#06o|sbcr_u(%VS@MjsjOi!fVj z1#osgva$aWDOy0+Z&AvHwyUF(oE;>B;m*s#k!IQqT3QjvRJGNIxyFN8%fSai=D6dv z`%w9wysyat%uD`j1C-?3f1_Cf`{4BW2gskBi{qW( z-Dwu(d??I#z?d0{8{q40zJAI(r2hk)@%)qp4p_Npa-4G%FBeuKgXRpxsA+#w5Qd{2 zrZTwG1T7MZ0FFYVap3=YIGGY<<2P0#dtad8h6oJ7&UfEnO3SS`J0{@o-}zC59+xJ* zuOZHD!(*MT$`98H7E<%UdD18lp%s*=Q&rp z(;I&cHX_!LC(rGU`&&?N;XE7g+J^@~jr+oedkR4w1N~9-X$x44z*&OrI296J#h%O%bi$MWU2{OQu31@!uN-Abj zFvcI2$de;n)Xwhdx72(<|H1r&GZPZo3VfdeqwM`uOx53%eLzVEcdy5;K_GUQ@PZ!w zYHDwz{z8pW{kzJilG2@@Tj0aeanD{mH@E&r?hePz$U5Oye|ugf*q+i^NklA}NdixZ ztO@M_hnF*CFn_WnhvFY!IE1RwbC&@y%BsmKMM6RrW_Xac=|Z#zgHop__99adRj{6; zVHK|3A1w*nGE7%>F5F!(U?%l3YN8p;vFppiLOT{t9QQwpnhhS?v%8{0`H_%iI7uWM zxe?@j)I^#5|C+iApf-c83jvDL;@092G`JOa_h7{-P$=#Yyf_poP~6?!3Ir)mTU?44 zT1s(u$s_RoH-Ba)lS#gr-FxrZbMC#9oqcqv6*k~oVHc!7`y!7lB>az{BaP_J-}ttXxq4e=C+MJP54yijAph`` z%+FHQN63BFz!as+L9j>e?Y`pfDtuJdutdKFqozmFf z{l;AA)Ym8unkPt`4p1Gon)qceUH7vOovVm(XSDnaa(KurxWWY`oQ0O}J~K))|R9@i&h3JEgU_a+Vtg+*OD;k-nN|YY%|LEd@;R zy3FzW>+P5ckp|~0I4Yhw2r%e=H}Pw;^ZNU6f--~^3*aj6i4XF zziZfAw5=B1D`HQ)5)o-?qlI#-VOM28oW-2&yL^NwGJqQpL>aInw!s~6m2Ai2pqS-0 z&78*XXk3l?qz_wx-Q8C;$tM9jpR7iH?d8PptazUxe}64I+HqOsvc%;v4{GT+AKU1( zU!u$aiNffQFNrgO(QS83UN9t=8@YQ`|68`MwN;&+w#{L7<4Z#*Tal$#g>a*LP^F>4 z5~uCr7R>R$Ft?z5ROBAWop>QcM~xmv7hgy%ju8?rUc{tUX5OPo3DvBE&gigQ=FUcg ztD2fWw~dc%yPUZ=+WPE4zcsJh+Y;6QlHc^9Jwxu7M~g&Gh*z+Z@IXn^v3t5ML&&u^@`%Y$6Bld^S~|uhf-IJxA0g=BgDMyUj@JE@lyVZ zyL6I1W7)2|!0}0y^?P-u#Az&$lYOg}aot6YZzc3OKjw{mAhLY79&%yPG--wrZkzR} zVda(oA*rz-=hx4_lrnIMLRx@`cvOgZ95_KtCx73L+@eHvs~!z3lyl+gsMbmX=)?hG z^y+~LH~bYznt?aaE-ZB>!f+*r8+1^+2ZAEgFpvSW-vowBv{cW^#g;;@=pycVjqq91 zVFsm) zL-QLhe&20|org&u1}?f61Tt=S03OWB`f=&EWw1AqB>LoPC`s1G+R5ZO&y+`o?Z zr9bNazH!bO(NTX2W9Okc>+SoA2|f*i2fKI?Ynb(t+Dx?hi;~UPx0_PyYmq0){eRll z1}w5{XQQ175cnN%CJ~iar5RcawGso37lUI5j9pojbOc>ejAG66O3^!*RMXr=%K6^Z z`Yd)=wsp)t<9X$a|87+VDgvZglcy6ZdN@jPy z)_|pcDky~ax!Kx?aaBh9Im+&Qo0i_Yew#rFznpk2y95OE_-;m8s#m|$7wppmH=}1) zBmS_)G@I~O{H;MS@t4%$${y9EyQjoc)``VVZmJK6v#75zPI?)O46T=MSUQBoiwJ5F2$i(&UCvwTE;{cOnXofr`PHE|LUvI(6!Gon zWPR_S*^uuq_~9mLQvb%;>m&CR4vBjMPRt06USf-GLi#*=IDAm%gEZ)W@?s^TG)Xf= ze6XQQ&PUwB^_k}B)dw*~4qG_Tb7=uMN6qxpp^h`l85|(Ru5HRY#o~gPBF3j}@Cm)L z^|xwYNij3hA9Avc0vNfyB?1flH`5mYm98)NGh_R&Y8%Vu0&pWotjSlFuffRQs4Ue@ zipWr57kahad4T?t$q#$u>H1mDpi>L$b_%&>JDCcWpOj*@ybQ=NI~HJ=)_wK@lxE5p z>QaNLJKB?%km6hfQjUJ7@Zdfw@!Y}9Ub)%87#qg};!batNtY9Vl-rd;e-fxARPxgU z7z3TrYz*C4^g3H<=Na6uiYxeAkyJM6_n)m@Ho0|Vqrgy-*U2UzIL=uQy7mbh??P-`+_twjoCyQdMlg5 zB(*vD428thiBLSGpTQ8aG2JNy%)QJ;HCuFIf0IRw%i#WOpagZ$E&Fx!PhZ?a1G-QW zd1uT-hHxzN8Gd08^E5P~PTe6Po}$h18jV#hauj`N986(@+YGWV?O0<4gDhB;nm3c3>#)#42U(7H9nP zme=iLz^WVX3b{$zoSQ!-By^_&$B#A#58K}1gAB;kRV^cb{o#K*m;6eCUR6Mm(?(rP z@13zXad|Zszf+(ex^Fm%UjJF0sNi#yQ0{mwRq5|s)+KhvLQhZo zs4C+kh@|RA(HFkNF!#ARESa z%*5wk{#vqq&2&etHPIuHVXo7P4h@|eCz7$@k)HDK`-zaFu6*^@J5`!UlGCyK#jaM0 zSQ9~_j!Eq8R*r;us&a%>qlKyzTt5o3yS+;=-`RQmcx{;)^YK<8T~vg>GpM7+deurklK4V!<9_Ln@4MLgm`@VuqeqYA1n;D{%0CXID=#8Q$QkjepyhK+B zWy)0Y%10&RRe{`AjgcaK)Ll%ZGPK5G?EAKTe%TBFtf`-RH z@N%ro+~_QGrlAhc+^*d39mk$G2CJK(w6Lf_O1WxIVfE>MdhjG6NG&E%%8P-gu}Nyc z13QV71vI$C9zZ|N^Bvf5OlnoZ$oPa(@s(*8U*{=k{Jvih&E1N@E>s5`BOnHxnnf5BTFg&>?P-+@u^@QgcIRe8#8FI$(*-yhV9 zOyt|JOM!dH;Icrglm#6)NfR%bVr$C+pXnKVPBegO99U6*O^d0BVJst#ezomKyzhrD z5-cXsP=+TUvLL2qdu)WY^Q>p#kCZh~iyo#U+QA(kIEv)n{Ksq72qy2FI`)R zyEV<%Ss{==Y5@yiPvAP57(9P8sZn>-c8}-LfoVWRognydK=G@A&RN&t)b_B%)+FAB z)#x>?giQMGE7>cRlVoac1}9q<96y*@BT9ZXYTK0pdwuzbE$$@^<`-3HL@I)au1bol1OgL$@AK-T8`{wcQU%`i&@xj9d#+%q-5OuSAa5 z2Hm6Dtoaaeu&20!m}-wvBxC( znn67-{ngDct&s0u-|<&#y}4H$WbwKXAAirxT!NuQ+4(9eFTuktRR^)y?v|9OIxGBR zCAFeLN*C5iP6sxVq8Af!tk{XNSN|Btu%pmKWhHNwd9h|3II`+Goq^Wm5LJl`um1P7 zN%669B-i0ho zQ$z~;ATm?>yiBq0EHW|mHj#7kFC?ZFG7+kkx$17EBu9eQiWI?DbR zKDXCKWSk)AensnTwT~$mi{ne4&6uAgz*~IMU%e4Ra%u4Nrn?(?myONqeht;R`o7SL zaQLg3D3F)5n6Z<8BGBw1hcO*THm~4Aml*`Z&h{5J54UHDQAKnQLZ>(bZj9u${Gaoz zb+ctm6sI*25XP+pFS%K&ALyrG+q+YKg_M(u9tG2Y{Ae^W7^vx?UGCtUmA#)ZJ9nqq zY38jBMq&OAh5&iVEHkwTJQLdk6_?7W2NoNfc-Po7do z4m0MYuXH4`?|W#G=!onu5QluVeHY>Doa(lCm-?EPzYz9SB?pRl0DDia`pSL|2h393 z`0m+w{d!-3T|cU0Hfi_I?s)2CF_=>6g0HriZAtg?bk64U16jGh0OjwvBXlZTlDz@2 zmvWwzbEhM+1J`mKo@+9C;acy{G9q8WIgi5y84?2QNy8D^T^H0hO5ozYtMO(o#Q!6vJ?@tcLzNiTiU7${9#=b(!*s^|Ho( zcu=l?VS!(qKo}(|hu?nNlu6v(x2drjsP+0^7PIwUHX5U}G0~z{`>{t-;%%2fYoTXk zWU-#Kc^s6H3e=w_XQX77z5gO*{=8BGA6$yqRi#Hl5aV<%1-34+ z_VpXdN3Mo8BL(zlhu$oRUw~i7hNV_H=!0vFzv=Ay3 z)utD6HXV{gK-VAE$*UV@o(1hGecPUSRc#d9Y$cyOxm;aB!iQNLqhp|v0Y~ktA_1F66$g0pk=9Kp!&>| zMBi<>Pl+CU2pI*crDMVt$QXurI{g;Oys}(8S_^ZuAcW?s~fAWo@(Lf}!xa+Jx zDc4E1;OmaX2FR^-nHDGS8;^t?_~StulDVd{c1x{yHoG@_+ybX1#D{9Qw#@1aZEX{v z`lLck6m3~^03jHWZ7YR~bqaHUGd!FL7g~K+l6;*c|8#BbriXpEUEuz1Qa)( zNR#WEafX+_JLH8*Pr!5}t=qb=#RB>_8 zRyqGk)J&@9kEV#o6m;?dhdlH5yZa)#s*i0fY7Wc$BF1!X;)6YJY}n;;LoIL4%#eet z09Bk`G-=t{%?e31UANQd$ix>0bjpTz2*Lntzv~)T%#B1ZSsXE za%;;>_9GO;Ts*Ln?b;9DS*G#;E=z~21WU%7TVki(Cf%dilYqdcI*A4%8w$3mJ-%pK z35(-IjPkypB+O+zNk9K*5!@(dfF|k2p_eIf6r;D7_L0WSMSLD}F}p#~ylu6ti&gRU zbZX5*=>NhnUp!_VB+-Fjbrt-&l93dU)I(KRx^g8~`c{2-W~fDrD=|lSAQIBS@i`fz zTsp5Z3Rg#)6~4s(4GulPq{ zXja_z-o`rIC8no%B>WC9F<3S)5j*Vb8P{f2r$hpNI1Nr?PjE>V#{Azq3F_1k`h+)XcsYS*e**E zQx!HC;UJhwsgYjWRU9I&VEu%y&PkYtJq9PFhcY-s{EY(zsC~gy)&E`AH`yDqVM$*N2)qp@+tLwiVGGtA}KAi&8=7A zhKjV61oZ$h0aZgS+V2w}r}6YS)$Di4{-kH9k7bTH;B8i5ufqjP>8L@K(@%S+1);Tn zjlI9!^5qHv`?yEXe2wE)Z+$8o*014ScpBZ-$oa}vsnUR#i4pAY@rIo&iaT2WSoA&g z8bo;shxO7F!?<-`BqT~D9YaW4Zfg3j-5cXxLU0fIv&xO;Gh5L^NT*Wm8%4#C~s8QdM-$vMgY)ce$P z>VCQ(Zq=ols)m`_-CNfB$zDAns>(7bZ}8thKtQ0#$x6P5fPfi+fPgwhfCK->HJ$wx z0s@l7LPA1SPC|lI)zQw>!rBA^g7zK2$cWwE*}>5S)5vIGgn{mjqucx7;PCfG?HwJR z9V4W}M#DyFiF$hLxajNM5HRld?uDfI@4TA(IuI}$OOV)+G$Ki(7#cGRUq7Hq0qII(Gxf5FLpLKvvyPk{vOSx?%=A!}Ksic|aBH zBFTF5Cyw+$exebZXD4-)vxFTmfZKcWAIuuQ3de+{fQE*K?f>zV^Woe2;AYJ9@+a7_ zmhGo3|EH{;nCZ8fI>C>kO}H>*Z^X#*jLj`9ej;M$l0$~V2RmdycWw;Py26nKR?s%q z6d(@*KC_z|k@m0==Cey31AV zj;Km%z$1ugFRSeY0fEK$`xi2*OA`_TLKH$yQtZPQ$fGnABlVqy7nT}X$W9NJ1%uAp;MYn z@@j=|bf&Yg=@ylBtgoLk9x3;>DV3Hel_qOiiJA5e@1J%ARYn_~e6PkoKw%+%hxjjt zpWHg7QnF~!3jN>uM!7Blew~C=kn+D=ob-3+(f&W% zdD2LP|9w_SFJO{Y-AYiEKq92X&u~`~XW}8$gaVpTtF6EX4kwsA z9Z(48C_gp019-*=n$4E27N=hN3gCjHaXB_V>9KiI`>>jnn4P%Z`7K*nY{2|TDC@to zt@`20dxLu*+FO^4o1k5=o0}1WXG9W3%p|@ke=WpD9I3Hh88l1bbZ`1Yx8KBx z)MO2Ps2TxtQ^93Xz3k4T?XzHDWGo03kJI*N@UNq zYU)hROb}}kGapk~2PTz`nj`w7 z=wFza35zi27AuY8bcl`*Xcv-!0YBg7!bsO&?q#V|kG|qL!u(k^==CrMVJU|rfYJ)Ey3`>(=EV?+sH!tax{^W^UX(ifZAK!I20`x6TXY8$>lWH&AwmV6()xIipX92^*YjtO?bfb!lj9i zr%7S6?sS(mltVwUD$TI7)uvi@OOv$X)}r}^rGF81xAQc>n#v{73agg(RpNa6<~h?) ztoHplJ$7^l*Sj10E+blLzEzoXOCPE_rvz%ch~M>yRuVAc(|}?8YrAvaU;>>V-V?hh zg-z3{AIQG_xk0Nxt2R-P5zq6@c?vWa<`Wa><_0b`vddbG6*B$7>2&SDcNYjBh#u(n>x4aZv?6 zrp?9kq1(01Ewv*{C8>D)4B)iMe@N5s6Z_3$_9ibwAM5v)g zAP9gZQ2244mE-uscU*ghPKOw$-YGXuVaX_Z7e5E;Qy1Jcd=$ zA(Rh5n@oEIKfZ^?DBglTUbrIZI4~vb#f{4OiC2%_H&)L5ljMG%pyNS7kAAywE6vA| zUkg(tAksU`L}GEXO5r=#p~^KSdUB|^FH;t^u&#|`PIqZ#7!n|HN6NZpZsVr>dtfIQ6bXUAh*tic_YavU#r9YSAb7Wc}={M7~gQ-u3{{y+Y#5=yM zaXp26%YCmY4!#jgNRDt|RQouU7iPSD1crA+!5My(5bttLjiGe>JMESu0v0&;m=YxJ7`ehz+Z?up=rO$K5xwAWyvWPwz zNCd7r##a0y6=q6C;c&gBoSTTbw|hy!A?qh3h3RT|5j}o<3pA9k@feQQy4C2drulG z-Bc$o5V+i(k&TzW{z+It1hqjmLQmz#_9zpOOTngL{zcV@K{sy!b5kmt_f@U-OjK2< z`a#p%htq@?2FT<>W4R)pbt2(=Xuf6U3-*_r2UlshlQe5_NL(B1Cq(mgGa*?q^xSB&>uUn(>bFlRpmaBH-y0Hc$ZFdbM3H1+ z;hfU~(GVW4CLH*gl9`5w)=8WzdTR?UK+SE9mFBgd6U&6sP+C?o_D|WGxJLu$S}T>F zh22SvWsi-S^Y63$Mj1|eOc#wEVP5O-`2WyHQ64JQ;vL@1!hATU;u3BcxY-LXkNd?c z>sH~8pKCUSWzaXeHi#dDUm*3HE%lNqxw@2kQzUa;Rk`zSRMSmd@S4KbHp&>dL_ z6zk3C#g)^I(ffd8C*(gTUYqE|=(&9^Hqd=<6H=VYq~J7RuV2hGG2X0-&v6B1gN?g=djI3=Q5A~{hR<}xgI~@02)@v=Vs}Y=j zTh=r0lSNezA4b!hO1(7YSkc~ueSKY9uxRL?icEE)e$YL3m@?CkwG!e-;u4VuNBBND zi^24|CtQOadO9t+7SAa#O!E_w{M=A0QqJCXc2ypBavhEifN3xl@4?+#sqeYDb#G>) zYX+B%OS@f$2#=3eK=m7_=-J-%AS}ROG`+NDV4ASBu(fwOmA}E8PI~O)Nzq(-rKuOHFW1mJbtMu;R52n5KWW`9y4<7Q6F-Hnhq#rp0a z6XBF>F)-tMW@k;S6~~)%EKjS=d2=$Os;d!*Twh+ly&crD6ge&^Geu^d%7yMUH)!@r z!qy!{bJSWQObde5g#bz^$c@b5jB>v|7~e%F9_z8VXr`q|$|d&#KK>xxZgi&tB^52h z;!VS;KBTGot(SpV(!)uyrc)VQ9LSSGV8nYW!LD!G7;)acBJQ;Fdm%~J8g^q_ipS^Ik<_W4Oj4|3<~)e63x=jV$E)9&=cVxHqcml&M!-Vv&~xc)M3Ogjun=$30?OZ^fOW^t>?BW@dnA zCHmg9ZRe$MPTtX}rpog5HX_s`e=W+DMJW+F8I$L5Pu+1SMXi*(mW(KQ?n45jzD&47O-*h7XSxx^jL?;!2#sDHWrL)-7Ir61oQ9>F**vb@ zqT<%|-NyVi;bc?!uTmib%T7Xa7Wa31b~h0M`Z1!1$(l1dd4;eGlWrMJ-6cA|A_<3< zHRqP`M=DI}3QROKVgRFjzxMX5O7xG4k=U1S-2-<^kqpY)>h+y6xsKITw$fcjAM`dG zPkhHdXYQu7QG=|qE0yt`#^8>kpTaq9@(RW;pSo-%X>7GrWoZdKFp0OdqSF$x8A7tFlY-dSnB_GW zGgvIh5&-n$U*{IMS1jN;i;X#r;csK~xFhoQTrKKG zo*tmxHVFb}157pYDt_hhjhw2#%7TWRmi_6>ld$D#-A^By2+s=h@_KeGMR!#_+B7~M z$Fl8r)2p8)#JF|jJ4YNek+Je%rgLRpn<1W>D#D$ZVFA)AY>(wEbZo^M6FMfEu|;H< zN^rng{`kx2h7&Mjj5=8Tr3P)LwNvsRS&@eDIZi057fUIs4idE$RoC=drfP|BQk@o+ z!_E+82@ZZxi>bHln-XoVZU-(oXB=Z=bm3CA_G{-H<*QV$ zCso4y$)wI=PzF&!$0TdxNgp%HkPD`f&MzL_+(GI-TBoZ4`l}O!^RzV3t}%`V{f9(!z`ml?3V1Y+2mUo&N^0=hhi9 zR#MjV8L4DXiC3?ZV2Qg-8dM^wxIY2?Y!jA@<1*Bg`mMZnKmXKx$J2vvY`LI&Pi94f z56Ac_c*1AzykUZZb%shvGwHegAjACXdI=|J18Z+%0a|$nPjOOb*r0akvG=3M>9fjI zW%qom6M-g+lGLY7n5+Gb(;z4nys?jhdnKNwBTkub1bo`3-zHL-U}vU7aWDv;o?CU6}^|h{cG0YIv zTsSso4h-wA*l4O(Z8-zM1Lp`VM_*d?gi*a``$S<)AdN}`V4!Qm-Zz!Olcmzm-F zsAFvL*UALi%8)u7c+AJusZT8K%qMv?zA)Ui*&OPN7yl3(#y6ROLJU@L z+-_nHnt`v_+T#1FN?{A+wkL5B>x~SS>&tFxF5h3g6Y~!Kxp#J{v*NxpsyyYBK@50D z3vQp9PEuQQkDgi0yzvIx+BIQ`Rdo<{bSnpbzI?dJ2^V}4_C|#}N35Xwva;<(d+Uxx z?$%404 zRB{4+G+C_H$)4KY>81kP=B%VO=Th#wg7ZK>Qcn6qWm9^j#|aiso;7!OdJ*r7y>i54 zl}(xcqO&y#@#x18DT3u>yUDr{RGAp)QyZ7$%ZXwINWvQKu{eGd%>&5yIphNIE6MYW zvSJWQtP1sH<7j{*`Qu92mFz5OB3bs-mwKudo1bOCuW-#}v>Wdq+oe$fE8%z#dR4i5 zTi7C(s`yrd8xIagtx+OYKeghgTuw7ZGpbN2!>xtOnfRVhXCFjXW*3a+!c=GbZ+=te z%+j%n+QZ}aIo;y4xqm#;CKjMrNQ1ysB>6Sw{`?tX%{4`Z?Sy8U%t;kV;0AE%tLuOp zO(8Umv~mk>h|9k6bvq_4z-K4BLBqAia=}$m?a^22KR59PSzw32>G=W<@po9&~)$3ZyVt zkyd)AJFd=?ROd|PBN)F!S(uqDZzOQQp1F?5s7RQ*RxLyCx-sj(p$FxxOTCMmxA8pM z<(vzr`C*kLK`$vd-6!&)01EcF<{^d7k4DgT`7v%Wh$SCs< z#!&<5c*-?^3mS{85=q?pryhwKI;PbUv+U|FT)6F&ORQI%bLtKvx8-Z2GGSc+_Q~CW ztdyT8J*0}-6^QT``gv@+35vZr(@eaRnmi9EQe9WD^B?Aepy^s2%6!R*|c*cVsWwf(ANTIIcz0@i1fi9)jy8mCH(lmA0PZ z5JbUjQ&PuosC|=_-phS&ZDNh*<+1l|PyZWrYRJt%4(hSqH;dwF7Cw5DgcYRXM=>qv zbl)H!LyKxqkd@kpjs9|f8U_Ow;Uj+e6eQgxc1eX_s%{d7EdKf=-8W#O=btHJ{bs~p zy6^9(i$XzS;ePki&@Ss=A+NuWH&3eF)4o|(Pg!oKROxeDF%5kV3|H$@q<_cMDUloP zwSf9%+jw-}Lyd&Xo62(V@~Sf4PPuO#k)eVnbED7r!zcQ5ma*AKsuyc0dhA`bil$My z%*p30>oMBC!GpHQCM5<7k*{Ggo132Bb8Gqqe+A~$C~hYZi9WA#;htXauA=QJPnxmE z=Zj!2c-9y~Rb5P7+rBGJ4rO{=7Kc}-wUg=GVwfPlZ!;SfJ34qH0jMmgfO9|S$|C#~ z3lBJ`^A`KiRkHF`R76j(`}(=ba`BmWdNz~b#<+D-Gyb!Hr({^NK9lCu2cSrEZEq~I zpuqO&d6Ty;YgeVFU0<~R?6UJxJ@hm6I)CN0CY}4Uy8K-AH|HGm+Y*AixNf)9M$?YF-6MkJLsu8ELTESZt6_p01d%wE_ z_(XMn%vuGgvpcG3YDak3un=pvBTvypUOY|;nM~N}-4?+^a9*Fv_JF6+~1zA0J8_qEJ~~ z1LZQCv3*EiablV$(YF`H4}0-Wwt>ja$;r+dz;)OL(VR8OP^f=HsC z(FlbJ&mPW_MT3PRqmh})_?P1#ShU?{*k4xBEtMbmEFTE%K!fxt=SN|}*+P1u??Nal zyKGb*XKB7!PaJ!u+R+*Vq4et{$jT2! zp-AcLdBLKlY+zgGWxeq{6?@t2VQQ*C{BdED}{OUrKMOu!zi| zWTN}Ep$|`}ekET5A%?=p7iH1yNN zbkCem8!z5zmM{OtM*Cs8K;-*j)9JLH^3d??Mc7cyGdz85Q4xJ^xmjEAML6I={%|oq=iLXq!vq?zc%=qjSkK|FExO6N+cvqQ9bWcs zj2~H!@)E9IQgt`B<56yjFJ3&8Hv^73u(*t6J*=|FrR0B$We8d{=ajzcgEzizNM zJzC!uKgn5WgVL|sxj!WE)pE950WsrqKU=~+7JCbGjp?DXyah= zwRtm4S$^tPB5eNZgarz!SdO9h{RqXc)xLK997dUOdhI9wc&cu7_)X1b;1EzT#uvfm z#mt%+RB=pZGL16)kKZ5)WlW=5X(HT5Rm|%?n4tYkajC;LQq&!<{TzJ+<9NgK3FxS) z>2`!*X&H5xN8r%bqQ<-`1~O%K32p2P&p zn|ep}6^suWb6IkWguJ-W4~?m`gZC0M5x9bS*WG8u`%4zYv7+!8=Bl@QG>&Va&Cd#gAqFQc32Nf*avG<5g<=~Wp6j)!m>^WuCbJ9QkDSkUujQ9fIqz|n}BCsSgDhiQyj0N!hv*(x@i(c?$ z(?=_ZSq9Z-x~QWh7CCqYTnd8_F1i9wJN<6>eT9>H5eYEq16Gh20PDPYWYAtAH%mX4 zzKsODDI_X`nYsD-?Y@(DIMntTx7bhZFYZ;`f2A)Z+4u+GHBXcOl3Do+1_jqe87a_K zZe4eOt*?Qg&EBL!$w+dJTlJhB+)x5}$8}}3SL)kjmw1)X=RegrKxN6y&C2V%_ys}B zB$q6eNPQ|GEBgLQKNGFtx86h74cV{l`~>#)x7d0jHgP@^t{V%sujVfo7lgk(5@!r3 zaGzJ;AV8SexgQdFMVl6xeZg$f$Go)TRO3j9&2`IEc`Vd)Ha zPo10vN6-CA#(%tS4yd4XwwmE8jPUq{5S?s#;dETe1}fmFypgInJ*b$a7mF|DEPsb! zeYxnx3l5+n-_O*p7p#q#U0|}?9T-G8jsbIbOh5$EBgyXO(<@b-V$`G5M*G{mUzlG& zvB<1ngAX(-sqmT1SzZ%BwU^as9HLG1(tsSuER~7q(hnLRpWY_cAbdc<^&^cBVwU>) zl?*sfJ{|y&`G6Zq+FlK?LQZSs(IWvoPi2oY&We!TME&n|T(YDf7+eHW(7*!e-&^>* zyZ@&^O0)zk^Z&64TFx+jM{e1_oa}#8hM^yV>%qdVRf3->{!2+#@b7}OtjJW9nE!G0 zzkl}pe#56{tWU51@Pz+ylBEpZ^e><=`}z+E{g0E%-*5Q%8Za;N&uad^Pk;r#d;0$m z1$d7BUm1$=6-2M=8Lbr}$bJdx)pCPF6ZO!7S;opQx{622aV~`5y8YTYgD~&M8(5*tSY>|1_WKdLGm+Ozk^PBm zneiR%f7eE40>CVPw(@C#|8Mw;;*gJzkNd$QVXsffkl4%>w4F!TtylIFcfZrpn<(@6 zmIk*Av#YqEymqT@vpf>+AaN~^#%l6$U$JmK@6p>L;G7b)G%q?f) zTW{;ZT~?z!TeK?QcwBoGjwBJWty`=&WEqxgLeMg})K6VxSrxQfanIibJRNV~sA;~p z5Q@;QHa&PjDs}&b!G&{^4Gc7 zDgP&Hre*kp$PU=^J!by?@-Iq9~vAKx3#sETr%3BL`Q|fWYbLD z6S-Y>ns0Vv`Rt?M7vARm`04I!BazpE+~@2o2`L#F8R3XjPGkfDkH<_(6(`1dtEWdA z((d}^P~Uo2So_xm!!KGaH)FmpSdX`-L6UrF6?=D~n%%^BS zgua)vAQ7YPKonlDJNp!U!M#+c3hM=Sp~Vt}UjV9*zWsyzjL5}BomPc9IWyzyK)&OX z)z122hePqG^~bX2ldjjNvDcbr1I>SjD!MjQo+cNNCCn zF;A=oTZ*l}01C7H=)nm~agy4NH@AC67^MhOu2XtixNxcX&1#VE7kM4-Pvl%C@+s##E0 z+o`}99NM{gP;EVjS!y9=@QL%)zFP2WeXI&9Gk(5Wy6(Rw4Y&_3?q%dpU3l{+PdZyc zWog9!_z_@_FpT_NIUmby)tzRxL>rJA`k<+(K=F=i%8k{~r!WUakrJH(rbqA2Oyn_1 z%g&TOX&l4>^eG(xO4Hg@8=}GautR)&5Br1IO@r+A7MvN#+EA&CL}jShNO&fH(PZTK z#{{U*lDLKP;Nm%*uC_+(g4v?7lxW+doEy`s=>lY2XYI+*{`GA)`4iz({h?+nWp$BVU1frCHj)VuUu zM6W}jFV$6q-epPQ989;j%P3Gy+En^rnn(Q3VLW!}2_aW3ZHUWx0mi_*PI;tb3@U4NfI(!H-|=6GLv5w8vi(jf?4GGcHHcJOCqj4eN?oDwIXPUcpf+R2K`n-y!~PhFcxRhGFtv7Bjbfx0 zN?7ahcEy_k0ReSY0XrTw zi--zl+Hwh0q}0C%xUE_HZ&6Gf5`QYPODN^jb}lsi^qa9iZ+kpJs%>&jh&4~2{m}H7 zdCj{0{IF%1b~YWjI>PlFUA{82L+zK>Hf~qztcBIK|0z+k(M=Tg6#yIVnC4-kVE7b%I+NhgEpmKVTcbZ`FNbO4c&{xodLKyBj?h_Z^QBDE z=QNySv?E6@b~y0mdF&0L7F797p1;|u=vD$TR#|a*TT`n>mEgaD;Vl9oJ%%Q?j3KD55z9 z=(YEBOD`mD?q(B~X5wCFxz;Dzzj%qH`8*sX@;R9{gFYMomw;1%vTM6H{g9;)>PvN= zHj5!g2({+(c#L7?F`g;baKD>Ep((;Z4Kzadk&uwGFs6zBNg__-)jkL55u%j|NMCv! zko{r=l2umj8~-_II-Qn2;Em@gPan~pS%;~(?&c-}jnRfD_r%I>nJ4!3Ur`T+74WFv z>~*h#$=dIM7tN{`|z3b}sz_O|_%h-k-L4@O6`-Nw+j)ObZ-1)ySKq6+gWiU zTKf6890GEG*#|lm)Wqy}dOEt!Z#V}3g0_Of!jMT?{gL0K{d>DT^+e76v7Il$D7`ig zBG}CV=k<{bmyOny4c!ccA!i9}BiMu3k#Cam8GMH**Q!J)Se7FGbl_ z8Oysu0;Q`3WTKvMHqU&otB4weuI+fwW@&c005Xb=6nIMW=0Sh*^%;6Fk_uPJw?A+K z9>ZiTTw$*-m$4Yamte}X1%xN=$c_>Y9UfK;_RS&|VjPVjpMLob$rUq<_*Q`sw!NY6 z5(VAa^AwXX>mIZ*5Ci0mdV(>&?8IqZZQ5lZ)W+}b;J3l8+KB-VSKQWmx#sk~l7Okh zz~Eq)Tia#Y?z@2da2PfR{xlz}XT5uxrSn_%XBGC;Nd{CyF7MLa68wOSwuh!B0;uwRi;`nlA z6pU647E0^CLChehun=+1&{xR7`&uK_p$`xEp<9p=oec{72F|?&c==x-A}U@2EI6y~ zl~-6ne%uN$~_oL*JVwPJdU7fl0y`_H=UUvnB)vxod{*mN(X8q9i=VQA& zFf$%;>WjQ%wxSCO$u|$V{Ny)po)za$jR3&YoU(gPPvY>M+Wir;IVwDZbB#h9jsbxU3MjtDY>B z@~3?f1Z>7#Fqg^gX~NzQMyoCE=uQsjF1#LB=D`r-O{h<&uP-dF$BR4VykpU#36vFk z3;okV?N|bCINASD{YZq~198=_*l$6viA7HPjxNK61TZ;8v*;OYziRiFu@3d|V!ZL~1 znOHf zI+k9F^lfR$Sl1NshFL-Mph;5EVAU(~@Omlcd+Q|7;4w|spmxTZoud5>L;l<+pD2g3 zRE>n=g#so;sWI;qs3}DbD`ac4kz_9PjRwm5$`deGL@qI6Xhc@379zl6vj|}cqTD9= zf<7ffH(}gOJbWr77ola5JVUQkot`UuB!t&S#eRuqv~Zy=&=MV;^>QT<$M+ zngSCNKl}*7=k`cJVqRkf{OYehG|pUo+*2x3bkq$`GOYy(VMw zAr-cILX7>jiA0>@J|H(_#^wE(!3(c-Z}^1S&-c1tH_id9Baqfy`ph?yz9QX*!T`=b z%~`jXim%VFmGN=MZ{egL`vjO69u7dd3Na*u0J?jyvd*6H|9 ziEAT>tTggPl3B{g;oe3cu&vTeu@G{vk>)sUMd`~j-&bo?Pj1c0IiFL4i|}-nFEUfB z>t7mnFGs2e(-Yk~OhOSi1FaCZeiYyT z8*aR@yqGvl_!QnZ3ubo{Q9hrgw>UBCuA;TwbRQof9i89uww1LSYu(JjYh-|lf=SdI@}>}+Op-sG!BjN zDa2@0_Hu8G$F;laozlk%A0kfk@7jj0u60_Z7?)7LeVt{PqXAcJQW|gKs*8*3{c7bs zJNtOr!rq9U75b(e$()fPd8^n&CMqmglSege?ZVe%mLVC_81qlP9{69UJenWom#b6S z+m={egTMF#ELp(RZc*X;&OeWNL6M`|8pm+b3o19OR3M^cQSY+!gKIcf6i2{`9 zHc;I`XmhZmv#vvNmj`(8KtqN@IU)Q!_LZ8vEqSct*B|l&BVe?&)b=!+$xc>5Z zT90kvV3;)}PpK4QeqUS%0EsrN>Q}P%eN2a6dJix=%!Jv{DdEnrj&9afA8Z*-Na;mQF zd+8LWeB?c!qm>%u5UhUjS=M_30SEqgp(>omG(0Ap@XfeBEaFQim1^$o_%$SKwO=)S^GNm%8Z=I z7C1f+pkH2Pygr!;KAm;{_V-AWVcH%*3@y($OC!l|kw$2KAhl)Ccg}ywt#Iju>^Ns; zRk9h=>KD;6O@&=O=OgW+qmcWo7$Qwz9z#lzh3EQsT<>`g8*4-WCg4%FRdm$TKuSb% z^0rTupugV*a5*fio~PJ35KR%$U17`8{^SUzJBuaiFo9WOmR#4|ERI1iO#+Zq?=TgH zANsN@l#(;a=WkRGrRnLDr{#8VMKnsugg$&&LX5Rug@j~WCR{N2$mdkTKNa9D!1F17 zTP5{dgurbfN5^~;gIWxpfs%v5CFCipTm$UF0^h(dhi5*d!85sMgJA*@(eiBjEzDE4 zo0I_fA$e6z&Ait$EuYLc*E(d_l(fEfmYRs&5PV4+S>INpF)^cRvp;uP#`fAiyQ z=AHx_@Ynm%eGZwr6XV|tJT`HcWwSKMS^7J33f1#LYKLG}BV6tie0Hqj`w&|ucG)lT zGNC}aTkcb=If(Gi<}^lu`8*Ke zZ-PL=Zg?@JR_-y%Rnv0&Lc-0Y)Kg=MA-(mZP7jJ&IW;&S04AJ->n(kAdR_ia&oq^; zfGw+jj>(&mrUCe0#(z~mBNn>x2RXTr1*vlf*Yys_0flV49663y6Emjzp(-8mnY)`M6kH@%jVd+g?!>X-eIm>VF#hOYm0Oe2f< z*Wbje|Kh@KCXtOLz(FR!+iI?i%gWl^d_4OUZ-Q$CY()SOuqxhk<`$+2xPgSfNq!x) zbloo}z*N$#DUw%EAoDkyI8l_7W2+ABP=qlf7e%5D@$ELt@YcM=bDh;TbAR`|C-RDz zG7|WXL37_U(*S1$eij1alPOa)6GhVJ*6-X>>$W*9ErKP6?W?@o@!%~2LKwoeZr0~p z@UvP1HrQ28RdL)2=Lz4CT!gr1#{DW2S3BQVf0ajK){uIM-(RaTDE(OZ_7aOG^OcnV z+lJOsIlEANt|o;v*$K5j@Ff**`?*<`wgkhxD!NM>I zAuId}Oh*je0$NQ+@P$BWErMb8@~LiNQEcu9FS|VLA(eoBaTuu!R&%AzG1(e*WM7F3 zN2ciO-*PAz(Uet=v!TvbW8)~!kmDSSdXZilvop~=>K9AfZ+lpUAFaG1Z#JJYII-vcUNj|&H!h>zRJwnivCt;C<|NYM zcgyml8Dp0Bv<<9A9fg4^#XVn9%Mo(rqRhdilYoZ0Bk`C5KQro)>>Mw+o3+y~D<+ zR&^L)>Ee>x^jnr)dtAL`H66y?V%=Nax$UxnmBA@0dsgQh(Kop4QDki~F$ z<=RyUSyI7FzCh}UBTSS@xmfBXCl#2PO)bSNW4IJBT4f_O*)<(d3pt*?KCitdX#|t1 z)C^EQ^4m&TEcwjxj@X13E)vAkLJPq0A8N|TW#3)BC~=?Owo(A6SMjy*=QlK-aS zY)-zVHVLVhnyK)2|r^)=5yG^Z=g00hO- zOEReyB(g}2^lCg4d~5RggeKT6<{3#ZI-RDK@?O#<3=DNjz|dYnXqRKs9anuPv0grW zY~UiE-4|%khRkLq`Ym%hx=!Hs7wlAY^o_o2pDfs2qZNQwRutpjDF2672viW+1%x-$ zX-u+ERZB`Yzhk~1ff;o8(#@)3H|N0cKU$H{wx@cMs%bDUU*_Y>jF%c^rTI6_jz7j0 zZ9S@v!>Q1oeBKWOQdBj$k2_R?(Ow#`V5>zOS%Pn|=-cYtPeiK6lc}(G&2yI#jpr6=6d&z|Ff8R}NHPb(w zY^YG$+OLB6jG{)uHQg3Kv<{QH_F{{j#7hve@?16Avp|IBYGRGeYFlG|)RG2h!7J!^ zc~t*%fdC$a2x(Y@Y4}(2r==NSgS20?ZcbGKhZxrjrXk37Pm&NKmi5==I=_-^+o*&I zuE-mG7BmJ%#`_-3RMV@_+rRHW{<3WXB6^#>JveU{86hMyK;4pZu*T8G{O;GfYJ7kv zAOd9brwc*G3%<8|Zsie0j=oH>&7Syu@RbMf(D^mJR$IKYv$KswIVtmzoTdE8%DeFJ z1X?f&`iV32&QSO>=0&)t?qs5&Qjp>qfm46xrAKj9>2}!m@)=x7dqGB9FDggsA@$g- zq_ii+S&K+9CGE{csf3vS%FBhl5S3ejF#!+PkFSXC(-k49pQxdIY$}y8J@@KY-&9c$LgoL{7R!5F}-$h@fYLq=tGbQ16ExfgPyBekSjG z?|HsQi}pfoV=YVjeWI&P=s@@_3`kH8Lw}5m7u~9;9qXRHaTr-*qoj#tOZRVr9QsbC zEdC;oE2wvx&#^m~mn+p{jUE)CYjrC7;$cJ34U3}`YIqWCOJ{=yF18(ENoDZ+#W7Zp z3fAq<(*Io`huaLc@cciw0N6za#e@itWz}S)3(YCBloDmou@swY1>SB|nBK^}?fJhm%EmHVF=7w%DUZ$fisK z2g0tEMYL_4fn5D3%|`1p8AC3I9m09{|4a*^SCrAwcs5S*67s$5M@(jI(%q=SQ%EIQIZ1hDR~5T$}*fLwOgx(1V~TMX#|UQFPExvd)Dus?aS?)Xbl$P ziH@i!RMK{5JetsZNqHZ}>! zC5co@hNUzY`vFIG=Qn)PbXD;_9_t_kS9K6zU*xQQQR^F(FfXstsJXV*M2K(R{~q7g zWq0T>R{fCUGQ)CC$H1`6*uD$8$#TH&!I2yIjD=6)3?trzyX2;DtrTb*uI1{S`)J(M;P3*ehh#tRui zXBU^vZTD|jnAq6X6Lp2_C?(s4G=foq*ucsffeUHAW4n{S^1J)8zAgm35r#x?+h@8QChN`jGdtY!NxpQnBhU*U)awh_f zviB`u(NAKa_oz&|%xm#oR$A0=+Dma6ll}Ojst5|6rgiI%p2;%SNOZt%wAcAxKO2z- z&dB#$>yYSe)nZE11xKa??D?mor zf-CFoHaV#w35I7gs)ZBf(3^$We}o|uS#y$Q(XgMgIysgRL5rE*0%QfJO@x>dN|W>C z44y?!!H-pEe*&xdLUDuM<(QiR+s~c{XlySPWqBX2oG7K`1b_Fw*mnaC-6PitM1 zd3OI*B3-E-8TlHMMKvYqwb;pzZs>;xS7dque}z=2Kcj5A&HB^BC1N^@oByr4g6MKF z=Wz>_e|OGhz!H$DV;qP;LpfF8n*=>5(T@*p^lPo5USbGQm&}hAy@>@)pD1;!~!E?e3Mf13o=%x~*MilSx0i==d`H3etM~#r&Ys%P8zHSa z-L{!(l_0E$0O40fmPDa)=<2K@6-o9`@WKv}$JS4*kZ*;Bpo)qLXPOgo5x1e$Ld;g{ zxMvMY0dxwJa?5<}H>jvS4)ekio200fELN&(2K+Qrw1wz7gP;W5(@3hkxfab>Q+sJZ z2dGZ8E86DVj79>je12?ZM;^1`##dc2+Ksc9)-OxHJ>o0HGQ!c!+ZhGCvc z~2uSzgX0i@8T^Wv8c93tM#NL00DzXO{4u=iA=!6VAa!$0RHknqdZu|-gf{S zgRzRavL)((LXd(&oNFC37Wcfa(L{sNHyJ)X(dz^|Sq-bZ|JNSCf*fM$FaGnU-~;*r z3DW2&s`aRR{!f1()YY14e1V1HELSmnL@)G7|&@nLX`m`(P>TBvjND_u;$R^RO6>IJ0ZO1uI(-yi7E*L8?2@d%aZC=}yzjzi7 zRH6PAMQ)JJxfX*qS8Dp{A97MV`20_;e{H#pm7TEs+A2P}9Ccf3tQtJd`a9Ga9J|?Egq8tO!66gQoAG0_w2z&y3ZKQr644+wi)H)M)VWxNLiw_b<>|6otN%EF;x}iVxxzS6 zv^s<$7z4;!yMSlf0$`~C_nEG+zq_kS(Y=0IoB zT1jwhM4!g4ShfB@I{>-_xFzt#4K#vSRDe&N?r%@D6CK|OiOOxQ@>Tpquj)pnFMS^L zx3t&3UuDO~3|gpH#^a-|i;U70N!d-UbA{|YTj8-e)>xaNI9hb%-C8vC^^0gy)oLo8 z`!o`=OK4_cYk{oU0-JN zWDeJyrPBhdcCnhHed$Tzw${YHI_0*k^eeBg(cTPO?MdUDx0dm~njnt;!pJUGM{{X| z)MhHsM}7oT`3|hJl4HH>-t)KfYZ|mkLVtm1_?H{W?b(NI4@0@zYZU*-V9maiiqv=C z%gv6)Mpiq(F3s|ySxq6Hq%V5)u#{1iRUPD%@9}xTMm*EXBlCN|$+wxHpQ<+W-%;vL zkuE7EeSuLo)Xh{kQ>=4ODNR^e8>0$tZNKV_4q&z7{QbGas)A;*A6fF|={VoUbTo$L zh2loi>Kn&-`hn_3$;Lk~=Hm5c_FepB6S`$`KY(!C?D#v^(d8G|rJd5^nf2n`@Ey8Z{?6Ov zf7i(SnPh=$swvV`d!5+Ii>!<|d4TfU@ibU{ z^FJ0P;BhW4eMPjAajDTaRmJDa>TZ9rk$h5E6obHvTbi5uMpMHJJO+0;?n*Gxc0fOX z%G&JkP2_NcC0>D3u@zv3fqwvqKl`Dg^6&f)eg%>BisIeubuZZ#&I%Ke33@QhR;z0% z9Z8IiZ67vcIpL7-i)k1Z-4VB9k!cw|*Ev-fL2e8L!^KW+B?*ZEoa~Vo4iMKaY}1xg zQ^*IO>C$JU6z4oei5e;oC^%E=quNYpMVqsI*O}G2-%n(AfTTpecChC0*@EV;)3Q9Z zIx*MEB=yd|I=1kAqpVmA{~HWcK9TXaHD2q*mC?!7SzadhH!MgtXvEoOX7KmIys)ux z{j(cCf=c4Ij#;yt&36aPBkS>X2|S-;^TY4(YTb;6V5#Epa8ScfZF0tBF)(b6{*{k5 z&ic04e@;Wrsm?Z}gHrg#q#yb7JsqZr^D;rmg52oNqSQcZ$U1Au;FJIJVt#kdJ^(|i za;0Lrl>HMDz5>HI$XcXO`QIsRI0ONsd#V^jYdcO3)*Y11xsP5i=7^L&Z+F`N*1Y4| zC%P;oL@8BV05W&JWxP;f4u*`za1IzE#zme1I;m0f`-SB|45Q49OLY<6@q7NvDsRyy3+!)V6*I)8^yJ2rD?1L*ZGfJ$@ej?i}GBj^ZftD-~RLfv1v8IhwL&wTIbSr$b2F+vB>^Ssl7Tmkinxw z%lUi}j#sW1%k#&a;9jUYQJ=nw6Q_j=2^L?v!A3V);#GA`eJn>K$hr?(aQ}oR#)Nmk zp#kiTp8T8CBL*}5?!z=)V$k?KIa;}5F7I-+h!55T{^0-QK2bmY_Y4~Dq3Qj8-zGV| zg_+J|S?tPrSf$uHPL0+M*^RW>4E~JTUTbyYP_dve3G#AWhC+J0HSZth8B4H^PV3h; z_9f6%=Edn)n(||Hp+&FFsY<8UX~DuuE6qlJ`C!=MJ~AeX>55VbM z;?yMw9TO{QYL-c+vUw5MFs&^xWlxv#9dkaf?bL+i;~=3_sU9eusU4{+`K`J#U>e$e zIvj!rFo6JWLk-I80=2qyG~)pHpE2b5>6XJ0j&YIDuW0qwaZy66T%hVc)_DP{dB;?% zmLFin)0@A{mbX8gR)!TFk2?O2?#}zGe(%6#t%dlZaSWUPM!xsb=~so~2gYvR?j7=sY>tOo%S8pM2{rz+4iY=jdDMDL z?{~sIyxvDlEO{IBJC>eepc=ID7l^Lg{#JK09z-Q2rGVGhwD;>}jyuhqPy|1QZ{a3R z3cXbvcS(#iHAZBm%|?|M6J!3;-&`=hUX9?8Foj55W4+@Z^Lb@wWWcj%f7DD$-+8(@ zZktqFJc-W8x!CC;_?LwM#0BG=l{V5049Iu>hb@f8@go4WMDon!S~#!ki(6V!+n?Kt zT9H>)PONPydR;vMn*$KyFkpgJ+Kfhi+XMbE$Bj#w4Lb0sCVH%}-iIe5UI2^7#>eZf z>^lv&?{F~pO}9#k_B~mStFvL}>I3TUhc__oV=4 zdq$aG*ih?3>!tnxq(DTT^oI_{ldpiM_*EbARtOqQ%H3pLUfB=jw^|5{(NbJ6I38?` zunJl1(l|i|F4G)dpMfYD9^0Vfw+pt<|GW=ZD-g?1ZH7@kV^~mq+hs~u!vc-h`Lm(S z4rIPEhxJHiZA3=W_q?PMtaMH5JA1g@*q6V8-kPA#i+7hPhTu)<0QldW^Eg|?Kdn(WKDp4#*_FQhF+KLt`K)!za zYjbg*{q)3dxDQ14VuNX{0AMw8>?%!tzo~ z;>^F#36 zC~3KspVp9Sx-@@*u(sp z;kpHhrAA57OeuemjqX$HG@ld!Q_S~Er!|hR_9+*ryJ8T)_31WRG?rxE>Wa`^Ay~=3 zSRe}0XKfaIwp%9fRlfe|x%4%KQDWgSw#@Q8hh(W-0{(E8S-&gY{l_iy>-hpI8XJQ` zqfGK%Ny+!cS1aZ~gfat$yDK!2!6K1ol>JkBH z#MfnmSOKE+b^@-8kU|Sqiy`hyZj39o)YEaX6o~}He)yrBIU@Ibnmo+p_7m{!5Rff& zI9nyV&L>ul?iV^qoa+jB*d{q7kL?;N?T*G-0=~Q?8h}L@`*BGo^ST^RATVjPYsgpv z8A#2qi-1!P?r^qfVbOY%1)?k`G+VfO6bFbto5Mz;UP6@FIV~p`0<=;EHR)Y>coKm| z_ECsQ!zmH7sXB0g!$!g9tx^ixqN=#v--}MI8Yc2fb7_aVo6%3VlQ|MJK6$$qIK=G^ zm*Xfg3hqh<_ZsDxVhAZ@V#FPk?M^pU`gH)Nbx;q28rBxg`A(JhRivBD%-6>Z8W&Qc zp8(BhL2ke76D7>=&kU}k%E??fNraK~POEi$){l=w?5&<7ay`%B&ceRA_E};l^d?Ih zJDmpF;E(6Bcgu-yZ=y6hzee{hr=Q)LeOC15!Oy$cluhraEiRF0M$oKeyNZe{=mN9s z*0&~Xmn)gNc!oU8WjHN!Ff2t^x-0VmD>YV?2S+sUmygP70@elQ+YNBki5NzBLleiJ z;viV(NXA;|^KfTBn!W(rzyS&GA^cm?dO&GsBK$wWgfyX#*AISgundl&iy+8)QrXOqLBF&41QMszHQe&Q)p9h%;TBulCKkMFJ{7SB zK0ka&A@`c5nJwmE@}%6$?C5L0Z%}w;b(uiv zv!1Y}-ZK)3mgvST>Zj5V)KSovltap@6Z{df<`a9H>=SJ9Q?LV<+MIn}`7$&i@3G@A zjYE?Pf%xw+gbFD^Gh}Sbv&RxGhxinM=YGmIUX&}Co=q<{+mAY4)BOk;t&vfSXtD9O zP%4ok8X)5Fg_}MtB%3~TI;h*otG&!*RM@8yS1H5usv>k!CY^BQrVTGE z#t*Jej*kf_BmTByWGa^@m1Vcny-wlC)07_&0FlM(R_VAdrsUTWP%oz2t4Fe7g+N(eZ3*pMhhpp)TTK&#GW0nt!db7CcUNwiwdaSJY#gEZ%^~43Qr%lZ?D3_7xGr~7kugl!((n|wpaX2keaGcBz&5ZwBcs5au7N$n2 z7%l3jDz%5(anG1&M^t(|(o_!1(RkCV*9z2lH3_m|saa%w z6L)s*6p8jn2zseF6H`oJada9ZjPNtl!alDrb%geTZG}M(?*Ipq26MfZGGg^~y=U0n zdXM+RQrpJ~$AKV<=#`+DoKdIJ=F8wa9vnz5c7v``l!f!iVr>-_bP7sJTA@cQ`_dF5 z22AQL5oO3#bF)6z(thXLX|5lbX~p`Tt}G`qGUXz{utPXjL_r_*I1>|ZfM_!I=CwSqEgC^PUH8RrQN@^ zP<*0V`YIbP5~5D&5&MrD*B zn}lI*REtQWPtW{l0{Vy>BKG!8Xu$R#M@COMw-eu}7Sl{MtuEH!Pv0ElO4ZMa>ya*6 z12`4!><%RC;JrsyFs%N9h?47^?~=>yd>W$f*BVpX-@RTmL$l}nP8t^_nVU_F+1mcP1xavnW zBx!u(hJnj8?ox4BIE{WNZ^p!#c1;0IsVulW6H2F2kNHr{1_nzG!kdvnC)Mc}R!Q^P zAbtKUNUKAd@tWH?bMOEo{W}_hmbQl}XG5RU3;T=OtQZ7K^(P>WWbU|FDqmC}>Xw#1 zA&Y4vwuA#bBoNW^k?@2GADkE=4zyU1~fWX8qs;^W6zf;CaF&doK5Z@`X{>1mr_x*jB1+H${ zaIAph;qS_1L4Q_$8VU?dctkblzz5^7jfJ~}s3Qb6weio?Elpw><|m1)N6f~s^jf%( zc4&W4K49D6Xjh1cB8{fzY3}{e~qSDF=90R0?@cI=;3S(Fw zL~Qz#j~Pu%2vv(b?R zo<=;D?|ZMEWI7A^S$hKeuAtR8^gy*MekkjtW?Bf7*a`CA&c+Pd2?7R}o$F%Hy#F@R ze@kmJno=@&O>d>zN+2g7Ib0;2$q+kxoL0SZu24L>`a@pLU51DVA&SK1SO%}#8Mz*( zWE`P%Z?Bi2YrEC}Z;4KeH6y|=r1Oj){(rX6jOrZCqc$43#*=Upm;}UNzfV zN(r%(@@24Av>OZF0XW=5o~?EWxP%y?IH3UPCc-ORPxSwH5ie*;x+P0Bx~s&kX;l(% z4=lr5f#N6WhJMnH`zo?c;Az$oA(bDijjsSjWmz%9s2}jQ-y{K zp+joIbKcRfBSLZ@#3go3I=fD$F@!uwXLH(}jG>8&e6$x%rcx}ONTs9OE!}!nrs30( z>Sz&G4rBr)3@mI19S5|7$G`~b+S*2ttzLK5Pilk&>MJwqwaSN}D0@wE9%bHr(0@{@{570A7 z4<7|*hCe~)mcehWb2{wEMU1PJ%HgG>OsYdJ=?k!^hmWyFKC1GFFBRkIL7y*G>@QSY ztT%;|l9HYPd50;p3V7B>|GzJv2i~3msgZK{EctL?`|Xzakm+gZa(}4V##K?g-{pF1KZr^tPN+>2UOr`hYY+A_H=9XhZ)7qS*!zWvkwzRW5|) zu-iyM{V}Nx;$rpaEX&Hq3Ce&%ghtE<=AcQFSiIznqtSHzy5M#IhEjyrMoH|LLr+Z9 zKPqn}9O0!^5^}IQnBpN6@XBl$r$RJ|GE4IuPpf^XQ31 ztx)FZ{@T|ju8q-GEdE_qxf1_7hN^vggHrcbC!KajlDpqO^8bbFSX)o%6Eyi=B0!!< zkv!E_3F-8;!Vr)Td>)td@C!eO!vFPtoVT`T9)Vv0GYoa5oGT&+8p1_BMa11+;XX_o zgGli%&J%w&smTJLZ1DL1UQ-A%BR0?@)F(4Y=z&3poo;DJNJ!4}hE_;=uim$N-RIq4 zloOzaA~TC5Z5@z8#Y_2AHEsd|BK(T^|KFJeBvhp?=Yfd{?GU`)?U7Pc zJW^n5&yHjU!ZdUFAA<%R4vTirf1ji8Ays! z`@Y?+V(Ui>p9ShAAC7hyyFP>&$1pwRm6%#z*STI`M!*vphjrFZxQ5EMl{lHZq3j2b zJP5;AltMOQiw)(`ehf$Kubh8rkAWzgnH-+Xy@3|n^`!UL8^wmarNDP{ zpV)Zf^DE@JNzusHGcm5&_|%AY@8?OdO#G)3)rC~16B!ZJaR$}jHZY)9fZt_VoM>Vs zsB5~Qy1guki^0r4%%!@+#edk3u(!7%0@dH^wOC`}`9n6?tTLiG)B|{<8Le2L+r~vz ziQkX)vOryP59mK!@m%m-7G6WDM?9>c|9u}6#Qs4dYIO$uC2(5Nz;Bec04WcVk7TyU zF(p!oq$d0Pwrk8?dtLoOC4O(ut}pj?rpxa6WcMFg-^}5JO#*-)nc;OSA9H*J5~{db zkUTA-T(o_zr%CW~C=Yt3kXKOA6bZt4j5Rtfq{CD}0a{__#TrQ|Ddc+m#&}`@rvv9B#U@bCTBytEJd z;0R(xUZ>^XUXc65JsREan1BxHdGejaoVqUF_zP=CZOeE4vdj5^4F8lf;y~fAjqRTx zhm%Al%v2E^c2(YhIc5<^5$Rv|hPH6}{TK&Dw*jlcWEs!%!+uuhJ1DTMM zSh@mCWraqK4k@btcoIb_XX%GWD8hyXpp&9Zt_@8VZhap$fS;L}dC^ywF@W1XAAIl>5)awk}6C)!)*YNraL3e{BzG+7g%p$SKz zgwf|U8U@G3y1S#=b|=U$;sHk+bmTAZ?FDY}WWFR%P}MYzO%OO*l6x+f)sO!7dr;`w zcH;B(#-fR-SAZ~#txoDfZIX;c#H&V*mOwY;>;3-vM9KmO?`kUSBjWSk9LEyjvG7ee zI3%^mDL-xI?d@&k1t#r_lqCofGVUR%gB&{Tw&VFzHTdc8Mb=M^0mJ)FlmZp9cuXh>6rRAxBXLY z{tcWp@a4%60X`E^%)#$Kb3i;NFb@OYX}2TFums8qB7BZPkJp{t0T#U$fD)0w<7%B+ zN@o~N|qfx~?#5LOgYzMN9uF3Wi_MItv7-kfWd;!CL|`jECdczYGEK1XtBUwy#-r;`azt7#2;F)&Elbr z2TbrGq5CCSqw4FK7qPZnvGQ$PlBg8zE7dyamn$@Lt(QHMWYQV(Dri~HjHiRvM!i_v zy~D)uS3bZqXfX_&Eop;52pSbyb&d?;^wbHajs2fN)8Bbyk&^v8!8xSD%;ljG$!LLSsNo0 zT9{y0Z8pY-Rp}CWQ(on0Y3b(cwfLo8YrpM;#=D%Yc-nX&Q|RK^{Xd;VR#ms*_3w(t zx-XCEd*u{FFHWT6pIv0}RTS$A)9Kv4O(5=-c&H=Es>|>2^;myYKdTSK9{wxCZ!_kA zo2>psO13)>Pi%m-hsh(l+Z!qPhz2#S4+{FY&TS5v87AMyWA&rACoq*zk5f3$vvRA) zTT|;{U_wEbMzfZXq&8hn$g+Z<;qv}s5sq|4rA?z#D2uf8b2aeuk@5_{+Kpj9E@l7I zD)T$D2KlPcN0F8<>P4h(=T9Fvd`RS+sTj(8#nYMei)sY_Dj*(@EG*M`C5?Pbas}Ba zXmJdhAd)wAL8rw7Wja{frgpR0T(iHFj5wknQ1YmtGZBCGp3Pik6WNZ&0wk=s*6)eh zlF>Vxv~fxOvuD+!379d#J`!WkdoO%D0k3py0It@hd1hkz2DM`2NOcbj8Tf&YK@rC+ zeWXh_PO$tF6FtM#3tVPNF2%3B@ypDey}Z-?YH_o=9cT6E$Wj+?hSIjKP-SrfiLX|h zq*=08%@#8bw;M}GX1>%og6VFoz18o6VH#2JbMp zVQk*HiYBv4FYA&ej^ob!tn_4rD^EWz(Cf~Qzm;FZH%kzQ&dB($MREK}I8$nLH!mv* zFeCdQCDT;(J;gs>=3)~1A+)CX9S5DZF zwc=v?C^7+O=nbHoWKd6mPRT`}ElP$$CZ&F9is1p6q{!$`59GYBiBTP7DG?5AS_#qS zh%|U+h^Ls56L2rdRhkAt+En2vpYv$;$?4sUJYm`d$qoOA3F`@R7$2RVi~d$5lwu$w zK$Im@HD$yi6w~)^Exyk+EaYL+obO)md<=8ePY3~cY0TRA`d8<+as=3x<;)5v5)6tw z3i>(XdM$4NEgg9oSM&8S6eScV|0YE2t(`Ef^d^U*->&$0mH61`9RH*;59*+i-}}z^ zL#C{1{Mp^vyM2_b{0n;B*XXgUr!Q{3z|kJ90ZMp^0I`}3?GwarV{BkP1(@8SI-KVD z(}c8y(w>U!Ip$c6-@V}A+U#cQ336pYzYW`sd|TPYZFZa5+V`NHpS+FJ>C_16?;Nl0 zbFekfl{#k&Hg4MEW@~e{0#xaN--$Y417D4sWj?US7!76me^TCvCBmeUKv&?aEuAMz zzPpsR!%@|OW%~6R*yNUZ!*cW~ClZ2Q#lZW5L!_0yMto+pti{0lwnCb%oh1mOo_vhD zERHI~Xe%$n4P}M8#by=TzV6zmz9Y<=o_=^KaQE;R=I&;)C)I=y1v=k;--Okj4-N+0*j?pruyWB5m zBcwTuSIU*iKs?Ae>V}4=@gAwgvSRC*HdeRac_PL5{C|)ehk25DH*leWX(PVW!k-I& zi7w0c)6r_Igo7_cImh{pM>(}L=9r1*NC+8EHcjo1bjBf|H)XrDWjHp+z4aQhsOV5! z!2E)WE_<_@pQhOOg>rZX3!4m+H&bWAUgb&D>TX0#W@{@N)U=c!4#VJPKA$>ie>}|7 z@$t*$7#@5->)>sC%BSr*qp)Z<2&^fl^>10#*`L$@+pJ`uQIk&q<6mQ?wFW2wD^7nBcB@5_Pd3v(FsipnDE`TTVW%BRa;RI^2_#BpiSPAPAY=gP=*YEF} zNARN-Z0eU?=rpN%rKAzbM&@i@8*+Z-mTsT%Gt;JYf;nANFY3E1b8PzQFT#H^HNWFE zhq2RX)f1CEMbH!Pe4#P;320ODNcilizwta{xJ_c}#Xy-s?RNDFL41`QwIxyu5LFyz zAMF;3BXttW|r(piPL zaGdvYwm#B$%>JilaSB7EeIz!>p;%iYZTdzPk;UEw4vu9R%C!QVI@lFXC^+OP%*~1A zn_!L)TTqfQxW>D^%fO5${I#M6;}3-x&@&q%K5gj#MzQ6)3$9FCsnNwq;wFBd1%xVo z#&11d)iaD_8aQJl+D6Mzh@z;I#I#?0$-2H?nRGfIMt$))WBZ20Po$yF)7CQAA}L#W zlOwQ0m1Gmp?9^7i$~J0pku_?epNnY*K)^w2?hifyOTbtig}Nz&oj@(c3`weKr?1iB zY%aV@>NMHub`H;SXM{(GsAusrEb42Z)K0)`9S3ywR$ly!IK&lyonfE-9745DL+$Q0 zlC0&wJL-rG`XrroShp5bifG#OHLGVLidwt^&hnBwH*@_URo3+~_o~S9?Ps+wdos#! z(o6+gk{4Ii31Nyh-Tw@f5M&9#r~!8%&>o*^@85UtPM702wyt`aHm_v63s|7aX{o|A z0RrM+yjZ%kkGuLzYhR{b0+J^z=9_gO#|Kt8ov032O?j=^0TYGXQ`8k#B3@Ti$em^0 zo(8=hFE{fIDWgD$>-r_(`_5m)#M5Rp=qHN&FK0pcW2?V-ko|RW9L4PkFp6H1bOb!U zEBEBh_c7SuI~5waQBDG_E268l>*z8P_7&4;E4@J7W8nan2REnr@S29cSDsv(@M1p(xjMxS21jZ|85lHzZ;c!fQrXsn~~x9 zYi`7q7r0|Cf=ra#bw7=l8hmlic+pbiVNrAR?AtLY&VVavBPGt*9PNptp@ z;Ql4fRtyXO@6CfC`vwVucl~V}*oA9i+M&C|7}IK7&vFTrG2=rIg>wWn>FF^1B-bq} z*EcYZW%an+l3SrpyCpu#^a_SWWXC1~!M~%l(V$K^&R6R!`_VCInr7wX$N_O>jn3Gw z6^vNaC+*P}$;)&L`>7MuUiw!%O1x~U%#_pBZXIErjpmaJ0Oz_^VkRf$(x}^D8F-_~Xq+iPV|4Db)j@QwY z;i{l}+knh!Gh*9jx}oPt^|P=u?8@}p_D^91LsKJB>nCrxUdUf$t&T-}H!BrAb$)Jv z)@d+8t66K7Oz#)!)aV3ryG5KS)C>hgQAn2z>8{RsOlr>MC3o(YLqe9Ejz7y&H@Z>l zFMpR^rfUdLe4ch`Hr=Px>3h@Ub-gaM#8B$^)U(pS*AiQ9jgj_@+%XK#jRW~2*v4z} zbaa~AY;uaff$Jfq9h`dJQeeMbRIaS=dNqJtzMiRXR-xN7-kQoD+=USo20V1_+u&)&Ep{L?nq%Il zqy7jSzpOHD#f!aMVyn}tH0w%%L)hjP66bnEE)>g5r&UuU;SxlCEHF%2OyUK;s`(4# z#g~_?f58u5Eo3WWg!g^y{`R^3y;FAS9M81gO&%yIN-;|H02mk+m=y23Gx484JHJlN zQ;k69sxvn*4aHzc%yA%Fy23B0}xUk-c}~8 zYgIVA4T^?nz$%0!B>f2z`f^DO6$pK1I&23ft#SG0mZ@(sJsfNUPlDD6ru_D}INS;oJ0nkw5G7c-M^5 zfYDFwTKT+B%?ZZ8`;f}2Y?q@D8+3E-P$5&sC0`)Ja66!zs_V4yS{7S0_l{_}pUzLS zHoU0N)0`~PW(idLd>t4a9i1b`unzT0ek@+x2_>sAU?PgFAb`t@EY`!9{iQd~P_hZ? zY-lgrAIEZ?c=DGJf`TS|-75W5=Ehd4bTEf&POlQ?HO>L|`Gq@+&x+bDyF0Ykk8kjR zPtK~Cv6yl$MPb=et@^pJmXNAu(rLbc-TS)){v0*83rz3eKMtz>xkE&6IliQW+r$dC zCzsc)5a_c_ur-I0CD-~uyqL=d1HBDs1D1eh0K46fZw{mh+$|i3DG5Rdpq=84(BKi- z(|$zrg|Qa+Dy%xVM~)drswq z30r_RKD^<)p|3(yo3d-9GkBwKy92LyK0#}90T0tm6Z*>%zbc%?+oi7?Od@JO z`*0Vw{L1zTXAVAfwGy)%?y{j?KA&evW(Y(jG(^#ca%oK&X1{vKXm!Hu9BBHMd3f1H zw@gV0))<@DmSTI2H?8tnRxXuCzxN5#t{}0<>M-psuKRl|QMt84#D4ZnwA-$*c#`^3sqnp>CFXaaj03tGOf5hoDCC419W;VpuLT^aR?6L9|DfRtXReXDQs=lz(D5vS%cR{>q*RZ z7b1SIrL$JOc{Vg=gIO33$N!J5w~VT?YuAPW7ah_m-6@iSbazXmba#n>gmibObax{i z(nvQVjf8-JfCBGadf(4}_Wr(KjKNs2u9$P4b;Nk0h4s;{)oP0VvvrY(98l`7nDmucsa)rz5tYRB(l5gp2(|!vEeMSFMo^J+&>F z?Lc&}y_v&Tz~9Vu?)?+~>rL@T|b3oEU{;^(r<(gyO#=SUm< zu3jtVhs9T6GR9V5lKZ0A2qrg0E~q*>`DQnl1UYcPRY{5jOxiUsC8jndd&-@B+8Cx3 zx~H7=InoV(&7qL_DaJ-jUJ>)lX3@8HH#HY|Qg^it3Noa%u0} zBAhKdN~h(_jUf6le&rlxt!3mcNW^-Di}LA|7=#>i!|#vWkf=`3L~WTM=j-ins`JQ5 zOd!uf=F$Xd{PSd(N=zue@;lVc?VC>4<>ch_bgQ3y3o{NNanCF$ApFL0GK!Ie!L3?t z<+Z?x;8#HQZJQ|cHPWbjCH(x`8p7i*U0ufL!9u;gy%N7UVz;8h(~(|!oPl4(UzKgri2t*Mgx;;)KQu& z_%%3DS)8Mi1c$PfW&%%Z@Do}7fH<*^T^1ejO9tB}AIG)q2D?dD0|zy!&!rg`dfOcl z!ZWS@!O_J!=K5W~ZCpQ;5}D35Sk=06nyl{4kE_-NpDgmk^}=zw-xB>y3Q$&M(i7zy z&5OGf)cZyCD7Gyt_uwhZ1xDf6 zuU9>mMc7zER*7slr~6R}j8y%Jqv{lHH;&f9c>6TrYs}Tw)CS%oNYAlgA^q+_E#OtS zs5;cBe$v!pYg|`VdyZf#kN_`w4HV9)bHb@vnm0OcfIO9&scFGbci6QUsBwS@uP0{R zj?QwbFQ|?#dQqQ#1{&db9d+C&?}Ke9DPZRMNgV1LMdmvABNvvIINLlLbR(J08eS0$ z>GjXJ6qhVCP#5ULQAtX;YrJKvk%4bDV*%qW8tt&lgBj@xMbpemgLWS1bVmw|>wzXW zMT=SncI+V*78aFG$gPlmyYJnl7897Q-PWy{absa#2mfi*ZnHziY<~z? zBvXD>#bsPF=$FHtJnx`BIBw(UbBoec9;&HJUk_t4PY-(>ByC0mNoA&mTx%y`ZtM!B z7jCs!x_@`94@{E8J;#f+f1O8UCpV+gvl=x0UNEE7!hrgGA}>BNLpGTie*RbzwGF-S z)iVjO5{tsdu@;5XFskrKab7Yg{hY`u3iPr{!@IeSbNGsu&ScM=B||3Wc9+0te@~;@ z{16{APA@l9G5&#hpj^YPM3C1Uvg$9QnX@v4{}sE#LH2j)ap)N?>5J(7?XJL=*5N1t z!P}_PoSjG%!)lyiO7t{gw}Z>H-!0>Z_bR<6r!jXrEcFv6iC_VHqTXB)tA3Ji`BHZ( z=7nTO&*1YXOQzKJp8@BWmZ#@)kcZ6F)Ksr7gyiI4ra0m#Vv!*O+cutnRhPz|+kRDB z8_0>xD`fT&!^4hzc+I+)TW!#;SwGJ0x(7fmEiRO@j{;37kO}K9eQUUSUEr-yms;(F zDgCr7T)CC32|nO2fzigr?!t*s^Z)l77S1scgV-X`Ar{a_0FuON4pcp|Rt?Q{J# zgeu{i@53Ga3CY9Htr+2H&^M6t_xJCOKdNkA&odqf53?S6fAg)pdO8S|Ako}I%JI=0 zze(}(G+LHFQZv15D2EE02P<5T_j%aLLQ8pfiV1OE*OoNfOxZwJdznr1mCMm7ZS&Rl zstHb2#S`MtC7-)K%b|Uh@T66C>G4$tI}A5uShdFvxvKnM#J;VqUp#exUEO~MW-E{W=jn8~ZuwDRuhm@=G%16Rk6!#JF6tP> z8Hmik7W9>!H^z!TwBT5~=M~qp<$5`?G0Ac`IW`eA2Q+_JcY26<2)LrB<2; zr?MBHXEB!;w6$PSZ!kfQu2y1aNM zl^8*X0&Hv-6u36zBmEGNK3+`AtG%~gBhVr);VgxtNx-j?rc2U$tJ={cg(J+Y8QoL^x}c$i__bIugNMs(1lMSw&Sg(2U>CsmD7CZs`baU789A&5 zlg$xoEJkrm-&U#@WBBj;+umC}jUrqV5AREv{BV;g;1hjHjk->9f`6Ir?kZ@)Mp0Hv zW@_`yv@GaSTOsm7u{p{4$F8Dia1RywtwJ7$VS=SQsu%}v>=zxGO&-5T+yd#-lBGu0_U+pg2L-lC1GghN+9&hfF9et zO(`4!ZvhXE0~?1Xr&!S#<|~)rm9Itzrfe-n9A1K+SWQj+QwXt4#H`p%qT3jFf5PGu1oKL=0Q??2$8 z7MbroEYs5AF8lH*9&4tNX-E23=`9r62$td@RfJ&G5X0K8h7&wh0!Al|ZBdnYdX3hS zu&{NCg_V_!rPUb37*V>mb;7@nIBQFj-TD+~UisrKfjN!{P?HZw(a!h&ef9}bj%N|( znSd@&NW<;dPAq>VnG2F56f5_wFDtn9j*l**><>Q&7WuaI=L+6BHkZh9O0QP#yruX> zvx(yaNbQ^0ERu>?=CDGN`-~68r6mr}Jg*tDCceCo{&eyr@1elAZX@e6XgS_!@V!s$ z)Aa>knPj7jFD53|->&ICYD%Rq@9NUTcJotvBAa${0vqm}03rF?cA(eNdypA5(kf!m z4%e|{n>)zl?yjj>j&PCQaQK88!2v~`Jb5H`tvVJG>2i#i0?~$7Orlh!dZ@qF;M>R7 z#qNHv77t>>5)c?f<(+)bg>M?<;3TWy-jkKsR5p32o7L~7VH5Hr=84K>=S*g%d%62m zys?iKgM%%36DNVOB3QFuhyOTpI-59V8ldlf<|vV)C5N_JQzpK?EhwaCq4{EseKXvP zCPmv^Q!^s=z2;(k{c)%%LKlJ7q}pV2|`8h6OOOJTcPMSMOd& z)N#s^+k7fsH{6@Z(!()EdNwFxOBN`k#1Ss~QEGv{JWLN@XT{)TRO@*U zzay{2*XyR)#AP}?#)!EmQ)o>v@Ptn^TO%1M)kS!+&WNYn@cQa%u?JDLy|&LXF_DqO zeM(MOlGpVlX*P$StLceiXe_44Z9A9JE!CU#H=_m5MMmA~6R}|g2rM506TTy==EJ(| zYYcN}lny)(Ct6P3jA6+gckwjG`ZClhq*sF8k$m?CJEd@j460o8=-dxLth5)@}Kez{zxqL}U< z3%Z6Wl}rlrk1;TK-mb+(AWd`uw2i_NE9sO~RV9cUZmC?HAsSb$7;3u!Rf}!!Pv~a- z?GZ8d-LrQc`|p@u_cTo&Mz`EuO#OPzoDgzJZTzNH?_{+#zeUxUMK{izV)m|Z?9!Ss zyBT-9qBHOtu2n&&tqR7)yGVY7pG8fy`81oh*ow-ThK}SX3jq7W)v{ZGgZWhIEqA@WuE9OJM?(uCWH%UNdF6a|#4QV<%wJM$tD*MrlpL+j}RtO|YzfQOM1n@p@s5UMp?apC|DY5Qj zhj`Fgv5oF&nGfrX2hj_w`Iw7d2;;9d<00WPsQ4v6eQh5rI~8uOqv@cSSTamoJ$De?Y<^-4*fp_r zyLsMvrXD80~QHgt0?qNRD2qH+@^> z8NW#Pv}aaKtx=LA&rPT)!D<`l8!xo3e%dI~b~F8Mn0A)>5 zyz`MUhwN}gn2y&j8I!-i70%9Z(UxgSNhur%&N7e9VqPq~r`q@3a{Jz@dg(be?{cdc z(iyKWSD58O_nAp=o!ePmhy#w8Ql;76K1j>yjCC)}!s%%7 z8SBgRxx#hR`WGoWsdaPc+DhI)2syB#ypD#x_;ZcFdWl2xPs^2LVH(tiu~(0p?dfKI zGBpMta36ELOCKRTbu?rsPkm;RGHlh zxU;@%KY&@|8hQWb$LcF!xEw>pS+Oiq zCeLq@9K^OXX}^Uvc1vAcju^GcxwNZIrT1B>_i_iE71F=XQ*@D1Z!hc#g zeUK$z8uglX1nEq=&saFWP1Erbs}Emgu&jt~$7U+gEd+f*wyoe-H$S6SH92=QnY4wy z9@V^l7|qX}N+b%BZ2+MwE|o{%_yiTkK8`KpUDAc1BT|-d50@Yy*GdXK{C2sdEl!%t zuw6HNfo`6{JV@sOP{C<+Tqi4Dfx&Vo>5~g5lB%SGG7`(C!+EyYZkDv~koBq`cL?t? z-~P(9I;8OyswrAy1x$llOVM=DM2k3CYcE!?b1jtBnE39rvpO&K#ksFU^Xsi#J*XKz zFVz=qWicjkF%W)YG(A*B7cs*F1zD^K=8#kYk?K8hTOf>x@9)?m(}8>Sw<<5Gz09;V zgVcR(uwi7@Q-UuN@mgMz{4x2va>uThc5WQgJmN%%yUCYDAyBw9ovdgms zPIS0de zk#EpfW+7Rvq33>`UM^>Rx690Zl5S)q#r1Xz;5nAr2XlHTX6f&Z8a6hQ(KcDKr6hyZ zM5@wfrKsrg@|IN;U(3F!lu4%>I+8(QEsbz4(d;pk>YhWQKv)9czW3df|c^7V8XLr z33$d{9qQ7q@?R!ascij31M)i0Om+uVY=fhR?h~v=Ys(wlZ39iKEf~0$l?w{gUs!tc zwmPwlOFq;t#)VApAJo$pt7Jj0%XBk(rR!eldCQG3F?}30w$f<86}R7i9`+om_>B3d z`sV$T+}=>A;`4g_Z(==)0lpP_gO6@YWGn^Hov{ALw!PIn8 zBOa*mAAmXbFn(KlQ*wGp$+)uFF(Az911#XAkbI3&xB1Q4r^@;UkwM88d#HLk719k9 zmX7utfUjqZmC1KjN#QT)^diu&0RLd~!m2y{_~#ejH=e?S$&4R|-TAlKj$urfz-;^g zz&0Uj0szvjbM?AF21{&y1yJ)pgWn=Us`%H@J8{>6MMJV@=1Wm9APmX3;eYll4$x5x z(VE@DloIJvD2Gc{8$S4L!}i>aFtO^Nt&3P#Q)RM_<6gJ^0nNee#H$Smjc7hEpr0ws zkmwO@T1-1IPtEpL3}3^O5xr+R{}h#=EF=&DCzfBBb%`yfw;JSK&}z``8K8DjC~u!A zgsbu~kV|ugr+;{XZEB7tE3G9ah8&L-OJTlDrsHkYeQO+z{_}Ug;YuXVC`KFzG!9&k zrzPd$cfamwC59s+ntz6yc4HjX)%D+W7Zm0dVmeu7#&q0pVQy7;)y`PCmdGWitem3# zRG8_NuCAARB{_A?*+fF!>N<|+yP;|k)mOq&m}jMXlzzgjBCEFGIbmAWh{-7`7K#O- zsQ!aUo&EUMrA&M3GaL@Sd`rl935z3dzxm1>9O`4+YcpnIQ~W2K=8@ukFjLzo&dA8H z24A`diBA4uVV$HsCg( zEdoQLN`u@T!}?zMBWTS_0&D~Yq$jvrvH~p1&2-kbCzYsbB6MT9qpIb&`z?-N;V_g4 zEG}FC&^n#Zg>F2F#Tp5%3b&u4z9nJ@`lz0%-|G|}R*deuCH77NH zu<7%zC-<`I+iqd{7aL$m3s=Jx;1jws&E0x=FVC)z4TG=3l}!sp8=_I z6urP<y?={h#Y13Xpj<xKOPQNDHMvKcK;WjXx{sV^Ey1$Ek>Sn*UQ#fQ^u%_zZ>A z_+g#W0R-reqT+EEv%mZwLr5Jg+}-VEKAGSJY~$DUuok(;QX(Y@JsokDP$_U426fF!-Demr7b7ntS)sDT+ z1^8Qtac0n;pyMFwho1RyNA`!nzgz`>+rVyU7u48d;M1X+P5;0^7~ukbXKOXG2elB5WC**%L+8Z3j0=cl;Jbkf`; ze)&yRJBnaG427Gqx>^W!`3~{t{`GKh>i#T}*cq&$ z-*{q5Ikh1Z$|$50z5jx5+2M$gu5WITZ9)P48XgBy7W^WPSpxL^e03ph?{?EH^q;v- z=jA|Az(&0af=fi4=ASe9TxL^Rj?9p{bv3eZVQebILogseg|$LG6>UV*iK^=>7kI5X zq8`ZZq@9=ykOse(_%%rBb8s7J=4tB?PurllIsD{#>6;e~YZQEczH^!X+4{2dr;UlF z%B=K-wxO&UA;p7{r03y0>kcuP5rQQ%REorIwM(g-tavZ5EqcoS587M+!5^&Csoo&e zq_b{oWcv(@iA>chzH&e<&wtFyvjh@Omw89L4&d*HL-ain)g2e$cdAT@8uDA0=#Wr+ zuTPEYf8IzqtGmcgciV z_1Us$%Pxxfy4YZ)UijDxb*=pkuZ;ewL{PDnV@yn`ZK<4QX@yD?G5P^t8dKsswbI7~ ziY?QBH!FA#fHNMYyx1j0-bM+}(|Kk>zPajQ?~S?2{dd#)eWE%A9qVwk^m7=aB`BI6 z1d61md^;uUhJD}Ag`-he1W^dsj!K9lj$mY6hgs zdN?YDbbWg}4TPWzB4=)Gj~xf3Dp)K~Rz>GDRl!0dbkNX9;T*f~4RAj9v*mr3Oqvaed8kX#&2}qA znJ#})H2=&{K(DlaS0;qZa5sgSnsYq1dP~^i0$v}Ktk z8i?wo(|3Ss$UnQDK3KKim+s_9`jhe$@I|IUf|Udd8aYU)YHnk!J4Fcam!84=m4A4x z$QsFB4}~gUuC)$k*C03XbDx6tET$`5JmFBzo?z*H#_~tq0W}mJUZV}qhs}WzhP^A? zk;F0??i||^t{}V>Fh@&rVf$y#1APLPVnNmH$N$EI*hg3}MhKf|kaU&+#K-tVJUcF| zG2axMG)ZhDt5rUTK?R?!`7}P1iv}Pd&;$ySb_XdQ51Peb*erLaM+IRKa+;%ov6>p} zCZGb$w!d-M(I0*9DJWi@Y(xD0L)pu6roImE2e^(~=|{I;bCO25<%0??R=D_$C^~%A zzPDvEYq9AzSYlDyr&FJUEl0#vO&w>hE0gsZNfLdXr0e&2#J`;JKMekz2Ktd7f3z0< zkop#AldZWXabqdWmO?53WCtNi&8FS>FQ!_MZ0zNLoJ51H$)|1(S@&L8t zDigvT1r*IBWo2dOHFo&<2w^eBflnEow~v`nLZ0m^YdYLh389+7NK@&XGY!B7du2ZgJ#78xsd69M=5eGynHc{Y_P{YF64VW( zax9_vLYiJP(`gCw=eGK_$P03rlIfK0Gr%`J(TJ)FhlTqn2@sG!14V>VkF(9IG8Hf% zpkp$#AOKVsX>RLTM|a+eFMU`@XlMm<njU~HV%*?pG`QDhB;J9Vn?IvcZLvQc z&h^51J5-bdX3Ob5ts&)Jpe^NmdqjNaj!9MZ3x%9-q7;GJ&UCMB)#kXCu30ivQ(joIprgC|i1p=609Te7MPk{n zsRkU)-6^Ei&UY(@-r~V%&z}EGPGGk1q5NI5tqV8cR zlQqTx6n6`wtW8Vz5s({_Z0s^v1xInSSg=d!80t{B=TNr7!sTx5?N8<75)jZ2)q{RF zQVMr$V2xNQnBH?+O$UbKGE$XKUFoJv}yc8TkzOFQ{R9o_m1#{vQ<{iZMs@y_z=va@zSHU^OAv{ubfzV-HC> z05zj5H~|EDrYCgrURf;I>rXdi9P(Zgan9XYE+aGlu!PvJjXWuz%tssxn0W%*IFPkF zA}B+dU`}?y?!oXcrwb691>t}e0aYi_5a$rseFs3ymD;w6)Cp^f!_vdqxz6#c8sZ}P zbI05orf1yhwgqQf7(G(t_|!EGE=AotN)d#%6HllQ-38%l>cpu0phjA;@`SLfPjb9Y z_(B3U%faw%tuk)T8OZ3Ph)^_QsZZgi8PphxdIE>y{+Q|RpdhG{g&U#4dduZw|L3So z-?XKa_W6BpaMu*4=zoX^v#f`Rl&TeebeI+}c?4;Qq_xRi>{9}pwFa@b?>`4DA?zWR zpZpd`XDNL?qsYP;!oNSf3ik{^C%w`w+91)UVHBxx)QN?~ou|)c80uUCu>SYVrJV5d z&Mc9BjF62oB!Y2}9KT#0Bz|8`UB?NUM^hR&q?BQZ#>gXZT4O*wNu|+x zZd;zrtzS2wF`LDgY)Mv;#4Sxm*g?oClthZT^cr zR?vZ&Vy#J% z9zfG#%`@tlRjCt~pjdo`If}tFJk!vfbolrPn6A8vq}QA&}M>f zKmp9qGo3Y$%sg?0L}9WE@-K9xRe;qD-MGC{Q0gc;rX0r%!A!QVqi|tem8lQ7I9! zfC_T?6dXLt5O`Z2Uwh%PgwLQ#RZD{EWcy8mHjKN!CXtdY=4D1YOm$n%O_nR2M_Ke+ zYHCg!ckV0|U+ahbS?A`f~)Gg5Q|8zQ3C>|>B&h%c z?nE|bW4chj0f>Rr7WslMNQ8ZANc5c)J}w>qhl4Zu2j% zNAJweVZOHQK~d`AYwHa|y&r5n!-VmbUzuYeFXg&Kw#)7Cn!5Zft~V$l)gP8pcE zE5xb}b#_aJ`8mK9j==zG>fGRZpq9ZhmpG`zXP@n}xFP{K&BR7Mb14FPOF1cJ;u3Pp(k z>xOt`Rm)#C<4tiSySzh;W= zvI!Ne2}EaM(6w>VLYHq(k?zA+`1 zh-@v+_8d`r+4%`cI|zn_?2HeAGCMvmq7PmSe~-eSHOx~mKbza84KPJ<$;qor-kr4W zL1YW{9mQd72 zZb<`{LOIh~UFscZ=^-!bbNMU9w*pted z6WALhRQ?N>&3=beza~gLZdgMSxBD z;>W;gpCIh(Lktw%Ll>Id4mHp01w2l3wZF)u`_Uj$jKrQ148H_D;2V}MTlSLxfG~uX z(~L;l@zWoKM;H^nzWwun{&xDQbEP&wc}$h91Jb4zwD%T3_O&4TQ^1%r2x8cp8jF6T zXFjHaNVn)V_@>?cs!hw_?l-MKwdefu+hQ#>)b*Y3jXSLGpnv!KOJpAwov3a&w5^~^ zWS#|zHf=)xqhOedET|qNcOT=b{w_bs5z_}CiJ7!^g0;(f&kUVE72Mtw?L?Y0FK%EA&RB)k5ZxRwhhQ1NO z`onax_gcKf*Y>}?xfuo$7Dy4!H-gXhlZy9!v-+GbymyA*bsP|OG+#fy(V)TPXj0O$ z-_34p@}|LO0tu zSU=0(d|z)c?19_40yQ-&sJE=1tDlDOZ=x8fN3~L#Wb?tQ1{WZu3WwI^=2E!4yy2&g zWyKVU6vLDN{aW?Gh}2i1j7aBRt1V8hKrv${yHE_s2$cZAlHOa~ZC#*8z`)2TYX$et zQPI7rJ4iz@kqGDKw|mj^VSKZ0*o-&$Qr)CraTnDK8c(wvK94)O_O@Q42S|;);kN&| zmu_k-5sD+Qof%EawQKWgL`1tCydW7bP0JGCO>ue7C@Cq=X%%BAMlfF}(v7g`w-{;) z^GrD=KvaWUqy{1gc=b{o*x{D2U}KFz?uf04aem#`+Y2FL%UN*_eDQ4c=~o{X)H-(` zAewPrx+_idM5o=oQhR2o%_fYY?^;B~P%!{vk*DzEQxD29kkI0!8yC6u3MeG{#8tlq z-=p|H7fRtp$qo>Ww;)H9bcTt1!j={tn$8}eN66BdDVkau#&H?E?s2pz5ipbx#u3eE z9*LQ=M+2#uH#RnwI1*Ws!=2B!K8r)EDN7oWe<^pXHue53LyT*7<}G>FFW#)prvkYv zKQEA3BXP{2RbSwn;CU#9P_i|t~l*hSI$l2g?jgRv8;G8pOS z0_`2PUe~$*PiQ!WftVVXgUpiV6Yq?*Ht@M`MM|tp64I^!FDH3*3{$TXv~o~XHOn*@ zD}%c|K_5;!_wA2e$j@)U50xl#j`y*u#7kl8D-GE5(TpXxT2b1(2{*E-oIP@tqntMf z)^$dX@A{GTI=%UN8g)=xf!LZ@A|+86Z5|qaxpkgG`7_IBkLP>-TwXI(cR^aS(A#V% zjpgs5XNE)(LB`Q%#R|~buG%}GBLT09P{;D@*;IuPCYUQ=9O6JOJ|W};VV!D1u(7ex zite?0R=LcXE=;e`x!EC}`9aI%;IwywAHgc#md#E8<9(41302DhLDf&{$SofWO~BP zgk|o)`?vt6Yi!kR*kJ`iH51SgbL6 zM0uZDY6=sAL5e+3zo?xf@OFChxkzHqEkx4{c{rff3%dyqPowvJ|R0*nu?)WecY+LwPfO%0LH1uQ!Xw) zH!xjnLF7;(9n6=Gu>>U))U|195@g1B5p`gMt$hS_SbWJc0vbpTwQOIYa??~T<|Fa) zOkCx!M1&fD5uT?$71S9?iGDS2)}RkcFG25Xm3~6;Oipf9b4DIx;IxDuS_k^X%Ytl@q`nQO6#DYV@y#tiP z2=w$ygWb`T7~n2wZo^q!21Id{QpO18&4&^ZBRCrIB1*AGO)cwP0OwgZ z8r(N-1InQgloiZRW07)BH&%>%ehTT7QcF;vJjl-wp>zMl_;}+cD-jWxM*B70iUZJH zd5i93h>zL%H3{%z*VmZ$?tb{A29hUdLUo_{elDGl{c~8>A;V?xIy6PF_gz3$(F)~~ zM(8-*CfVz$Y8r^=K>Hp^coz}|w{3V23JNZ*sA^Eo`oTjC?+TmNz4HRyll7Go*o z^}VYYoP80C=0$X%T6s~X-&$9y=dBn7!*SfIW<8tWe)#5JEr5~_m}bfXXMxP^yPbgK z8{>E8ncVr}vx33_6JYgsoR2UYEA~aL>+yu8`;UY>v%o0}UgA=&e*vB1qo(A^i46)y zBo>8^XFbw?NByiXgoQE&CkoOO9i7bn4IJHkNZtJg6CaV}!-0{~G#H?v^N(h|;4S5i z1nTb`EtB*`SRx27v}6noDS$&m1|VZ^hM%-Nmg;84^1H0OHSqYA?Oxy!A6EsUOIVA! zkB`r*FmB~^tRWOw?Ll74&lKg>-Z@x9^GKE!lbMx98$JK&WMuny16N!|E-o${qhJTu zTGhlt5QzdSB3N_)f5E02Oo};-k}tP5h$H>Pk-E$BKQfITEgc?U9H{dOc^Q1a3Oj_7 z41gvCl2h8Bo$?*jzYEPj$FnT{qqO9K1_x5+;y7w)H3o+J+U_dnzDa(!zl#>jIC%Y2B9vukABLVY6C|$`Y6I@?{_D2sv`+| zv|f82(C%cSCS&=v7clG7QBza1J`Vg2PAWe-q>=^Qw(YFeyMDot%X_aq%DcDsoq5F| zS$fi?AR6#j+|1)CXji=bUcZ<2&$DbHg6*fz5O~WmSE?QYJRF|}Oi{qX3~_18j%V_z zLspWQYm(9;123gF4itJg1NB({)YIht0Q2xCv5oe*We&107SNbDW~>l$3?VD)&FA}- zWi(0HivBlJ`W<2oUCLlDycrez&xyJo)eV{kB|us70-U2nF#b^+Kz2+VB;L3S>S}5d z@BKhvh;Y%)Arj#_PhkOKsOeoNI6Fd zq|fRaiN0eY>$bXq-ai(zm0W&I;p)EIsLDeLuGh|8=OZ7Bk%wq=iEdcvghF6*KZ+8Ak=xj&A??;3k1#3 z9>!0V9;X1IPzW~bG+>USTO&dsNMeYfg+5)b^;9(TD}Tjj;A)5rTc&d)Zg{ajb#eBI zd}U!#qMVr98q9**OTdth?&O>nY-C3|I082PTyUo(+OhbosTr@{6;aHg3|quqw>iHs zh&2m&`YqtWq~cmFAJhjEPk0XhS*k-)AfP8lk>Nj1!a6mDfT^7nfD2>`NEAn+SCL$6 z>L!C`U@Wg^M+&pn1jSZcMH@}{6G#EhPt8Wf!Q9wr43w2c1VPmBv={4=b=X-|oXznv^1Onk`zCG$RQhFQWfc`hor}PO)c^L@ zlf|HI)|--F3=-NMiM^4m>MWm+h=720tFw8-?fokQ_3U+Xi}6|1>%N7>+jPn~-_PSL z(-Rya4|cDkYq}N?UdARQ1f;z39A;0r3WdR-z)64Lauqk=917-r~tJL|GK%^E6n0J zPkd49ivT(cmbSK;841Cjv~UuBNY4pjC*&d{N-Qhbf6X^{2JP;e3jntY9Hy}r$}f(6 z=mqTcXR8(#7F%HI+w$wAoqmp^m|~YI`yRObQUom(45=DXvAaN8U>Fq@6@Lr9{BpaJ zPS)4g_qO<(YEV$-SKNp--&O)M};z@dC=ihymp9(a%D97AIomBhYF*^_$ z`NqddfXl&z{YBZ$TDaO|N78TntS$91hu!Aybtj?5mK}+Nm-Ch@xehB>p`oFEVjsFe zV-qz@0*+1FuhezTZGn=D3%B~?IZ78=T87?9AvrDh+M^JdJ*`u%u3Nm8q5f5l(e7@M z`d(Ju1|+poTzci~z^k3$rCwL)*Mbl4a{KfB8YtFa+T-?+CKDLXqMjrhRs!or0cRtT zeYKB}0A5|#DH#(ax4qz~xv01Dibi%ssZC8KyzgW6-|^-I?+9zPPS<{T9*_`6n4h=$ z5hQ|FQsWB;qU6Wd922}m1l~bHCxFrDHuweo7zOlwC`cGiB$^Mt z?>Oy6-B`a>sZ)j;f2OARq#wEP$S^_gDGp}OFuPu2Gx^R5mhxf-v@p*AYD6ti@*RVF z%eZ9MPM(o7xg7M}3c*f%Vfy(&0`kwwErT3MStZBE!?B^@(f*?8POF}j-rK$6BcZ_q z-+m?KSFf=7`zvS<4T>~?NzDeZ!excJOA~F+VEnzjKM(59f4frlJBmN{jg}Du*95&u z6Sz!Tdq>C9=RrgSN4p>df~B+7NFxFD z63$TS@9%vARM$a_f!gD3eE;qctNh~6GW@SEeoh9fmd>}H8P_k-&4?sWUyU&AuigUR zNZ$Y6BGIFr2k)fP<|~_AT53p(rSlHGy%P#8jirEk1Ursruk##*t`NhgAMspcEw>8i zXzNVM!KhpNZ5&Q0Td7}OwuFWQV%|ow={I_|kxi(Pq^YT?iUiL)_6Eg&9?w6E^GRwq zZnie5M*|H%coXj8xjO;W&(|;A@Xb_S7gV2Kn%Q7tZKGcBeAwNRZ@Is%-CB74fG5Pv z1|z`PeVDCJ5KTI38%Aq96vf&sK=8hODJqg#ke8@K@zdEEfo(-Zvf)m1z}N3PL9zQ9 zT>7^>#>o`{wsZq(C{LuMs>V&9DlJ_BJirZYO6H93UH=Vdc}Yn(@aoFXQC#`|Jg(=^ z$8`;fzn17$MNTZD&7BQA&HF8tX(m@+6wcCbmR{SAJ!j%OzEE3Jxt<%e+D2yTpkloB zKB?rha4)W;zB69u?)8J{ER7_kqJvH;jaKmak)_9VV7^Q;Mp}X6pjpFXd`=;S4A)b< znIm&>G=Y{~kMI2ta)XQogEyqW87CMV*kO!sE*sk0up31G_k#@eqXy@9+TXpM8Qaal zE1Kofrzq9gb@8dJL_{vrUpJC8Y4a;$YEb$&d9;I+#&$ab7X87%Y4!ABXBrQoTy)wq zf9*P!&nGn5lW>fI0}iPz`d-?yva4A)%fJZ@UKwsZyq zuIf3r5(=yu1(8VOrOBEaC*ed&5~sa|9`AgEm=!*1?i4eRFiD=(!Q|1qA3wfOvyGpt zNvo;}p~UB7)huRxcfNKeiN-gjs_*?=H&iFU^PKu$edC#;G!Nee{cYa20HcJ0AhU@MWlEJ(s?BNb&to z)Ymv3b$ZXzzC&DdYmbe$FmJ6mNXN)nnVSm&H7oIIhmtU>P93DW>zOx&B)Vn61DQSoSTivlowNj{pe?kB`fft9rggGiQ0IVABqDSi@QH{<*7^&CMtK8q z3d^z81;%camttr`X*b|>-%|7*@s0wlR20tq;iN{&7t5A>1zo~$;1m4z^qO*G4Dn+P z7Is(M;=<>#rZ2ZrY^~^0CPoh<)3NgFZdm(6t_gR3QB$y^#mE)P{Lr)5CRc*f(Id)x z)PZ(*jYz?u@HFB8zKaygAErXw?H$HorH7d$v+q3%52Pix;|KeSiw}sAymTN*&`(sj z!^Ob~n4VVi@bWsenBf8Y{(s%Cl0x@~WQ+SR4|h2@%0VuVqjo1+_ukA)f0}>o+*H&@ zokEF%F6DR%$D$n@d8L|MYAt#W*EC5*j1Uq&#xzDVFisq_#Go5L#9uFrB#{orDQpcQ^_Y2KpqQQv@+XO3Ib0)5r$41W1 zCQ%}T>-qa8@7r#I^G$QHOya{;)?VhTrlRM4-B6NnbbGIpm4!I9TwaaF)Ia&SsV>_9 z^T&^GXC_ISo2o_ZeJZFvahB_OcHWYoMi0J!M=T~>`S`ZpzjthA1_y*bXqcBr;rx`L z8K3&!FB^o)2tWUKgAKarK_jc@3U`T_nP}wXCWokZ*bRaykmx6T?}Z<3CS4feE|0e% z*Ril;qU|-Y=h#(_2(c_buzBqbIvJ5fehV4WzZ_C|9MbVsCj~638}CIX&heW0sZ? zK{Gv|l0kt_*RQN2AP*w>Wnam}UhX~HbTae>gv0?5-=`K9EZ8Ff_WMkJ6j*y%|NjK3 zvBz*2g5s7`WBj&XU7x)X#zSxj7j;^~ z@td&w!rfGQmbQ-4&%I;OD zp4bfXe6eZ{?>x$(l0p1FWcrPm1_M@N?AXZBut*VyNwSw$D$7Lb{mrB=m7pLgkdNmj zTn8H#DxVHbMq5t!LpX+b2oaC}YqS5^`RszwyUtiwDgCoML?|f2Fp*K1&7~VD`R3s8 z->grMi1{-GXHdp~_a1M1{te}eUUEu3pJ>;jlC~R!iDV`C8`vK`Xxn4h zRT+%qzXiKsg+m?pSTi$x@zQN7wOog%ou?XV@r~}Mh9lWL#@3ycnCpT5 z@ae?;(^nr$G7#;lQ(0$NpFWjViDlFi3CL>@|G&W}0UCS~l-bc^z+%RMa1Cm{Q>69a zD90GHOG+yU94pc^H*cFV-IKUSd)B95w6z;x7zSy6`W0UqE+W|yCG|Id%+s_C4&S2J zB)uHh9*)g!1l}{o|BtSB49~NFwuT$4v2EK)V>W4QH)dnow(Z8Y8r!y$Hcn$Vc+T|x z-+Lc>@Av(f<3g@;ewdjxYt7*Q3^p2ee_^XvhZ_neWc^k@*!7dyTBD>4-D7Yu5J$~^ z_yx#dVY#`vT_5*SdLY(C{~(Z!t=bEZ?lG1_q;W?e0!Bz(4_s zVIP1>)c_L;13Xs&Px(LYcRg=ZB2)Tnk5}^>AQFSQ=Hrc}r>>dciG z$4j3uiRX>taohRqhWSnR4!&kn=VBV}(Q>EoNICJ=%e(YwI%vujLQPiy6MqhVMgwAe zD6O=>VbjD>K02}&lHZ=n$5$imrB||IoGYSE?W-F^xoRa&G1FR|F(}a3c9Yy~KcDhgA+c5_(MHU2v+RZsG}8xFr)+n9ER+33Zb72XvX|}XLA1Gr z4__VY$`;8Ry0{P1U6cAc7HH13IkovA#uuJSjrP)DEN6Nk zQC~`IOE|YWmk5WsklirZy4)3Cn-(^+d-N2LpGinb1t^ZJphuI#Tw~q?=uY3-ng?A% z)Bo7vfBz5%moPU04}4$KR4vdiMH0KSZ#uOVRXBUF*i_zV!`??ks&DS7o}K|UJt?#w zKKi`iKHooFp{qBG_Rx(Y{DeaGySPFW2LA&s8FZG6So6mX<{HK1#|`6GXv_ylaADHy zD!T35SUrlGG-49v)}{QvL}NFYk@j~WiU2b^^#Q}AEr<+Vgs{%3pC>J5VG(IvWy3hANb zg%Gh6V%sLqV5YdVL`x4-oGRH4RXHfFDAbla-`S|KIJZ7%cRV4oC99@7{HWgAAZs>O zZ#L^8J249%V?iF}C#gj%53qqmMOAyV!ze_5I%fY~M5~T}hb*a{Wn3}2P~PI27bgbB zjwzq}@lt3U)~W#?0w%hg1*ouN0mZU#faQ<<0R|EAAnSz@KyHX~od55V?L&{l6|Oi; z@r3~JBdV^~Z4x&^q5v3paPS~qkfx@lEudM_+86_J@Ngh{TK72bP+iLN7ZoJR6%;GQ z9*ETtwO1YT2OYKSSmDc!Wi{R;8&DpuQ2Kyz0Ih10^%)j#ar6`rH-<(>bEGB$as>iF zMF)ECvfXwv990aGiM?rFJ=#6`3>k{am~AhY!Z?N;w-M~GURn7oF{AEJc-bTlZ-XZ# z7E?r>`%`6i9S>Ly&BNC~uLA<`2m5R!;Ho`=;d*W?wHUgqNpiJuBA-m`zs8tdjPbrA z5BkQ6kRqmtHlI{WHK*j}i6bmCD1K@gtAE+YviaM8kh<>|M7kro< z=?JG{@&)9k=@E?PY6NVF+R_vU%hTsLgH7jgI)l=R}=s| zXz{UCoyz$EA}E;t8Z{mar+%hXj>JVrDi%>~z@HKN^?iz% zVih6K@!^}pFPh1EDY05tGrHaDkM{VjqdFOM)HFy(Ecwfr--WLe}2)w z++rS2&**Hkpa~!lx_1+b-Yjp5+dba9#>1of>`I8HgdBeS_wqeW_nqr$;uabwf~;M6_gM)`p~T{mJf62ksd?#bOlMG{a{RNIGd2|Bc2tW#5l{_JPE-ptX*h3ti?B*3D~uhl^1 zQKK8;d(@RI8+z1tHcs#J#XF@<#3=><9&Q=kueIHD3MLewb*|fw(L2m?I(=gH1L#eJ zR#sN1m<1^BZwmHp_$Wzi>9%A>`8C2pTh~MrYkl?%nEkh1lwmzRp%jjb z0gI@aZOsXb7|TH69iKX!UcWQ0I+P9P28V{J#EV6ZXrmiw8G0aaB%>msmDLSJ7iV1e zfhcQ1KDmGL!M<@tSmkC-sw|p?-V1i<6bH&!qPkYbcU-w}U8n-`>{S53D9`{$i45t*$3&UaMzFs(l z27;G&%VGMt6|>bdK9j*C+Y*54zCA6?Y;B$Mg4NlRb#!zLkB;^Nt<$@?vk38>m~sfh zg++s8b#?a7&aBCqX#kMQR|cl#k+iWLs*FIuh-F9$9Y$avhOkr&t1`8Y+QcFfdK=(! z6?~|h!XctoCStUpJ~PJ^Z-N-o{X3U@@Pk3h#M2V6A z-k}J>zH}3KM)?xbBzot7D=;``Wvjp9&J5ZZ-_2dLlF)1D!v8#qurquq-t)-$ZH-Yq zvL#B6^pnx^Y{|?k#wjBY2)MYulq1qPW91)5bXcjFZOa%GZ`yu|2Rm4fiy4Z~Gm7Z5sIYIZI~$@mFCY&vwMf5Ss5)Z8QCtSEk|O4Yr9 zBrAMF?0q?yY#$pCfOOI@5ZxOgUULEM0AfydW<srS6<=)FA?MTd(# z|3oD2ma@sEICjto$SPLo>;i3!ibPG}<*PMV(Sg=zM7V#u2ZO$8+8&oJ5qjf+@KbQi zt`buT`gkSo%u#t~$WQJOcyc}B%8Q1fH@nkuhtuJ_=2F8C{ZMlUKFOBzIYv=EmC!}8a)qAPN^d%7KKM1q zB`uE$xM!zf89R3;C=Ljj{Z1zKqTu2L*h={G?TNn6)2>2&i5p{Qny4_64bWZ~DZVlJ z9XrqnNwsaK-XoN%DL({w@d=~2+Xt`_;{GW+)Jy2?>WbG>eRL9g2h)+Z{jGp2*@*W5 zEACO@8OWRlfuUJW5btPh&(xG$^Z7Wg_I5NtH||Xfkuc0K1kD<{*K#UHN~ z?LG{XSP5-}zz;q_Th(to(2*(;0{TjR=WsHFXk;ce)-2-c%7~<^3T}<=k!vU!!QlChOA0#CTiv$O>uh zyzCHZWzkl9wc9xpW)kqsI$8EvAs>`vZG1=)1vVN2;%V8m(oaz1J*h^Uh=;d%!5x=M zns!@koHQP=RD!XT^%U|@XB*0HGI>nh{wc}YpS8g<% zN}$!>y$ZJVLCSkj#*golDgEhGISu)K(R4pTmmE;x0|WUc%#Qi9va(RW*aP|n9k;48 zK)*y$@lMGM_>=>Tn+kL)>0U?Hn~`z4b^>(rQ&~jw*99=|AkV(_@72bHs`CG! z!J)a*Sa|$RGt453o&_m{ij~Tw<~#94IB?PF*t2?`?ggVi)^T?RZhl-&d;(0HGz8ZQ2B*bO~hq?YE*Zb6v_HZ zrqq0UQ<`+@d3>T+Qht3~B`2{^e8VU|DC$yG&mkI5=j7y7-xjezD34I`V-;s%;3&S< z0iC3=!f-kEmue<5-onVY-PO~0<0GS_g5nrE{W@%T56g9m)ieerglSu? z#`H>@g<(>~gXD!$9(Ty)_KzhVpU^BcG|o~zD>8fd3<_kjs>;Xq8(b@_KYy3bs&et6 zvS7nYyYpl=o3b4=dr#&`wx_|F)MgWAEvtstT)GO_>dzl*$?&Ns0tw52KOB>%nMs($ z1^?t4E1GlIez~`$9H}-Pt-O>hEt%&dDjI8Zgm6S`Nrw`&v(8d@0daC-Q0jHEtHhz1 zus`|a_}h%L?bzN4ao~B3U!kh4Po0sYVw%r3-2S9dh4QvCVXYsmaRJ^u2}T^T->dk_ z$uJfpZNF3%vBX%zFuc(qDDSwc*jk!9miAk{CK=74D>&WP3~BWKXE*an%F?#%uIRxS zg%LEaPXp-%Z6Z70<2}|RRZZtlpZ9C`4 z5{kGw{Le%a&8%KU7EURjq{?qlj7u}1B5f3LCP{UUwI`mBXj^fd44F8K!KBR@X2(4+ zPORw6r!Zgc2BT*&{+^sm77zd6bhv1H23t+g3H%Yf{<{jX zV+3OwG<1pjnT*zFK%{D8YQt1^9<+SAvAk{5@+(3^jo72_p-}l1*E^CV4|g#a>z1+( zIky=v@dVOZNFs%q2R<3?%Av@3CuVZMr-SOXwuV9~Q6;2Kd5#x^QS>}HFFx(UmYTfs zqv2(&N3DR8`r?Q~I072%%-eOepp6JK%VVJr$#B~!lD;Fpxt}Ze;I5ZgS`DpKzm056 zmdq-Px^sdg&0gC%3d?OCjTOvgZ^Yf245K2wY3SmSlEKr(a9FPSbFDG})hryZiyWj2|oI78@1g z-A$PH_Sc_JO86$4Omda8QT&_ABS`5%`;;v_jBD-(?3~FqG&ciKyK~ z5wG6e#e+MG5`;zq*{~!mtI{;wE+x23d`0TRAk7&RlRSU3euk zq{@NhuZu4TFke0N`@rkmwl8(ksWPuLpT)k1DK~ywizT6)!K1{D4oI(40;TcvSi66s4t{mczLyv_e(<24F+cVm zyZW@^X@1sXX4Y0xE{6-mdPpFqlzNZVS@kj5gP;Ek&RWLyBi3%MK$cAQ-@|!HGbWbd z^^!8Upl5|;NU(~kS%=AX;@h#Pqsck)CXEm>PgQ=&;(8lEZ)uCJ9=g$Ld8r~=W|=N1 zJCREd(Pchs)p4~aXuU!dqOwElG#=<{i0~l`Pve$^b=co@v_sPW`W36S%*v2nRl3)I zFc$JRH@~00b|+CJ3C%GBZSYS}yDN-bHu$TbKE2aTOmj0#6HUMTZo^sy#SqYqQ7Jf-V^U^Vj@|L~Qwh2?0s~tDdTT!VbN6#|9uRA~u1`tJu z3CgJ8f8xtfUqe)^C(t3cF)HS8BY{3XU8v5d(e+q1DC--Mnp&36ZWG#t8e;;wcT~bb zjg}OD#;x!Kom0`%=@J@;OtVvRkXghGz%9^%&zjdqw7PY^{zkmNRFVGGqL;UrDe5rq zb*Xr9C%lWEpzNM%mR*OJ>IoJVD!NXwhw1N4NodPa&nOyH`ue)TnlreV z7C>$}w;vl|JDa%l`8?Twf7Ct0c<#%3L#Y^}rtSz%0^7b#o$%R9@sWM4^(+g>cfS+U z!@^8=`~9&y?Tf5Y`pB0S;qPpQUruL^J_chaXqKI~9Fzdf)i;7w#y>UqQ?sJ$h(M=B zim-D~Hm{?OzNYAF4C9CfLdlpWmoWUj)2Lx$fo__`NsvIe_7^P#YN4*%6q5>#!ojGf zP{;-%QWYx{ySNxJ8}+-Ome5c+QD}*V<5j1oPu@dZ8;~gs-uhX+Ql^b0y|V*2x67e} z7ILKxB`rvw^4|`?{5riSJvRG~=c@MQT=DW3Tr?Hjd0XKKzg18M*fd&FAAuhn&Zn`Q z$iufyc|bCiQh~42SyVh4h3sKTBe47ccYZ#V|69@5^G3=Q{BUiUZJzk|_y9W!FVGk2 z3>Mv>!Pq2eO@r)H`>7wFSdHW1oLe;tBrIsunW) z5@-xC+RF)=AK=DV_spqmm=V~S^b?c;Ow?6RCJ~8(h4PAB+LfbJQz2G;H0G=0!7Plw ze5xC!Rz34zZyE}kWevz1Sl*kt5fkUUsS=xqM<8Uxzr$GAKQerWJ6=q8= zAPasN`YX6(wziqBV&qbS$j3id&_b$l7lJxelKnHX*97e$n&lUP;v>ZL9>_wgg|(fC zL|Ui=O!RC}l?9xvM!Ye1{sdDfi8aF_yLaE2d^q6qX{2Cj<$Q0_!>POE7P=e=t$tQW3QB>mY8gy!!y!9|ex^&Q3^S#6 za3aBJqc`7B=F+J{W&iDn%A_apPDJ2he=nxr-pp?Q3xjbHf!rafsnVI58>PkBZnR)l zb_64um-WcvZDfGl7sd^j9@UkoJ5Yh5K3L+B&>dP+edvGR_diNLw_B@jWNXocX+sqfKGLS`mAx&DN2 z2&?{bXWZ!e(-jUfip@s*C5!p&lQEN;fP8uPLbXlJ@9u*V`c(IzpH36)xojX2{;am` z9b3NBhw_ZkDMq!!Es!hEvZC(TY%bcO4DbWl-z&xMq0xVwk&n}kDvp|x}ufFlNLWswt9)-h^~i-ouO7%?&LrbbQhv%}Z)nHG*c z6fle%tO*r44{2K?Y1+W^^#nogp+tubZyk1pkf zeL2x zP!Y#G#!QJ){Rt&Qp;N7^AGszmV%b;;)YibQBW=*9j?^EHMH%3y)1r$bab1~s7+Qhc zpZ;3ytuyrSWKz|*!u7ebwHPaAcD~b&r%7+LSz%(IJbEG`A4!2WEh`(ycwI5jSx{G5 zDs9Jsn7NqXBEo2_9(_15wA%z7r8IOyE99oirV8{2;+8=l5rro9VaB_Q(JOw(k&979 z4E`+|*;;_Y*=&a#?3i2oCFAxgjmCX1#iDCYjcObzdD*_78Lu3kHkY#BRDau zJ?sDoLf!?gZB;xxJa$9)E{AtAR?lZHU45i6y!kxuDz5j^Z=07A?(OZZE6}dnDde4( zT%pwXRr*Ok?<8G%t5`Sd&j$v z2=Z+H(=?Mt`=viCZDtKf4 zX6$Hp?cddv(l}6>31Gd3m_={O43pW!-jekx5l}pYR@Jd~zZ@J#rEe!$IT<+^Pgx85 z_xqQ&?i^UiHA?8ef4Xx zS(N`%l>OHWjDrGxVFJ|X*#Cci74UJve*=zM=vb8o!wF1DDR4ahZlF7mhB8M$jU@u{ zy>g(XxjA#m-OOjg9RPDI7Rs(a({6Jd(FhU$N%C$QBtH!zqQLyrpy(2~f4@+P4Dh(r z{@*Qt*RfFKOO@Lqo0>TK`}+^?4(0}dMlAbtwoZmG02R@3?Y~G@-8i5(Lg8}Oe`!cD>TagG0Z+ONK7t0)}aU5#RceqX=VuJ z2l#(XqxWtaJyh{s6c7*ci^giNiLYX$ttNimPD^nS;~L%4I)5#Y>FW#E-rkX=hsOr~ z&+kt~1U|_@Xp%N40S>$!*x#W$JNli)K=+Kql$0sqYKN@>fN2r|^i@}&90pp-)Bxi3 zjZRh@GD^yyH%D{NNE_?xnQkR)9RJ$m-AwP+27WBC6n&D##HfC0L^XgIQ>Xsy>`cAY zzR-Frjr0BR1?+fTXKZD18s1-EX{mn~c!Lx)3;90_x19@a#upVJix>94xPSjHI`HoR zsx9ys4wj|H|NCeB_nV-|1;<$N9d{!KRBxMh1K!R_#ySbcIvoi8J%^HIuu}@X{{KHa z27J9qT1+3_%L6+#nH%iauL{8yzQB9tN8f6+qtl|*ZwI`8zq*WGzt!Xx0fVpF3)8_ z+=_nCj4a+9)i~D?i@{l)2(cvDzrU^ib8c5X%(*gtcW0&|bamp#KNw-;p-7*1ew>^S zdFA=pwwo%UTd)uaYwI(i^+7=Ae`pFumG+Po$?Y;{^i(ar{2Av2PrAAV|F;~+i;$8) zV^>5*uvlFU0G4+lL~e*kj7`47bEw`@$;4Df=w~|U!5*R@7wLmaN**dKsuYPNZ_0#C zK^|I%SXj!+`U+;Sw4}mLYuNpengX99;OB(?`K75Se&*}1#q)$J=^&tG(Q{PujTzAw zNjF0(Apso!EQ;P&WUSvd1D!dNajF(2G^k9)sZ51kcaNd}kQA`V_+CNnelu{?TwU5M_;9*;TdfFYVNn&NZ!{#=-y3u|Glq(t$a0GpO4sZ~5{B;lw}CpcH?GwAC^=PW`%b zUT-KXH0mJ%hrs3a)W8=i%JTOJ;Q0%>^nGo43U!2KwNvCjM%Mwj6x+ZQ&pU27_B8-i z6r52H$*&-rXhEZrUJJSwL1327iiKm>s%pbqB?6rk&Bf&&Pmfe`Fdc~|`MS0H<-8#0 zbn5GGr#=m@(gjj1{d@4AQx`QTd{z@B<8n(N^wJ64Lr$d8E_H(fQXi^%CEEOIGjF6Nwgy?^H?noeOC_tWUZzjOM2-%Ii#y{tq6XI z>|!qG6%CjK1A~Ku1E?cE@!rlW0YXSdJO?H+_V-P%mkiyYW~^skP!k9jo2xk!5*%+8 z3VmfN#du3lzS0zu!sy@2LFD8DGvDXMrII#Qa1&6z8Sp?CCbI$9M=w}#ZjJ0G6j$+E{k48*_bKpbE2mdYZZ_Yu2uVLTgObCX zoYxs$SY4l$_Fxm$7>%F=sJ^$u0*8b&07g*AmsUKc8Ovw=Q(Kpn(<`|py2dQ9Qzn}H zJY-965J2|_epbltTHMu>qn?ZUGc>*3ynY@$2~#n+dkiUclm|7P_(u69Ys1s>Xiyfx zYv`6vMAZ`sJPO2avW-|dy1)oz1`S)+T!Q?T`Ur-9y>guImXp?8GpFLc%9U3#1#d`E z82mij)ePplb%7FHJiwr_lys=Gsy*bYglFg4ToqVJZ@aRhY#~HTPki;FYkq?2f-OcQ zF!FMdo5c)lVtvY{2CW8)Hcgf*{y z!|RW0m+p+@@?jc)-aKpG9Xc+ULXI!?bKgQNdj`_2HU}&83d*!IdENFU9gXtUG)GjX-z^#W?-A> z#h=Aj9g=+1%;*!yuNZsHmRd z)udhW?)QKVvl1F1@hkDpk7^7QZaif;t1{0z_+lqBD5on*{=ti?Z1ahFJmr{u$F{d5 zBnGjy2Sl14mFfutFOgWk2WC3fG&=Njws4!z#gv-SZ>_|ma7aMsD5pJL3AaM2zTx&- zxk$?CGIc^=ik_Dk9`iD&dq`_`%_p2K%-``Ni=kH;yLqd_#1#j3K)7o{}SUryPc za>cG9@!&3Q>I#z8!|bT|V_#T%yjq$)S_h6oeUu|VoJ(?4EQNH~*ncU_dHzLt$3S7- z`DpU|XX{N|mk{$va$=0<)2Hyam#wz}K#kr*&~~Q>=+gimJ`1fL&lz*AFpRfpDH^NG@LB9Xl|aeBq5wucr~JWc z99@487L!Gmip+{cZqSru{ajDfLB|h$4=qd(Y3pN*`Vg49Qf~sS+QGf!qGaef!V2~+ z*j=(xOtW+Tk$ZWD$WIMXwk-RF#2t!${trY}3|^=pJQUOL*ggs$vmJ-1GC& z>!)!1z}G(R$RAtW!utL4frtIRX!O^uJFQSVG5xTZi4nr}`4rtkd~d}?;bAZoN?mF+ z10s6JujL=q66gH+TeMcFAn$&lt(*OWxm2E8z0H z-|W7Gq8!KW4c!1X1+FD)fORGCOPTRxiT!#vrmprP>Rk(JL{V_ou#7yI)G*p z$T|;dx7tJaJpUw|5V8;TEL03jOhf}Z+67#PIB$8~a|f=Ik^Q|GR>8%qcH@iUGCK8t z3dmQ%1tqJ25hjv}^iq!(JeoZ}qP9GK!TsvN4;yr9V`}cj6V5>|qU4v$G)7Y{QumAh z!Gjr{;96((%9~i2d>p5W1i6}=FY@$)x!g8ufYSjnTO~1S`L^8l4~kq1Ff8wpawv{c zQa?LVosjxudYN8i?#v9;=R~U3z4gLFFi||nz zeJuS#_*^!hg)ej@<@8$c{ew33V$}tX&pLxWZw-sT`GgV6@atO_6YXrQ9WaXXB^{l7 z9P}ScF#eTuOR9qz_P}HKPEXH9^#6NaR2ecBM7CY&VPCf?OSIw z=MO_h-!I>&eOepn!`zt}p)!{@%$4#x&Zi?Ti&22HzcZn=-uU9$21#mYgRVuxVy5ix z{M)Y^=tMO{|FZ$ARRZ$wR)|E_8$)Td$SrIhp9XU_hb5yfL^@d1Z}G(K1${JJ5Zsk--5C%W6H%KmZ*!DehHOlPdK(#3-FV%;$viL&*>s z& zQE!Ebd|jNiqVok@iIU*^k%9+9U6`s05~Q`l9(Ge{Z2LT&`g!6la4F1|+Ljk=&U?@{ z->>V*+C1jfYd<`zJLs#DCFahR38woYU5|ZAyLu*hK_E1g)OxqEUmB&}&!sw6#+r`X z?ng#$70SNNXSQLkoI(>tMQdgNYW1HM)a669VhySsK70LzwYqlfjGF|~VbkRTubKCXWmGNl?+`Xq~AiAmnbi^yC-jh!j zKrVXtqpS>8MrQGmnz`iAmj@po|5NGhfi5s=MO1GQI`qQHopXN>+&r#tK^~dtbmKM|*xC9d=Mnx7d);Juqi(%j@P#k?PTWC$rWp*la&qzv=J3#4lk zTd);)!vd%N?FDd>dJd8iue$U7n7NBua@Kb3rAE(Y0IWAZZh~K<`erv_Ybn>!6o)1f z%^y~S7-cy`O0v~hBHAa_=La8$`_S@k<`E$<+Qm8upMXHz+nfKqK5}hQZtG!?^Wr+^ z4Nu@{CT4rrU=CHf3%9(v`fEoS?b^mh)O+^ZERe&2z!Y@tDgEAEEvB!Cd5YZ+S8yEcq$M>a1#p)mg&`{fnwciCI#j<|;E zkLI+f8eid8-Rrlj?PwU#;)xi)^|e5`SblBddjXT*ozdB#HX*e4lP;`TV4P z9M;HkIk+}xE!eoxW9#A)mE<5(5TsD!_?13F=-KFrOM_vRWBILKt)4Uvp;?5FH{GeyXK$V?KW}=$$Ifv(hOYe z2UR<$na|r~{Fbrb6(~4Mb6M##kF=Y+Z-J}Pto;nU4)#XrRUUqc{3%v zi1vH*-=&AnI9!+yq?gt1<{<*lXtK-Ub!^TS%?srgGzPc%#;$WbH` zG1F$hFwV6>-hOf0zB9H=`PQ2sEL|OLJqOQE(dgF(3Sd*Gf!)x-3^4gvD>f@(o;b{q zQ{TT%r|I9)ykIVEEE7_K?;%m{?S>%{s@L^?ZLuY8JGmo{*A;DJf_+E*v=bwboQKkI0-(LMV5UblyR{zZ@uEYA5Q@nhn9xhkwp~BM?QrYiO zPHR_WROi4sz+aj!1}sp0=+L-OVc7E^({J*MCs-~&MV2u$glAuIE;1x2=H16 z6Dyq)Ep_XiQdaK)YTc3}mcKudCH3<{b&!c8 z?E!>uhDp3)AtKGn46XRS-$18m@F}oE!sBQNr9$%vaZ+s6?2k~()fnvrhaeQN&Gc|~ z25{zPf=n>+rud5Hs*TMnM#v!jTApwbY9ErdaR~s@vvCoq_k{^XF-=uMb*l9!U;5tgphj*an@*X~rC^HQB0~&Ti2kh1HEN zu20F+3@&4LuFlP)f@$PJK8Q_avhlgB--?&`HK})a7S(CVXIda zS%@@VphS!i31_-Kx@uTvvd*tXe<$ZbuI7R;kPZJY@a3YmZwX&$#N|}G~dOI4jZhlVa^h#OTy*eyXkGGV)Ziy5! z8IhrR3L*|`X5P>v*}`=1v+RIVuRn5(56KZATGoW|Wn8V$#cIi1KY(J`^Ovqt;w6~~ zp4#|uaEzDx51=J#2O#fD*%}!=f374DGDJQ&LPa{r=?&Gis>4(BnOsMx&6#g!9^EtV znyDWSL(r517%hwSvx$yEuJ$1Fq+s&%Y9u=OkA+oR(d}r|b|>2hN8zcFd|aEe=JibH z0j4K|p5&n@w&&6t!^#0><|V@7(o7K)O(^k+cnPcSlh4|V%YOQ|8kP|kOT~g~N@ViN z%KeN}(qF#x0=<3SxTPUu zfpM%`f(j!l3GOvD5Oq~CQx_AV4srn5$W_Yss4S7kM=k$xtqbS=VQLB!ttpk#qVE;v z5bxA5nvR!_2*;w@Pc*TFns|WrXs2yfg#P>j*}4^Tbq!_c$_ZlJx@9NMYM^&!)-v;u z{jK-q7+vPV@A1)Lj84c@0O`h-gr{8Hz-Mdoo0*%kWGyq zMj_6Ct@UnSg7%{M^8EVm*6yG*H;1m7C;{#EHC#53&%7JfMjmuSqTE%lndF?vn`cMD zOJV&Q+6o+|-jH7p6+G&M@4)ySxz{m8_5N() zG~6NJB&DufCsZh^3DxcOz>xkvgw<;7;b!EILfOx-^7}&)C{B?pzr`HU1kbQ6WQ^Yi z8N9M~V<{Q}k;)XgfKhZlg7UD>*%fwmWuKUssJFthN##)|Bk%U^a=P0|wlP{@tdjiE zTk_dJ_q)vBwo?4E`IQzN-QZf~-A&@luBnS&_Ixx_(ZCO00->n}Azu(mkKWEl{(?*-@Igt3Kbt*KPg4zR<=l_VCkQe<;iL zDJ90El|TlU*$RWAJo>gwdT56h)_M$ky7Cs526DgK_eEb)v!CY4$_z5z!Nj9lh~s)= zq|jf45Dpg=$g7#1k5wt`f}|9@MnCa1ZYJ1Re4Q;Iwca3;+YkGd|5Fvk)-R=6;JJhH zsDE;Fv2^(tzz4Cs8XsJ^xPTY6JhskMp`BTz5~)gj8#7gwolLeF6#CT+-NQV><=PJO zve-#C-2^ZrT&|ZvO=a9YUvyR{%)QE=c>Z`l_f$)?3+bouzq-e4n zY-i{(9*4V5(+ig&wtOMHwS*zUcr-pqMssPA{B<`8-%Yhu$5}okEQ~12JGY$$*-X!B zSPzsT#&62QU*d-Owc}aDlxJg5kqMY;Dcnkai`-=kX4Ki{K$zyODkgXS$gq&V!iiW_ z((jSUmk1}hb0@v~ao@WXAd4x*dxZ6#>;u>py`OKWa^y|rK=~8}NrKo*{9dg@@@XG^ zTD*jqdawVKP8(Zb1g3Fjj9y=VMTw8=8-B1`dPf++s0ar?6pT{)h>`h)*0k^QF4lwg zAJ9f~Zo`=>4XwIFd3{}F@TeiIPAE5hSm6~Y66S*P06W%Kzu?($=C{0EPsjEy_(5k& zO&dy7dZYTL%#j`?iuYiylcc|{(F}H<_M!h#C@@eamngniPvL?;p`7n#K(z+0Q#!Ao z!UV0XgJ4z|Q@?9CFaupRj@Bj8k{+)OOqvX-STU7<1%3i6@^@HAWo%v^^zykiFaoF{Dy(8yRS~F)u zq`CsGL+mQ-MCnob!TpEN$QXhqt&n>E7Ty5mP=pI~Q&LWlqZSwqLLzZ}tcR-}EL)K% zt1Jn#M{a0X@6#`qw{ZW0$~DWkAgFQ1?D>X6^ysT-ciuNMvM(!r-X`I9=N(wB z3)S04Jp6sL^0ID2c!>4G>0&>!^}P*RN_1Dr#Nt~(3jx+W+7n})`pYP9I})MEEWAGy zV?I;_w;4CgnU;EM7_6psa5M@~cHNG+g+cjMb5e3uNx_TT)U(|-51adgzx89$HjaqeZxKS20S2GevSe1z-v@mjYe7x{K1Blz59%{9PiWF!DWeDwRR1FH5q-nu#l)Ak1 zW5UsarD6y-+VTAKi(?1%hh1($-Gp-JQgK6x`awY~mNByIC;6r| zM^@0zaVWghCe@nDgyAeOaEX1|XJ`c0;tfHjvN|6=z`%WAVp!d$`XslMUnUIcW{9X- zan~0F$9DmTL3##=D*D%NBh~04^KTEm=bIQs$5pBQ+ttJJWR+j`+3~n1weflocZ*Tu zNTPh3g^bZPaW{K1LSk|+{o`}=<>Um`8!1rY&5)l0D>PCcOC!@Xy6&3wd{$tzfBfcA zpF+DOT+HrH+rK_kuo&F^FNMaD$Do-&^ck(l(!@RK$oSS<&*U7{magCGARH)^oLq&v zGp4X=W#7x&%LmZgGqvvlzqzj5!-Ba}y8tz3{hhw3K-*^SI^74Sb%uF|L||-5g~_~Q z&gA4I3^}*+X=)X(+C9+K26d6!ZZ}XU3z&!I*!zi9Mb9W#N=G-X+N2ijqftB>yWIMo zM93$p7`0kqZae%4^2B3L1nv@?&T?{7k1Uor#ov*!uvPDG9ZLat_d!lg zEv2p4Mz9jkgt^`_Tins>_U2>Bkh|Wa4t|C1OV`F1c8}Pd>byFRyaIrYKQS^ivbr!= zoZTld#6c+nj{@=yCem?)%}1;M_=*49H*TBOxGD?$7$vYt9$B0faPX(&`EdrtYtFuP z-4K1JLS-t_(K5hisO6M%hs4jP3vjj>wnga&rPKfVHMWPN>*_UeOjm0w{r0$_rxjnV z8p$KDa$QvPHO_xzU8&eaHj=IG(1)@Vm)$AEL$-7-Dmw}OX{KCvp0`S5!lgXk?D2tHb?Sz6wK>HDXBEfUX zUuw`^#=K5BRZ8bhc1Q*=iM!LKUv)ZP2h~6Lk_CMD@V#RQc?UI|w^jaAheR+3#=7`O zMbT3mTFE4)W-uLUz8J3ED%EDUuv`_vk8DP7U!yK6?=(AaC(<_ePx5MX;1@ff=#F0% zg$79GgC8pc^xCO~tu^pJxkIY!f*l^|Vf-|1W!mG`_Q7jf&OS@s8O2Mrh?j^DI({ z%M;=f_j(nfI=5K5p-0vjxep1<#8nRr3+?fAsG5vt;Z`=YH@;ouk1eCv5mpbZo-0N6Vw9{hqysd$;4_7j#0V@p1wgVN{K?VIpl$eGc;oi{e$Cp^~H z$k^mf+nm>ECs#S17*SvGLkMac!#k)oXc&NwEliVC**!IGt7#tHboiyWHP8o#_QBz9 zKi&PGI{x392hfwt-}fFe5#4SP3s!=Kwj%V&R4fBQt$?s_itIv>QsoszzX)Ch-&&dM zVjl3;{E$5y<^G7!degIJl%wo%xe0ZIxChsCH!x8uDRdBB=LGC!pD;cUy@b_%_|O4A zPRV&*CkL*fhE4J;^KffO@Ai)#V!DV9xr>>DbM8iU{h1>_%HP-(V!+_O30nJ;W-lnd zt)#yar3-1`wT}r+D_!sJF=_l7nQWP%_EQQ{saj@iuRe|)Bg44ZykXIv%d^@5ie`5} zi6K%_g1j>0Mx;R2p9b|@3@*`TTdi&(qkG&VTvCXQA4|#4)sd|DZC93{xg6q4`(Vmguy{G zl~BsOQ5u4L;PBxTJP)lo+GPwbqPfF=?caW&wE4En?Evz@afV&x;lUO`S2kYPg87(2 zZJ1tS)PLL#lg{=zq3dGn=}q!Vj+ncD{b`qY7!ewlSXPMFWh~Z;#{}c3+u#aTN;J)6 zN?SYBE}?>*thM<-w} zOa3Ej{qJYIj`0<+j~V5%k)VpyiVmpsKks_%&U?H08j8YwpgexOU-KvtcI^idH4`g1 zOiKOC4&sJ$yxM3HOWdxOK7xr>SWGluppp+oAUUx&Hk@{AQr=WnhF+MK>5MTfqf>1f z#}+x3-W{o+;Bq-c|90>$U+4^de#jXF$YNq%WA;uHGqFUQ87V5pR=UGYhS}=;h!t{Z z`=T?hlY%f7+AhD21p?_VHEVJ=Rmm}RnakYXVgpxq+#uj+;`~3l-U2GC zZT-UqL?s33knT?DE-C5m?(XjH4(Sp^x*HLsyGuHx8${|(_CEWZ^S}4pF+TS|$M$or zxz?O-{oeP{n3?BcU^SJj1%>=!$4gePf1aO+9v^DcXHR%*6YN2+5Me_KFfldKmsN%p z6NDBS}lEav~zMjK|%1; z-tt{YLCy92f`V#uSAc=h!-DvxDoc-4h77M_trxRbE~)YQclM@~E^c*hiM(@O_Kh_| zS80)oMNqG+e&D;Mm<1ev4EH;jA3V`nkGQigx@}3SDG@z1muMLw#=R%*gblgaBQpU8 zZzA9d#Z|g(<7wEA%LF@vsjydP??=;FiWcsBWeWbWEre zR7JEg*VUHKetI!{MSlF;aO)Em^Dmh$IWlx1^_&G_@b@Wn7SAl^SmHBxLW4*y)U^ z-C(g9v|D{iRJ|ZaqBACXD=vwAi-&1Cn~PxIp>&x&%y&4`yLoBrHWJx2O0QwMH*&X9?FSKBYGJ3> znY%1G=!Nj7!QgYR=LzZ<%))uM+rckQj$u`FL4F87`}Pi>12lE*k$_6piG6MgeZ(JE zJNe3u>lfLOo5#CGzB_%i0IS+(8Wg>zxo%@#NHP~xAXy~3{b_-m$Lno_KB^>+*tond zE$x{y+Q#i%#)wMZ3&Uq+U2j0ce}r`ZehopjoB@)a5Q0EE<+fBZOqj^lguH8~h<~t!53H~y5t)&=w^JeK?#ElTvl z-tt#w64G^-Z=J9i*c#M&VjIXAVPf(Buw+jts-{Y#?MxxvM~?n#$3C)iG*#O@(pge^ zN?Z#WIX;R7PJe-W*Vks;NpNt6L_m^-7pqR<-)&C+ITJ*_Wy1#bSU^FU1xv3+$uS(o z6xlNHBqSKi5z<_}O-#`jj5V#6L(>|3Zezg6XL$3UHU=5KMa%x)UBa?=v_A$3l-1?7 znhb^sO1c=WY7D2CSon>1+Pl8kvF>7{ohwkR|M))npl%~EzL8;2TJbV-~FXQVdgOHljD_u#~VR`rx}yu>NlopIs34h8U+~cr^>4`Q5stK2l2~ zl6~fV;p!U{_@KC-_cdKU^tM|Pa9~ZoFqobo=G2?T)AUxw3o@KR8uCiujCuPFL z_Wy|GfsN981A2k<=?Miy$>Aa5EIA?ks*`951r#dMI0I7yks$O0^-+;EAw-L~MPO<5 z{Af}04oMQktI{|tZZ6wE-;plv;0@^H6m|*9gR)_Ek@nJ^DXZj|#k^=luTNOe4<{9m zPz(8rIhHTPmbh5MBtC_Ap(e##C{m3yge+AEqvOKzXx{*d?!ar}jJKKQ|3jNm=_4GS z!$i>Cr=e3lGh#)z0c6a6b@!$QRvan=h2(8#8N4=@+p+bMNu^(;fGS@ zVw6ji3bR(|`Td{7)$?gZjR#b)2gOBPBCsNaD2;N!?Eck<&Q90NgJtfUQSd9fb>!sM z%jCG()P42FYmC=*=ON2fJ{aw?qF^hkypA+$k*3BXxku^awP`ojF}0J-XPNgsjT$u$ z-wN;}%jZUEUeGTD7=dBWwN%U3IS+R?#hfVbvFlG_4PB2GlXsYCRLbl)Ou!f^bNp_+ zfDgHVYH)qvJjQziZA*B#UuPZyAu!}Sy~AjS~lP7cB=HrH+=j+-(*r^_|xI`&pPPm;l~*Tq%GE^a8mSxIFw>=mEL7j5Sr?`}6VG&H_G zT=1~f($q54a6|4-q07)}*&4YhJmp}BnPh>Q)Ael!@|PdASJeXa1k@VZN*S(Dt)y-^2{?VGjABVT}eH%K4-LCc_xO zBZxR8pODPE0$!d23d9m_Hj8PD(pQy{N4^BiZ1){(^>&YXKGCo}Run?W|N8hFLEWEl zF!RCcJL3hSN#J0selit=f+U5gEC97WIzQf5m0tmMj>Aa@2-OiBk2xK->^NLIi2nOK zz|a2T#Bo#!G9kM{P^1F?`}0UBFfmMkw(#n|mgwIv0R*fT**6sR_%r!jGYssxcyJB> z<9m>ap}$jOs)RAZ|Ik0>3fWLzPy6>K{O=#WM+j6n;B5_ue-3a+eymK#R-q&|sUjlz zlsCCD!d|SS-~wPqZ(n3n{LiaJmVjoKT48mH7Ux=uvZDuo?do8Tf#h_DGvU9k_dhQ% z1Z)tbf9^Kx5I_iEt*s+V{MXU&-&e&{ioHpKsVv2$SMST7*go6@e&kHWtq8qrYL+ThsQ(wW?=m}{SxvM)4S zpDQ5V%m@y#MpYA->V)mD3Q4~Hi8;enp#+H> zE$C~Rx*LyXe{D5R2@$>-nDu@r`aSdhLn>l2MRAJ_nf(*KL|cG0wMcEFW0(m8`ls~F z?jr*Oqt|+XS*i-!x5xofJsu2x9DGk(LH0lYL4ODon6GP{2NkXKer7x;78V&8=W6xk zDB4=IiW>$rsE8)0qiUreD99vl3$U*s@R^|GC|QC?HEwe41@wQBZjmVotD1 zym}G1i4+&-;*@(eF)P!zt7z4#Rl`lTNgdZqC-YLo$G*OH@-A~Ye;dv}n82%^WVVy< zorMSRDLtGar0#dXy4l=pwXXllZQW#3uK1=osIQ0c>B>gw2CMIZU}d^bIaKCq?F za6Ved)CDL!aTYw9FnsZ>x7eAezt~!mVsGN(3z`F*ys^m!YfUPJ!m#HKyUA+94}gXd z({w@Y$d}CL{ilD3q4Tiq)5-rnF9->tzYk6ga~7+6d9@ubH?t_qEj8N8I=6PRjjr&D z!*R2!sOf0Eqo|eYnHQ$VV>@71pE=2{x&9T2qRmm8RB;;wH=zVO9>SKK@MlmfX=ZBX zZZS&%A7{f7z{iXL0_IB$vurlcdFRuE?Bne&LFfZvS~NcIqCVbg5T?IkvCII-U3XiK z@4Y($FUGqVEx-`W1au~S?ZS>TCFt1ci(U8l|9gG`g9u*5Ooo>|$#PZm1DZw7dsZHl zsxpVE-lYfvdjaWWj%Q}+?5zb!=9`~UzRh$osC8)OFD0W=X^AeDI4|J0GPp}^+?)$$ z_8-+(9%GR+0Nm?WsV)R!b-)YS@_&sVH<(N_1QO|_fv*r9IB4KZ-f(=s9?9nQECj4A zeQqD!S9}INFuHGctN*{&gGlK4VLrD*63d~L5oSlcJjT0n=8*kz$nE=|Ikkp-wO zcx(3Dgm%(%74odP}ITy4YU78*)RBZaJ!hutTMHGb<< z?&27D*~!9ZCjJOH8=^Z~%e&?DloR#omo{lB-m8+b27STd2J zv6PH1rvb8QcdugQXwKe0DvI9cD$Vg+J{CtNFUelvDmmCeU(%7z6y26X9K-2eGCOl$ z<6DHMY_J*i#BiDmzQ$d%qNhzrYI#%L@xp5J;pTf48ja-B>M0>gn*1aWJqf*H$TaE$j zN)9UY{AL5$*sm{Nq8QIy+SALt=L}M`Wh6K-kTslHHH|*B=*6+r9nmrQ1uOBSk2cKu z>X8@{r7G^*)tqjna3^TLkT+jyC_h|jUs_ozlt|nHPdpwRW{aJvhq%qiF3>Rq{1v=o zXSw7|O!CaegQNf`9|vUA@I8uX*?HiY2zm{<)%X+OU+w`#>>Mzz;2jhjW-ZeKdsfxWV2q}a#=%`J(&JN0sy7SzkZp8qCwgqZM*{BtRJeM0GrRfo42kojH@Yf z&)JhPki2ezk@>ETxO4xq?0#g>-%F9ff9FedVbgxpNhVXvGhWbWvxz~Gu2Dg5ed{qo zSTP2dS5M%wJ&2CYY!r4a_Gdd)_wb+@c@bKrLW|0HC?y#Xu8OJC)@dKXoXr97`aL7A zTHKI%)XcyQS53z zBTW2>vmpQ2kD7+~2B?v-ywGyXCWZ}Sgmim%v<%))sWE zeTy#+mX!q(%|x(b2{`0;(-=K=b&jWNf;o%pUH>}n*I*&Zoz8+#ZqCB|YRyde!UEFM zU!?*A8z8v$r8qkQz-)(V$M|SKykCn1Nt6jfr$oRwkkx3gGS?joUPBQ$d$m<~k$QCu zMi~UYx0vqN{aAws761vBu#lOYTnscJZLPokauV_y&kPAlYZSn-Gp%%UXKv*d;Zf7dBs@&(1ve-LqRG*B~q^uvpTTn+R)6@Y@5Uhur*xIb7 zZSwEmZgg?$t+2|AdL`fHOOVeu=oSE*@qRkO~ec8tipQX&^K=mPD29J;YhVW90sB; zKvSl%UTR1NY-sK%n1I13Y?eZuc8@m)x8!!v0Qi=lqop}vn89XkG+U;w!aBkxL1+ez z91?ZHqC|H?mbs9M>eu+PR4d=1eF4nur3Rw_T*MdjSB?+P0VHZ0D1EQ%vGHZW^Ck12 zpI6Iz{65r8Y~UIchO0cke!I`WuxJ+;*vWg^5a|@@HevNW2sWTtv+f&et{#Tl&S5;b zNp7f;Iu2m^vDvKBgw48~=d>vp+D-$aaNbMSn2sH57c4JPK!IFOl~Pq3z@NPgwtP-_ zCf zq;XvSz#$q0%J=v`GH+YZ-;jcsc-%A;ZWW}u!pB)YKGezN@Eyf5a9p)_yIS^OLNPu~ zyjdMW^C2nL!|<7lA|dyW5{G~q%oPlLwxkLzPUhTDXiS29bWPbLN^uw*P-*v?=E#yD z^I%BRs<%XnYzLLr6cmx(=deFbHAImJhVf!zIKn1>9kYLSA_TVHhW*Yo0Ui)~kUK}{ zs>n(aLNrT&Yp);V3!~qs>v>FsF#*rnDjq^Pz*fsj%>%ezX8R2hPpk=s&U<1c-pjb> z^YI*X;xB5S@tP&H76w9m9~>6NnVbmv5DI`;N+Af?8xtMN;n3gX1*~S49kl&k`TSXZ z8ko#_YB@v5uGOG~)i(rqn|YbDKB}!L^A)#AWLTC@}I= z23@w%j7p5R{W<)d0HD~PKf6Ht*WvI1LO%^=FE%D71)M}ffUfCmDsIQ^cB_VsmpEFc|mLv(B z#Y@wy^*$Bf#ek{Dp{t9*t8^%mj3X|GfpW8uQV)N|%fP|@c#D4cP)3fevhMCoU^ad! z{&@Nc+|C35%!L)&rdBFJ{sQBs^L$UptWa+mKKWaHmQLRq;aqD_w!i2faUA zPQ&eboCNr5gMdO}LBK(73^VtR*Q3tqV0NS<8~%~R@3odcBTOD8jGtvR^fqM%J}lHC zphebyi#%0SMS=rhZ(F*2L2=>g$iMtIZ&Ar$(38gmw)uxwB*-P69w5iVRr%)Fl=N3j z5?r&JS|Az}@`%;~$RJbw7JTWswcB%xgvX^6w;K&C5xCwp0;sn+Q^Wje^KgHi(*k0| z5KQ{u>|%C3)(V9#-W|)CdkM8x^uj-2M;8ik%&%+%ztn|DxD?A|)tQPNFE=}5*#P_J zM8>G3BC6ABR|BZPdu?7*pFcWMr8RS<0ykYM^b2qyQ=0FzAUC*jU^BU0M|atH%ptN> z!O1?oq5{ldG9WZa0bsmSJ=odgJ$3#J%`Knb}Xv;yB131Ngm`N zl#ib6ANX{j_9JJTbQl`I|NVRHcma^KEcCeA=Q@1N>4+e_VeUrpj6)CbHhU!UOM>fS z45H7+g7!fOw6kz4$EpOKMqjHr3v~b+oO~VchvBSc}{d$qj3o^33799zG zvd~b@vYS4SfMN`tH+PJHU-&zn*Rx@T%yDN}p+bxE?jgz(S0}Zk^c}4*jPt{vD`Q|~ z_<*U)GP4AaD}hE8Ft80@JYR2|JVI=Wpky|nYoatO4rrR)B3wnU!s}?v8;P+pl9QK31$T_dl8CFF(Df)YOw;w zAa!vX-HtY+p_Gxq^`)g=4jtbI=Q~eb%x`$jD{Jy?0zqu#iguU{*Ke(mk{=j=O5sAA@wKsJOELaFN{gA9{|Lgn*VI6 z_h|||p}~Pw`F`NM0s$r95$ zkuMzu5n-fm94dBl!dH`{?#msih`K-9Q#?4xnJrU7F)6&WM7HG7Mu^XA&I|uee3cPG z_#1(>C_#ph_zO6g=5I%kC&4gRid%r#D{ka|5?Wyi5?FtotgKH-38toR6Bc3{m%Xjr?dL$v&(ukWs$w;&Ik_X^KQqX%#OvP~LT zHKDZy^_u%`wuIYd_(@V%<#4I5i81yyW|DPf?X3;dSV<{ysAOl3m=eOy2i6&OGSt`7 zwO4f>Hx{0AXO2aRobj=v#;Mol&f1nCw7SzWZI~JrCD*luKRZ-OIvXxZKITZQQZH=l zJHN@A;tI74wU>B%`lx||`!9FVNDm5T&);AYrL%(-Rr}@CNw2 zE(E&lgjS2RN$E>N8SIrlx&EOzAw;HW>f+t)5mn-@A$JG{DeB|YI&DpXaj&XZQ<25p zn*K~#PTy+D`J?JgnlPyGAbjIgzvA8Cd8K+1Pci9bt##DxtS_SXCpCewB3h|mS)S`A zHlZp0OKV{CiIR5Ca$tFDL?PLyAJ>VW#!D*?Q_~?ah8g>%_HA2zW;&;SJ9`ui>$*g4 zwRtV*_YI{w4hlzGO6hM$b-14iankSbRIvANvslui4l~B_a3*N4)^J5d*zyqPJ`NQB zJ1ylvya+TS0GK(+ z8W4xsk}1YMqhz$cI{-(34K@Kwci`(p3V!<6UCBhE0_fDR{ifgVZ|xe(95y(snvqkYr7^?3jODDk7V#p64X56 z_I*}t+5jKViRra%?1j-2s6~A|odVUmcJ2fZlDtOFG8Q5t!D*vqYe!(#k?)6d_;pVL zseMX-KJq}Mz7sgZ&DPyHxZ|-X2=@!5OEqB-e6E{)v8lSg4>S4rJeC-%!J~P5k~}Ee zKZ-6gimq>O%3k-iBRf1}9l0tlyScZ>d)(8;Ue0j7ZL#Vo2Ea$x#&#-MmMmOk-`5C$ zf8A(<$~hUuV@@d@z+<6;3pYzhI#u#!pI)_KX4B4wU#WMUhrUCW!e5z}%*XeGxpqx1 zEAG7fA70KGQQ`VR!-7eR2Xyq$P4a_cs;S@KYf6+wC8^V2$D}W-ND9?+v}lMmFLwQ4 z(h`MwB(=k1)%0cSZ}?XkhL8l+&+OG10&L01Y_8yobUyDEGh^OEx0J`{pX+N0HMK?7s>3dp{67vgfBwcV;BaE606qF0=r*JU5P|I|!LF|?Bo z9*m@tyIDcNt~ok^(c}JlxCq_5C=wSx7eKTp5)9q4lzDvB&k%m^k6ix3dd(VRB93Nc z>772clBSZIi$F-svIDh<_RY)cH>>k7~A9_oagJXxVCdJh{eD&pO^QLmLzN7dnN6VFB62kE}^j&$3I7T49S_W%mxT zxs=j}OaizsP!N5tq?izClOoignCJU0aTs_PXHyeyYY)J@UOVY;B3Kc4q1LqonWz2cblnM}%d&zlBS{OjM}SY1S_5tZnBU%)3DGDo+U|BMnMG9f|? z7Lk)9MYH#r$&*!mGmp&O*(!!%%T;(+V1~hX=yTGaWkx<| zYIP8tu0XVeuFjU5&+0%8!^pW_Rw|M%6+@3pE{cPgFpu48Y|=XEustp`ps*p)XWu)T z0?(o;#ik6A>zcxCoVlVQWn!B}Q%;l1EFjtH=2U)3s~}t-QDifG<`)Wuo9dfD-zjU1evzqV2iR2C7n+|KAGqB7DcX^ELrm$IxLVmYy+g^c~K;S{ozm7)$;C)HHicrkU-fry4s&n_$-Yb zN{&INQ7BBn4;-yZp7wV*XWh6#mNOuG1~}dnIo{Xlgv$q@c#zc8#NmEaKj&oJtjGJ~ z%9S#Uz^*!+ks|lpb1x$Lv_dj=UFTbS9$Y!hcRobnU!;o_I_)hbg^#`axoyMo$G>)J(zxPtjsw-@r^*LYS+eJxO&6!x2@sxMjM7g{$f;7l-$FES@4 z3-6sJX)Ed4DQHtrRYhvhE9ezRtxy~lcNuaQYMAw@pjpT$qmN1JGZ5nHL+MMe)JZS2 z?Q2d;^;~S88kx(it?9na*-UEkX*4KC&uU1Gaqhj%;wFWsK!_vXe+paNwv;N?cif7@ zx6%ps5f1mOp6kuhg1ijSy2Fq|mq8j8#~7~2jQfG=A?1EGp67K$&aC%cNWIl6B1%$} zv31sAi$R_>^4@Zo2grw;sw3Ajt1C3SS_DYC2v2$Cs%-8vD>TEqmm*Alo5A zb~;C_y6>sG^k-ZcX`pe{hXw>mi68LYMRY?|b-k72zV=sjK0Rt?`#!qUX%JaaDIw@9 zPE1TNTg@}xf}T(3ra91?ZVX({XC}L+c-YKlaP9{ZNMFD>dpZ-gIkr`m~RdaDuFrgruayJVqINKM@8xIvF09Qyi$O{5m0q zF~|B^+EE;QCJ-Hc>Ob;_Pw9t!HPRZxqEfO? zI}Cl3(iqXk*;>HY`$*$RKafEu|G932f{8lFQF&SXi+p~B)Bd!o%~9@m(V3VJb1^B) zuCwuiwF$g+DnFtVucPy~9!6#dOgb{B8nBPHCD4Viiyh)BB-O|iU)RT@G-vc{l!Qi@ z9j}Q!y;eU9Kc-K~CNJXk) ze#qCJ*N52S$YyR`6XKxQ>m&$qZ=0xvOxQez<=!CZAk$ucvEqm&wwrFLtPT$cUeb^7Y&v)4|Q8ZoW=thJwGTn3dQ==HR^`7YAT`}LhP=> zFg_-zAE#f06pEAW=Fdd6MdYiCb^PIR{rge)Aq!BB0%BhMui)@peevb!2v>A!rE!Cp z>XmyqqSq1$>*hhEi(QiE*7%B5DNN};U<$1Bdj*&<@oaBVvNBODA|V|4qqq8+{qdWl zYwL~nn__fv*b-jVC&BQT(8}EM6ZF%G3WibsRRUM2VUGF)c@hxWvL)ki43a_0#KS17 z_8`|&S-}_io%(bTrehG za97s{;Z@aDfH#_Xa-}B4=ElaURO4&<1Pz`w}a)?Woo__mw{U#!vkGa&semk*< zwYL4MmdeRAxBBm)Vn+w|iYD1D)uVT*+HfuQII?f-_$gMs?da|@T6dWLctn_BKcIfQ zp!loi1%u@es@w1hLarcDNTmLrJTh&$3w3Z|YJewQNF4S2cHE1^&yGqdq^A%k&%mVn zs>@l3($oUqZ^sa-{gC#J+5|t3E1}$^IN>`OG#VA@Q2PSAmppQa-8WPYk%F|k)ds=U zTw6LX>}C0{=M9NmSKi4Pp{l(%D?JRVRj-IRw<;^uTcGC5{L}O4<@{xH33H*> zeg?rgP@?HoEQj55_fNgZgVHe$A+oP)PJ^KCnYb5*$i`gtx`=+p&jr{qb+5IDIGmrV z3IA3t!G?wi%2qcQH?nzLR$steB8T*SEAT^=C&&yWiG&kN%+xD&c#rB+(qLRu#+lvT zYFZyHH8!@uH^@TCK${E{e_g*wR<}URp`aTw^QBkkB5H>4n;K=dOp-^=jX%uBNYsMR zJE(_(Ahti>gzVRoL>SbUOxjtNlP00^R7Ps~v(XmXeb@9GI_np|=c6Xx?CYIqfILMZ zYs6H@FVKukboT6Gj)b#yZAE3*B8$V35CX;3zg2iuVFDnrZ~IdY56Q9yx05HiuF|O?7$=ze=BrUnr^C@xKiown zl-Nff>SyfK;;lY~e_c;m^~A7oUqhn%v`vjqXQ^anhPXQ}@3@IUTfCcTP!U5i@KJkA zV^cTI!ETWIrnaYbGYeX_PHXaV2AjB+vb5O6e9t@u2hZl8vN$~Kw^mqA&$zL^!@Zz; zZsaM#EkadHhjYTh@hbTOG3-dMD^y4{6tyP1Jg>7Km<|W!bJ=W$2Dc#o90E{!65!sS zDcW|h2THI-2&?JCXp62x;=l&6KzjKSlm74Q3cEFL#GD1Xi6vvlYRcg6M5v9YNz z8=pT-(`q(YwfKcyxDyj(>ia%;s{8PsyMFU|Y+mD%9>qe^B?qUy{owVPeX zD^%4s8_aG!;YugC6lVnnS=VJ`WanzCu-`yhNVhG2N=?)tq*p>KmfGWCloK`CH{0@n zlUYW}Wj{T805cUjznP?9Da78j;^Y- zx>!G#-a0dkJxP(#p!!E%GiV2?7c72WAdE|;QWR6&W%blfzo~Y>_Rgn-rKZHB(@=U= zrAooI6W%Xt`So&8WO`Dnl|+%Csfp}kt=!Qz6Z+9Zj8A0+IbI$1eM~J^B`IspFb(u;aF*Z43J z-1^iX8p}guh)9 z_&VYU3ac#s_wVGIyMA?bRz?laMKYT^@%+QvHLWQ2igpBzw9F3`M%^6=PS;^p8rTdp zb5;r7rFa=fQBjrK8%+pJ5$oyLr}-C2e+#^XVg=%fMU8IHw>8k76$%wQVe()rYGixt zQ`yUGAlTMED}yd&NTI}%C}i=|*{nZ}Mh0a%0I6zi4m!_^w?gk-4%MDz7~bv5@Srr4 z@T@=bB=rFiN{V!8bhY7m3z#J>Sg(o$D}v$=%2s=?5s`xc5uC1{2=)$WF1BxXbLLh$ zfVK-UEOZ;VJ~w3V(xaOcH_!>9e@f8eUz zsa{iT>r7QMp$rMJYu__)`CC|_$Qk2yI(n_v!YIn<$vw%+yE0OVQcG^oe5_{6rGoT0 z_13nX)H(@s=Dk$hYK&GqoapjP7K1Tm6h~7RZ1hIu77^7rtzaoB=c|GvgZ#{)OjTHU z35Pwa^U}-5*|+hkkI5u5h;F_zVlOaf<`b}!SX8tnl14686H2Yca4ai2gf1QL@A!@f zm>u+fL1C|c4M!VORx6#R_c}_846WmEw-)WyZjs^LZo(MPw&y$@{+H5@&I4H{0X+-a(be|Y+k2pfsWF=% zrc27($Z_o4>5xpK%yT5t?|kw4O&YswH0e zFelXmC7GF^zRwgG=%^&ONIe6k(6QGkX=(SR8ox_fU1LhoX_SXB~r`&oTLPyN1$_#;4W9<~9Dy6yaId%Bfr&}*A4%1hl31dBx$ zol_{Nn#yqKa+{pKl58Kdk`3C-b84y8aF*l;zp}*eqxwrk-gQHh093pTM9vLpJeZ#@WdHGyPd&Bh1B&)JZ%e4N|H@e>y*7>D8@?er zR!4WR_5_>5x-XWX@un5bY1Uk*F;?WAU5=?PJ7-t~(mh+-P_${@(@gYEh`or_k}j~3pJxUkR}5Iik%r%?EMKx=M~_2xSTjcNsL ziJYa|HP>O>V8*J?-AC2M=ZdLw2W!v=>a*ePw@5svez}s&$EI6a!dE(8NLV2!NkB=G z{@ya>S-a9}kyR*}8Ot6CI|GmLHAq3!K`KN%?}1iKH8Zd(Ok}l$`U_7zEvNF< z9~_X;$7q(Lgh18qX8R)2^Bh{B9s#QQk0UJLBas4$%~YOhBqjq!W+Znv=NF3G78nr_ zt?{5V5Y`bZP4zDr)oJGnK++9T*>duZI&2SeqH!@^gIX7KAOv3{kkHO?b`X-2DAlM{ z8w8Be;FGEu!`SvcFnj-T(k{YUrC{6+Cx___C_!Yx2}t4&{1vR_2UE=2(f9Kx^1^%~ zKe*0nK>=AKD`ox80BGOoU%JD1A}_~p5fCqou&-vSI4;ZlRZWeFs(JXgq5) zUXh*xctX)qyUuoQ4N)D#RTt`v-F1>o^DXt2E(px0)> z0SJi)p z!@4g#7FN^~4Ie*t@LEUQ#R~|)0O_tmD8q{TZWa|^WA83zB}jI(T7zsvgJEBcIv2fi z)+~_j)S?**{w5V#SDt|Ie458xPa4Ff9f>mwQX`gD48OnLL6~Xt^5s7f5Xc z(*!n+!d?gEjdiI(OL*VCUbL}uIqw(p%C~W%2WfXjo>a6RpYJeMUW=Mp1NTdj6jH1S zt*s}ig#46}+i@pq$wgU*Ri0D6?zJTFSV=8s%WQ6(>xuaIbW|aD9D1tpF-!HS1yzt= ziIMor5&3OhQ|F}RCKy{}r;Kg}->5%5K3v5nqwB=aw`n%o$p40rOy`uNjR*z{TAU@8a2;V>7|W=#&ut8au-YE%Q{;})BU{it-<7>HfmxJ{fKTE zZ%h1LsuMwhlx8MsN&CT?%&37p814-7L4;DC6i29lP?9qg*TbWt|LSV5I0vqWk_J)v z%;WMG(V%BB=2P$9j3iMdRzQT^Tdu&_y+J%f zg#DfFcRo9r7Y-vYQ%R(Rt)Hl(V%6vv*zm|TK=gEBL7ylf2MfR!AK=#n@u0bps3A!*`APy|1a^nbF(CR6|JnRrM1F-c z{i=x*4zu5u41oTqBNtVA5|`bkj3SfTYofSe84m$T(5-u&tH?xrbQM%Ec)!Giho9~3 z9fEQ?NvXzX8Dg8`i{sXl*%f0XzTdg-?^7iRe?b$TlCaMMskawSBwYLKN@711%n5_< z5@4l@1S=j8)0~?ovre4@LpgV11!rZZI>} zRZ#frA#*tWl4EQ!&LI+kKw_P`7-tBQ=_<$r4IeP5uKy0uU`YLC`EGkmyN0nC$n#)M zL?%x_day_BXzBpiuM$*WlG__Fe}&fqUdI+F1<`$dMAj==;xK)!)I#BcL9DDnN_%3#b%HWcpKu zuEG^g0&U>nmk>h*NK~_bPVoIpDLE>8qfG3a&SV(E10;qktbN`jpp<5KIv8}5TwAP= zD}$=PaoQ9Bv1J9mZgxds^uXJD^|{l5*;umW)7j?)Iw3(zptkvK<4|D}b#DL@`vISI z`7Rsh4*`%oKkM_518mvC4ob)U1k8%#1xio^V)(ZpSPy|aI?6}|X)#-7p4q^Y4bnISs5zd`fx>tnkS+U4Kd8{yJ>UpiWqxir>e^fv!Z&eYHY=}|_oX%9$yoyD8IP3hwVz@$Qq5oSpr8m%`lTfHx;V{S!tsL=8;2gbTT z*=rchpIJtmpiK;4^byU?+lBx0+<_RC2tpS7D>MrN5L@%h#J?WC6+yNM4`)hn2OVk- zUpZxMq79#8epI6FJNTJeWMXp|llj-~G020QX5)m~Rb3o!B<|(C;bp%nUT6p>ca)S+ zrgPZmKM*+8`O7J8xqSD4GkG-=b+>OGRy7mE818f(6oi>lOOf2!>;EWsO8j1U6)eOw z)cfO)XlvhYAkOi)aH$uBeuvy3M3n3CA0l-jygHeHeDUg+WJmv)2^;AdFot*aH}jzO zHIi{Sr1qTVJve!}dy)jAl!FuOExp3O56Yl7kUOmKS(aV#{}|Q~inVA=H4nWmMd;f} z9q%jU(C*lGoW!TIg1^XkR25jl)Y8s7j{s`hl0hzP zjZqja1E{Ji+CW1Ky60cs$ciz{87ppYG_s;w{lA=Qp;(}z9&aUo_x2y1j5t)?m@sn9>HF#H+ zGDNW`zVm8_KWoY|pFPZHf!p1q7NNf!h2C{(okQ7a|2rNxyV7K<^&hZZ*x%V5n0|r) z%?Dcnv@IJ0e_g{XRNUo!OX4Rz9CH*dyG*3P==vvMhiWVQA|vwu*46uulM77H)>R^X z0P^3zG>OI|LmjRumX^DF2pXB92z(u;~(y@VjQ9qzr&-QNb2!kY|KQ76QGV<|n=CjDd33 zAUOpe#Dep;`}hkL^u_*}3nTfvxJBrXAo#5x9_?46=#Lf(w@*9UX5a%kT;+i(m8 zX0(Eg6d2*B(e?SC?{SSAQf2Hii%9d|-NL6YAPa=D2tG;HhJ&QR{(kZ)2e;SnDw(F( zcOw31!^q?j7M>-3d!u}RevwdVHk9@J_|d!+d>}z+1Pgth?V>!u78iicqbBd%Xz(L~ z{w0eRDG!$49PW8ZQHmp-5d-Dxzoa7Aap2u2)OFh879iFml@JwW__UaWE_^}x{Q7#8D9ADo8KOFMwUTSKX`uMlwxgKaO;Fc@^o=JG5) zJBl~B^Z%=l6FNsg?@0>b@it@`ronoN95hLxCAIcWH398S&@vV$;lC2PFwl@nB?v*q zm_h|md;SI{jm`%uN8!w%$QMcsjfF%$3EaeFY!;J#(2B4Pyj*4iYIqD&rDF}4K7y2Ml6CiAnn0Srv)A?|{v}ouA zLZUZI^2(?i_i;qR(42FG+3CC|YtFMEgZxGLYs%rz*12^xS6k4!KuU=pZ~}X;hkfub zZSCw^JsFmRd5`HD;2u%mjsFwVlAs9mla?wS&sTR(PKCOv=h9J8NrWH~(A&11iNTGX z19js7Xaj=YwZ%$Er648!72%hbg!{xmN=_~b@>S0TQ7DGwCEL!Q93q{M7i0VyO5HBf z8`{dRL9<@#b(QB%x~Y@HC9v@;d59gbzEI(PK1xtddKCu?Q=Kg@F-kn?uB_Idk`t|9(2h#^er6s6U&gVR-l*Q{NJ6Qz_TTUp0Joe z@P?jT`#B>Mti_e1nkN)&biOsfbvZfXwif)jlVQ96PCU)5oBL;#ZO5(HGi!&g=L(Qw zD#eYGkZHmD>3P{;y^RUgJO-51or>K)_Z_s`G;-5$as|T*ktd$Wn?{8h>{{B zTUJ7nkiBO{LX=se$S$&1l08zCy|Y*L_^v1Qf8X!@zT@zB)c=s@zVF}fzQ%c-=lPRx zd?DsIJm?`&OKgeNo})_`-up641;$eRze5_mw^~_au$R0hjJKiKy$nqW&d!q|2GCle zvA;i*o*%VQo)2LLDPrD;O`r9coWBS<-!|UDY8L1{ zYJL{WNWwgd{4w&lL5^DBG)cNE`%NS>n3p(mPJJov4czQ!x0E&ghlQ~s{Uouqxbl`h z{B-PBuXuN7Y*^s9g&4GG$cwXl&lD<@=5N9bq9e4z=u$C;`)d>`FxGkytRsHEI={@? zyckSnqIN?_H2Q)G5O@Us6Cat6c2DIb5m zTfil!!xq)~bO=z!G<2_*Z{#Zysn8J#1Lj!Xs;0ctND@e7pR1cmyw$u`Cbsiy)hbLIk$Gu% z7vwDTDWPjPnE8K1H2sfmKtlM#6p*}d7MB{gn0{ga3Rm%)9!HNL>p-fDK2BTTH)Hsd zR@lZ-uu?>lSSKw8VlQ(H#tSh$|FRkQvCuL$^w@^8dwZ@GE`}4``^!Jq?4{(WcB=WE z*rgJ}lWYyGaB{&Q{sdo5$Rq|}34P3F-uqi5Y-Br04H~bZ+ZAzL(DA+SoGmH zhit6g^K_Yuhdr(MF73K-+hqB-$o+ExS7I1s%WRUr6IMuJjLF;5j=sX|8r~`V#lKT z0_^nh9x<7=SSJMj$&oq2V{x$f<+eK-o{~=~EBE)(M9QH#8V45zKxyc1_YK$>Y)UgO7P`an0q@1B=+h+Mp^FWSawigL$TL_ zy~ZHOO|iY$F|^)7diYZBf2O9bKf$n0?G>@pTM$dN0R)%S*v|jn^2X1H^F7S;j^PhG z6Gd&V(pa!O%**Xk|j*V@1yEN0lz6 zXNkhA@A?DKoq^Iz#_#qF97$?T&|UUK9EPUo6o%l$QVTi)fuIINnEuqS=hLim$3tVa zlzJ|@uVtWnz>CE`tD`mEDTgb7q3g9MXZe)Qph*$Y%`Gh{dIt_3t+C8LWDGCxg2SbG z?tm-mum{;#6*I;JrbljDB0uli+pLcqeh0`XrUweB&2QtnxH0bqYj+i)F|(FGjMtKj zUK+QGqc%_WP$QM%NZDF=I6Px#H5(7 zbp@(A6%ndjAMaJwNX58sIVp`xM9IZQcKMV1bw;W3C#YSrimM+Pem0(nHSQ3sHPgmp zHu)U=)>Eh@{a0YbqC_yKZ56r5m0!SK7-8iDWnf`}k)2MCvTh|Xx>|wiLm%zCRQws6 z?6T97jhC2F_>H8>eh`u`mJL-nTc%Pift1fxf3;A;%-DiI6BK8$KnJ`G*n&6dApLpQM}ezxfU-B)z4ILlpg5aJ zjVoA)79G3Omg(v2H}8n@yJB%+i$2=2t?7r9$q%4k0+6Z2eh2c6M)a5>2171D=ht-D zi5k^ya%^Y0uX=OJ_;ww`%{PWS$$js$%)&p$syC;}DvKJfA#zK4ko?Q?P=4H6!);3Y zmq~)~bHBWIfSUTlT$*Rs>uTM*3+h!zt2u3f)>Y8nIq+b2Y{O>{JT zwq;aZ9aS2+pSztcDzJ(t4E3Zhqg!GT~X_fdv4FhzG=OX6jm8)`MA633dzui z*5#GNtHy1*f^brgyETt~mw@EVG|p#1E;XIh0od$L064^53GW~x&{ZqFa;4G1vL&jR zTcO=bemCl^@($7G35IKFlHxPMzc#4#coXoFuTYSy^+s=~z30vx8DRD?Mz<5M=6|N| zi2S&oRz+lqYx*)qGr8c#Sk8C6B~_IVKMo!;BsDSZwOmEXMiq;S7Ce76Z`m~V(kwZz zF0f^A_tCN_!xz4%wv_oXrl^DLoaB9Sy#8_PpRcdb9w1|tFm)aO)?W|--;iB95yroN z-m2}X`Z(d_8|K5adkF$Y6=w3-%z|r;C==$gxwan0C$5lQ$0qcb@;Htvv#el}Xhgs8 z-F3GP^F&_X;c|X#lF6el3yv&V-qY_EK@Tdar>U~x>W*Y~im&R{RmNEBSFg8Uf{{dk z1(QJu*A5{uAHeTF%AL|#zQS>jPc9mjLRs#3%l|WNgvqm-OGTplp*DG>a{P6Ut^C%m zIM-f~-7m@%oVE&VAfw#tGP|%a!mU-tU6^!ceQ|0>ynUNL)Kw4B1gry3rb@y^xmYX= zDLL?$_;rr~(ckPwEQ7Dzdm%Q&J}EX9*(xxk7e?Y&&Lh*$I`XWauT^hw>3VdgP|p7j zth89qu+NjBoN(XC3`{rRF@E(hzqU>qa{Cw&2a}M543=z{NuO!D#%3drS5glWq}?2i zu`wKQ8<|K8OzY&3#@PrfLbV+8AL@XiXM+MZfX>4j6PL^=Oawa1T1ull-8{n)^B5)}gVK$2l|aLL=s#*Ykhx4>p?`1tZlDBRwU}r? za`2anF{7-@l{(#s9R5hEyYa-FUP;Vd171(aJ3GVziZpxq5IsZef6)&v5D6(?kO{L~ zBi{#Agcy~k+`{)lL0~7xgP@=d8Yy~r^DIBk19%em^;cIVp6LskRO@?EVBfcJc!8&Q zQ>hm?Xn3A|N7v{WTd;*GO(NIV*SFuk)0aoLSw&|4l;#sd&o0>GeqneVPwdA`PKL!^ zeK6ad?zBpRMbUgO=nG5pKJ@b24*Gue7^@DxGn-8R_pY$C$U@ z^xl8Rf{Qj{IO7R#L>n_rl{vCtJRxLx?OrD$)1~Dj-UdY$O|^s_P2is%*CxaUcMq$` z^OROcpoK9i54rvu1J5N#dY#W+J0os4iWCqGK_5Uqm@_zANYRj{Pdld#!!1kUS0)pk z!N}};7u@ARRI7Q;cC~sl8jjWuT6_~UT|qJ`;^)S)ZJA&ObOFlaECz{Z1odJEOH9ki^B15>%?NbqgHd2+oVCdER=I!YcHl zJK2&GGY?VG^}t@&>8=D0=A)EsR3pXT@(t9p(Rm8|b6u+KqQ<-paH6?*NSZl*!T#Gx+oivqZM{8~+N&rb0lkO6`q&kM?dR%# z7Ht2vP8BY$g>3)3`8=~#m$#Y4<9nj+4Gw^#X_eIcVy!v&2Ab5G(2WW)1ghT8tskwL zhQXworIX`TVzgJzud~kznr+HLSqQrl5dHlzo5EEOm-|dEBNxZt-6p3MUe$Z6!=3tk zp*4>6hYLko!A)AlYcxc6FU<7ZISJ=X>q(a)N$?+~OOLi~LN zUOlgeIs#_^ZHQwZDjs`+XPy$m97FTg22ObWZsR1XnIGpg3$;`6+;!1?VH4rv5Khu3 z(sLju)#pdg*)uE>wAp)V8)D9_RR_jWMSQFC^`(DE6=5mbQidjAMzFK7QRgicwkC!4 zjG=UG%G_S~Q=;YX-m5Ppof{|eJ9oqyGvd0DCaGvxQR~Y|{F5ixnMt{KDy%!thsr&c zSmTHU)i%1zgBSDh!2Wwsp8V(^FwagO6LT6J?&iPb=mKSY(IM29%-#VAiI%x<1Az?g z_&}_>d0zZ3b>CclLma9CiLSGGXAZ@Z)2tP0nU>N{>48cls0}9_W|uT3koH{RT)Ncf zs3}LFgy1BNU@fFcoAk77X+BS}iH!KwcKx|N?ZF)2IpFH>21mj&wa!$ym{>Q^jQXwB z3Uhs9;8MSq{1ty<7A`rB$1s2>e#{HxLj2w;x7D-&d6@%T0%|5?PHYnrN6rlSi8&>6 z5HCUuJs&h7#Qsh@Mgr?SpetoeAEms6oY9tiW!sxo@UXX0vc>uM&?D)mw=o!_T=m%h zl|!E*%qeVIWs#n5`ulm)L*B8&ojQAB*8C;T8iy%Q>W}M+y^!dc08Dr-a4FELG_!IR z5S%ihcB$-Sf+Y2N2`qx{t;gxzs?we*#cv;Xs}22`Z$MN3N*#mw)?-?UZ<*274SYZ( zMhgM#ok(>0l(M$jH4@dJYhHFZhO(%y%}uNSI;3D8e!xNoxW7N5xzAb zjyz4Noir{1CHS7%Vb$moK(PgcaeIsn5YxMQ3cSn1!^*PJ}<#19Fn6o7X z7=Sq9xBfUuP$TgQwrE)-_7QURT?zX9K&B7RJ7wljB=sl8dxB>2vsr<;ifWrt7>zsY z^nu+k41}8AH@k_Sr5;};#S!gxzb{C-_7p?5Q-Fd|XGF!_OG z5>k+aAu&;2<14awe1x&xmBv_d8nVddW@S79v`vD0S_5nAUF3H(al~%mZ9=|j-DoX7 z^F#NT%ieCI9y#7M#eU%zs@sH;-M0B8wi}wyn;1Xy^Sm)aaAf{O7G$L_x$CMeHgF1Q z!a~Km@nt|)XJ~YR(ZAbVn!-@}{<)t~LKc5T$2eg0SOlMm7?%gTT=pIc3A*k>N2w9$zWz7QGlcj<`chb&W z0PYl@FXlf2+DLR@#Xb>(F&9UGk@aYZMBV;EUZr=~(y^FOvy{}W1Rh;%3d%r--O=pq zD;UK0OdkEpCYNS;`T&^eXs?}|!4`D%K4u0hI(fSOk%+zYd6Z6<`VPv2>2JirPR6KK z5E?oHl>|2amN8AUXwVfFs%M*QlY_;U7nRY}YNF6a%X3;Q;23IE?(ta2Uls8kXj2bt zqey;$gj_tem3~UGm5t+D>kZuEj)?I0!uQ_Bm3blLx>NatVC}JFLP()&` zDC+Y2!DziL;wEaQ`;D~ZAI~|DDRnd*MXhy$UhjJ*;nWo%{QP$7677~@{dj*yDXvQG|FRrZXbdE4inLUm<`s+@ID#Ab9>eu=~+io1-;5HSw^R2>p>VZDF@@py+ zcLmSa$vdIBn5!i2xgK}W?B%p(X-=BZoaYnVQ*ZfAIltMGop6rJ2o3!%Z=l5q!0hzI zGQ%sB55q7UgaS=WmZ4^qT_n2x>lqeNnY?lH**DClWX%rl&I)3oIoX@uet12A?(}9p zOA8TvnSqP#;vStssB6!h@{z1=BK+Gcbu!sB5SgYB|Gwh^FtOq8?y|(mEP0?CD+4m) z4hR1sKI*VJkOuK(O;SmLyn{Z_PVI&t7IjF2#@7Ut{oR;XdhS-VB5h>vc-5-}23UV> z4X3qF;{2R$!vB81WKw!qYU}56gL$<-8WLfsU5o7?3CXdUt>#f+IR6(QVFKv}iYN)( zJaVKJ#P7#7EH)e5Q>yp|QOC7!d&~V%lRcrYr;<4igxg>XwbN)*aOR3|VSn=7|0fu9648t~={` z4-5*v*K;3IyzM`)8cBz>))k~rcDJ~L>2fdM&`wxZvSqFE)Fz2~As;q!I?t}luEn<` zz)ScadaMS?=|Rxdz3bXUiXo+DH!~G+=%a)2^nUIAJm)0ho?U;YD^)>8VZ(RnoPRx) z7WtO{HTj^!WE{~b`ZS5w)R6UYnHqe&pwEghe@}aML&(LAUDlM?UtBhwv)dD~D$9I{ zIW%1`fCjPQ>>eiYyeGj9!$aP3op2tkIhqA(6{&iIOOeg!19b9bfAz_ zb5%S`%^rTHcWbS5T`Vh~1g2+Uqy}hMs75?*4^pK?=Wb{;bg4r&Q+)vgrLE;xVUi7nA=z_O&6K^j~bF$8_@8ga`v7C6xUV?~huGT{HeO$*cV=e8uP;Rew8NWB^>n?#lWZaane>8i=bU1D z?3teQhu@9e?1~pNLm4(I4akwZYQa*h5=;>hudJZm-0>?PqwiYIQsmrlSCAB?mC@tW zO1svkn_<8->?k6^`c;?d+0R`{L6yo$Tw?;-l@_ic;ETW53AcA>bw)6hmOO6s80tmxVa zu3fZwvOX#k;I$#)IWTTEeFv!hDHWS2fAgug~hd}MXkxGe``SF}5wUXrctOwSb zc#s*zT_qxguuokk`pCQ_{#&y<6AFjFZwdw8!&|4#+JxfRDEbX#+v{lWErR;iD%|1) zaJK5hKhXYyH18ax29WeKA#?KVs`zly8hNnfN*h!4O9YmW8S>|~3qb1=yJTDGKbwa01`Jjzx0KQ8 zkSEuD2b%Kv2v0R!C(0Qn43>OS2DY^G#}cT_4A^TRxlt>MI;Ipbn!hRJvT`O<2Ax;8 z1*%|V#r;g`OhoL?@%H#-LCUaFD7zj?(piYSJv1;!*2#Q1aD~#`KkTt5n8QVEfH%}@ zWCs&R6D^T>mo1qT)_TMXf9odMc{(14(Um(i4C!mQO06Fagp{nV>8w29eD?9qITo6; zN#mgb$-Ih2e+IV4?2zZ`WNI8Ar&*h@PGSD76*MvSB9OJW6B?AzV$26a)xyO-YYlj^ zY5Hy}!$S;2!F1-(Ee(uX!$OtsU@~U{>iNveZu=r^M{Bx%F(P_ zp!&1HvS!Bpd4J{$Wl=hMnPKy#6|p8bPn8)`U7zBM;;xZJv~OMbp7Tk{ zJHQMt?&!a{p52l6@vrlc1vYgW&Gb3NUcOc;zmZdix*&2)_qzsmAwsy$?L~oj#CNoW z+|!vtpYb*|pC8x&ynIirMtVFr;MFvM*7+GM>E>!IM1`b5_7;uHch{Jd4q8L4KU__{ z_ytlvPMWnC=~__zXXjKM?Erh07|zeH+|p~RV!pJ~%e#6=k}ghdI%YF}r&8q74#UJP zrfEEY0U?{5+^B+=cnhX~75tvQ_x5_m1g!7Lxd%f6XrY;V3PTr}FV8}8c zP8f(0awh>8gWt`2^oe`vGSd)OF5Ck4D%v*fM_^}lT00vu8apIiX6<1f&TsFLOM`ZcM;veubiLOpXggX7&^id~9NR z+Hj*|h_z$jc;y=fpZ>LT4-ir+&S$13BsMB!*Xqw0vW14ytk^4YsorXJ2|Lmz6n*JF zSz+1wJ5H&7JhlN?}f-4;8n>!Qt)CoQql241hvR z8Zrj`kE9(!#VcTTS>`zMBPaa*raIHQBR#{D9)DS&D1s3KuiTxEGx*tQxHj zhbQz9?MFu&E4(^X%Bdoc_6h|P>P64<9%qlN5X;T75n!!i_`dIBPmj3F0@7spZmSal zbNDAude<^I{RUmc)Jf{(@-2@&=_wk%#uzxbX+v)W1C=L9(? zDdLsxZexT?kXmp64O-8OF8Lz_9z0*e9jW!MDzpE7bvQUdsZ42|#Jvy{dwq!U)tsT& zI>DFIqhgWPDl3kBI|H8mQ7jfozC&USKwdj0P+kbXA$D#fAsY>oV*Hyw(O~&+Rn&q* zX13?f%EedhZQ<`lhk>edIZejHD8f27D|A_mG>ja6>Y`mwhW|E?C(JLfD6#N`Wq5cl z*B)+YICm`vu8O+9)E0_-swLXpl#G_COoKn^J(JEInlRKQe2E-r$%!w>b_RH1{wsM} zk%(9B!90n`E-b6u`Os>O+(w(@pH3@`Qe8-vvreLH1VHAiAVYkWH3s%*sy$z9^)*+j z%;DC!`L;DPy2e8w59ggDzb50N6291&jJiakX=KbAWG8)YreJ%jvXu$>%^%mOY zz!R7sW;>2lUd*8h5Pom)-po|&5U1iL?q^$0tz?`r3N>4DcX1r zhyxQnoNwn935a#Hu|vD|PK^QTZLqe)Hfb`Q2%o_e^H<}_&s-0&zYWuFItrTZnKaK8 zlDzhB-*dH_P6FNfSxAq7tW|n{f6FdZR_G&ItMlG0q*^t%AUQTRW5-`#{B|pH561sx z-)Zir_Nvv>Vs+YUty#OZZxep42@z^Vp-1i6dW9wzwUR&90r92{R>qd91L)zh9%zmc zD3BkaRVMiQ)o{lfeHw!J)rK^0M*evL==>cuOepxF0hOz>oL%IwkpI=LJ%8HQ#299z zw^>P4PyVxK)OZa9Qn9fpcZie^iACLCIE+>s06ly9-U`@*0X_+xgo}Q+1lzbK)6>RR z9hM3uUU@2}Vaxsqo6(OhIrT3I!;d<*!H}{&nqGP-faZhi@({@_JZc6a*G*}=XV4=BP`Ua2jJvyM3vHs?F4)j;U9I8JXE?9wB} zzzn;QIbIXg?Fb#=5)}@XAq;$%R^RE*!$=thA)g{%FFQ245bSJ0QE=BZ)P+{eHWoE3 za5MiIzyjB5y*w0Fn!%*fGQr2V^1&O*rtkR%Q$=Y`r^SU!n!l0aILyjM3C}r&~-S>4` zf)d%#mR8=Uy2%`gZ{6`Mvq_F#e2oz8RXe?H-}fqjXqwdqVm&gM@!yK1lcb?jRmcb8q6|- z;veWuS9I}Ufles)yNPG#+gaDDXRVi1REC0`!)zr+9>nW;vz$fydRCUalVZVUcnj0b zIKUbj^0qsqr{sb@)5u)!3|VD%e>GA2z^XMASF0bc4}ZmhoYEhUmm?26ElHejfZkX=J${VxZt%I>qMYP`F1+zpy-&~L-B;kkAqGOdj@DH~IT)HH1h3R)Fv88+mVNbt;*X2Gi z={#Mi?V-7JoiTbTIk-Oc`S(#&BXbvp`!b%1u^$yb&u3JDRe^Sbm&i@*}W9}mszd&ygu;6X4Po?<=cXz6Qy|49&F4&%C9KQ zb2meH4{!BET+HBiPsqSBMCiQ5Eu&N_#0jLZx6N*N#B5Y3G0v)kxEa77_&)Qd8lg`7 zBPDZN^{2{H@BLHMu9LAhyh=MUv6+40ahoGq-C88R!)eY%WUiRe-nZK_y{7Ajy{QNL z@!_2OV(MtYWu6G;3K$QcrF-z5^c{#ZhbvtSI85Lu8Qsq@WjY``ck3)80`&RE=RKHs$D=+ z+oNgjogdbyU_C=_70|)4;P7RZ72homRgRCgSsSc$&68VK(GwvHF4KI*tW*J>TCOAa zujJ<+8W^>?qA`dO)hvR%r(LODMU*omFm#M)X}NTRfM%IlNw$Wv>-D3M99a$_dk|L+Q^)vKUFI2?v6v{#XrKMTb&H6hX7e z>tP(ocT-VOsZ_jd{f2<8RT&4c?g5G$ea>#gA{uF2v!rBXBW|G&jLD6G*AY@)oh62p zw>2ib&bclVJSO`}JfcVzliUg+Lb9b+kGB(_o zZ})~)*GVajHWg>jj6cyft7yO7^mvlQn6FRVX*3GWjkNUTk^rXU%UdB|l<`*-LIT=~ zSAxLdmUxn^xb%x^$bS|N7dUT>?u_$wl;fV!Ef}vJ5#0gc5ns_ z4p$hcW-|55jyBUzmK<*~{Q4B36Os@DlUr67$7UH!9=NxceSEWwXJpTyOfZ@=bAqH- zt8b)}y70r|=jK0dBiZs(T%H%c;7qy792|9Si@msw)caph2gL51rIaNfb zjltt)7@4<{uz(Wl8SSiaMWYax^E;&-k9@nN=7MIhb@lX=Hr!6Akr4sSliQr~d+J;z z77T3T3VxSG(bv9N!CpZkfHhgYQ?oPLUs?xC=$g(w!4tXn7bC{N$JRpS*MIt)+~Gxa zS4~}ASbDf`J#qOO_OG=_c{c5p;EO%R%DLepb+#;PARV0-?TZ+=Q}jpAmPJ##9Pvbh z+$$52)$?4=`w>^kMynEejD-VdJW~Z;vKgDSwDdSJ){33ddjU%CNh%R3DJdNTk#q(& zR5ne5b#azh{}YC_;IQ#lL^OQbY)7v~2wB%xv=8manDVUw3HR>D(R zTU0vncj3S>0U{qR27)Jq!RyB@Rk6CjVMx zl=UG48v{2JhY-)>SvT2*p54g$9f-EOmbe?h*hy}rC|RdQM*ofL)d3PS5F}% z0hhM9O@Nh^b?`RkCy9+je3Ie7-Ahk1=i`VjT)420>{DxR*gzK;^^CAj4h9<98d zc<^;dsmFle@5X@CNT~LuLizI-s|DdS*UoGr?yW*-H_}lyI12?yfV~% zvXn*ndT-;7qn_Hgd zWagb5J^*inI|8IY7=PqLsgk1aw7*fq6|s8{k6VdvA&RN{BI}Ene6wA&J!69e2Tn)7 zD8+a+jNxDfOwxHmY)Mb+pzgjR^QI2FsTq|6*zZwA%6zADUO#M$c$9;~JGpbSz_ydc zXpLh5_53)tC8-Gtor0=mBITv#~jk3FBot80({yS4$h3&+S&pDM+VV~w9Qm5OPTCb zva@`9rGAdOFViy`&!d^K-s5uu6&NSGpXEEaHu!`;6w9(d9)hT!Mb66)( zi*{6WQ7zq?<7`1tyx+l^reR8Oyx8U``4SFVz($)TtEv8&S3WNcR_PD#j#krvf4Rs* zoFm@ls*ZZj_86Zn))BoY1SevAz0Eu`Fw6d(ybARVb32T=UNR;P@mGm3`*(Lp4{=Ax z@pNAFZ*J07NAcAHXFvGG3p!$}n-&&X1$<=fr7PIQ^_o|%3{ZW^sjVGtB3*{_ml$Tw zTcMkV9aigIQnEbJ;%kKEjhbyIzDJP7Uy0-R2}%#Y_4hQxEB?q&Oc87b7Q)yY z#TkP4Ce}I-=_*9sQET{Se=V_vub|{ij^H7-0JB^6;ztqIQQ+3z1MoihMGRMO^J*L) zS}Equ7f#w)q#Pgp8N6L*ONZ08PFPD9g=0}ptZUF5D|*FyNY^HrKdajxC}dK~P@)Gv z`T&q%%tuc1f1}ExCi_&uCt?>`*kz@m5h?cul{>SZl&-0%n4K(tahQ=0GdVed`2&7) z+XYm%K~SNKn}+CBY~)0I!{lk}ViDoDixc1FYG?OGwT7QEQttWB3TIhSBlRD?>z-`w zpk;c++dJxW;80V_A5jMCp#DPAPNg-Z)9#Jo;0kJMRV-%KHAu(f2V*C-p);=^EHwAl zNggI;#>$ca1wG=HQ;n|aE16<8*l?ND@<7qmVmGX?k_OFrMWorWuyXw^Oy2L_`CQt= z@07?UyIbmyi!*Q9**P@*5y-8<7T_1i>p&5ZklpC=E`7Z$2y^zk*?JWaGF zk;DA^f@kQFB6)>rLfDBu&dli6-iu+BXMh6@SBvBtR)16PiXi)X=#@Q zyI&|B(bY4~76j^uQ0>vO(-b%zoU3|9W&A*Uu-y!kjS0J1%hba|>b6>uzEb%8IT%s2 zSS1;UbUiIUIjB2{lwPSgqpPdSAOxO`k6QmUN|~RJZ3oq^UTpz1WlJSQGn!7SM75dp z_1}owKmu1SX~G<{EN6Uy6AK|_Vhtyc0B+e6iU^Cz0&;S_b?JMTK_{I}Uemn@$IiI4 zw9ziNxQ=_Asf!U2XIU$cJ;Bt`0(3C*{Fhmr=|;&em--%*`)-am6M|?L)A<`(p|TLc zzdprzb)zPbE2?g<(@BwTN6@GOIVm}Yu$W;k?E)&EVR?(#c4YSJ4gua{EYGHE#To)1 zu|#z6}b8Z`>>ztlg)Q0{A|zj>>Oq z#?qwoXxB^}6*a2-?6X?4v(`+eg*uEIfuqT0^B%4Bj zk@?{RWvO*%pUukz$xGB1?vN{Ea*En);|3?U^55@zCft;(IOy|fqboJslW{sK1(}`s zG5d33_sMUS!G$kT=r@w-D|T)+{)Jwq4VN7ojxV-$l-M$tPQkx0d@KMu2i`Tw0qG)R zf!ATlrCT<%u~BhkGQXl?khO;3#*bo4yn@>OJ26K!=1NZ?I$gg(R?EF&g~;7Cw*X}z zDH7*9+;k{}8u9tN&`5@_`s`FyPAEjR#V0a(ZrKO_Cx69QU;~YfKvqM5i{m53=O=WBMAyc18`~%_HAe4 z^5OfD1=r%4wQGdKh$WHxcadPLJEx`sho`D@-qE(if>(g==8$m~ko3y$;K zT|9syd}ZP%Z`7x~PD66*tX&#F%v9-U&Q`duPdf5;$cX6DQbytk)@(hIqx*TFU*>cI zjlkDco2j(KFBpy4Vq{}ZS$Extvy{=Yuwgfq+atG{kwjv#^J`tgY5{NRC!P+LUrhCm-FCguiG~od?KZ$Ey5dB4yi6##?F05BdnuM zSxe$8HU`AyQ@(-fmo9~@oOBL~`?$EcI0G$s5=7c91q>(X(3yXWbKAXvFBbh81~9>v zKo%*|CnQ>Da+~g);GWQ_4Qdf ziWJww=v>%&vRPoRA*feryc6I4-b4BB3HOO7D}%qz^S#-U@6gKcsC037$X3pk?Xz8) zo?Lq8A$d9P=jO^APnTP2`083%B1+JNMBbqf8mw3MtZef-kSR}|V9_{wE)5nxTn`vp zPhIs=-5F^;RuTG2orHd>Kgcrr@@3;0eoh@tYNlJuhNR(~WcrmM=(hE`_%Jd%58$_TbKs-)WD$J3nh4Z#1C*>?NDJR<`}i*S0fm5sK?i$4ej z69$$}3JwOhkv7CMfs&3tx2Kj-P8vJmKS5;i$o;nR6 z@cSEKG-pMA9pauZsF*xEa#7_l4$V$o-E~XcrXAbGMIRy8dN+%J4M~J$fw>1xAt9{x zqFUNKM{ugx=jHM-#}yaTICTWn_P;U-Th|c zO7;c^l3Z)(3xzW20X}BE+ZNEVd@Rkbc_1a#`hlQLxQzI_OTTs7J(@ zG__nxnT_GmC)#WH%zfKB+ z$s9phWAS!0wZvZ^oiVTrF`P62H(+rW(LWs?p_=i<#Ke@1a80pVV|iDvmKF1-&R;W( z+~Y|WJSL5G~~b$dTKV>_NSaUyf)Gqa^WR@ z{d6dwfT030FM>PJAA6tFKdoebAu4%y4qDHAnk!EYvIq*_y}Pgz{7B_6$ruisM5jK8 zK-lVL@Wu#FEui|v;=5KykL7hz51WTq+Ia^Uy?-R-&6PXqr0l7X9iJ%^J11|Mc=~ZY zRR{>KvifyCveoi@!EY%iE!9nyXw)(UNh*H5eqID;y?|u_($s5RMfvS^wxBmY0NTNl zKO_RUdi(nO*Zb3tzYbDtjG-%lc`%S`-LwjU+h`!!U3T)NH42a;t=fOd(c0MKhF*aN z7Y+29mfD_9*>c&0fSwvQx#W91=iA6K2I2|fd*k&*)18{?Oi!?trPwvffTXa- z#>U6@&(LhM#iXAc&Y`tHl`mAM5k{fYsc^eGWD!byjPk&}#+&?4)7ZUjy@#lmfO1>%WYcqMz z3#2j?%O8FUxvdxxPzz-^oiSfl%Sa*v112P}Tti^YMMyzG!O+$GsbO2MZeMr!UK633 zhDH+rg^x{4Zbh7qf_rHHZ0OSMJkz5#OODKig*&fay;8o$LV0?C6~I}6eqNi>9vf`R zaU4n?(499Nr1hI>9iiM#7rA>!=zoeHuw0rC``}FWF5x z&Ug@a%fQHfB>H~BpPYS)Jr*m{$az)N?nX8p!OKQZ?F2>sL`n-#U{uq9?}y zTMHV-Feb16F6eKB`M2=&NGN2^NKVL2JJ?ZDsW#Q!6SX=1KR@&LuSf(`>{yxPuO%UN zkm+0h3DE!FA00u5YzfSVoKpOql}a&of0qib=+D>wS^xe0{o3mk-vZ&KBmNUcIphC# z82y_^{IIhE>cPiK*l}1AefQeT#HT?s5UNPraKvF>A0Jka!6^;qp9&!s^%dz3XGen> zBf-~!feBFOH~F8z%eo|fRuLUpby8H;&S(6u@bl00n8`nk3=p__P;(Wu_x|{S21qEN zo+n9)#QM{&z;ogY_>P(IzvBIW7YZ3#{&-5h$bH@7Ml_U?)Q~P(&`0g9O#M0DPB{WD zIgX{U&6P3D-9YK0*!nw86<;TRsgPKql=MCgC}=kyO2(e)}+#zAn18!Iz&Lw?n+ zG#c{nbSU_HB~nPp{1{yjGIZp-92uj~Us2aEFMf8(pub%#ZV!iB1^ zdgE-D{E1h56FhsrrKP(r&w0_yi@Xf07W~9I)&n-2npxCoyZLX$*^vF%aCq>W?lc-k zMyLBoUmLWc#oq--P{3|eR4F;Pcg^k_j;-dO@bN!e2L5Tm$Jz5T(5-R=f>!|6*j|_0 zT6x`3=kVu08wP=c{`U&Pelo|<&`+W<@vtx8c_LW>=hyys1%Lj4Nm2pt32=+giaOr7 z5ih4*4~5RZN54V}0lEtNj2arDW08`g0bT`jL_|af`7vMTsP~TXNW%W8rDIrB4E>^qT-nRrFqzV`}%?Yr?E`0z0yMC5?P)?hJ zz$n&~6+LZX*XZ!`)O3|?d+<`)B zSAUKjivx2x)0ce@{O5jqi)7zdQNg(sMNM)|DoXkPu1B%jII-{U8SBmzoF*Y=0Rf`m z;NS{4qSulHZxOi7NER&EizKblC4ed+OG-*QxJ0=92&%*w=I^64BN=^m&8g3d4wZun zk*5P2`^^7)zoX%!ao|Xyu3n9%Q8Jo1crrVXb@uA2041NnpM#e&?O~${UvBiPqz<0xWAJ1_Db?-~FVC zOo?5!J7Ta|l>HK|WwA~8zNC3=sF#ZEZLGtn^7&8TdS9ThF1c}-rfqILIq@gF!CZu? z=VoZ1@$=#bUCPbGYK^hXO{~yUqAk z@y4yB2(v69q}i>p3%L+}AKyy49zG^QW8;Lq?rI$bC88yT+psgFdIrGvpt$-i3 zKva?&Ptec^Y&|6iU&d@b*Zi+&a9RvLvoE?kLOF0pGT_slV3qfKmx`zyK3D2$2w84W zvwhNPj(CSx|7|pb?4#+2-^P|&*#_S$T%`sCeb_$Jp80DDB1ukY~Fl+WLM5q1{K zfwM@r1{IV3+i!D5ib4wKQ*MD%$m?N(r7>S@)vqsU7#DPGj-#C z^#078| zRxVGG3Q2lMa%esepNVk_{ALvSB;c+-{|(_V!-cSPTUa`2oB~d@?|q`WR0uRmCr7W{ z7I@^3$1txL{a^Nxy^?`bRV$CX_`*`OWJlNXXIrvu;r(@fy^VQp{f`dv;(XL|(pWp& zT>g^TUp3X${gVoABsJ99$B$}TgyyJf;hzK42Z8Ai$2j8;4GqER3-8{c`=G5D?AfaY z57aWU@8Gq(DsX95?^JMI6z^c5GP6&QjpBCEvp9+SEZntu%P`bx!^|B~^ltUM9#3aw z%ez}|(3$rT2C9xpp_1>O8eBj(tLQJLANXQV9UEolrf<`HxSanYjdYpiMy&v8*y`%} z(XXHQo)mu{>r19%${Yj-=3eR zNOp*}6Pk`VW+}%C6d*dG5A-DyX009DxLvU@}-QA&a3GNWw$z9oJ zpYw3pK=KSRwW2CgW1stgJCsa3ZsvF;q6IFh|&PFhWt~*_G!_H}6RvNuT z;|exD)647X(`JtQzV7L$gL5gQ$UW>tEAzk5PT+tQOLQcs8t;wImt*nOtCmye#9Wvw zmr1PIAkXyqRlY$5VU6+v;UPbE+eR)00*Lyf{m52jlU3WEk*E*t{Y{{2PI1EUr^O4XB27#i6h(+NtJs)}}@|VJUy@8_x>R)^OiJqw_CT zfh6~L7PHE!FS5pN`v)f&@F5eW?ukx8j=Q6HKl6nP^73;w5;H3O>;&q^onJs$rCZ5G@Q{(Pd*J4X|roU@m zV;Q6wkm1Q}w6}&Kjf6-}rNinlWipyM`yw=VcG)@)ZR0?8AIp5@sqXVLQ-h#^XPA(6 zE;8hnxn~s7eu7p|^W_LF;N{S`-uu?x=lRHR0O%ZwEb1&Kh(DW~ZvvLvp6=DBJyyEE z``xx)YZHUzksPJX--$O*&xwtt5scUKtlZZ?&^KWtz(ukx9&2doOOm^2rDdGtQQu0cKH-agqk}f)Q+VxjvP{>)gE_S ze}_f6hb)>AKeln}oy)GEHGQTpW|(1n8b)C3(e`iX)zMZgVX|=3Zky8rpJelKKQK_Pl%}hMIp?FG|6kehlb*%dzA_G+)T6Ipa{%8Q)~WnZ$dxsty?? z57J>|9$OcL+!XGV@-MNgW4rlOXiZ<%;#cg2JN@a72L_AGBTi>c>$=sMnLrM6ry7Pm zyzDO^U}6c-mI-D$0WSCG9G!I?YAq6 zCh^0-*9kZnmlqJKN-D$tZ*I}`zLEf`c(%3NRp%LdzU0J3lTDkQLszB=w7gKnIX7>8 zv+2XQx_LNT;6-aaI$m4MB0FzumZ)Ruk_%F6R~ni8U=Y7nvV((@nh{WY3=652r2Tei zQ`=2g_(^%kcxYvh(kd6cr;dR~v{GxE2GXX#fLcCYu#LtfQ`U&kbMR?$|V z`8n%4>tctpd8CXzic;k&_2uf0`g()2-`twJ{Fn03NdA4bM4m+F!AdKiBi1UHd}O$& zBN8%DY9iiksh8093BBg6LXDqc64tJj)q@N5@)o(MS-_EB7^fP&`UHN{z>&=`XWFuh zxDeB9Gjt}wa5bx5Z|{sXONq>TtM3%3?ws8p+79Jj6r^D}gbkF$Hr$CU7 z&Qdhtqvf=D%e~WJ%hrv&asUJbYrl>pvJ7ZtYO7)I~qqtKih2R!k88WF-wVL54&KT z0WB(yUijp)+g>YMdv-{F!x-XM$1zVKV(oA)iU~JHG#_%N-9@?Q-#X;aw4naTIG?nm zSp&{GRp%?f@t~;?NHou^V~g4GO5Nch=d^(z7l17GWg$bT_E>E| z+4D8!2K0lfvW?abE%oK+CUom?#NNlO-z~0_I-bl<&iwzhoDH%z&p+pePYn3uG<|7+ zpu6i5o<2oeyjp%>G7xz$jU9psQ)@JFc(Bjy#8;qnW|YGI<{P-Y^I?qN@t5)k{H57a zD~oKPxTU`Nr_!C(Ucu!tsI%dGxK0a)NXd{RC~}wn19y;-PkQw)+FyM~*Us!j++I8T zonbP>Aij2SAoTnOCW!UXiQoddAdNUeJSdi73O(`1mwmYlM-S_xZS>cV!J9~}L;wDm z+-tNFh(ePOMw0Nj6@{C$3qzC{8_$_*Ke3uyUjRic!*@8yR2b=bw6 z==V6F@K$V@BPP535-p-SAAhfyFYt}`l3#cz$)&w3nT1T^R+#EQu0gOk6OB%*`uCOW z`b0@DB66ju)d8jLishwmR%wK~f#2tX`5JqcU7-se^Xap!aVEV##Gcy#Qo=nk`U0qb zr4AS{f1XCfa^7a!9tAiy^lglH@XzBeKb%e?GudI$jp;aDBwL3b zOVH3A8dWV;`i@AJ>D0fQmnRM)JAbY;nIWFN_sU{VuFdRcZ5e*>_%SgrM6OfzlquvY zuK|tepNsj~hlD!!<3GiWGVD+fVTSB-S1#G(Td3-_RGErhtrvX1nsBO>o1*VXLioKf zqaz{RQuJ+Wtzt1RJxkyBj?FI2icqn2aHK2VjIYIatd%&3Z7#*#F-@L{LNjxI0tGJ0 z(Mb$O8SfFx>UZX|1gnNmXNuI!2cqSg-iM?N z7f2Sr2iY#1;Z+^Ht0s1&fg+uDu=cuMGeqjLe)_4QAWpXNtTW3$_vyF)D^N1hZM@UgOGOv{IZccKo%E9(w}sMW{Z9RE%;9>2HJt8c;1vJEMD~Ts zM0cIYtE(UyAl*`nhqS*PZ|ZRR)P5M=(`?CaYI@JPV>US9=2M(VK|mtiS9vF~pu4j- zQ|xsPwW#CWv9%4#hYoBuWKLQAq!ec!^F=`~{g2RN5-%WivQ=D;6_Vw1Of$xZ{0Ccv znT^Q|A%WcJL!|OUd(zx!oLb=@-@~5tVg$4cGrLW&*(*DXc!%gT@*gmJMV@>;n0zHT z7)<>digJx;Tkw=Rgy!Xgp+$tpMagdNfvegD_=* z7tP7(Bq=yp9EF4%5gBCvbxte#Nq!H<*mj_dRjqO)X_KKVFv+Gw<@o`_532H+@qjtE zxf%af9_6RINjla)GrCuZy#fJMnq!VEG+y`;B*D89UY#LHheXCBvsfr^!yO49_@jAX zBgdr`VG(h$(d>uV&l7~O#$slv{q~eOY6-g^%$Bd6kwI(X^y~&f*<`UE1`Sx_jDLBlg*U@?1NRM>0K)D);He$`@{C%8x-VM;Kr&6aAOA(2-;o9cpU4~9^-!$C%Vm=h_& zp+-a3O(A+4mdV#G-0n6|NPy6HQ*(T5c@3t@K$pl|} zEB;9%urPMlxD}4Cg1EQDFU+-cg`YxcG*jW=ZQIxg8a z*kg4%o+m;(JhA!Rn>h}VqVu~>=!uk}9h20bj9{bYd;gGBvb%)_If-ejn($v@A?DRG z_XxZ&989208NbxZE5KJ3)lONJrDy-s>5+#Fhv7Ig>?6hZNcZF+CDG}5?7s=K=R%Gu z=RiE~+1#~N`Q=h{`=HrqNBF!^U=PD-@$*2BgT`+CDgArCq(D}$@XiUp8}=2S%Q15+ zV7y57SFxDRi`gS)1(m9XhN{R;y>7I>F1Odev^^RdK{nDoLm+R~d`zE~_xU;IS3Ain zAj0`A&0lwi@3o!lLXDLm*9pz_IeY`^hevm#z8v0UA$@tsp3y<%^@7XKZYezrupKH- zr41%{+$q^yu9^26*yQ?QY(ZDYtiYy}KX}Y$-ee#kh9Aa`TDHpS#A}CpBUsMdmyu_2 z37@Xg^ciV%>S!a@>UmeNsOWN*bdG2#=-P&Rqw`JXqkc$3_F*SenSS%z8EVA=)q4$3 zCY>s5Z}sOVy#40GqoS~c8Xc#MqKnDJaqUQj7TMP|6pn~Jod1lj0*XGyMkO+T;K!mE z7MpB67pq*dcs$hI2lGKe5Rl!;Zf?xmh##>BwW>?W&2Zj6_f)Rt&@W%!*c(<$tjmsg z<6=I9Y~?Qugyy{FMv$za~U|S4{c%! zCarkQuJsef5~ZvaQ|Q03!*0EpYBYjd{z^W@46Y^JZcr5vpSSL;MQb=RmOGX^9NbHWG6$nPGtio@d-8RI zv^POZu^~FXc6A`t;NaI7}_39iT|V zD>n0Gi8y>~)KRxs6Hl*yD74w}tk(z}{c34Pqw0SQ7|@wrS3JP3wBXJ?~20^&St!u0y{w?})Lna10NC zYOVzo$m;@U&N1^9h9oK}9}PW9LEyc`3b~2#`D5sH>QT9)^x~3Ug+@}6|V$S?YYYamo;Phcg;EG=6~j~pj%xSTRpA|hRO`$ zoaUVu$I#9uD(IF&;GlJNMVfN?ZE*0sDThZhS|}X50j>|kGKa@3bMvLAm=`kzBKiO6-7(;8@7TgjYtXWzzA<^I9MaOvqZXBnLEYi3 zv$VApx2q$94>eU^rqiTTRQ#_OfL16&H%tR!$Q9PO*8*6fZ`+dq%EZ~CY>a13kT$iV z^(`K@aGm#&PYc^_CyD!w`O9C|rvL*PQYKF%F2Z6dOWik#$s(mDT=uSKj9@JQAaA+V>YxSiwsAl`|CWaRZ3q_-;hy<8~JsD0DZzv zQ~I~_9}WY?XC-u5Do05NCaTVk!(xTH;Maq5Z0ep$rV6_2PAN0TflkD&lyF=5{bBN!jhA2k8QaRp7TuvP&}JtyQrBX!+hp&z zp*Vls&gRt5G@&$qIeg6`rFKTsZ04mC2C`06q(7={MccNlKLRuUwidhIB%rTbK`E>Q z6Yy(I*ZPal zSO_s>6vgf71QE9G<`X(HXp3m(PGM|tO18weji@EBw1Q#E*=NW<$i=Mb_&5(#zwtM~5}T3k^S82h z!xv4BKYP3teSb|&PuC>uc&31kZal_Y>e;w;AD4^uFuFAAD5ABvD@X%!Rs zDHfcA-o>eS$B0NuA~-uckHU6bBv$;I@PAe->{R9xG zC{2F<>Lp#GufMiE%6())+8yI;w%MfAKf)_)`n;8bmf-^XI+jhgKVYHCamYGqe55x4 z%b(ek@lGtWN1UJ0-A)9Fow+J-1x1oe;|Fp_*`9+T_v=z-#;#QULb%z%mSF3yG-lqz zq|~;2v5=HJndI04bGp^9vy5)XzDI&;R_5ClWA`B>At{;oh8Yw+a<*>=t19DY{r{~Z zU;=n3&_G1ca^G9$QbXAIu)@xBnSML>s4^-FDsl5%WLxlQ2Os|TN_EWXz~-NX7rZ{3i+e1!Gd;c)P@`pPg8KE{g66Ja^u^y z+k{Q=pz|gOJ8PPgEFoVJi_M%_Od{H!fv)B&CPB%mwVs0Sxa2bNRhxUa-6Q1}F2ODT zL*~3b1b~rj9mg$G8!mQ9g_o{NQdPBp`w#b{BanEqm7iqLouAbFHL_1C^?R&$OhkA!^e^3M4-|LFF!b6u`itLT&va%=a3JYL1|J3 zMeG_oHw(LV#p-AT*956WjUrOb^9UPrs8_e)l;Dte4*fP~!&BB>oT~6!Y zZu3<-KvsF|Bx+t61`Pl_`uVJy5?)_%&j5`7^CUJ(VT{WERAbyMd~Y0}od73<)qFNaG$_%#9Fvq9bvWZ6#x@ z&3`m~7qR&=`jTQVPg8479I9XtAV#EgG-%tMQBPrON-~BXLD-+Kch*|Lfavq`;0Qu=U z)w>?xldGk2((YZ&mpReYmudeAuo8Vx!xOScI@N2!)A8kX&bU0V|J&SVFil4t*x4w# zTh~KR3#y0&BEfFn`uv-yrHu1T-|NOdUxugi5UQE4MT|Y_sx&drvp&tXzeg8R+kHQ_ zE@`L#N^QTS7i#8kpsiJ4Fr4(+pQS15pD86EyJHC`$zYaj@GAP-oe0@zh< zBOs=NDwu2M$qxX;dHhxm*82szKVa;8!O@Ks+%ZKQ%e|b+N``3vd!={maNa5C<&SA3cUom%%)8SBOmC zc{-wHirWwQo2WMETT^U)DvTd>+8Ed_#fhr6fdd#WoizN&|A}umryXou95F zXM%nIw%PLQ;feP{y7e(Xs0@-fE4hM<>G$|wg6~t%w5!nMKxYS?Y~?WgY%_!}aE1c|LszP05`8&|E(p!0hLQdMI`_rRfzro zwD@0~E1+G1w0`-)pp3&tOilm8&6(EsgXd7A{_5!Qqv^obY8{3Qj9=*!)(GMG&kx3Z zK>N(IQ9N0)(1Kkvw|MO%h30Dc3+P6rvRy)lqmc(pW1K7hi2m499;+|lO8~fbq#l2* zZZl~)_wa_%_!24{o((tjx}PRzR46IUqjp&?oYISQZVVC~2lSPNwXh1OTOw(6l`omv zlQwKlz{G*(jZ)VSc)aFU5=W}6M&n+D7n{?+MQK4wB!M9WM7Gs8f{}=bCw+Uw|K$z? z+LcJ19NFT3yqVc-JNvTdcOk zzR22Q)2S~$$hL|#od1-!xJ8>RE}rs5OYRhqSW}hnLZN*=rntEt&eUMNPl_7Qz7*vmF;eN_RGFbuI}-}Fa{$v zIaJCMn~}al+{X3WQM*n%vyxoh9eBn>Vg5$LduUdxc;|s`AiiKH32FmLyl+Oa@^q_7 z{`uJkGB1nd>@)mXW@K*ITEIp%Gpjr)Nq1MVhe5}tjA^cVx=g51q^7{ahLY3Z+J(xF z$$gfbvtcwb<$Il^w?WIpLwLXb{|1Nu{_@ThE@4xiAD}r$U#64VuP7mnjXa0jc_P5y z^{FS7;vZBC`nKEasb*=j2&2dB!U^nmeJ*pCB#Ts8NNEv!VDYBk<=HBry2=*JDF0|S>WoEZ@p0Vav|=tc{Vr#) z7$Y`a`CeM_%23$<-ck7fM(?gDVS#|qff5bHj`Z0_cH;9gw%hJ;s4zw z(xgal0ShPuz+Zq!R#sNtN;OewfBk8fd%ut>!xVr<_S@jR&W}^taQGZoHb3|LIzrQA_)iOAZrEcQMoW(^P}%}K9xnTt z#`Yrm<~U*mtBM7v+5&M_gQ&A2YdNNH-Y&rA^h**78^Ke@M0&tcMOJ8qu4E9CgTC#@ z<~h{>s9OIPwD_K<(G%wG%YIYMIj}H=*EICyRLl~x!%hX@ULYtld0Fp(3dYs9BH5Y$ z2AwD}W}SwQZVxN+_ZiKf)h*n4*ZFm#50e|L_3K2N5zRqYyTq=A!La|{5hCm4lMfNC zKTGW0RV-XDBaOzsGFot&D@mU+7>^PO`yM8j)DT5OIPN2wevq6ZajP56h&5~lFlLsp zeS;#U^TD^QR;?S<5>=PY>{OAGpV5}JxahpXv1t%Q0^x1bHg)^>JO*5 z<$H~C;lp?TZF$NMS+8U)W;5gkclm6R9n~TR(84J;|3@1C_u=RiC0#@8P7TuGbaJ<` zDgB<07hT6mLq94NdYE>UZAq@m*u!G+Y*5SN9(pIRJjAR_W^Kuvm-}&d$yz zKYo@7wI}9xLjf#JvPjt-*#NZ57{l$}CoJ|B>Do`lFiKsJo<7n>7kW8-`zUF1#3dj3 zH8CZ;|AxQQg@|$rc&FkqMOpuSB}l2>Qe4kNlmR2UNk178lM6@r)TUPiH#S8~L+$`b zU9VBYVsyY1oGZNMW+GgGLJ+Y+{_;Ux(BFQ^(dZe9%X@)$_hUHzg{`Et)M-K-&G_X0 za)w*3Y^esHynx9`$1$%9Wbpl&0BpL{h(*!(T`5AY)5x_#F_{ZlT&!DFBBsC)u_`46Zt?I(xb-Or!?) zzqcifM5i&~hj)<)w8)h$P58W_WFuhDMhBogu?mX#yj~}xi6O3G(J>T5ty$bfT^#i5 z5ac;W_wP!I=ab>EVEzWkfqGK4?zO2Z(QC%I)F5lK5LRzJ@5FBtUQ`sO!z+mk{uFF4 z0Y1Z6GyGdD9v(=l@xC7H$iEgbX7)d+d#13u{JYLgF$LDSU zPo+!WimGNhk`o_5q(!d&YC|5z5h?%ILHs6VAru~^cRZ~!UX2kWmTGBvzrG$P_U8^} zb&T%I0VV?9N!ySYWs!U~)~w|NFlRkbWO7mlD)1Pj3#P z4?H=TcQQGkKyjd!C!o7I>qYkDS5e&}e3Q!Y560)wWt+#N(6Drfw$aoY#FHD!DJ#4? zIkSA7j+C_y7ex@vZjo?#41P(DztNY+JLSEpf|SGZUmA3dw^8m&&%DQkBsW;Hnk&g# zQO6C$)FTisw4p9Hob@}YwN^FSc2nrNoVem%ZIC$q>cyi7SI!~~WJ2Vj}-b`b2>@9-KG;wk5g!>6IqxZ5~3^I+% zZL&B4s}mIfAZpx7KbF=(%CVI8+5^Kbn5dZdjj;nWM?AsrX6El?NCa6OpwT>ym%D(z)I zJlVvx8?nXE##FrdZQBHFWC!VMO_R;IPSbhIx7b(mc%CMRInY`#4$@ic#6UoPP9OV( zK+s>U!}({Bf;J94$1SOB4Umf5c(&=4j9B|Wf>X3E{_w2v!KIa(A=#f?65Y+aF}b>S zV?T3Gxu{j*sY6>6=Bzbh{*U9k0rkg2%cUPIPP>FY%qLv(xN9#>wSwrRJrA|4>&PdJ=1p?JrG`CLOG1xA z3Qb&BE{x_p6+Px-V6$t?H#%;{t}Ta+<<}+<9_}%Rr7NG7%=)J3g5tQ%_ATw2I@#}! z>#^%4iC;Xj?qc@hA;`DJrV(O_axq>89M-(p!ff8Nl)8@{Yh1a%s7`Oh@#x8H8{0aOv!rxEt3m_3P>a%lW~wLyi7(JD zt$;wSp{f;|SbY3T+98Qv_=|?Rx@YG(x(DR~)#lbAW{GrmxgbNe@dJE7DORV*+UZN72SZ(~xkYa4E;o}4;JjA^tzVvA@kKP_He z9J)2iMzV)E6}Yku4OUqJ3zri6XdSycefmMSA2?PzRY{$@O>8MTb?Mz~5K!XkidJc{ zHirJS%-B&ipeKAkPcHp)(Vi~PS*#5HWUX$YXvdSbd#2h>$ILEwPRlU&WWORA5v(;G^H__bc5Vy3_@?ifZ-&E@ z$toN3so`-wzG>n_9RI?7)+uTKFEPm&{y3?OpT{Sp%2B$P{-SquwK!e&A&(ymX6_iH zq1gjMm!G~x{#Bwzqb1>!L6wPH4Fsa_R|b5ww+(bs70-Znd-AUySpmxmP({}q-4|z8 zMMOmG*5?_&g@M_6&>S{gm#qQWWwivZre-h74r>rlPk?pC4ALv|TiUquc0JaS`vvK5 zBPB*gX(xw=OY4>%yr3XW8_IgUOMQt11F(GBz+>?2q~RmuM3B#(AdM9SuSq9@{cI^! z7Lr#fG9q7RY313T;7MjQz~BSuv_^@(Jxqg*t1PigHEP^EtBNZ2Q*@e+GUnQX-Hyf* z;UDCjyl|MGhGC~-b!%JSb!^vSCj2Y*mHtTpH;QJ&%`nDsDS?~Xk~3KAp^Ws?7ut^h z%j+7WWXVDwF`?LVISrS&qN#nooDO6a6uAh-%V7QOkw5n|>YnEi!e*CzVz0K&}>c21FieD6gu=aa1A zXd8wjyEP(*Ccdn`KhbD;7vQ>*Ohx{a`pA!oI;4?qB^RGP>xoH8ZGAU)-6TZB9SV;g21W{=S_Tv0i+(nS!#~IUE>ED_86!%QBYMPt+J% zi%*x>DdvNh+i2hD&wHirUTd+;%q8)J(vLiLK;;h1B~^zx#aQ>e+Ho)3HLd1C9xHSA zq@)I}km{`KQ3j+C94K@B7HZjoKqB<)^OV2p3!Y4@4Ih{0NB2tI*Q)>}td5UQ>%cy(r6}hu+T<+t zU1ljBc)GZEW3rr!_(#17+is4Fg^k}FFR_U315>1?VqW_0zv_O=2JTGUmM1F~Fkem=;bZ0CGZX2@yyPX)8 z8(Rxpq-`g}Z8=P3Aw%WjM3CrSYglt3axB+&@e4=^)x2<%!u$=DiD}3w=$|=>ah5tG zi{+Vv-C>pwIjPk2UN_c!~p@p1D6 z6)@9fFVFwQQmf?riaidc*nNf2_3|lAUX(J}-EmI(y-~}_10a;-`U-;^Z(Ef8av(jQ z89#*DT4~kvuCrJ4FnF-Y1LKcNSOOfJH|xmnxPKAuF{sG!P>Z&OKSZUGw)~a#0Mq4q zFgC|5G`qYU1{D?6Qg!)njj{o>CB6&Aq&1rt--rZPz|Bgdfk_HP-&wZ?#83mQ8i3xp zN4=aA!F7adV#N81q=^X+6e&!e4aya*d&N4oh@PtLR{^(|@dYYlP*-X6fmM{QiD z-h8Z?;VUql(DzCruCP4Ya6^(28`@{iMP^p331z*ag^e!cKb?g)Se!KYW_i-OP4VR? zMD^@9Ip0Li3@%Z^*MOkz+z;OeeR+JixIQ0RvzK{U17TA$WS<#b1on_Xex;ZpLG;=T z<*0yq3o!ELj{Np5n+mSe^t)gv;2`iN@-I}E(Qh1F{#V0iDrIuS@RYX2nbR}6scJ5w zAf5B+z3e)=j!kH=(|TaFty))%yYTT{uK1CaSbZrgm%w+-_rcl7a7AkeupCrm0hSa% z^Y(^2##0>c6B??B0Q5Cs3sU_Ua(d*iSMfzOb|lALD{;k^qcojoT%#ez%Gv-H-aqR7 z!Sg1l(d83~?{MkCvYh{|jR26MyPNP}{7vml5KqMNUh*p_(iR-{&n7^-@VO8FqiP(~s&r6DF$Jw8( zqZTQ7H1X{1(d+}UW2l2Cy|b-w|IQw47Jv1*O7LYTFFA{fKNgzEu=;hqzIrg=0e2Tp za;P7HYojs8BvGP9uu23fg#iHRQf+O?O7f(BnA;FUg|jKN{_n}jyo)|Za#HreGj zI;dLA;RLAHTU7LkP#l&zb-R)%8dkx~xw0$Q+JCrVXftTg6KpmOni(#8*GL-vwQ~B~N_rgRCtaaetmUhNKwf8?Zf_A$8h@*<9$wAPV!_SP4P^V{ zEJ-B|Ml^aPEXCi3E2i$ih)Eq{XI93bmiX_|eY4s4Yq-=n$YzMKZww2`N>^VOnZ<0P znGEEAS=yja+Et?_VwRIq%+{7*F-TI(;wz{>wzNfs;#eEzOIaHd#GHEKTNR`4pJd1( zA(zRDddj+nUcb~nI4pl0kra&VOAR=-sAh5ba;EaJK2s>)P)H^k#%J0+;1@pP6-q$! zv&Zj%6EE$#ko9gyi((uDA#KPai}kcdvJfN6@LKOTRyAKrrfTYqLk9))>E}=MFHWQP z%0i4vV~)b;sZOn<#_&1)FGUwog|(RA?^zx^iRRH3lQ>zwgi1#BPu@~=KM3x`jnu!t ziLR-9d;yITt*$b`R!E?aW_$q)Zc2_wUVIS%+eu87e9W`=AzBKJ)9M&KM;SNt#Dt7C zdu0Cgpct$M2Gmag57775b-5J?bo^^ZaYe{0Q1i{`wfvmV;cF?)g9YKMcKJr#omYL+ zxy9P{M81*Le`Xjva%Jj`4ySOooIT791xm}*Y%y322FrMr{TzMomR)YU3VqJn>N)OC z(^Nt~9>4Id0=iO*wN0t{>A$+tKzy_hl!cY~NNL!8JtlOcWrO4EVTVhSoDswCkJmd$ zq=H>t$gN9GXhtVQo9(+H>fsUuRWHyj71j5VqOfflBAE3r%Gv_^(8|pL`*tCWA7KaH zcFZ5h?gB6~^&SRVgcqYsBXB7UBN9zcugSZT9Q#ug>;^;I@SI(rbainS!r~79Osw6% z5?3kkf3gt`gpZ`f2U!+nXV@5rYI_Cf%V;ts78H8O=>O6vTp&S|JL^{LKE#?p+DGz7 zG{{f1@n>Pqt3?^F)T5JT&7Q@=#E@m(zqu8v;qN)FZegfwIj)}2ZDc=dJ-ci_voK#v zxlP&gU8`225KToS=YNr}G=vvAlz{7YXeRk&M-_s1D=@#~EwlQi9ozNfE>f*cuei&z z^Cr)mE$SsbgpHHSt>Cv8^`~jcm41-9z7T(#_ig`4!+H#0L?^=YXF&MRy;2n5X2k0z@UHbqh{_3JxQ#rU}q6xIYK`cAHR?Y=iZ^j8gX(24?pL7IVAL$e&nt6)CmiHmVowpooy<+|0y zLG#U&`0dzGaciZesx8~6VU8luo?}lvxv!iw2iOenF=5jrS6@&&rOu(}F!bT}=SLvs z_v_H^Qu*fwNn(*`eR&=D%Gn+tN{%RowouRgkI`+4SA3X@#kCo;f zzHipXS$DF&t_Lr7xCA1E5foZP3VA#e6BjL0AFbtV>KK1DH|Wd zVUq7vr)G7POG*piRg)xhJmp(vCRH#r9Z7B47xrP>W!6Q_7p z@>@|451vnD0?1csbJ7Lf_MqI9gzlr{R5*SBD-1EZS7`uf>4 z>E~5=F8(r=`l(*nUDZ(RfR_&=f^oV`L`G_MO*V&cOnTFne2}%fp?1tQql(Okyl-TY z;U^f~l%_il2}#L%ZM&I1i3jlAPnx_Gtgh0!hy~|PH5ts^BX`tI_MA`mXqpx8+DXbQpXs*crKAJ|~jL@WT z=R;2OIFIkh@I?GGAn*Od{DSLq9UIn_ON`507L%PeUHzvFA2}BT0@Kc6X#E(GzFfKOXuIHh}0pPnuhx~G@ZMdWg|EI%KC)aOb!`>-(7AdW@#*ZJ-31sI9g zW@%mTz+@A)9E!E{+?-p%*a;^ucA7To5K-!SHc!hc@_&g~1-&JF`P(uXx_g_)_4j`ocJ zt#+&Lef(mAvfK=!mF6+P1UU7}=kb~s;{#~Y3%a-XA4}}sN3nx~owtPLoXv@opFg_P zUq6}O_pW~U6KwUHNodt3aI4BnDOzHEIdJQZY?a2SiJB|*u8GIQJzVfT^&2U~bSCz2 zsjSK*DE=@$1A7Kn0@gO(uyqp3GgE2x+u%HUElOHW4I7fYJ-!{Fn2vhoy&bG~; zLnW{9WJ2K}Pnn9Y7>^iAmu`IO+yOn-d`;unC!EtZ9S$mLb7^Z^!HFqdx@+6B*L!v( z6r}yf)6e#2nUEQR>Cp$)sg8(8Rr&|+8u!p-c`=DJT895zll2oO!CtcEa+p< zLGoY)eltCF&)Q{6J;OvR8a{-KR-*NFa@)k``U9H}8?75(U^y&OgL2`H>Bf$ci^0lYpX1DiV$Q-6dz{h4 zlbd?dJKt~~%c3c{E=$eMxiBj!9Gi2l*WwcuLH;3Z3iMdA2iowVL=V=b^Ue9TW*zsH zqDjz)xFPv=j{_`Z&*W88S)nPHgq;-n6q*W2dIPtyIzxnJW>xiJ23NeLJb5vdkdyX; z9GZ!roZd1Hz8iDB6m|E5P(57}X&W_gf2_9U)2xO{EXB3vqFu+fjDk(knz6K!GU~X( zginb}R8i@*--1!>L?YMnlb9hr^A;#bNQq9EOv%AB{MH*TU?vplTwbxp2^FuxR~0t@ z+n#Dl_S}-`YHzm7d=73h)udp&(T=Grz0p0<@~Nwbj)Qd0kzty}4GKS_Vb7vl`Nl3% zP-PqDwKUtp?ZT4OvSQi6V0`O^J4YS2xYjMT!mcBQxaCJME$n zCt+dGfo0){`TsEYj?s06Yrl7cHfn6!w$aA6ZL?9+*tXFmjcr>iMq}G%laG^BG<3t+X^8Z7-cqBKc&X zYh7H3`vL=&eo&jFoT}FLGe<#+=p01PugF@u9rbk?!>dJwWXrvt+!5>eB+HUw^pVEK z0O1HvqYq{3xJIcs3q;ys7PSRX5{j)g0u_jud71fq&k#Z=l!E;^!#3h^XIz@bQn?Qt z{3>;a70(X2Q6bLpxgu0^QrC2oK`k!ODA&uLhDux-KN&~8WaAA(&tc-WrDce?R=6s8 z>a=_8vjiL?oCWrjEmB|Wax2wzo3w=cEaAp8Y1qAFiuq-ZM0pWrE?alD>PvJ{5Q&S_ z9(*^uA$h(Em7}bA$cI@=w2B|&BqFLtk6fQwU)3M!ar|L+alQB<9_C7}`|K6Lr|k=b zm%V8>9s+*zKN5Z>%!OF59Ia?2nSDnG9asuyx%)76xGB6%@Re0KS_$YV> zg5#HXU^rYUa!BVRT86$>lxfR=7&t}*{E+wPkf52uEcomkg_O%B+@c^*4TuAV4aFXIisZHXqDyy! zzgOMIs7_uwU1zXST~0UF;~|_4Z@nCF`6vpUG>F1^wMQ|I+XvS62x`JvbC%5AR+U_pIrK72rm#ViUVF6Q)cjCC;|y@Hzc_4 zxtg64;zk{g<4H$`KnYw|mAX(fs=sq-7cikr-m2mm1;VuCKe= zp{#S2j(lYq@ldGEW_Z(mgTg?EymjY3k(_;_69b~vlTrKnblIENE26&Du2V&1eOo*8 zO>VZK^_v~VrVMyi7bmV<-2HDP$gmI(g_9|eWfChx<34zgTv@I?v{6Bt^5i!-nx@WB zais$|@!|@9<9Yz?h!Bp>tQuDlg1>>n*dd%hJLnMZY@?=jnbWu7RU<7`AUOgm4Mcv} zGq+p-33f?b34@{<9W(PfqJ1?An|-Q+&eV8`wPWrI=kQ}}#ud=as6U!&u=AF(`zXX> zt;+VL9$G%^RE#pS$x+#y27id_yxYb@v1+#x78XV(GYR`k-|P57DyNo8o>TnvvA;Cg z1OXN&;3(qT6w{g8RCGEqn)PF<=Oo$*Y{u&2G^{-HQINaak;)7`O0??AiNpAHk;6J6 zzN3M4xxv`%sBS*ev~lhsYT#0%f>uWvRYh-tdam6zyL)N1pk2giNJq(cs%KsBU$fG# zu%be<^#xQ?j3Bh!P`bBM-fsnIGHP6SuLd_pSJ3wG#BskR^ct|6#KZg^NJ`RS1ma3{G&3g?=t4$a;tQ@@T)fQ-1s~K|WbEAP-2RLeGDzt-Ns)q9(Toz-ti&S`hQ?ScHyk$Yt zV{vo7S~oWs|J9C6U=*8MkXke~R?+TL5wRUFu9#jTw+^3l_}|;oH>_Jzrk5KMXH;JW z&Fm;hb@6B2hXfE7x|s@aI*V18;qVgk2Riorr>*Or96zyv;PW}7+0nR{6JAK8T1E54 zh>q<^gPTd;yPL@6G|+X$d%zo;gl3zf{G1SaX4d3X4b!H_u2pHt`i*%!xL0NZi-`h^ zrzR6K2vA-i6)*E{SiJfQ&<~T2a{@9<%;;~OGTbs41#Kzs)fxK@0SKt=-{1+6G{sC! zJKKE(GVzkL?lFX>HdGHw-tf{-&m>=ep%Q1(N^fQ1LoJg<4}0J!+y0FBpo1ft>?CB| zSHkg~+aCsK0pLkLpkCU6CskD_m5-U#C8fD#%T)3MEu>4)Emg-r$_87!eiv7doX`m& zT_jc?cicQKl~E1~1@XH(e0tNOA53>ZCOFpB$BJ zz=KwjPQ}$UoLK#aAj<|M$QoElNXD$Jj}aEMqSQ%#Xya;SM04PGkF@Oc2TH?g<&&4b z$fm`ozJ87zFh0UTW1JyXePC#<^+*_}wl^}ucHQR-GVYS?xg8>6Wp3XofLyL&skNDV zW6A4P$1OVgbn{FU8H%hXJ+OWJr|BDFF)clBTT|f@as-hxd!YDVwNNBwl)&khOQCpS zfwo@b14GgB(%odI)6pSz?iuxg430^*86)Vkj|)!{r}niHq7D}b)S4ypV6RV&>fZ$> z{%4gS_Dgp&rLD68o)q(^P-U$)bfzLmyIjr@?$VASkbXSV89X{Y^WvcM4?=Y6u+7(_ zU~()HUU^LK{AhK%?$up*Gc^_{+EK+BioG=d27t={z9nsvK!9xXyma0 z8!yM>jJ_*qzC`uwihZJf|3$22CRjMP&ky0^Bwl1KJnIPCVKKt6yzR({@YtAsz)5Ew zU-3Q5;54537GYz4c8Gj6bB_r}Xq5$3 zjkGuee#N>qLu0XsYP3N|uvUC3>1A){@GB3)$(>v%F|p3Y##>BJY4QyB&9-SN_>183 zYp?S7>BsZRg`HK*%-Wpy(VS_v#DO@gVLn6&?>8DK5=UDKCCVc3$8VP^+b$Y7)@%Ot ztkd|z#z&;Tokp5iJ;}~FWvK@j;kw(0{yCw+z>Nyd{UBC+2yehpRz)Zb5`6U) zpdUDD!iom*4W#Q3vXp#gh`f-VG$zR4(gN>PMVX$c7KmNENl?gxG(S&sCg--&X;X)l zivcRvw8OACF1W5XV_)MrK<+yFkuR${E;pmj$&uK$otTf*AzxX_7Zr|b zM?8Tfarh~bOk{%8U@f)v0ME~}z(S{TDKuqD(Wywhy0_D@^36^<@m5Fmq& zFXcQId6_w(8oQ^qAXIULye(%qaEND8HoAVc>V=%fCl9uJ7|~QM`ZG?Xgu;jaB8Ra|kW1XsadqDekbF1-^sx%*H9q!r9C0}{BNUOhctfAg zY_t<_e7pp!A~hIaNa+xR6eXoshRaHY$mBJrsEmI3aAQ+`YEeSlNI8F;;GV972JokiGgUcA}cg$VP z&<(uZ)fg;a=~q!+qG@1tIjoHl*(w_Q(_+<|YKDLi34^V??=E99PHEQZ#yMJZYy<_T8Ev$IT;; z^MdmlD&Bgujul&qri(RT!$7RKQ3aUh7RXp#C_kc8%QBCxrE16;1^3J6rT2;eoh)M= zTLSXL-7-0K;!zXFMvpf%$op{diDV@_`}*yVk%{Y2BQ@0KFh&8CsfDJT-A*)3q#xRj zNs(K}AG#`jDA>CqfNeMP2_P*;8U-2WmvQq<8_4zjUtDo?xcQej%wTm5b>lzEoZuRgS@Rs->l-6tW%31({4nWC|(s_QbaMyBaega3wPIs4RyOI?zlmSRy^# zaa%rbZrd~W!)F8mD$kENNv0}7t(Co%G0fs8rVX{pabUMhsK)tbe%9iIZIYBs4EKkK_O|4uS7R z%tZMX^xs+lKblDORR?c-Y1;7iI4&9g=?wkD6?nIf>M+&QW9pN9O>Kk7m-p)2wsUX= z8g2A^raIM1jcl;k0q8$)k(vzI&$H`fMyf_RpAbnEacptUv)sXNltpzEV$%efe{|k` z{dF#?xvm%~jRsq@?OD!Yxxu!~4XsR_2pZG#3%1^_eyZs*()4D&<(Vl>FLdoEGbP!n zf)z3hchjj(AmOHAmS1r=F++{E_OJj4{N-MVg8~}eI}XX%5p>ePuBr3 zGyia!eAST-O_6$NtwxeefF_HA>{sa{9VFZEyu3l3*9%}`%CD#`B}KQgv2%`LJ*LTRutJ>@>z6cXy>ZrW@Bim9v&W_Nj0cY0_YOM zbm_!Q?2!+8(qlxB^_6Ep;zq{b5&YS|rX?%*?%&mIElh{*7;&E}_YN9GD17@A5`JS5 z@?+s-6y@LNH-E?n+HsGGmfd9F`)}?*q7`V$a2z*MlJ*aNceqLW-=W3-tOvTp-Ed&e zK?Db;{|Gs#z$R-(Zxn=d=>M%Hgm@t}Eawr31jhbtR{fV*2JZI*xMcswL8kV9zGeym zOf9_uA{3GJy?xu`d>xTL+Lv=Lr2W%N{pY7)zkeF`fuHq1)_4aRus@djDw%YV|M!)I zWZh-vPn(hl6u@UrDZ|v~yt(ruq4~#tP|LWRts6Tt`iI-_kN*5e19bDhe=@;@4(z{8 zihuSyCi3P{nx1oQbLqRw@jvhV*FVmQyQNs~9rl&}y&wMjP3@sTf3zQ(II8}8+yD19 ztHQoXX6IXD1^?kTz;*7(K&yTKJ-l7Y^8fxSz%|Y9UBB9wCT;kCJ?{T}jeinUiKT*nvvXTFM4FPbgZ;0j$LI0`sWV$1OFm^0L z4nsZZKVS3zsjCbM(lE2=Drx?6cm8<46@O+6+P{tKe@+QT5}-F3c{FuY|MNBXNWiUr zaDAix@4ox@p4Srrdh`EJ14odtl=i<`cvBVf)`K5V?)L*IrY`_n;^A&V|96$XU?dhZ zIB>kI;r8I-;zHGSdjuF|Q{3PHR+Wl@0R#xcg#q}KUo9=zJv}|myYbQv0NLB{dUqK5 z^XJn{hl&5mSpXwy6Xm_*Vbm~yvEf^V!sYeE=Jf5`6+px?-01XjWd`^_A8z)?`+zO^ zP=H6XB0CBNOpQwc@O&ypDk>p0H4K23VcY};&@Kr8Wi>Url%n(^e>abw+_ZVak{*q;3Gi79sraFv)1HdQrxb0 zB>|9|!B8wQ*9qN*6TA2sz+k`u#SmUx9fZ*iLE0c|G+3b%0L%hQ*uo07vw_JhUQEXJ zOK^|-^EExC_qfd$3U#sGG9jOd*X39xL+cZ4!D3F%%_iedFL?8boW?6kjc1Q{?zYWp zGcqwTv8{;%B&q`^SZgw%(WdzV@D|P)T31H%I5;^twgD`R2!N4o?GQLG?*?EKQ2_Z! zIPZ6Jod&~b1oRZtFV1!MJn4%ry;+@tQWH1l!zp6Oj~^u~)e|8EYJYe6_^bkDdeG3& zmthz`5~u0-`0zHopDW~j55yYDg#sdh)7BklV9|!)1ag}-g9Hh51DT4lI8^p)tx5DU znP3T~On!Dmu_-AbZ5N%mfPC+|xA{(?Vv5l;xv-<I?H|$n00^+cb=#qG)IO+D+Sg3uofCCd zxC6sC5iL)11P+%-G_SHvP{&QHMb3b{00Ah zr0^SLE!umA&jMCMQWC}@9fEtkP}FEB2Bn25mC4XwJmc@*+ZlNxI7RNyd>1;FQibgs zzc(U;pSI*tEBgiAyh%*SDNg~|nqZCR<8^fS_U-?%w(AnTr^mW?%7Is%Ge5w;=>gcj zBm0Z>4vvqv$ZtONTJ#un8bt7^wiKs@5CB-zEV@%y5&oL6SBh0V)->(sS6{Vpian+Z zoM(VQxYAzq5;Qj##kmfBeJ?K1$wxmmVz|2?jPR|h9sJ)rbY|6`% zsxA_8@{jc+697Y+cTBiwW^07DDzVVcg{uyZWs0UL+vlwXjw~aBOw_Nc5@amrfA4|g z=ku~d_)di>C`p}P77M2o%w?;^7$TwP;aSd8OJA=uk5QVdF~rPq!>kxHG5n3GPh~IF zb?h{TaEb7*R^L||+S&HpSGu(SD_~U)B_SmYUEguO6;12MN_7%Tu4^s}TfkxS)8r$~ zSp=m-FMxDTREFZleNpz&5<_m?a(d=x6mZX%KrIhgkE@#{mosUwlHsxJip~Ncr~Aal{kEQ(1Yr zN;v+!pvcns@ux?Q6Y@YTZFt}yu(K-Jf%(6>YKI?y`R4KggA$k{IXjSm71?5-9oto! z%;YB3Ec3b^ro`g+sK+~t*S~JK1qiT4gVFfYIjMy)>}+h!`E)iCKca~*{Krh>h9(kKf*=wwH{Crf68DzhuuA;87SJoF0xA4J_aB(gFs<6v*}18 z$M-DkFY^U}ypNJT(LGGPgI@y??MpfSi3OhQ@6SlE12g0FN@_|NPXFzr|09-kAOvGn zOSjCCo%#@RH-&zJPlgc?7-(>K63~f_lkNr_qTnrXaYXp__LOmb$ZtA4FY)~)Vv@w35AfArN!JU zuaJQ}-ob8W80u6W0hHVhKi2K8ga&cPoN~yP*6nD{z;De_zV|fa_4nIJcf*%uI#7QV z!w|qmX=V(+s1VJwKanaPC}1GUB?y7YPcmm{vme;qX@$o1MS@zCaY-aGzzWFC_|58IzLJ@Ydvl2yZ!4!p$v8cw>JVXls_S zYcs2=Gx-+)5~(&BMrj8Q4>{E${Eh$HU0cf{YAKgoa6HF!1_RW$e$@2+y-kjWknQ|a z^q9GZq==6^$pjJG=rsTvf2SLgFW7goGcSS_DZ#(DNZG+lg{vH~)@e%P$A`8JAAI>S z$&ZK${QISQ;my9?9h0_So$~K^d+Z#IJU2#rt~zwN=Mh|Iu(-(Ekf`gb#NA_E2)OB^ zxN`ntRUMd+vM*SpqtA!lhjN{mkK?9OtV?Ey@@HgBI#)Lq1<5s025Q-C)ItbuWzK_M zz%yAbki>L(Ft&1uu~2z&Pwv-Kj*8aUN(Pc8X??F}OAipF@ z7rw2F--N;lKysQrxA#Hv#rwu|=24nr64P>Hh~Yk9;FZwzRmM*`f_FH+ZALgAg-U#e zM-*1DfHX2^6zl)KVO%`F{klpjPo*9X;~d@n)3(ac`tJL-mv+`)%uG%j5~IODFmXVb zc7Zu{HB~AMa9}V`C-e#?Hdk$%V*C;mR9@MP`l2ux>F)r&g|(l8-73PGB!t-!T_1+*3q71 zC@lQf+jdnh4lArrvKot8qXDAsjx*m6brolE2DN3n!ysFY?zSY@kTucSzxb-VB{<45ok>{Kv zP6(7??!#EuR6;6AtfBKGjKXfD^iOrXUXL>F0N!pLO){PIFy^mvwu?|xYU*v1kJ!|s zVv!cSS&2^&nwEcN()1N5(9G3!J>%wDoyo2PN&}>>0c}!3bsfi`+DV|OW(MYAn%6_g zkAS;wP@*Wm(}|WK4Ld}qqk%R5i;@`)Ad?9d9Bo40%>ET`;T!ZWLuP;u?f7q#WP>V& z^rZbFL$T6~Gx{~|PxJY?znnjkkjr}TH&%X!bcCxw24h<$%V*FC)UMaeje+bca< zr>asOG((|@DUU-Ob_M^8zm?6$Xc2+ixH=(?t*?F|tb)-36zvYI2P zRk@yF8qov*PN^%nGvRc2ofDSZ7LpFqYIF=3cTLs?En_QEfd52ajac3{&tpWB0&HN( zeYqM8pU!+i=dsenK}I$#WFzEpF3!@zLL5(JhKRtRN14Acfk13;e`bxuyvutlsLQS2 z=T(^}P_G57AF4TbkB2Ps5H8?@aRvEOR3J^JqnRaXCmScars21E>=wd``ab~6jAv4J z^@lvE4*3n@bMeha=IT$lP5HR!X0k~d-ZNBmb$NudCr-@bFOHWo?}?DHn{B{-%Va_8 zL5EMp#1547negRs0oV9`#@0?Z^a%_nr3Yi_v_QvdE13+kVWz^=zas$5tU&d&7Gf_u4raiNRyg?Yj!CG z&YWh=%%7=3f2YAY+Sxlsi+gCCbl%zrUj;>>=L9FMpsE{nd`y#gMkQhv-zZzgCYeS> zQ-)+orOQq;BEqB9p^VOEBxUj&VQZ3T7%Mxm0r`nWO5d-yolRBh-i~5DBSedag$f6; za4;1%=1XHG^mogBTUm)oVM)=XSx{*>3mS&W*NNOp7o~l%#I$Kub=5EWBqpW>J(;0A zQ1;I#fP@7?b3x07&tLxeO5KR-Yo`oYRvepXLP0$onXbEe?HtRjdKv=Fu-`P9SwNLQ z)Q{tmA|12GtA(yN)nVk6Gyq^Z08l`48~*L>;lVS(d%;)W%INci9e(fD&iEqv zpe^lV&l~;swvNOakaklm3Zj8=4}iIq0RQ8a*a#6wG2Xc~f;$m8JsxZ_s(5hTf zRngynjI4}Bxz>eh+BDo4#{@H3^5SgzAzuXQuFT8dU4J11#4xAoftF9Hku(@5(_IH` z5!X-yjXuq_%qGT2Bczn^r9>>B7VZ8k0SoI*weTvi&8Q=F@84Mty<|CG>2_jlQ1Z$w zNkwJVk!?XIQ#(TkdO52L?=ilikQ}yh>zp$EYl3hOZ$ylOe?%fu$1b@tDgHbk;QEru5JVu-+Eu3V@9=+ zE|*E|2xlV3w~Nf+VqeK(p7syOhNW&!%mSJ<>ERu4y7BHYcW-OetbU=xgAVN52P0;vN5$zO9Fa ziXO4#lL>Z4Wm`t*Kx=F;k`}8Ph>0--_kE}~(s*le)Hm`aI&v%EH)E^`+xD8dlBr?V z!;=`sHV@5ycC7bvT!Dnz(oSOUc-=kbJwDI zenF-<82wuGrGcNmN&qJz3QZ}M9ls}-{3@O>d9g*hx}Y5+%4N_&VWh&Zl4lZ7{VW+s zW{+S}d$1-Zke;{cov7d*G6~Xb$}*OxdBD44;bN%7N)m7*soUV(&Z05Fgih-K&U=UjJoc61Aw@~o1mrnQzbI9(Dr3E^#E z3Gv?);^#cTGIYXz6*{9Pk41^woNDxgD2@B%cKET;xZ!y^e!KXqXpb^ihtB37#+oao z&L!LT{CcP{qT|E38kVlMe3siS_i8bQj|v-LdeUPAjXt+BahyAB7)*->#`cz%d^+h* z{V9OQps7xCMDGtGi%8=Z3(_o@vhIg{6}P#@!OOAhV+5yuT*9|5LM&*U+ImtLEhvm% zhI=5gjc!h}D8s^rJBo?Sq9)rKZp+KBm`>s}@IMtH^q`}gH+1{*aegvdPT)mEC zy&l_mLZVasp*K(;-|NUAj>x7n!F`-xlstHqg+4W7$emjx?X}`(7X{ zKd4phWVBF;OfL1el`IA5c%ZlBt!oRjmx%xjF!Mn+1-EKZSBd$7Bg(Quhf|c{PYq-xLlY|L>UIDZnV$UDO9-9?R%l%j%I8lWD|Y_H#Ye#Xs5G}CM(&l%ySejJs4Q? z8+dN_cPIp+=x6}At%2bygPwJJ` zZSQ@9+eFk1BzoIK4PCXkjyfOrGiu~r1mj#r0XTJ3x6m*_oA4Q>ptPi<Q>hVL2W5h+dIl#OAEG^Zx7f*P4Z=I&*MxR)A*{9`^*3OR0JQsw2d^fzs-kvJOKu|zs-qjtlW)N1}60#QQ>LhtV@ z-j?4EAFTJNfMM|S3y~Q`B@AkwV(zJ|I`!CW(&*HBK-L`zh!F)QJljrcD~9+uu<&f# z)X}{l{ygM^jNDLt{8y2%qx2o&0efX`0^m*Ml&(x1W>^ely6#g3k#$qdTvT9LXSfpH zrruT=HBZNL+5;wv$W~1|9GZq=&uG{Bed;KMaE`WR$k|s|e`%2~Jqk3ZHscU@vSWJe zgM%cZ&TV-f;nZzofx2puextc{GrY=0V!Tg{deqArQA`enNWLY*aIt)hn%DXw0+I0p zZ{Rc6(?U7<=$v7qkHM-#DQz&#oGg%DAbSM4KGD0LwBoei6R;BJ;QMFA;3e4+FvTdiB zR+SX2rQzDjBifAi%+X#D&r7KT$R;>j-m0rsEfnVGr}kH_c!@M@(3tQinX#3(@Z;t- zzmfR+av1~hf0|Bh-2ms!iW#f-LCbBLMDynNtluoH3uVdbVY-UlerIZfAX8LvvL*(N}p*A{cnIOGGAp)29%0KX8 zb%YHf-&Z!oc+p=&Gh=`=HSJSyVbE;4f%QpA)5_% z)|?BC7V)8cwO-t5kr0O;kui3|H-@I*b+fZ$Afgh1C##~XQQt{L)a|D5Q_v(i7X5y7 zdrU#uo$E~P1$J4jowhRP9$?n%6W0y9P2O`v1=$Ac4A4lr2 zjn{KGEOx6z>CtMmP5@p!Od=Cad9a#~6@i54F)H*n|8^N7C=uP00%$Y3J`nzWfO-S= zdFpz{B4ICXO~Bs;IH!u_2@AqbTvN9CCyNqaqwP-1Ucu~rG)qiKt>u+?T3CVIFt8ZD z>rf@Kido4BcbQBs#jpfUaPaNDl4 z!<0H7?0&FKvT$Fp`f)&7G+@*gJBjA>P*+4BO;p>h@z80{ALGqVP9oxR+Vv-?>n+)_ z)4e9*^Dpxhl&_6t`oi3P05=JHoDxQ)rhmM0MxEZE?Jd#wR@}y0Ba&{owG}l}Gn*xh z4JvEv{d2YwmzqkYmdipV3M`~*N|Dsa930Jp)nLJSKajxmJ;WD;GoI3KW8&7ald!pNF!pI1^_YsFEdgwOp}qRo{k8Dwpe{xYQ`rvWPq4 z9H#!d&6KhtRkGFf`*JGtRnHRd`y=IXq|0F5l?*M(nJsJ^`TSvmRT`U#k39Wlb-I>J zUcr`|yz^wc6)-nGYGqr1tJa zF~U{Zhw1o;oCr!`ZMl@$&~wIV;y$6em@XA!yi{0X(G@^Sd32L#yfvCP7<;|HQq1e3 zH7w)}W$BMWiGjs90B5snzc zK|5rIeDNU`IIx^&Jv24Tg>wq+NL5KQ(45dW^Xf!1PZiee)<5FH@-TkY|5OSx)RV)e zfvp>V32NG3Afo2SC0BJ>Y}2Kbz1#21O^T+p`QFmht(PWcc zOP_H~RLPqhBAIKL+F`}Jtu%6_IbfK{Yk~#}ua9?jBy}Ac^d)P#`TL+e?_1-Z`;~G2 zBJmHI;kePqY=NEw;~}63dC0u6FZ{ls>!vAd zPah98JYVdPS%sWK9lpB}@!~eT)kY}4J(=O0>bgX=T8bU@R!W`eg!9BtHXVmpxBUt= zWwl}|Qv-0p-&JQw#Ud5V9f$ukZFtjCkK*EJp?kg8=KOTuh6x$X3qpP`l@L7wB^M#& z?F8zHb~mm6zB!QZ-bx4GEt(Y5kRXd!w*_U5MjLcpZn(dg8{}xLxlP3qLnjG{;mVtw zAzvBldFIG#poF=p%Lq%%+Su5466~2mIZCyQBNCJ#3j{|o-tOT}Gf{S(eVlqLPiXu; z1`M{fOsESNNwU6}~zfm3v&RWCR0Kd(gR!2$uf$UAmNb{2Hv%f%u$R`Fot zt`PCoBUcSrMVuj@2ObV3X3ER8%C(Ep2z;E4({vV#Bc-Mhexh(YDX!a4r<6%LXmn z`&BgpmcZ|pz9zy;xJ}iX6YL(|N(A&%1AoVMWneT1W<8KjmjJ~+{HuZm*RO$W6fK;Z z16@a%wkad4UrhLuX#pYT`Y}v#0#d^>q0{iAOkc`Xj6Xl7@<}k#jpz+z)x<%=5C&*m znSYioUaEtqB|u5HK_;dxJgd+!99b;L&Mv6=@qvV7&qE{J`pat^O`_dO=xo0!#+jWg zZI~s;`H9|m&wLo-&J(jCbW9|Q59k3zTA+c!shKw2>H(s{li3I@Pjt^!fD?AM(I^wj z%^~d*goT&ghW`?nZ01T!<#)N|6OET!B{l(%j*d>0ZMLh@2FT`3pjXEK0ru-9yyLx} zjws1FsHtIM`2VEVUJQY9zU&ite~&$dW|x~mg*jbms*Rv$8+08atl-ajHeTvrS9c;7&g8AhD@EuqtN@DvTL(CV z#Li$S$}vSyK|IotKL*GoN){U3*!d|z*lS#lFNq5~xsmkvhmcg+WOs4*MRm<7O#jTb zuQ4zn=sp%dY7%TUg19?XBe{2iH}$DChnjsbKrlfDsTS&~(+u$<#`idh-j_VJ2l+H- zCpH@nrSR&;wk!23<;c$U4I6;T`;6cLXgjXl>{z8Y>6^gW0y!d(K0&@5PtE$ z0m^-gR8e^V1_DT^L;-u;B~SIWt4A{xGo^U8sE;(isM9hy?ZsSN8sdQx5-qJTfVMoL zyq($bCD~?oDuYgA*I|^t=^Z<8ns`B0%aHqpGe|T7!w&-jIDqXL^)B2Vmz~YXp78=+ zO3%gH>%SX&{Q?B`s!h!4|b;8QZ3Hil;fgN|W-vFNE0mu;kPJ4ULO!uCz zwUKMiRu!x7s={sp}I%TR26*Tow zpKp4uEDw|H4y<9LKyL9~qy+)`{e1;eEk}e}Sr33u<*00_6WN5+eAc*{w1>G~F|+6@O~)b|h(^E9 zX}0&_3P#D5uIA_^*Dt*W7Z(*3bq6YCmQ&~p6i!f!?YTa6M6t|*^EcQ7<#~`6kux*G zWjr+@^(raZD_hBEC#TdSv)kcu*(cbDj4lZ$m~8nzI)`;S_m6@ zyg=>LD;g=r_EMW$Rbo7yF!fI~&D`AGyMFF1pcXW{zBo+FQ0#RPs8(E|4#J+3;y2BB zhB(NiOx}Kk^gP2~ySfrRX_>){K)il&T7wO2@=W#Bi)^bXDI1HE;IMvs%b7GC82H*V zH}@V-q0e{ElgS6ytYzucZS_Y9n(ukAPoqxg~b}e}KZRUZ75u zJR+yl>)8l6I7)Q#r8f31h*27!doXie08A;7kk4H~(iy1oA^|oTfucwd6rtybEV&Tf zHdjg5`UFaO|ME$og2_`vOib$j1<*W0s6CGtmXHWF$&z^u_q114Q=?IC%5w2y&}l}p zZ@&uGDOaJ?!s~ZhZiw&pgSS^o{~ZWw|E}Xb?l%3S1MJ0~v>YUBgCJY8e3wfDX@qC+sUIR3zs?lOCcNcC zUm;SL;ni&8a69F*DNw+nvcMDy!9m3SvIhR2L#?k1Qx8r+J+vzXVT?oM=JMMQj-goy zN*ISmMp!a1>@_h8a=%H0-;zrw8vvV7 z$&&8b|5Ed=C%s8mRl%*E3A`3VnL|J(E+UdDD_L9KP72G)jYw_zo+*gkOTZhF3MyB# zs9syT1g61G5T>3&Z9>&Az-@xsr7bD@!Y}bhM4I(_PAvRGI*qx!A;mW$=>(v#TQ!|S z+8c}aBtt5Bu!}$2S1U|TE%$xzdR#04rI0@;Ei;WSa|rwnC`wJvBl9ur0?IJS1;0v} zKykvt!jh54W=1_xrT^kL?1khmo#{G$^F1$UZEcksbrKchQ8&4ge#0zN^(M1T5`AUx z(jlTXsKv9f*|mRb)NthcuMX#m@qIvGd|7zTP#bansgv9fISbip7N48kzmi2E^p-Y6 ziOP+o+9lzpm6KxTTqsY&C_0r_Dy!@v@VJXwAj^h6nr67#>=-XkTx*w8l7%J_CcN-l znz3W|(pS3@b? zQobnskQ=_{({E=U1etCeu!iYKG_#3RIr1(Ky0?*}XF6F}hHe^coP>F$PM-%CCR(TR3c{kVBOg?X(btQLB`<(|Nxfi}n znP&^QvPC!!+t2ow5ta=$+~Yv2+7B$3*m7$;BE^d{rHyKM zGq^qc-5B3S{|0D`4|d$ITdU~_WfMh4WJSd(DkHI6eyub9``qv`q*Nf_%^3fWSEDZX zA0QVdHDeDH@Big7of8#hvUi396Tir^E6I$2Q@z(_U>M08(Eg)IpsQJ#xdQDx&{R}M z^Vq@wm3@*)KJLiP))xdpcK^Av+poy{saNw)Jcl<>53Z-ATv#p?sJ-&-f88&TbY>F=-;U9v=u#% za&1sfQEsW7xa{pujvgF$74R2=T_Xfca$l_!&s(-~NT}-)tR4v>)Ns09&HKq7*&;;C z%^n@`Iour4o`B=JrBjJ%`ZJ`M!0W6yoVS?`_%PU%o`FR0%v9p9H#0m%WvInPhSPMe zV3?SLW-P~0!E2E{3AikuTMGv5hB4dJEuKGs(NZsTRWVFU5}zo~d;znG(Rus!iPm}7 z;s(&rA}~Jf@<%O3ExtnxFT#JPi_g^g$6>fLY3WC7Ay)k9Xx5}yO(gZcin|Bx{HIGt zN5@RPw~G7N=Jhi|e>Sj9!^f>`Q$LkBCI>K#KSw3NXvc6Q^}q>avOU}vx;tRcFD&GX z%bay%fq(OKf>g)5G7S%M!ry{Nb}5W=nN=e2rFof(9?5I_X1#ir-C{}x%iNoM3eBnJ zEf(hOJoAA-``_H$TvI_K4-Zf867M|`HqBJty}8!fNOL}X`EejHekjI>OlE$MT?!;YB~w+{D*T2C*91PTG7vD(Zy=6dm;_ z3XvwC#%~`3>iQEn>LFhsMVGbX`}rQxhE>DhlONDQt+uO?1WDn~kG8Al9WDDSmlx=cl13#BE}N<=a0J}SPRgo&pB80b ztUuGTOB5smR~u z7g_4WIEh``hEzR&Vo16A5Z~bR>zYiuAa-|5_*-20S@!LJ|KDEeHo(fE5;EOG@^q&q z|6YH-^sNJ%GczQsIoeNWmBb&W%MwtuO8cxUao* zgw71CCs=*TbIufJyDDDO^D|_yyCK21Fnew5&Z}6gHU>t7yY->&i&OvUM;myaBK!t5 zC<7Dedc9!w`?2-4%o=n($M;*b9)DKWDhjieq7b!?5Y75uq&ao!Ibv7LxRP?Azq^X> z+C4aJ=nT38p11+qMuE4A);Q3rZH>xdQND>zC|A*GMuWcoI9#KGwNk?I{fj|ho?62G z=D`{mT5lF%?5-!1)7GY=jHQgXTp(UlWA41NpyxS0rWJV``sSr~0+#H*bZ2!+er<{) zeihn{X)9HytY-J66^gC1JsBB9zyL%ipMHHO)IGWO$0*BTsyj;7$C0sziRg2FVPe!g zI{hmYpj`T|$&XZ_%ZPCtmO-NN34F?rH1~|eSlS)T4PBBx{3Di6^?CkwW@d(J!+2ck ztC_KZXxZ-c>2HS|e|pTAmFfgoqVv`ePRY)ZTAI!G26@|#QJ)Ksc8--%Jau&kO-ju# zVn_^$kN>Lt#A1p1^@rg!xy#NP!+tc2QX0*q(u(rpPj0|S!)h3mV)Wxj4p6$A`3j@c z4sBR>ID)CEWpOE`@xFx595Oy-lGR{cJW6G8$2^PD3ape z=lI#COb83MnwJ)~?Y+QEvbZhu;A4?zC4aI6`i^DMud2wyo>xcCvYtW@e;k+HHlu7` zN1*@KL9nF0?ea%l$&0DU2YPpaIlrW7#(vl0RHDhIk`pWO^hV6_%}D|%b5wbCc>bVW zUtuM% z|9UhmcB`;fbZ-z&el}y-a@13Lgbft0x!3$o8aEESdw1u5&_hP1b?-6JP=g(&H# znyw2@3~3E}mFp8V%kwl$MMjSeD^h(;e(PPJ=W#p1>(@4!%Y%c1n^~SmyYDtzQnXfH z#)!Ue_b?%laDCN!D|m6dQY7a7UADYI>oYdp^M5!*)@>gBqBdrc`=dy(3_lM`nz!|> zMVo)FIv*r}U5msL6jltN(2M8#T^Xr_9haV#ChX(0{9?!~zBPb<)IlS?DI*#Oh$~vO zE!1^&87tzf7Q-7?JxaspapV=Frp)6;!iEP?pDj&5Xbo2MT7Y~UZJ zK>}Rh-bCN1F{fFN0F^+s`CFM&gE`&Y(o&f;Ta)pgjiPKHQ81nw7!Wrb_S}2OngId@ zK|^zpO$QB)VPisXup1M+v#)Rm<_hnNr8qQs{|r9o(zn}lRh{>Aj8BCl;F{>&M16&A z+IR9I5@@n{v_4zAI)ARdCrtCTc$-!-x?5^JG5L>g+ zR(3m!8yeJ!$zBTi=#*er!kxlTP5~oGnru z1dUBqU$aB6>TMu>Ws|7I&)m| zC2#M7TN^u5fmTB-jjh9ZUp;1~6d+|oX zz1~GT+#UlPSSV;JG-AK&25!kRH6sBqzEuU=%r53_SgQtwlMBf^y@khWPeno3`(j>~ zU`Yh^(N-IqnOU)wcXyoWy4nrALy0^!m$#h{cS6e^`xP=eVq%E4ZNDXVEJXPjwh(Pmq;oA|PH{3paH;Ajg+pnofUZcDO?u!3)m95J^?lO6^i2ad& zof$F9FVszClbIy+^s+?5p09QSNQDq+$Z=`yrdVsZl<%%?LFr1>PXvzcx8Mxve7N`2 zz@M#LewNPSpab@qFs?0oo{{G3!+P#D4(t5iT~na?euR(@SlTui1fg4Fj? z?sYDnVPz=C*HchSHy3RW^#l?XnRJjWw3$HGM+E?47gnFjL{{x$%Kcj>$; z?M4OT?x2p`I^ZECLVDQtnWT2Y{xzO%2BGdhE-)EQItPYUPg2s8hP3BY@2L0&c!R7I zGk924eO>-k$KZxI7bK=FXwGc>P3`DR$&$QlsLFH4Lzx~^R#ZL>^Sv;d;x-?KSLOQe zZ_`Iw4u>nNY5C8u3o#Y7eo2b#LfEEB##VtA-T8ZfwY}g zVMu%~2T~{Rl%0p#m^jiA1Lf1eAOblm86L0eFRP+4kwip9w+%w{Cj&RYxPsbZV;$*7 z8)i#j(cpTl_q+8iD!*F>CJ|fTenfyLA>T)`5#g@x%i}d>*hCVOt|i3iyk*K7OKlLb zjwsc4{JC9CiTYZ*_nk}bHZr#Me9oE6?hJMHe$Neioyr+sZ;@1(Rk2&i=~JB4WEBqV zy3No_6|Ft@3$ca|-h=%Av#59o2NpjI%ld6hke)~S@k!gx72}Z44O@P1 zb}WLt7H#0Sn3^PF(f3+J;!lQK@E7u;`_gM=-xc7oMm3N4B7f52y|i5o}|OHyN&nR}g&9*pOi3*se@{#7q@wbmd!^4eV(HS6D zv6y#n+?o`Kg;2r!9_zfhYpBTcy|NpPV-fqQlaZa6l&{(Jfj1J4vJ1k3@UK7Lg5ly# zUC75TSP*KT8>^w> zE5It?u=&b}?U*i9>b_=Vls`f55bW{FtxtwNjI1Jr9$n3`qpI|VC%7$T_J zj`5$yV9d|6*)LAqoc)RzZeP~x^zn`!M%(&u z&qS{8Gx)+4>a?WhxfS~c@Hc`ZxXf~;tT(zf#$i*g2-P%)B}G}dtCmtpN^Nc*n$wd- z#NoEe;s*l3x+-svBsz;rMiEOq0)ibjmuN>C%bSF*`TlV4s||w**|!CKlTolqpXPqw zN#=-=0rn=el;@(R3Y<8oq8TElnkH_;e)cYt5B1kV?B2Idbec8BNpC52UXUMLzt7K# zXb32`{@zegVo;{Z)w98$8}S-Xva8l%YbbWEK$al1ajEs^26v8NdQTaTj_7X@%C?2ASC z(1-TNOo4l6&BMRDiHmI={8SoPy0P*F2wJDrJzV17c;JT|)9|`AgBy8#{yqQXLmxb0L;vsG`rg>Ql&Nt zqsM-7J*#ns9EmT|h!aNHxgpG5u(r;vR z*H{N-nR(Cg*}#6Xp2zV12h6JskB&jX9g=Z8Dx;rcsh3<75lbGI2PKW@oiCv) z>Z#8fDi?x(3*${sehJT?ZR=r!h2b!Y!^F`0(ThE`ifdOxK~@Wjnz{?EVj9P5u?m!T z{?->4SLN~&Pyn)g`seQ$}<1Pcx^%RzmdbhuGX;WdE9XgG0s$3%$Fryhiynydz| zcQ{B;6LR2XZxci=egVwt*oz$=E*KkRGkWZ@ z7_g7{3VTuv8aSlP*R7bj%4r%&dKHP9r+Pj&CTVOY;d%t3O|jdRDg`pPg6tZ$9XF;k zG>J&<%q7a~faaL_2)SG+#Imi(9}xV5iIT3m;ohqEZnPZ8|!-`w)6ia97{ z5vx=rP3~V(uBYZcTR_frO{2{vWM*+e>Yd9clTYKQmOECU%)r!S%`FQ(83@57(V98@ z_U#=GLiE&0em<3QsAR$vz{TnJE;JUfTR?C&D!V7l?;MeXRp|cONV((B=l12Cs*c<6 zDcbiKsRB4XukV;^G4E6)i%BD(8atvt9ypxF^AC_k@4iI+$^_haCe&~{-`>)$GW1vN z6Cb0}ui4xtO&5fJm<7CV>&$(!%Bl)U%HEx|ZE$$aFzMxOdoq8LH77Ca!Mw*0IOtB} zw36~f6P40& zjcBb38{*N8b}DA&tqd{=A?7OH3*I$hVuMTXW9FbA6zgm0Tl!r5fDTb+RB@@Ml%M#y zMfKcb5p-%F0`=h8o>I^gA!U%)F-Q}9JlSjv@XdS7pBTC{r^T8YQ3->UNwFJu^#+L% zvvlWI*{-xWBvZ?pU-(%E6sFE`b|XRhh|agY>ZRnb;Nlge6ViF5k(<%HQOD`KrWj!2 z7Nq1pe~kiNFlk1bCiUggh;lIZkcX7zOhr=47EC}fi6nX$Mw^ub;I}9;bGu~!x?r*v zjsj1aYeWT4{046L@kaITOLS`C^ufp2R_ca{2AdSUgXCjV%+uCz5cJ zkykfNGhQ+mn0NOk3(5A@+<(1D1NxPAK%)P6z}1`4vdD65G(&Ej&60gC%bbE1ER)7r z;>0Mh)R#u!QpmLw)YGxpCQmv3HAN||H5~G(bxC(2NZX1W0qWFGH8YNkfT&54495L} zR%N;s0p9Q8=ExL;@CCY}^GgAg}Ri2UK&a4#|KK!y)o3m&^1BZVcsN2~P1IU#~? z^Ag7;DI(De(2o?U!Lqm(%9Bo=(za`-&SQ^}PKyHV3xb%RbZytkX02`r+()z^rr;WK zqvt*8B5WIsPuZrmEa)skri6?u*k@T#QFbYZ=@Oz);Gyqah(*6lWD7B~CCe}i1w{m; zcpNqV^0Gvu8J;1boswqz7zosN5N5G@oS`jbMHqN!#4uSZi)ZE14msYC?XwTq0UV<0 zx^9^pEwW#Frp z?~y?^gpD9LnDdI@`z}Y4dz4;0;w-bhhcx?^q-71sE)1kfuFp`WK=3#+g5hXnA@Ac> z;nW)xsK>`H*=EJ%cN@Oj;u$9%h-C``y=3WfgJT%C>*hcdyV?yP7tLnS*`CMrBXf}D zKM_sqj>vc3|C$P6k3l;bU=3su*;Xup%}njWdPtCU|^WZHWKW8UR}bx*p=eU~@CH*{e=*!FU{qBGR{623VEnw@c#qbToXrrjCbeTF47lFBt8+nSG>H z>OJF$aE0DcVQS?us&Qc&g8+tJ!f7A5>%aKx++`11row!#N6*{A2u6?CZzvQ`?9SR) zo#PIeEs!%QnzqZBZpdUpFo;B}zz9K_nZ9%v6W2&SX=Hb#Dj|N+zcp5hESHrP z%AK0dJ*`i2ZBFC@74lM%^N=a}XT~tb`|PlcPbM!g{?}gK$MApL`e^y9Vg$r?9SJQ?s zZ%=-%cah19aU?}>b_Rc?s*@3rK9WN@!vg*87kv20R9iwL7UV#$H;|@ZUT}_Z;uILf z98!N2Yw(3DBXJBibhry;Kn{9LQu^HqP{du-@S^vKI#V06_+z%0@?vtE=DITcVl-9`j zoHQG$6ZA67v7Iw$se4nw)U4MXEEd7FKCWFuNj}j>c&OQ)VD0|qX}K8OCK6Imtaz8T zcBfDKweFC^IK}jbN&f^D)aJeUGIZ=0F{<5MF*mAp%J?U zva$3S%tMj?ZOs^CwIT46(c^xC3cve2`wJaphDF4@7k*Vzv_U;G6ibOp#Qlhlkl8;z zt>fDwfvj{5b%;h&hEcc)SM>=Fyj0g+ENp(%VE~TaWr@-@adtKWHhQhZ| zf(lejJxs{+{xVardJOYTZRg-IT-NH|1B)igJ-4vqL}|6w*Irq$pCGzOk7&iYEbD}) z7U*GveM)r0YGC`nY$VClaP(&w)T$V+=#8OptcVMHp9F7*a#RDcL`l%|F=?#3A;(Iz z!znqbI9TBrF`z%4t9sw;upK0*9fnJN?yaEd|Ah`VA^fg~Y9ka?uxGA5VeC~^F|SKs&vjQh^TdcexhYkD$)xf#Z>oLKA}o2mWrha$Wf|xo#AVX6wZ-L zVG&slwfrv3M*f|i$L^_9Lqa*>3bLb{~ z+sFKF%BP*<Fh(D**>?@q)bWBkil;rK%NMNS=XWo3Amr;lkMVImJv^g49u~a zj1;%Z>kigS#G%BvutdsRQ`KGCAL3N*TbwwA*1=JseES#va)JUcMYc{o3^M#OrI`>W zhhmVrBIaT4ZXU$ z-XCY9qA{A zIPUZI4_%v2NFA;jHErlDOb6Wj)}GS9&)83jxpe{*!6A7w=>!>kT8$AhW%DO#!#{kD z1%t@{z3*RV)&}Vx(q6wig@!P~upwe?gd#M?I&XNt+AY;R>OK_^P^*Z0E`Q_p@1f!y z(YpOA!o5)7shcYxx`>*IqCUXzo6%s za8T2Wu5XV$|}_=aV7Wa(;m-5z?%>l6Vt4# z7(d_ca)gLmf??r}{m4$wY5qWh%#aSMEuf=9Jl!5b*zgLjfwu_yRw1;d!pKKui8~*^pPPV(%iDV$tvjH%7 zbCdg(z>+c1lmcQmWQDsGgPARbgFUa}zG~p%iJQw0-!xs~&jVjio&kU>h!|myFtB5z z|6phObBSKNNAB0RN7$6>Jm6$IZ8Oenzus33j)!dVIg8x z@V8c>*e{=nUi=GnQv=ufI9K3zkc3DXbZ9cEOiGgk(9DF^=k0yz9gpk_)Qs#_U&<8j z2L-(%t{q=3fFf~9OX5s;uy38oHlY+IMcv=!;!F=T4QWpo7oLbdBb6@gR>irikb>lX z-E6M(NZsJywLqr8BrSo`g%HrR736*XjJoN6o^i^6?k!q_5R@m?`wb=6yOD4S{q*?) zLP!T$!V-HcQO77rYIdGXLIJ`J+H}9{t`i?`m_mkNc$E{P58<7_=Us@{ghK_7endHzcB?SPsl3C&fPN~M$-2?5F+%^_%P8ec) zzRIO7M)`cbAa_zE=2(DMV#*ijFTdP#gPWvHo&Es49W`Mg)IQ8$GjL=1Jk*SzyhGeT zQDn4eI%d_hPw%EA>{}ds>C2FIB<=RLTCh&)=QhbdhMjC z=7cSV$! zZhc?hOVNX&$dLOqixYjfi+PQ4hMA%oaO-;}T_a_hpor0;Ug@Z)#F2e|i#uK;Q~Ube z1+4#=w^8mhp_tF#3yt58atHCD8=(7U=wbdu+Wy1jiT7g)lBsIhnXA~J7g_q2*=Pg| zl81IjlG~Lf88#^4Pw5mE7TQuR`#B2m_d7mE@h6c)2QLu!G;So4V5mCNiLiSe^~bJAtadKIDsr`LnCi=HJt?!Yf3h-fe3aPU;w%X7I`S;6rT|~DFOsNXnE-v}A;F?Ca*!Z{A zPV(FLyYr%FnB*tWE$onzP58Y*&N_Z^4#K}b`Tn(a|HZ|S{egm;^-m@5RSFJ-5MIt0 zkrN7pF38MyLB#$L`!aHIOb;Vyvn=&VlS%0x7?=U% z#K_;D^ef4!JZ?7`jK&pACK38n{x`t`fKH%>YM7SCtZNBCN0f2 zDE`$PZ*84u!3ijgMi$zL3n3Z2j*=g<%2t@;n3*q{ONiu~eqrscHo?y`X(Z*L;pbD_ zoZ@_}Go8lYUA+F8VE>>yS8vPMvP9VjC5;?nBf`foc(&i-Y|JV9R;v4YK(|} zeboC!L-m}FKy2REpQk4MLffmI#&T_TX2we(^HU>{zoOtx&|{dUM@5RhXu1WUV}i;( zY&kxregA;XtX21MslmpIUmTh`_>D?n^>M=52iDi-PFC}hi3 zx5&In`gDUzLb)F1_sjc(T{>%4Xi)#z{;YSOT1v)77=Aa(TZ|->5!_ zgsMIOz;MleuX6u3+&2=q+rE^qw7MMP8eU-woX40rzf)$BqS`_K`{&H0f}DO&X;d6H zM4xa-4r=zu70jPe@e*ouYdV)M>VOzs`}Ajmc&0UP>K{~-NT)@4X)A8U4tnhN0Mi1o z9C)|L^*-tPAxbWetX3`J>kt$B!Co0MMnq$_Af}bMx8DMA^pw(vde{VUA;C# zf~~1H`Fv)>{<%SI@w&vcT-Q?A_H_J*JVGrH3DRw9tW{`TTBzU=dx+_wkv72&D%*z z1A52bL37nl^@2Z!SIOJ6>gCfsdPwg$tQ#^t(bA?3)E>2MC@DckKL95q-MQD(h52Jt z;}qSwCAh915p4kCmBwwyBsBnX9zw$jn2!idJ2LVWfEuKaU`zD)opFO&h3p}Z^K8`x zZ0Vs>fwOA1ef$L36M%YiUkt=LM|$mu^BbsY+jK>1r^iCOU^wn|4~IBD7vyE}r;>Wp zxV=|Env0ta90lm72mUCeP`zBuOsk)QZ+}6J`RYQbaUQ)t#ULL#cTKq>6xcA3 zGX*@=&p8!W|IFoAw6<;hZI{~JwWzxd9lp<+_!aybq5emx`NQ!e6zK7m>o1!)bCuV8 z%TrxS*g#*OK)S_oL!!r8uStKkD<`BD!2F^8&MAW4E&d$n$v&GuL>m?2Oovhie(E5p zSJwIX^|gPooTqC6#x1X`9Yaxum)4mPdCiQ7!ygqv0_pDMcRK>|l_rbs3;>2gHz$jZwP z*JuOxRw03&n?D%hPUeKYQ6KT4yY!)A&&__L=Lw&~7DN{?znTqEt}P_~J;CTCd{Z6X zp1ER{--z-^HU~c~WHVBTq}6*;iF;Zed<{M`!^}_pC7=&WYwW@I7Y#-uYSH`b?mL1< zM0&=X$Z@B*t!2Q@Rjz#+?9WlVN7QR?#H-(1ou7*MrMd#uZD<*!ciLR=)>HU=bJCyk`JkRhC7~mVux0CHN)6o*NWT|6O6YUbCS@iG4?#(@)XTZ-2+wCm|~D1ZckP zy9tk3Fm%1E1`?uM(1c$UX7M;DfD(1S5m8f`&-tCu3nGT!cN& z7x=Z6);vOy#V71obi@-{gZX1j@_4)M=3EHwXFu`DBb#I2XRQI7nEQ6~+e#f?ZK+z& z;HN>OZ$KpBbR-dbkZWi-ew)Cm?}PVmk)~&D@-5E$rZP?H>hZ=j<7(%!x>52h7f?Vr{QXyOI4~iB5iQVzc16{0TgPJ}_$uhfk1UFx2FuWDc_Dq4)rS3#8~BHnv$+Upw^HQNOM(;dF@UzU02G@M zlb|q@m6|#MIm@^lg)jjHXyrT*QW1kBuX>AFNvOGQi_>p@ol(a)jLA4Y=RMiAicn>djcfDbHs_3?{UGtl-D&i@suj%1{~$q7)|n7D(ycTpQZnTdYy%+ z{hISo#*Jnm5Y)o6Kui@pu`yF1+waEWd1)=9pupwy0bEjrd*r4Pc_egOn4rJ-|AHBP zBvH7>hIF06fq)Rc3@1j9sMHml(xQPVcp}aL^z8!cL_g*lJIKxez$lkeu4R&vFM6y@ zT!{{oKk2T&0dAk5!QpUx`Ym$9Fnh)rt7|`oou&FR$U+N1+1k(+mYN-D$Fqg9lR$jN zD}r1{vi+$-g-kvi2u8z+aE{aNOr7t8&-CJFGo}dR^l3+^{oq8_Y=me{1kh$N%jP}b zMDRgWs0Svqm{qSc`RTU&_)v?%BWJbGX};2!N)V@M^~wj4V_OijD6vzp?4OKAcKn-& z+xEFPP(vBQuEC=pVyA8^4gL55ePelbkDP{v1{b$C2P1MYZp#aJ70p53ZqSD(qQ9Ho z{CmOwg?-rbBlx$UI(Z&9{{@xRIw!(O#+P0);}swzVpSfXss!RHwF)gxkv!f+miVTw zvQZhJ-GASZPY&q^YB3c#V-W{(b{i)rr^26PI`m)?x?FjgS_wtmhvHkcJ=6#S=4XQ< zAO8YJ3jvnv!O*}!I`R0{8rRn&W7one2 zN>$W9kqQ(8^&drF6KLV*>MT@a3E_ew<5Ai5AMF7M)zmh-$7efRyG9PPzqakH2+V)I zoS*ar@u^Qzi;tDmb6SyY#Guufwp<6{`96kDSf=*RlER>Lm`2L{5!O1Vt}jlDqsWigp0a1NqQXMlu) z4mJjx48F(dX@Raj`9uoUKY158B-G}q)o3dZxjG#Hg((oWpW^@vgYzSsWT&uUq!S6( zmYScbDEk@n;dxv-Jfp@z>p?pWdbQDVtaEBR(uW|qq#36CtRg!CTBXy)c!%VLQD<%6 z0?hA8yMw)WSHKZF-RP+0ufz`V4zdco4hVgP8{(EWCnkQ)eGbLB$E08o5YUbJNtnI? z!mv!TOonVS*g^D9sdgh1CJEQrhhbe~e)oU$ZIQyZpaxGO5&a_oo(BXe#(&&WObWZE zyLnMKBQ zSE6OCm1dkjZ5L}Kpf*z`Vv=KEkDE}8o)R#KPIIOyjnibls+4|HO8EWb=}%^JbCop7 zVj%BMIBMBv;`6*L-{4>U&!O_D-@%0NXkheV?tCC2@@El+TSb9HY1LaMnzTz2CTSmD z|K1F;FR|;m8JJ@tqjCmXJ9zY?-U#dsmFq4~q%qXB5!9?X?L^YrSF(REC;zh%WNCgE6vg)QAP8A*n4%j+!_1TPFIF(Ld_FQ-$1&lI68A) z(H0PQO9VIqQTV55D;Z2}=g(+w_h;sD1_p;Bb}gLNCxy(%^GePqk1%iDr!)|!!|j!e zLg)9Qe&YZ0WaU7LROH7%1zUgxftQTMun%InSYv|q=K292mP1B%D5(tw1PGiCQ3gb+ ze8$S6*$z98e-2VxAWWIp1Z~_K%C#>JX|l1=%Tw9(?Ab`VuUV$1SN$C zFr?NyAK1~Ci2gQws=?*B^WL^SyZr+@<6`|AA&fK&=#1-U5j+eJLw=V77LHD!hzXEN z;>V`Cm$();?^7K%GKa4I$h`n2(cIDH?_(!0aQL7GWcxKA|r@1p^FL{^s=onpM2Q@ zh?Ja8)$6();s`LbKJWMXdb-p4p763QPh`e416k%$@6WEc@ zNeis>qyszeVtu~XDy}7eN2@VW(7HgOF4rr28O~x%Ri@M{biw?h4WWtwa>Q*h(SDViU<)yJMY+#j*)#PK@zUa&467yEb|sPt0kYV@@(_!Ga#2#71T*NIBVP9y%M z-IhujX4^V#LAKfH_%(nx#AQMDw*v@;n};8{hX1?)&p!d|Cc0C6Sy-}56VfHyCHKKL zXs=7V-)2byVim-~b?(nr7uLyPT&!Jd8U>v|CgK5i^)0v%Ne>Rey<>;qR$G=XHok)Y8vEk! zo_1QTMa0GzVCRldvhP(0wIdTV0SI%@iO`X|#S{|aVsOj|g+%yN(}zuSph*yw9V3Yn zHE!xBthM>hr8=m)*ioM3M)ar!-+eE!H7wgGbn`6nm9EIzuS1zsG*+km&%KF!SU1WQN%rIa9LZ7B)m=v_ClU( zvQ`){wgV1NLkEW58T(gl(mBJ(ig@R;oNhZC@hK@%knO&FqEveF$S9*&lPXX;P-qwE zil;6K%5ABtFoi0W+>L8x3;7B@-lahpIo7$Ma8CQs5SZD-^LEG4t0P*kQt`xmDWERMA=45B(me9pF zl1sSMGR3W^tC>;<@D*PGC4m~}4JaO`+jEQ91H_&?POGm^jU-uWP$~DR2k0y?wOv2g z;9CvJXD26CTFKSh?Z~Akz5nWo1P`&Rjefy@tMe@TKYC99+WZ5pfg(V}B_vRfOR@w! z!&&HjTY@m6prKd;`BT}KKb{Bai?(9)C0hbvG5*$j&>PLIq<$5N-th9uFrJdHT>{nv z@|qn@UF>8#Aq2>qkj5S0UfKb{7MyN4hrC(h*ITAj$AOirz<`lu84z=t0OS zORwA#{NGI;x?(A+QK@zfN*6|>fY-cGvVs)SpaiVnk>#o`%Dxz0Zo ztA1VhzV0M+n{?o9Cv$=do8ttn5`bAYeXH$7|nNel|JHuIg0Cv+REQ&;op>eEzL{{)}g_-dfxN99?SaZf3}QGxqk z>9iks9c`OrmL(;-x*=WK`k)l)Im1|7fCdnNEO~R*o_Mk(Yyl4Qhc>{*sFlBC?PXP1 z5r&w6T5Bho|0oAEg!|*-G_W8&e>p(NC3LLW&^Nibd@GbsKPV2_hp-_)yWa!z*VL9L z@F}AbduIjQ2cr`dUSGR;Y({Y-8AL#B+zeW}XJ%(nDQ>2m)oNu8$9jFB3RC2cWrxen zvoXK9`IzqHQadiI9mWf9Ur#B+9JW4ojMM+Kw&KBu$C`B7OWbf_MPL$z)1y&oL>~`n z>;dRAB;XsQ^>pt1gX{v?z#0(z_K2(01FcxNQcX$

{title} + {helpButton}

@@ -283,4 +288,24 @@ private[spark] object UIUtils extends Logging { } + + def makeProgressBar( + started: Int, + completed: Int, + failed: Int, + skipped:Int, + total: Int): Seq[Node] = { + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100) + +
+ + {completed}/{total} + { if (failed > 0) s"($failed failed)" } + { if (skipped > 0) s"($skipped skipped)" } + +
+
+
+ } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala new file mode 100644 index 0000000000000..ea2d187a0e8e4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.jobs + +import scala.xml.{Node, NodeSeq} + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.jobs.UIData.JobUIData + +/** Page showing list of all ongoing and recently finished jobs */ +private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { + private val startTime: Option[Long] = parent.sc.map(_.startTime) + private val listener = parent.listener + + private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { + val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) + + val columns: Seq[Node] = { + {if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id"} + Description + Submitted + Duration + Stages: Succeeded/Total + Tasks (for all stages): Succeeded/Total + } + + def makeRow(job: JobUIData): Seq[Node] = { + val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageData = lastStageInfo.flatMap { s => + listener.stageIdToData.get((s.stageId, s.attemptId)) + } + val isComplete = job.status == JobExecutionStatus.SUCCEEDED + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + val duration: Option[Long] = { + job.startTime.map { start => + val end = job.endTime.getOrElse(System.currentTimeMillis()) + end - start + } + } + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") + val detailUrl = + "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) + + + {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} + + +
{lastStageDescription}
+ {lastStageName} + + + {formattedSubmissionTime} + + {formattedDuration} + + {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} + {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} + + + {UIUtils.makeProgressBar(started = job.numActiveTasks, completed = job.numCompletedTasks, + failed = job.numFailedTasks, skipped = job.numSkippedTasks, + total = job.numTasks - job.numSkippedTasks)} + + + } + + + {columns} + + {jobs.map(makeRow)} + +
+ } + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val activeJobs = listener.activeJobs.values.toSeq + val completedJobs = listener.completedJobs.reverse.toSeq + val failedJobs = listener.failedJobs.reverse.toSeq + val now = System.currentTimeMillis + + val activeJobsTable = + jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse) + val completedJobsTable = + jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + val failedJobsTable = + jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + + val summary: NodeSeq = +
+
    + {if (startTime.isDefined) { + // Total duration is not meaningful unless the UI is live +
  • + Total Duration: + {UIUtils.formatDuration(now - startTime.get)} +
  • + }} +
  • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
  • +
  • + Active Jobs: + {activeJobs.size} +
  • +
  • + Completed Jobs: + {completedJobs.size} +
  • +
  • + Failed Jobs: + {failedJobs.size} +
  • +
+
+ + val content = summary ++ +

Active Jobs ({activeJobs.size})

++ activeJobsTable ++ +

Completed Jobs ({completedJobs.size})

++ completedJobsTable ++ +

Failed Jobs ({failedJobs.size})

++ failedJobsTable + + val helpText = """A job is triggered by a action, like "count()" or "saveAsTextFile()".""" + + " Click on a job's title to see information about the stages of tasks associated with" + + " the job." + + UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala similarity index 87% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 83a7898071c9b..b0f8ca2ab0d3f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { +private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc private val listener = parent.listener private def isFairScheduler = parent.isFairScheduler @@ -41,11 +41,14 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent, parent.killEnabled) + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val completedStagesTable = - new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = - new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) @@ -93,7 +96,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")

Failed Stages ({numFailedStages})

++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage("Spark Stages", content, parent) + UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index fa0f96bff34ff..35bbe8b4f9ac8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils /** Stage summary grouped by executors. */ -private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobProgressTab) { +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala new file mode 100644 index 0000000000000..77d36209c6048 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.jobs + +import scala.collection.mutable +import scala.xml.{NodeSeq, Node} + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.ui.{UIUtils, WebUIPage} + +/** Page showing statistics and stage list for a given job */ +private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { + private val listener = parent.listener + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val jobId = request.getParameter("id").toInt + val jobDataOption = listener.jobIdToData.get(jobId) + if (jobDataOption.isEmpty) { + val content = +
+

No information to display for job {jobId}

+
+ return UIUtils.headerSparkPage( + s"Details for Job $jobId", content, parent) + } + val jobData = jobDataOption.get + val isComplete = jobData.status != JobExecutionStatus.RUNNING + val stages = jobData.stageIds.map { stageId => + // This could be empty if the JobProgressListener hasn't received information about the + // stage or if the stage information has been garbage collected + listener.stageIdToInfo.getOrElse(stageId, + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) + } + + val activeStages = mutable.Buffer[StageInfo]() + val completedStages = mutable.Buffer[StageInfo]() + // If the job is completed, then any pending stages are displayed as "skipped": + val pendingOrSkippedStages = mutable.Buffer[StageInfo]() + val failedStages = mutable.Buffer[StageInfo]() + for (stage <- stages) { + if (stage.submissionTime.isEmpty) { + pendingOrSkippedStages += stage + } else if (stage.completionTime.isDefined) { + if (stage.failureReason.isDefined) { + failedStages += stage + } else { + completedStages += stage + } + } else { + activeStages += stage + } + } + + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) + val pendingOrSkippedStagesTable = + new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = false) + val completedStagesTable = + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler) + + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty + + val summary: NodeSeq = +
+
    +
  • + Status: + {jobData.status} +
  • + { + if (jobData.jobGroup.isDefined) { +
  • + Job Group: + {jobData.jobGroup.get} +
  • + } + } + { + if (shouldShowActiveStages) { +
  • + Active Stages: + {activeStages.size} +
  • + } + } + { + if (shouldShowPendingStages) { +
  • + + Pending Stages: + {pendingOrSkippedStages.size} +
  • + } + } + { + if (shouldShowCompletedStages) { +
  • + Completed Stages: + {completedStages.size} +
  • + } + } + { + if (shouldShowSkippedStages) { +
  • + Skipped Stages: + {pendingOrSkippedStages.size} +
  • + } + } + { + if (shouldShowFailedStages) { +
  • + Failed Stages: + {failedStages.size} +
  • + } + } +
+
+ + var content = summary + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowCompletedStages) { + content ++=

Completed Stages ({completedStages.size})

++ + completedStagesTable.toNodeSeq + } + if (shouldShowSkippedStages) { + content ++=

Skipped Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

Failed Stages ({failedStages.size})

++ + failedStagesTable.toNodeSeq + } + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ccdcf0e047f48..72935beb3a34a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{HashMap, ListBuffer} +import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi @@ -49,8 +49,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { type PoolName = String type ExecutorId = String - // Define all of our state: - // Jobs: val activeJobs = new HashMap[JobId, JobUIData] val completedJobs = ListBuffer[JobUIData]() @@ -60,9 +58,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // Stages: val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() + val skippedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] val stageIdToInfo = new HashMap[StageId, StageInfo] + val stageIdToActiveJobIds = new HashMap[StageId, HashSet[JobId]] val poolToActiveStages = HashMap[PoolName, HashMap[StageId, StageInfo]]() // Total of completed and failed stages that have ever been run. These may be greater than // `completedStages.size` and `failedStages.size` if we have run more stages or jobs than @@ -95,7 +95,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { Map( "activeStages" -> activeStages.size, "activeJobs" -> activeJobs.size, - "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum + "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum, + "stageIdToActiveJobIds" -> stageIdToActiveJobIds.values.map(_.size).sum ) } @@ -106,6 +107,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { "completedJobs" -> completedJobs.size, "failedJobs" -> failedJobs.size, "completedStages" -> completedStages.size, + "skippedStages" -> skippedStages.size, "failedStages" -> failedStages.size ) } @@ -144,11 +146,39 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { - val jobGroup = Option(jobStart.properties).map(_.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + val jobGroup = for ( + props <- Option(jobStart.properties); + group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + ) yield group val jobData: JobUIData = - new JobUIData(jobStart.jobId, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + new JobUIData( + jobId = jobStart.jobId, + startTime = Some(System.currentTimeMillis), + endTime = None, + stageIds = jobStart.stageIds, + jobGroup = jobGroup, + status = JobExecutionStatus.RUNNING) + // Compute (a potential underestimate of) the number of tasks that will be run by this job. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + jobData.numTasks = { + val allStages = jobStart.stageInfos + val missingStages = allStages.filter(_.completionTime.isEmpty) + missingStages.map(_.numTasks).sum + } jobIdToData(jobStart.jobId) = jobData activeJobs(jobStart.jobId) = jobData + for (stageId <- jobStart.stageIds) { + stageIdToActiveJobIds.getOrElseUpdate(stageId, new HashSet[StageId]).add(jobStart.jobId) + } + // If there's no information for a stage, store the StageInfo received from the scheduler + // so that we can display stage descriptions for pending stages: + for (stageInfo <- jobStart.stageInfos) { + stageIdToInfo.getOrElseUpdate(stageInfo.stageId, stageInfo) + stageIdToData.getOrElseUpdate((stageInfo.stageId, stageInfo.attemptId), new StageUIData) + } } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { @@ -156,6 +186,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) } + jobData.endTime = Some(System.currentTimeMillis()) jobEnd.jobResult match { case JobSucceeded => completedJobs += jobData @@ -166,6 +197,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED } + for (stageId <- jobData.stageIds) { + stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => + jobsUsingStage.remove(jobEnd.jobId) + stageIdToInfo.get(stageId).foreach { stageInfo => + if (stageInfo.submissionTime.isEmpty) { + // if this stage is pending, it won't complete, so mark it as "skipped": + skippedStages += stageInfo + trimStagesIfNecessary(skippedStages) + jobData.numSkippedStages += 1 + jobData.numSkippedTasks += stageInfo.numTasks + } + } + } + } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { @@ -193,6 +238,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { numFailedStages += 1 trimStagesIfNecessary(failedStages) } + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages -= 1 + if (stage.failureReason.isEmpty) { + jobData.completedStageIndices.add(stage.stageId) + } else { + jobData.numFailedStages += 1 + } + } } /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ @@ -214,6 +272,14 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]) stages(stage.stageId) = stage + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages += 1 + } } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { @@ -226,6 +292,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.numActiveTasks += 1 stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo)) } + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveTasks += 1 + } } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { @@ -283,6 +356,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskData.taskInfo = info taskData.taskMetrics = metrics taskData.errorMessage = errorMessage + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveTasks -= 1 + taskEnd.reason match { + case Success => + jobData.numCompletedTasks += 1 + case _ => + jobData.numFailedTasks += 1 + } + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala new file mode 100644 index 0000000000000..b2bbfdee56946 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.jobs + +import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.ui.{SparkUI, SparkUITab} + +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { + val sc = parent.sc + val killEnabled = parent.killEnabled + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + val listener = parent.jobProgressListener + + attachPage(new AllJobsPage(this)) + attachPage(new JobPage(this)) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 770d99eea1c9d..5fc6cc7533150 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { +private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { private val sc = parent.sc private val listener = parent.listener @@ -37,8 +37,9 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = - new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getPoolForName(poolName).get).toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 64178e1e33d41..df1899e7a9b84 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,7 +24,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 36afc4942e085..40e05f86b661d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { +private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2ff561ccc7da0..e7d6244dcd679 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -31,11 +31,10 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTableBase( stages: Seq[StageInfo], - parent: JobProgressTab, - killEnabled: Boolean = false) { - - private val listener = parent.listener - protected def isFairScheduler = parent.isFairScheduler + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean, + killEnabled: Boolean) { protected def columns: Seq[Node] = { Stage Id ++ @@ -73,25 +72,11 @@ private[ui] class StageTableBase( } - private def makeProgressBar(started: Int, completed: Int, failed: Int, total: Int): Seq[Node] = - { - val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - val startWidth = "width: %s%%".format((started.toDouble/total)*100) - -
- - {completed}/{total} { if (failed > 0) s"($failed failed)" else "" } - -
-
-
- } - private def makeDescription(s: StageInfo): Seq[Node] = { // scalastyle:off val killLink = if (killEnabled) { val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + .format(UIUtils.prependBaseUri(basePath), s.stageId) val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" .format(s.stageId) @@ -101,7 +86,7 @@ private[ui] class StageTableBase( // scalastyle:on val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId, s.attemptId) + .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -115,7 +100,7 @@ private[ui] class StageTableBase( Text("RDD: ") ++ // scalastyle:off cachedRddInfos.map { i => - {i.name} + {i.name} } // scalastyle:on }} @@ -167,7 +152,7 @@ private[ui] class StageTableBase( {if (isFairScheduler) { + .format(UIUtils.prependBaseUri(basePath), stageData.schedulingPool)}> {stageData.schedulingPool} @@ -180,8 +165,9 @@ private[ui] class StageTableBase( {formattedDuration} - {makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, - stageData.numFailedTasks, s.numTasks)} + {UIUtils.makeProgressBar(started = stageData.numActiveTasks, + completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, + skipped = 0, total = s.numTasks)} {inputReadWithUnit} {outputWriteWithUnit} @@ -195,9 +181,10 @@ private[ui] class StageTableBase( private[ui] class FailedStageTable( stages: Seq[StageInfo], - parent: JobProgressTab, - killEnabled: Boolean = false) - extends StageTableBase(stages, parent, killEnabled) { + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean) + extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled = false) { override protected def columns: Seq[Node] = super.columns ++ Failure Reason diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 03ca918e2e8b3..937261de00e3a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -19,18 +19,16 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.{SparkUI, SparkUITab} -/** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { +/** Web UI showing progress status of all stages in the given SparkContext. */ +private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val sc = parent.sc - val conf = sc.map(_.conf).getOrElse(new SparkConf) - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = parent.killEnabled val listener = parent.jobProgressListener - attachPage(new JobProgressPage(this)) + attachPage(new AllStagesPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 2f7d618df5f6f..48fd7caa1a1ed 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -40,9 +40,28 @@ private[jobs] object UIData { class JobUIData( var jobId: Int = -1, + var startTime: Option[Long] = None, + var endTime: Option[Long] = None, var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, - var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN + var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, + /* Tasks */ + // `numTasks` is a potential underestimate of the true number of tasks that this job will run. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + var numTasks: Int = 0, + var numActiveTasks: Int = 0, + var numCompletedTasks: Int = 0, + var numSkippedTasks: Int = 0, + var numFailedTasks: Int = 0, + /* Stages */ + var numActiveStages: Int = 0, + // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: + var completedStageIndices: OpenHashSet[Int] = new OpenHashSet[Int](), + var numSkippedStages: Int = 0, + var numFailedStages: Int = 0 ) class StageUIData { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7e536edfe807b..7b5db1ed76265 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,6 +31,21 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ +/** + * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- + * and forwards-compatibility guarantees: any version of Spark should be able to read JSON output + * written by any other version, including newer versions. + * + * JsonProtocolSuite contains backwards-compatibility tests which check that the current version of + * JsonProtocol is able to read output written by earlier versions. We do not currently have tests + * for reading newer JSON output with older Spark versions. + * + * To ensure that we provide these guarantees, follow these rules when modifying these methods: + * + * - Never delete any JSON fields. + * - Any new JSON fields should be optional; use `Utils.jsonOption` when reading these fields + * in `*FromJson` methods. + */ private[spark] object JsonProtocol { // TODO: Remove this file and put JSON serialization into each individual class. @@ -121,6 +136,7 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ + ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) } @@ -455,7 +471,12 @@ private[spark] object JsonProtocol { val jobId = (json \ "Job ID").extract[Int] val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - SparkListenerJobStart(jobId, stageIds, properties) + // The "Stage Infos" field was added in Spark 1.2.0 + val stageInfos = Utils.jsonOption(json \ "Stage Infos") + .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + } + SparkListenerJobStart(jobId, stageInfos, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index bacf6a16fc233..d2857b8b55664 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.ui -import org.apache.spark.api.java.StorageLevels -import org.apache.spark.{SparkException, SparkConf, SparkContext} -import org.openqa.selenium.WebDriver +import scala.collection.JavaConversions._ + +import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.shuffle.FetchFailedException /** * Selenium tests for the Spark Web UI. These tests are not run by default @@ -89,7 +93,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") find(id("failed")).get.text should be("Failed Stages (1)") } @@ -101,7 +105,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: @@ -109,4 +113,191 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("spark.ui.killEnabled should properly control kill button display") { + def getSparkContext(killEnabled: Boolean): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + .set("spark.ui.killEnabled", killEnabled.toString) + new SparkContext(conf) + } + + def hasKillLink = find(className("kill-link")).isDefined + def runSlowJob(sc: SparkContext) { + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + } + + withSpark(getSparkContext(killEnabled = true)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(hasKillLink) + } + } + + withSpark(getSparkContext(killEnabled = false)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(!hasKillLink) + } + } + } + + test("jobs page should not display job group name unless some job was submitted in a job group") { + withSpark(newSparkContext()) { sc => + // If no job has been run in a job group, then "(Job Group)" should not appear in the header + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should not contain "Job Id (Job Group)" + } + // Once at least one job has been run in a job group, then we should display the group name: + sc.setJobGroup("my-job-group", "my-job-group-description") + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should contain ("Job Id (Job Group)") + } + } + } + + test("job progress bars should handle stage / task failures") { + withSpark(newSparkContext()) { sc => + val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) + val shuffleHandle = + data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + // Simulate fetch failures: + val mappedData = data.map { x => + val taskContext = TaskContext.get + if (taskContext.attemptId() == 1) { // Cause this stage to fail on its first attempt. + val env = SparkEnv.get + val bmAddress = env.blockManager.blockManagerId + val shuffleId = shuffleHandle.shuffleId + val mapId = 0 + val reduceId = taskContext.partitionId() + val message = "Simulated fetch failure" + throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + } else { + x + } + } + mappedData.count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") + // Ideally, the following test would pass, but currently we overcount completed tasks + // if task recomputations occur: + // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") + // Instead, we guarantee that the total number of tasks is always correct, while the number + // of completed tasks may be higher: + find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") + } + } + } + + test("job details page should display useful information for stages that haven't started") { + withSpark(newSparkContext()) { sc => + // Create a multi-stage job with a long delay in the first stage: + val rdd = sc.parallelize(Seq(1, 2, 3)).map { x => + // This long sleep call won't slow down the tests because we don't actually need to wait + // for the job to finish. + Thread.sleep(20000) + }.groupBy(identity).map(identity).groupBy(identity).map(identity) + // Start the job: + rdd.countAsync() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + find(id("active")).get.text should be ("Active Stages (1)") + find(id("pending")).get.text should be ("Pending Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } + + test("job progress bars / cells reflect skipped stages / tasks") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = sc.parallelize(1 to 8, 8) + .map(x => x).groupBy((x: Int) => x, numPartitions = 8) + .flatMap(x => x._2).groupBy((x: Int) => x, numPartitions = 8) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + // The completed jobs table should have two rows. The first row will be the most recent job: + val firstRow = find(cssSelector("tbody tr")).get.underlying + val firstRowColumns = firstRow.findElements(By.tagName("td")) + firstRowColumns(0).getText should be ("1") + firstRowColumns(4).getText should be ("1/1 (2 skipped)") + firstRowColumns(5).getText should be ("8/8 (16 skipped)") + // The second row is the first run of the job, where nothing was skipped: + val secondRow = findAll(cssSelector("tbody tr")).toSeq(1).underlying + val secondRowColumns = secondRow.findElements(By.tagName("td")) + secondRowColumns(0).getText should be ("0") + secondRowColumns(4).getText should be ("3/3") + secondRowColumns(5).getText should be ("24/24") + } + } + } + + test("stages that aren't run appear as 'skipped stages' after a job finishes") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") + find(id("pending")) should be (None) + find(id("active")) should be (None) + find(id("failed")) should be (None) + find(id("completed")).get.text should be ("Completed Stages (1)") + find(id("skipped")).get.text should be ("Skipped Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } + + test("jobs with stages that are skipped should show correct link descriptions on all jobs page") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + findAll(cssSelector("tbody tr a")).foreach { link => + link.text.toLowerCase should include ("count") + link.text.toLowerCase should not include "unknown" + } + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 15c5b4e702efa..12af60caf7d54 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -43,7 +43,10 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def createJobStartEvent(jobId: Int, stageIds: Seq[Int]) = { - SparkListenerJobStart(jobId, stageIds) + val stageInfos = stageIds.map { stageId => + new StageInfo(stageId, 0, stageId.toString, 0, null, "") + } + SparkListenerJobStart(jobId, stageInfos) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { @@ -52,8 +55,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { + val stagesThatWontBeRun = jobId * 200 to jobId * 200 + 10 val stageIds = jobId * 100 to jobId * 100 + 50 - listener.onJobStart(createJobStartEvent(jobId, stageIds)) + listener.onJobStart(createJobStartEvent(jobId, stageIds ++ stagesThatWontBeRun)) for (stageId <- stageIds) { listener.onStageSubmitted(createStageStartEvent(stageId)) listener.onStageCompleted(createStageEndEvent(stageId, failed = stageId % 2 == 0)) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 50f42054b9296..0bc9492675863 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -47,7 +47,12 @@ class JsonProtocolSuite extends FunSuite { val taskEndWithOutput = SparkListenerTaskEnd(1, 0, "ResultTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) - val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobStart = { + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => + makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) + SparkListenerJobStart(10, stageInfos, properties) + } val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), @@ -224,6 +229,19 @@ class JsonProtocolSuite extends FunSuite { assert(expectedExecutorLostFailure === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("SparkListenerJobStart backward compatibility") { + // Prior to Spark 1.2.0, SparkListenerJobStart did not have a "Stage Infos" property. + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) + val dummyStageInfos = + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + val jobStart = SparkListenerJobStart(10, stageInfos, properties) + val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) + val expectedJobStart = + SparkListenerJobStart(10, dummyStageInfos, properties) + assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -306,7 +324,7 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => assert(e1.jobId === e2.jobId) assert(e1.properties === e2.properties) - assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 === i2)) + assert(e1.stageIds === e2.stageIds) case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => assert(e1.jobId === e2.jobId) assertEquals(e1.jobResult, e2.jobResult) @@ -1051,6 +1069,260 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobStart", | "Job ID": 10, + | "Stage Infos": [ + | { + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [ + | { + | "RDD ID": 1, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 200, + | "Number of Cached Partitions": 300, + | "Memory Size": 400, + | "Tachyon Size": 0, + | "Disk Size": 500 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 2, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 400, + | "RDD Info": [ + | { + | "RDD ID": 2, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 400, + | "Number of Cached Partitions": 600, + | "Memory Size": 800, + | "Tachyon Size": 0, + | "Disk Size": 1000 + | }, + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 401, + | "Number of Cached Partitions": 601, + | "Memory Size": 801, + | "Tachyon Size": 0, + | "Disk Size": 1001 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 3, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 600, + | "RDD Info": [ + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 600, + | "Number of Cached Partitions": 900, + | "Memory Size": 1200, + | "Tachyon Size": 0, + | "Disk Size": 1500 + | }, + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 601, + | "Number of Cached Partitions": 901, + | "Memory Size": 1201, + | "Tachyon Size": 0, + | "Disk Size": 1501 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 602, + | "Number of Cached Partitions": 902, + | "Memory Size": 1202, + | "Tachyon Size": 0, + | "Disk Size": 1502 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 4, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 800, + | "RDD Info": [ + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 800, + | "Number of Cached Partitions": 1200, + | "Memory Size": 1600, + | "Tachyon Size": 0, + | "Disk Size": 2000 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 801, + | "Number of Cached Partitions": 1201, + | "Memory Size": 1601, + | "Tachyon Size": 0, + | "Disk Size": 2001 + | }, + | { + | "RDD ID": 6, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 802, + | "Number of Cached Partitions": 1202, + | "Memory Size": 1602, + | "Tachyon Size": 0, + | "Disk Size": 2002 + | }, + | { + | "RDD ID": 7, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 803, + | "Number of Cached Partitions": 1203, + | "Memory Size": 1603, + | "Tachyon Size": 0, + | "Disk Size": 2003 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | } + | ], | "Stage IDs": [ | 1, | 2, From cb0e9b0980f38befe88bf52aa037fe33262730f7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Nov 2014 13:50:20 -0800 Subject: [PATCH 18/18] [SPARK-4518][SPARK-4519][Streaming] Refactored file stream to prevent files from being processed multiple times Because of a corner case, a file already selected for batch t can get considered again for batch t+2. This refactoring fixes it by remembering all the files selected in the last 1 minute, so that this corner case does not arise. Also uses spark context's hadoop configuration to access the file system API for listing directories. pwendell Please take look. I still have not run long-running integration tests, so I cannot say for sure whether this has indeed solved the issue. You could do a first pass on this in the meantime. Author: Tathagata Das Closes #3419 from tdas/filestream-fix2 and squashes the following commits: c19dd8a [Tathagata Das] Addressed PR comments. 513b608 [Tathagata Das] Updated docs. d364faf [Tathagata Das] Added the current time condition back 5526222 [Tathagata Das] Removed unnecessary imports. 38bb736 [Tathagata Das] Fix long line. 203bbc7 [Tathagata Das] Un-ignore tests. eaef4e1 [Tathagata Das] Fixed SPARK-4519 9dbd40a [Tathagata Das] Refactored FileInputDStream to remember last few batches. --- .../spark/streaming/dstream/DStream.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 291 +++++++++++------- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/InputStreamsSuite.scala | 106 ++++--- 4 files changed, 245 insertions(+), 156 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index eabd61d713e0c..dbf1ebbaf653a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -254,7 +254,7 @@ abstract class DStream[T: ClassTag] ( } private[streaming] def remember(duration: Duration) { - if (duration != null && duration > rememberDuration) { + if (duration != null && (rememberDuration == null || duration > rememberDuration)) { rememberDuration = duration logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 55d6cf6a783ea..5f13fdc5579ed 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -17,18 +17,55 @@ package org.apache.spark.streaming.dstream -import java.io.{ObjectInputStream, IOException} -import scala.collection.mutable.{HashSet, HashMap} +import java.io.{IOException, ObjectInputStream} + +import scala.collection.mutable import scala.reflect.ClassTag + import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.util.{TimeStampedHashMap, Utils} +import org.apache.spark.rdd.{RDD, UnionRDD} +import org.apache.spark.streaming._ +import org.apache.spark.util.{TimeStampedHashMap, Utils} +/** + * This class represents an input stream that monitors a Hadoop-compatible filesystem for new + * files and creates a stream out of them. The way it works as follows. + * + * At each batch interval, the file system is queried for files in the given directory and + * detected new files are selected for that batch. In this case "new" means files that + * became visible to readers during that time period. Some extra care is needed to deal + * with the fact that files may become visible after they are created. For this purpose, this + * class remembers the information about the files selected in past batches for + * a certain duration (say, "remember window") as shown in the figure below. + * + * |<----- remember window ----->| + * ignore threshold --->| |<--- current batch time + * |____.____.____.____.____.____| + * | | | | | | | + * ---------------------|----|----|----|----|----|----|-----------------------> Time + * |____|____|____|____|____|____| + * remembered batches + * + * The trailing end of the window is the "ignore threshold" and all files whose mod times + * are less than this threshold are assumed to have already been selected and are therefore + * ignored. Files whose mod times are within the "remember window" are checked against files + * that have already been selected. At a high level, this is how new files are identified in + * each batch - files whose mod times are greater than the ignore threshold and + * have not been considered within the remember window. See the documentation on the method + * `isNewFile` for more details. + * + * This makes some assumptions from the underlying file system that the system is monitoring. + * - The clock of the file system is assumed to synchronized with the clock of the machine running + * the streaming app. + * - If a file is to be visible in the directory listings, it must be visible within a certain + * duration of the mod time of the file. This duration is the "remember window", which is set to + * 1 minute (see `FileInputDStream.MIN_REMEMBER_DURATION`). Otherwise, the file will never be + * selected as the mod time will be less than the ignore threshold when it becomes visible. + * - Once a file is visible, the mod time cannot change. If it does due to appends, then the + * processing semantics are undefined. + */ private[streaming] class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag]( @transient ssc_ : StreamingContext, @@ -37,22 +74,37 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas newFilesOnly: Boolean = true) extends InputDStream[(K, V)](ssc_) { + // Data to be saved as part of the streaming checkpoints protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData - // files found in the last interval - private val lastFoundFiles = new HashSet[String] + // Initial ignore threshold based on which old, existing files in the directory (at the time of + // starting the streaming application) will be ignored or considered + private val initialModTimeIgnoreThreshold = if (newFilesOnly) System.currentTimeMillis() else 0L + + /* + * Make sure that the information of files selected in the last few batches are remembered. + * This would allow us to filter away not-too-old files which have already been recently + * selected and processed. + */ + private val numBatchesToRemember = FileInputDStream.calculateNumBatchesToRemember(slideDuration) + private val durationToRemember = slideDuration * numBatchesToRemember + remember(durationToRemember) - // Files with mod time earlier than this is ignored. This is updated every interval - // such that in the current interval, files older than any file found in the - // previous interval will be ignored. Obviously this time keeps moving forward. - private var ignoreTime = if (newFilesOnly) System.currentTimeMillis() else 0L + // Map of batch-time to selected file info for the remembered batches + @transient private[streaming] var batchTimeToSelectedFiles = + new mutable.HashMap[Time, Array[String]] + + // Set of files that were selected in the remembered batches + @transient private var recentlySelectedFiles = new mutable.HashSet[String]() + + // Read-through cache of file mod times, used to speed up mod time lookups + @transient private var fileToModTime = new TimeStampedHashMap[String, Long](true) + + // Timestamp of the last round of finding files + @transient private var lastNewFileFindingTime = 0L - // Latest file mod time seen till any point of time @transient private var path_ : Path = null @transient private var fs_ : FileSystem = null - @transient private[streaming] var files = new HashMap[Time, Array[String]] - @transient private var fileModTimes = new TimeStampedHashMap[String, Long](true) - @transient private var lastNewFileFindingTime = 0L override def start() { } @@ -68,54 +120,113 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas * the previous call. */ override def compute(validTime: Time): Option[RDD[(K, V)]] = { - assert(validTime.milliseconds >= ignoreTime, - "Trying to get new files for a really old time [" + validTime + " < " + ignoreTime + "]") - // Find new files - val (newFiles, minNewFileModTime) = findNewFiles(validTime.milliseconds) + val newFiles = findNewFiles(validTime.milliseconds) logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) - if (!newFiles.isEmpty) { - lastFoundFiles.clear() - lastFoundFiles ++= newFiles - ignoreTime = minNewFileModTime - } - files += ((validTime, newFiles.toArray)) + batchTimeToSelectedFiles += ((validTime, newFiles)) + recentlySelectedFiles ++= newFiles Some(filesToRDD(newFiles)) } /** Clear the old time-to-files mappings along with old RDDs */ protected[streaming] override def clearMetadata(time: Time) { super.clearMetadata(time) - val oldFiles = files.filter(_._1 < (time - rememberDuration)) - files --= oldFiles.keys + val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration)) + batchTimeToSelectedFiles --= oldFiles.keys + recentlySelectedFiles --= oldFiles.values.flatten logInfo("Cleared " + oldFiles.size + " old files that were older than " + (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) logDebug("Cleared files are:\n" + oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) // Delete file mod times that weren't accessed in the last round of getting new files - fileModTimes.clearOldValues(lastNewFileFindingTime - 1) + fileToModTime.clearOldValues(lastNewFileFindingTime - 1) } /** - * Find files which have modification timestamp <= current time and return a 3-tuple of - * (new files found, latest modification time among them, files with latest modification time) + * Find new files for the batch of `currentTime`. This is done by first calculating the + * ignore threshold for file mod times, and then getting a list of files filtered based on + * the current batch time and the ignore threshold. The ignore threshold is the max of + * initial ignore threshold and the trailing end of the remember window (that is, which ever + * is later in time). */ - private def findNewFiles(currentTime: Long): (Seq[String], Long) = { - logDebug("Trying to get new files for time " + currentTime) - lastNewFileFindingTime = System.currentTimeMillis - val filter = new CustomPathFilter(currentTime) - val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) - val timeTaken = System.currentTimeMillis - lastNewFileFindingTime - logInfo("Finding new files took " + timeTaken + " ms") - logDebug("# cached file times = " + fileModTimes.size) - if (timeTaken > slideDuration.milliseconds) { - logWarning( - "Time taken to find new files exceeds the batch size. " + - "Consider increasing the batch size or reduceing the number of " + - "files in the monitored directory." + private def findNewFiles(currentTime: Long): Array[String] = { + try { + lastNewFileFindingTime = System.currentTimeMillis + + // Calculate ignore threshold + val modTimeIgnoreThreshold = math.max( + initialModTimeIgnoreThreshold, // initial threshold based on newFilesOnly setting + currentTime - durationToRemember.milliseconds // trailing end of the remember window ) + logDebug(s"Getting new files for time $currentTime, " + + s"ignoring files older than $modTimeIgnoreThreshold") + val filter = new PathFilter { + def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) + } + val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) + val timeTaken = System.currentTimeMillis - lastNewFileFindingTime + logInfo("Finding new files took " + timeTaken + " ms") + logDebug("# cached file times = " + fileToModTime.size) + if (timeTaken > slideDuration.milliseconds) { + logWarning( + "Time taken to find new files exceeds the batch size. " + + "Consider increasing the batch size or reducing the number of " + + "files in the monitored directory." + ) + } + newFiles + } catch { + case e: Exception => + logWarning("Error finding new files", e) + reset() + Array.empty + } + } + + /** + * Identify whether the given `path` is a new file for the batch of `currentTime`. For it to be + * accepted, it has to pass the following criteria. + * - It must pass the user-provided file filter. + * - It must be newer than the ignore threshold. It is assumed that files older than the ignore + * threshold have already been considered or are existing files before start + * (when newFileOnly = true). + * - It must not be present in the recently selected files that this class remembers. + * - It must not be newer than the time of the batch (i.e. `currentTime` for which this + * file is being tested. This can occur if the driver was recovered, and the missing batches + * (during downtime) are being generated. In that case, a batch of time T may be generated + * at time T+x. Say x = 5. If that batch T contains file of mod time T+5, then bad things can + * happen. Let's say the selected files are remembered for 60 seconds. At time t+61, + * the batch of time t is forgotten, and the ignore threshold is still T+1. + * The files with mod time T+5 are not remembered and cannot be ignored (since, t+5 > t+1). + * Hence they can get selected as new files again. To prevent this, files whose mod time is more + * than current batch time are not considered. + */ + private def isNewFile(path: Path, currentTime: Long, modTimeIgnoreThreshold: Long): Boolean = { + val pathStr = path.toString + // Reject file if it does not satisfy filter + if (!filter(path)) { + logDebug(s"$pathStr rejected by filter") + return false + } + // Reject file if it was created before the ignore time + val modTime = getFileModTime(path) + if (modTime <= modTimeIgnoreThreshold) { + // Use <= instead of < to avoid SPARK-4518 + logDebug(s"$pathStr ignored as mod time $modTime <= ignore time $modTimeIgnoreThreshold") + return false } - (newFiles, filter.minNewFileModTime) + // Reject file if mod time > current batch time + if (modTime > currentTime) { + logDebug(s"$pathStr not selected as mod time $modTime > current time $currentTime") + return false + } + // Reject file if it was considered earlier + if (recentlySelectedFiles.contains(pathStr)) { + logDebug(s"$pathStr already considered") + return false + } + logDebug(s"$pathStr accepted with mod time $modTime") + return true } /** Generate one RDD from an array of files */ @@ -132,21 +243,21 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas new UnionRDD(context.sparkContext, fileRDDs) } + /** Get file mod time from cache or fetch it from the file system */ + private def getFileModTime(path: Path) = { + fileToModTime.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) + } + private def directoryPath: Path = { if (path_ == null) path_ = new Path(directory) path_ } private def fs: FileSystem = { - if (fs_ == null) fs_ = directoryPath.getFileSystem(new Configuration()) + if (fs_ == null) fs_ = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) fs_ } - private def getFileModTime(path: Path) = { - // Get file mod time from cache or fetch it from the file system - fileModTimes.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) - } - private def reset() { fs_ = null } @@ -155,9 +266,10 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() - generatedRDDs = new HashMap[Time, RDD[(K,V)]] () - files = new HashMap[Time, Array[String]] - fileModTimes = new TimeStampedHashMap[String, Long](true) + generatedRDDs = new mutable.HashMap[Time, RDD[(K,V)]] () + batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]]() + recentlySelectedFiles = new mutable.HashSet[String]() + fileToModTime = new TimeStampedHashMap[String, Long](true) } /** @@ -167,11 +279,11 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas private[streaming] class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] + def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] override def update(time: Time) { hadoopFiles.clear() - hadoopFiles ++= files + hadoopFiles ++= batchTimeToSelectedFiles } override def cleanup(time: Time) { } @@ -182,7 +294,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas // Restore the metadata in both files and generatedRDDs logInfo("Restoring files for time " + t + " - " + f.mkString("[", ", ", "]") ) - files += ((t, f)) + batchTimeToSelectedFiles += ((t, f)) + recentlySelectedFiles ++= f generatedRDDs += ((t, filesToRDD(f))) } } @@ -193,57 +306,25 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" } } +} + +private[streaming] +object FileInputDStream { /** - * Custom PathFilter class to find new files that - * ... have modification time more than ignore time - * ... have not been seen in the last interval - * ... have modification time less than maxModTime + * Minimum duration of remembering the information of selected files. Files with mod times + * older than this "window" of remembering will be ignored. So if new files are visible + * within this window, then the file will get selected in the next batch. */ - private[streaming] - class CustomPathFilter(maxModTime: Long) extends PathFilter { + private val MIN_REMEMBER_DURATION = Minutes(1) - // Minimum of the mod times of new files found in the current interval - var minNewFileModTime = -1L + def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") - def accept(path: Path): Boolean = { - try { - if (!filter(path)) { // Reject file if it does not satisfy filter - logDebug("Rejected by filter " + path) - return false - } - // Reject file if it was found in the last interval - if (lastFoundFiles.contains(path.toString)) { - logDebug("Mod time equal to last mod time, but file considered already") - return false - } - val modTime = getFileModTime(path) - logDebug("Mod time for " + path + " is " + modTime) - if (modTime < ignoreTime) { - // Reject file if it was created before the ignore time (or, before last interval) - logDebug("Mod time " + modTime + " less than ignore time " + ignoreTime) - return false - } else if (modTime > maxModTime) { - // Reject file if it is too new that considering it may give errors - logDebug("Mod time more than ") - return false - } - if (minNewFileModTime < 0 || modTime < minNewFileModTime) { - minNewFileModTime = modTime - } - logDebug("Accepted " + path) - } catch { - case fnfe: java.io.FileNotFoundException => - logWarning("Error finding new files", fnfe) - reset() - return false - } - true - } + /** + * Calculate the number of last batches to remember, such that all the files selected in + * at least last MIN_REMEMBER_DURATION duration can be remembered. + */ + def calculateNumBatchesToRemember(batchDuration: Duration): Int = { + math.ceil(MIN_REMEMBER_DURATION.milliseconds.toDouble / batchDuration.milliseconds).toInt } } - -private[streaming] -object FileInputDStream { - def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index e5592e52b0d2d..77ff1ca780a58 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -265,7 +265,7 @@ class CheckpointSuite extends TestSuiteBase { // Verify whether files created have been recorded correctly or not var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - def recordedFiles = fileInputDStream.files.values.flatMap(x => x) + def recordedFiles = fileInputDStream.batchTimeToSelectedFiles.values.flatten assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index fa04fa326e370..307052a4a9cbb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -28,9 +28,12 @@ import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} +import scala.concurrent.duration._ +import scala.language.postfixOps import com.google.common.io.Files import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel @@ -38,6 +41,9 @@ import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} import org.apache.spark.rdd.RDD +import org.apache.hadoop.io.{Text, LongWritable} +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat +import org.apache.hadoop.fs.Path class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -91,54 +97,12 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } - test("file input stream") { - // Disable manual clock as FileInputDStream does not work with manual clock - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") - - // Set up the streaming context and input streams - val testDir = Utils.createTempDir() - val ssc = new StreamingContext(conf, batchDuration) - val fileStream = ssc.textFileStream(testDir.toString) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - def output = outputBuffer.flatMap(x => x) - val outputStream = new TestOutputStream(fileStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files in the temporary directory so that Spark Streaming can read data from it - val input = Seq(1, 2, 3, 4, 5) - val expectedOutput = input.map(_.toString) - Thread.sleep(1000) - for (i <- 0 until input.size) { - val file = new File(testDir, i.toString) - Files.write(input(i) + "\n", file, Charset.forName("UTF-8")) - logInfo("Created file " + file) - Thread.sleep(batchDuration.milliseconds) - Thread.sleep(1000) - } - val startTime = System.currentTimeMillis() - Thread.sleep(1000) - val timeTaken = System.currentTimeMillis() - startTime - assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received by Spark Streaming was as expected - logInfo("--------------------------------") - logInfo("output, size = " + outputBuffer.size) - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output, size = " + expectedOutput.size) - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - assert(output.toList === expectedOutput.toList) - - Utils.deleteRecursively(testDir) + test("file input stream - newFilesOnly = true") { + testFileStream(newFilesOnly = true) + } - // Enable manual clock back again for other tests - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + test("file input stream - newFilesOnly = false") { + testFileStream(newFilesOnly = false) } test("multi-thread receiver") { @@ -180,7 +144,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output.sum === numTotalRecords) } - test("queue input stream - oneAtATime=true") { + test("queue input stream - oneAtATime = true") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val queue = new SynchronizedQueue[RDD[String]]() @@ -223,7 +187,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } - test("queue input stream - oneAtATime=false") { + test("queue input stream - oneAtATime = false") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val queue = new SynchronizedQueue[RDD[String]]() @@ -268,6 +232,50 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i) === expectedOutput(i)) } } + + def testFileStream(newFilesOnly: Boolean) { + var ssc: StreamingContext = null + val testDir: File = null + try { + val testDir = Utils.createTempDir() + val existingFile = new File(testDir, "0") + Files.write("0\n", existingFile, Charset.forName("UTF-8")) + + Thread.sleep(1000) + // Set up the streaming context and input streams + val newConf = conf.clone.set( + "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") + ssc = new StreamingContext(newConf, batchDuration) + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputBuffer) + outputStream.register() + ssc.start() + + // Create files in the directory + val input = Seq(1, 2, 3, 4, 5) + input.foreach { i => + Thread.sleep(batchDuration.milliseconds) + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charset.forName("UTF-8")) + logInfo("Created file " + file) + } + + // Verify that all the files have been read + val expectedOutput = if (newFilesOnly) { + input.map(_.toString).toSet + } else { + (Seq(0) ++ input).map(_.toString).toSet + } + eventually(timeout(maxWaitTimeMillis milliseconds), interval(100 milliseconds)) { + assert(outputBuffer.flatten.toSet === expectedOutput) + } + } finally { + if (ssc != null) ssc.stop() + if (testDir != null) Utils.deleteRecursively(testDir) + } + } }

res!o$t4m)c9285wfdWKK~4 z1n9CudP(?^CK6?89MRyXdYn5D4^!-RgB}8t05s^O95zcF3)IXkPCtA}vNlD+*cMEk zQ{LanBvhL7U^^$iRvq~AiTl6DydRD|@uO0^j=ty6;IXd=vh8uq2jaL&sxc5kDFqBp zd@EoW7jh4V=yo77CPk2GFoHpd_V@@ATvlUErnCFk@$cPYaO5ddeALFH5+e5|5si)5p}@==}5aDHrt<7%vx18Z`m8 zkOl=@RMlfT^Eu!#iI%Vt>_M@y6`>cO3m0nJbvDREulCB`hN^XG1fZzGa70WvSy7SQ zr6PSVHx@yv#aq`_pOrIi5Ay9u3vZc=SECcvgLMk_BL7_$s0)NAi6bz1uvqzX&yN6L z=aM<5?RmFM+>+Gw1RelT#OoxIrNef}1rpO|S`*%Ri`D)F^}|)Ke$Pfw zB8nc*Bm^`=M{!Ov2q{Id1UJlI|B7Uu?|Fi#*xqZu%YDTTV8!EzxrEUNeg{GN^N#kE zESc+|&n1)6xy!y^IncMu_G>!uUnzB&F#h}*f`Bk|9*W_c6-3m-606CUfXc2T1o;ri zQ@a!+kQjwy&xfGhXj?E4MG}G0xfiGpyn7XzwIODuKXBXW2Zy;`_eJ(x`&B3OW>qIR!E;l+^{5b0n1Xn$7cI_yF z;|=}j3l6bwh(Uq9-H$v~>|2G)`bR#Rb1NAzF!OhW0Pn9lL!;EkM>wGijClabg%&^< zS7*J*CiXTqHkQL_N9pn2PZ1{Wz?jAi$^pgCM9pfq*sqSequO#WrWBkL0)N2kPJ>zr z)p#*aB4P)a@@N&S(t!VL78nJ@Zq7)85B^rn?*0-upAzE$sM7QpsAcO6yOAuf*4O>g z+O3b?gq*Sd{L)MSaB_NF(WBn(lXAZg2rh||bjh!e@5>igbsE8Do|*QoG{*X6=47XY z-f0A*4z_%gjFG-GChr-4j;{AbN}5gu2CEUtmQ~2xiudo7o9%WS4mXmn1=rfldbhto ze)ATw5~xIWer5YU8<$9`1|rp7MtSU}OgGPxy|gDfWCUFhV}Cf9%KrO5;f`G_lCceA%Huc`np_=GX_Oyrd->uARZH`Amq zCko)ig5V>u{I*)Ed+oeC6a0)xCxy}=L~yg>6i*Q*Qe9nr4g&Oo&@ePFH-Xc*PQM!g zI4CgQ0@z{H2w_DNmwGYAwTG*Ef zU_DI*>L5iR(G7cus@@#~y3vi-MR7*&6y@aVN}dc@u6keT?oH(eexS8uMGz5&Ge{vD zQ&EjttSFt2DoXKSES_aJm(^g&@cA>Pw^YBY zq4ltz?8LN~B5??SEUfexLBv;W{~#$VCA$_Cy(&YJNG3s#*)0mb+a&BP@;zQ}`4S8$ z&~YjktKN5(cQkmvbb`A&Sp2LQvYqb+Zu#brY}18zJ6!XXy6KgzM|@fxUd<=P!$;qi zFU8QYwt9d+G46XVfG_lU5y&E9o@h{y-Q(%TYps0G9#Q)^tmZ2&PD-a;9dgZKTSh&6 zZGv2JZHsGS-t&zml9_Bv=yvFpZFUJ6i7VNGS~TLjyvfHF^n5Y-r*I&2Qv^t`E~rpX z|7_ufTI_Ng^%n_m;HovfOCYqG%z(+f{-rPq8c{YYOn@x_(PG~5=Kpln;BEw!T(ipZ zdE4-|LO8CBcEn;R?hKq(z7fu4nDO5zj(0=_Lzp215e#ZM?H*bHS0 z3C$NP_X}g0u>Lg2=kU5jd6d_sChC_B`kd6-dp7Hfv$MMfx*L%%e zW8UBUd!FBOKlgnaAX~fed?sff`*!7k;HM-Z0U1@G(xK-1 zw2>A39m2>PNK~e*G4RQo1pY|ATe!5UeHlNK$-IUq<)+tYAZ$&BR+bmY_!Ok#%wKbM z)Gb;AcM>0NukdNGAkht$lptpB{fG^@DpB9`j_ub>^thVKltVo-9~U13fQ zA@`9ZWG{)vL0lvF+g%Ii$dWl?Y}sRM+W#*crzFzl1>93ANae8{tfG_=Aa6s*P^!Cy zg@qe;y}I3zF5!YM??~sTUKcr&ejL2(YOC8AL0CgWmvoQp<1KMoh+XpPm%lg}Pq5?* zE=|}jZ{-==(O1>Gsg_;;U|+`H3P-Re8O79z6cAqPW-3+m7WaLsZR_{I_3Vd)6%k>O zleR9L`vpSK{Q-jRE~Yka zl_|T>gPygJx8 z?;Z!1AoEBk%Cwc@9NTo**Z%=OIPjklP`efc_Zud`vJ{z%wAO$Bw0Bq~Ot#!IoTx;& z+0$gc(u+p0MI|NAqy8lG<<;vQ%D$Yf=EmccOa}?cV&92mADnDsdzJ4cVz-qR1%9P+ zEV)Rby-?iO-@KW||LaJtzc7Sg^BL^BK~!`M%2ZZEBLnV1H|tJbLFt(w0~p%Uqjl_M z*I}(-as&|x6*y@N0!%%6uQ-mqI~i%vm8H2O`9~yJt}N16)t+r$@@=!ox+Pk11Y#Y% z`*oS!Qz7)Y|7TOccXW)M@eo*7rA};Zsa90k^-S#{_V~SA%)8bUXc%#Gqb4dUD#A}J zh+*$O;Cq5dtE8}M-2H;Y{>V8JR46CI$`;0FazFc1ywO-)J8$1WPce=xKO!J0c+hDSqR6ow%7 zuuWw;I+nZat>XFl_$76cc?F~_Lqxul&F6SG?nrX9BGSpl`fgJ|!-gtG*y+W0W~n=IN^Sb?X(B> zAU6T`ZK}5d6vN@y{=R&#Y&HtpQy=MUWpltf$-AkNO17H?YsIPejT^5Kgo|*bu8t5L zfNcwKh}dG#QZw-y>}E>7+7Jjz`V;X<&fV4N4>b^8a$M}MzduM=fSN_bhZ(yIM>ukYyU4YxWJTyp}k8rchcU} zXS&=1g2A_ul9G~VWbLkl0Ep1nP@F+*!y?4&jKFKQ(hc*s4&M0D**P&BT(|mlItDXr zRqoW4pb)`$JyeAf7%O)f>!cbm-DOD)A+4IeQ5} z1l|W7u)Xmmf2l!O36Muz z8TWuwpbd6AOM1ORpR?>Nihv++@1xk1*nym%dCT}L>tic*NvLyxUnM!V+3b^y(GQ!q z1OI_nd-^gyJC_<@^`5VVytu9i^l~F;*oG*aJgx(GsXZk>UzOO%Vp*zHUUV98D`$7= z%E|o^=s&_s1HkO=r%V0Lq(DW~p~s6re6xmVS3l6Apm75$FCY7^*5^=(MVi1JVvuVjt_V`%V%y&42~DKo zjj5?aK;3G28y$8~(bcpNP(zEdxByzXmAgp^r6Z6}^+1b2(>Hg5e;= zTRVeQ$RGSTpHoJJV)R31PC7pFLo-`juY7z7&GY;NLAxR$@ZodAQY2(pNL`s27$VpW zC|Nlawl34>T%eTzqxNdNwrjYLt(XSZ`~XjBh2;=d?u}D+C$4N>52(MQSaAw{ZBuDpS9ug;?;9@ovof Date: Fri, 21 Nov 2014 00:42:43 -0800 Subject: [PATCH 06/18] [SPARK-4472][Shell] Print "Spark context available as sc." only when SparkContext is created... ... successfully It's weird that printing "Spark context available as sc" when creating SparkContext unsuccessfully. Author: zsxwing Closes #3341 from zsxwing/SPARK-4472 and squashes the following commits: 4850093 [zsxwing] Print "Spark context available as sc." only when SparkContext is created successfully --- .../main/scala/org/apache/spark/repl/SparkILoopInit.scala | 7 +++++-- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 7 +++++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 7667a9c11979e..da4286c5e4874 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -121,11 +121,14 @@ trait SparkILoopInit { def initializeSpark() { intp.beQuietDuring { command(""" - @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); + @transient val sc = { + val _sc = org.apache.spark.repl.Main.interp.createSparkContext() + println("Spark context available as sc.") + _sc + } """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") } // code to be executed only after the interpreter is initialized diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index a591e9fc4622b..250727305970d 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -61,11 +61,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter) def initializeSpark() { intp.beQuietDuring { command( """ - @transient val sc = org.apache.spark.repl.Main.createSparkContext(); + @transient val sc = { + val _sc = org.apache.spark.repl.Main.createSparkContext() + println("Spark context available as sc.") + _sc + } """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") } /** Print a welcome message */ From 65b987c3ed79b8362dda53b70434652d2d4840da Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 21 Nov 2014 10:06:30 -0800 Subject: [PATCH 07/18] [SPARK-4397][Core] Reorganize 'implicit's to improve the API convenience This PR moved `implicit`s to `package object` and `companion object` to enable the Scala compiler search them automatically without explicit importing. It should not break any API. A test project for backforward compatibility is [here](https://github.com/zsxwing/SPARK-4397-Backforward-Compatibility). It proves the codes compiled with Spark 1.1.0 can run with this PR. To summarize, the changes are: * Deprecated the old implicit conversion functions: this preserves binary compatibility for code compiled against earlier versions of Spark. * Removed "implicit" from them so they are just normal functions: this made sure the compiler doesn't get confused and warn about multiple implicits in scope. * Created new implicit functions in package rdd object, which is part of the scope that scalac will search when looking for implicit conversions on various RDD objects. The disadvantage is there are duplicated codes in SparkContext for backforward compatibility. Author: zsxwing Closes #3262 from zsxwing/SPARK-4397 and squashes the following commits: fc30314 [zsxwing] Update the comments 9c27aff [zsxwing] Move implicit functions to object RDD and forward old functions to new implicit ones directly 2b5f5a4 [zsxwing] Comments for the deprecated functions 52353de [zsxwing] Remove private[spark] from object WritableConverter 34641d4 [zsxwing] Move ImplicitSuite to org.apache.sparktest 7266218 [zsxwing] Add comments to warn the duplicate codes in SparkContext 185c12f [zsxwing] Remove simpleWritableConverter from SparkContext 3bdcae2 [zsxwing] Move WritableConverter implicits to object WritableConverter 9b73188 [zsxwing] Fix the code style issue 3ac4f07 [zsxwing] Add license header 1eda9e4 [zsxwing] Reorganize 'implicit's to improve the API convenience --- .../scala/org/apache/spark/Accumulators.scala | 30 ++++ .../scala/org/apache/spark/SparkContext.scala | 161 +++++++++++++----- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 30 ++++ .../org/apache/spark/SparkContextSuite.scala | 2 +- .../org/apache/sparktest/ImplicitSuite.scala | 128 ++++++++++++++ 7 files changed, 311 insertions(+), 44 deletions(-) create mode 100644 core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 2301caafb07ff..dc1e8f6c21b62 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -244,6 +244,36 @@ trait AccumulatorParam[T] extends AccumulableParam[T, T] { } } +object AccumulatorParam { + + // The following implicit objects were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, as there are duplicate codes in SparkContext for backward + // compatibility, please update them accordingly if you modify the following implicit objects. + + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + def addInPlace(t1: Double, t2: Double): Double = t1 + t2 + def zero(initialValue: Double) = 0.0 + } + + implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + def addInPlace(t1: Int, t2: Int): Int = t1 + t2 + def zero(initialValue: Int) = 0 + } + + implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + def addInPlace(t1: Long, t2: Long) = t1 + t2 + def zero(initialValue: Long) = 0L + } + + implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + def addInPlace(t1: Float, t2: Float) = t1 + t2 + def zero(initialValue: Float) = 0f + } + + // TODO: Add AccumulatorParams for other types, e.g. lists and strings +} + // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right private object Accumulators { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ae8bbfb56f493..586c1ccaca72b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1624,47 +1624,74 @@ object SparkContext extends Logging { private[spark] val DRIVER_IDENTIFIER = "" - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + // The following deprecated objects have already been copied to `object AccumulatorParam` to + // make the compiler find them automatically. They are duplicate codes only for backward + // compatibility, please update `object AccumulatorParam` accordingly if you plan to modify the + // following ones. + + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 } - implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int) = 0 } - implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 def zero(initialValue: Long) = 0L } - implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float) = t1 + t2 def zero(initialValue: Float) = 0f } - // TODO: Add AccumulatorParams for other types, e.g. lists and strings + // The following deprecated functions have already been moved to `object RDD` to + // make the compiler find them automatically. They are still kept here for backward compatibility + // and just call the corresponding functions in `object RDD`. - implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { - new PairRDDFunctions(rdd) + RDD.rddToPairRDDFunctions(rdd) } - implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) - implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = - new SequenceFileRDDFunctions(rdd) + RDD.rddToSequenceFileRDDFunctions(rdd) - implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = - new OrderedRDDFunctions[K, V, (K, V)](rdd) + RDD.rddToOrderedRDDFunctions(rdd) - implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) - implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = - new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + RDD.numericRDDToDoubleRDDFunctions(rdd) // Implicit conversions to common Writable types, for saveAsSequenceFile @@ -1690,40 +1717,49 @@ object SparkContext extends Logging { arr.map(x => anyToWritable(x)).toArray) } - // Helper objects for converting common types to Writable - private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) - : WritableConverter[T] = { - val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] - new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) - } + // The following deprecated functions have already been moved to `object WritableConverter` to + // make the compiler find them automatically. They are still kept here for backward compatibility + // and just call the corresponding functions in `object WritableConverter`. - implicit def intWritableConverter(): WritableConverter[Int] = - simpleWritableConverter[Int, IntWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def intWritableConverter(): WritableConverter[Int] = + WritableConverter.intWritableConverter() - implicit def longWritableConverter(): WritableConverter[Long] = - simpleWritableConverter[Long, LongWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def longWritableConverter(): WritableConverter[Long] = + WritableConverter.longWritableConverter() - implicit def doubleWritableConverter(): WritableConverter[Double] = - simpleWritableConverter[Double, DoubleWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def doubleWritableConverter(): WritableConverter[Double] = + WritableConverter.doubleWritableConverter() - implicit def floatWritableConverter(): WritableConverter[Float] = - simpleWritableConverter[Float, FloatWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def floatWritableConverter(): WritableConverter[Float] = + WritableConverter.floatWritableConverter() - implicit def booleanWritableConverter(): WritableConverter[Boolean] = - simpleWritableConverter[Boolean, BooleanWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def booleanWritableConverter(): WritableConverter[Boolean] = + WritableConverter.booleanWritableConverter() - implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { - simpleWritableConverter[Array[Byte], BytesWritable](bw => - // getBytes method returns array which is longer then data to be returned - Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) - ) - } + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def bytesWritableConverter(): WritableConverter[Array[Byte]] = + WritableConverter.bytesWritableConverter() - implicit def stringWritableConverter(): WritableConverter[String] = - simpleWritableConverter[String, Text](_.toString) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def stringWritableConverter(): WritableConverter[String] = + WritableConverter.stringWritableConverter() - implicit def writableWritableConverter[T <: Writable]() = - new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def writableWritableConverter[T <: Writable]() = + WritableConverter.writableWritableConverter() /** * Find the JAR from which a given class was loaded, to make it easy for users to pass @@ -1950,3 +1986,46 @@ private[spark] class WritableConverter[T]( val writableClass: ClassTag[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable + +object WritableConverter { + + // Helper objects for converting common types to Writable + private[spark] def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) + : WritableConverter[T] = { + val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] + new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) + } + + // The following implicit functions were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, we still keep the old functions in SparkContext for backward + // compatibility and forward to the following functions directly. + + implicit def intWritableConverter(): WritableConverter[Int] = + simpleWritableConverter[Int, IntWritable](_.get) + + implicit def longWritableConverter(): WritableConverter[Long] = + simpleWritableConverter[Long, LongWritable](_.get) + + implicit def doubleWritableConverter(): WritableConverter[Double] = + simpleWritableConverter[Double, DoubleWritable](_.get) + + implicit def floatWritableConverter(): WritableConverter[Float] = + simpleWritableConverter[Float, FloatWritable](_.get) + + implicit def booleanWritableConverter(): WritableConverter[Boolean] = + simpleWritableConverter[Boolean, BooleanWritable](_.get) + + implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { + simpleWritableConverter[Array[Byte], BytesWritable](bw => + // getBytes method returns array which is longer then data to be returned + Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) + ) + } + + implicit def stringWritableConverter(): WritableConverter[String] = + simpleWritableConverter[String, Text](_.toString) + + implicit def writableWritableConverter[T <: Writable]() = + new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e37f3acaf6e30..7af3538262fd6 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -32,13 +32,13 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ -import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} +import org.apache.spark.rdd.RDD.rddToPairRDDFunctions import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 6a6d9bf6857d3..97f5c9f257e09 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ -import org.apache.spark.SparkContext._ +import org.apache.spark.AccumulatorParam._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e4025bcf48db6..3add4a76192ca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -21,6 +21,7 @@ import java.util.{Properties, Random} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus @@ -28,6 +29,7 @@ import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ @@ -1383,3 +1385,31 @@ abstract class RDD[T: ClassTag]( new JavaRDD(this)(elementClassTag) } } + +object RDD { + + // The following implicit functions were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, we still keep the old functions in SparkContext for backward + // compatibility and forward to the following functions directly. + + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + new PairRDDFunctions(rdd) + } + + implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) + + implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( + rdd: RDD[(K, V)]) = + new SequenceFileRDDFunctions(rdd) + + implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( + rdd: RDD[(K, V)]) = + new OrderedRDDFunctions[K, V, (K, V)](rdd) + + implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) + + implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) +} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 9e454ddcc52a6..1362022104195 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -82,7 +82,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { bytesWritable.set(inputArray, 0, 10) bytesWritable.set(inputArray, 0, 5) - val converter = SparkContext.bytesWritableConverter() + val converter = WritableConverter.bytesWritableConverter() val byteArray = converter.convert(bytesWritable) assert(byteArray.length === 5) diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala new file mode 100644 index 0000000000000..4918e2d92beb4 --- /dev/null +++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sparktest + +/** + * A test suite to make sure all `implicit` functions work correctly. + * Please don't `import org.apache.spark.SparkContext._` in this class. + * + * As `implicit` is a compiler feature, we don't need to run this class. + * What we need to do is making the compiler happy. + */ +class ImplicitSuite { + + // We only want to test if `implict` works well with the compiler, so we don't need a real + // SparkContext. + def mockSparkContext[T]: org.apache.spark.SparkContext = null + + // We only want to test if `implict` works well with the compiler, so we don't need a real RDD. + def mockRDD[T]: org.apache.spark.rdd.RDD[T] = null + + def testRddToPairRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD + rdd.groupByKey() + } + + def testRddToAsyncRDDActions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[Int] = mockRDD + rdd.countAsync() + } + + def testRddToSequenceFileRDDFunctions(): Unit = { + // TODO eliminating `import intToIntWritable` needs refactoring SequenceFileRDDFunctions. + // That will be a breaking change. + import org.apache.spark.SparkContext.intToIntWritable + val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD + rdd.saveAsSequenceFile("/a/test/path") + } + + def testRddToOrderedRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD + rdd.sortByKey() + } + + def testDoubleRDDToDoubleRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[Double] = mockRDD + rdd.stats() + } + + def testNumericRDDToDoubleRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[Int] = mockRDD + rdd.stats() + } + + def testDoubleAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123.4) + } + + def testIntAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123) + } + + def testLongAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123L) + } + + def testFloatAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123F) + } + + def testIntWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Int, Int]("/a/test/path") + } + + def testLongWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Long, Long]("/a/test/path") + } + + def testDoubleWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Double, Double]("/a/test/path") + } + + def testFloatWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Float, Float]("/a/test/path") + } + + def testBooleanWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Boolean, Boolean]("/a/test/path") + } + + def testBytesWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Array[Byte], Array[Byte]]("/a/test/path") + } + + def testStringWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[String, String]("/a/test/path") + } + + def testWritableWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[org.apache.hadoop.io.Text, org.apache.hadoop.io.Text]("/a/test/path") + } +} From a81918c5a66fc6040f9796fc1a9d4e0bfb8d0cbe Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Nov 2014 12:10:04 -0800 Subject: [PATCH 08/18] SPARK-4532: Fix bug in detection of Hive in Spark 1.2 Because the Hive profile is no longer defined in the root pom, we need to check specifically in the sql/hive pom when we perform the check in make-distribtion.sh. Author: Patrick Wendell Closes #3398 from pwendell/make-distribution and squashes the following commits: 8a58279 [Patrick Wendell] Fix bug in detection of Hive in Spark 1.2 --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 2267b1aa08a6c..7c0fb8992a155 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -119,7 +119,7 @@ VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v " SPARK_HADOOP_VERSION=$(mvn help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) -SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles $@ 2>/dev/null\ +SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\ | grep -v "INFO"\ | fgrep --count "hive";\ # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\ From ce95bd8e130b2c7688b94be40683bdd90d86012d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 21 Nov 2014 15:02:31 -0800 Subject: [PATCH 09/18] [SPARK-4531] [MLlib] cache serialized java object The Pyrolite is pretty slow (comparing to the adhoc serializer in 1.1), it cause much performance regression in 1.2, because we cache the serialized Python object in JVM, deserialize them into Java object in each step. This PR change to cache the deserialized JavaRDD instead of PythonRDD to avoid the deserialization of Pyrolite. It should have similar memory usage as before, but much faster. Author: Davies Liu Closes #3397 from davies/cache and squashes the following commits: 7f6e6ce [Davies Liu] Update -> Updater 4b52edd [Davies Liu] using named argument 63b984e [Davies Liu] fix 7da0332 [Davies Liu] add unpersist() dff33e1 [Davies Liu] address comments c2bdfc2 [Davies Liu] refactor d572f00 [Davies Liu] Merge branch 'master' into cache f1063e1 [Davies Liu] cache serialized java object --- .../mllib/api/python/PythonMLLibAPI.scala | 110 +++++++++--------- .../spark/mllib/clustering/KMeans.scala | 13 +-- .../GeneralizedLinearAlgorithm.scala | 13 +-- python/pyspark/mllib/clustering.py | 8 +- python/pyspark/mllib/common.py | 4 +- python/pyspark/mllib/recommendation.py | 4 +- python/pyspark/mllib/regression.py | 5 +- 7 files changed, 64 insertions(+), 93 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index b6f7618171224..f04df1c156898 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -74,10 +74,28 @@ class PythonMLLibAPI extends Serializable { learner: GeneralizedLinearAlgorithm[_ <: GeneralizedLinearModel], data: JavaRDD[LabeledPoint], initialWeights: Vector): JList[Object] = { - // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. - learner.disableUncachedWarning() - val model = learner.run(data.rdd, initialWeights) - List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + try { + val model = learner.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), initialWeights) + List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + } finally { + data.rdd.unpersist(blocking = false) + } + } + + /** + * Return the Updater from string + */ + def getUpdaterFromString(regType: String): Updater = { + if (regType == "l2") { + new SquaredL2Updater + } else if (regType == "l1") { + new L1Updater + } else if (regType == null || regType == "none") { + new SimpleUpdater + } else { + throw new IllegalArgumentException("Invalid value for 'regType' parameter." + + " Can only be initialized using the following string values: ['l1', 'l2', None].") + } } /** @@ -99,16 +117,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setStepSize(stepSize) .setMiniBatchFraction(miniBatchFraction) - if (regType == "l2") { - lrAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - lrAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - lrAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + lrAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( lrAlg, data, @@ -178,16 +187,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setStepSize(stepSize) .setMiniBatchFraction(miniBatchFraction) - if (regType == "l2") { - SVMAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - SVMAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - SVMAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + SVMAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( SVMAlg, data, @@ -213,16 +213,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setStepSize(stepSize) .setMiniBatchFraction(miniBatchFraction) - if (regType == "l2") { - LogRegAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - LogRegAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - LogRegAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + LogRegAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( LogRegAlg, data, @@ -248,16 +239,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setNumCorrections(corrections) .setConvergenceTol(tolerance) - if (regType == "l2") { - LogRegAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - LogRegAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - LogRegAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + LogRegAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( LogRegAlg, data, @@ -289,9 +271,11 @@ class PythonMLLibAPI extends Serializable { .setMaxIterations(maxIterations) .setRuns(runs) .setInitializationMode(initializationMode) - // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. - .disableUncachedWarning() - kMeansAlg.run(data.rdd) + try { + kMeansAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK)) + } finally { + data.rdd.unpersist(blocking = false) + } } /** @@ -425,16 +409,18 @@ class PythonMLLibAPI extends Serializable { numPartitions: Int, numIterations: Int, seed: Long): Word2VecModelWrapper = { - val data = dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER) val word2vec = new Word2Vec() .setVectorSize(vectorSize) .setLearningRate(learningRate) .setNumPartitions(numPartitions) .setNumIterations(numIterations) .setSeed(seed) - val model = word2vec.fit(data) - data.unpersist() - new Word2VecModelWrapper(model) + try { + val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) + new Word2VecModelWrapper(model) + } finally { + dataJRDD.rdd.unpersist(blocking = false) + } } private[python] class Word2VecModelWrapper(model: Word2VecModel) { @@ -495,8 +481,11 @@ class PythonMLLibAPI extends Serializable { categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap, minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) - - DecisionTree.train(data.rdd, strategy) + try { + DecisionTree.train(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), strategy) + } finally { + data.rdd.unpersist(blocking = false) + } } /** @@ -526,10 +515,15 @@ class PythonMLLibAPI extends Serializable { numClassesForClassification = numClasses, maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap) - if (algo == Algo.Classification) { - RandomForest.trainClassifier(data.rdd, strategy, numTrees, featureSubsetStrategy, seed) - } else { - RandomForest.trainRegressor(data.rdd, strategy, numTrees, featureSubsetStrategy, seed) + val cached = data.rdd.persist(StorageLevel.MEMORY_AND_DISK) + try { + if (algo == Algo.Classification) { + RandomForest.trainClassifier(cached, strategy, numTrees, featureSubsetStrategy, seed) + } else { + RandomForest.trainRegressor(cached, strategy, numTrees, featureSubsetStrategy, seed) + } + } finally { + cached.unpersist(blocking = false) } } @@ -711,7 +705,7 @@ private[spark] object SerDe extends Serializable { def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { if (obj == this) { out.write(Opcodes.GLOBAL) - out.write((module + "\n" + name + "\n").getBytes()) + out.write((module + "\n" + name + "\n").getBytes) } else { pickler.save(this) // it will be memorized by Pickler saveState(obj, out, pickler) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 7443f232ec3e7..34ea0de706f08 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -113,22 +113,13 @@ class KMeans private ( this } - /** Whether a warning should be logged if the input RDD is uncached. */ - private var warnOnUncachedInput = true - - /** Disable warnings about uncached input. */ - private[spark] def disableUncachedWarning(): this.type = { - warnOnUncachedInput = false - this - } - /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. */ def run(data: RDD[Vector]): KMeansModel = { - if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + if (data.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } @@ -143,7 +134,7 @@ class KMeans private ( norms.unpersist() // Warn at the end of the run as well, for increased visibility. - if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + if (data.getStorageLevel == StorageLevel.NONE) { logWarning("The input data was not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 00dfc86c9e0bd..0287f04e2c777 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -136,15 +136,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] this } - /** Whether a warning should be logged if the input RDD is uncached. */ - private var warnOnUncachedInput = true - - /** Disable warnings about uncached input. */ - private[spark] def disableUncachedWarning(): this.type = { - warnOnUncachedInput = false - this - } - /** * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. @@ -161,7 +152,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { - if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } @@ -241,7 +232,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } // Warn at the end of the run as well, for increased visibility. - if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data was not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index fe4c4cc5094d8..e2492eef5bd6a 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -16,7 +16,7 @@ # from pyspark import SparkContext -from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, callJavaFunc from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -80,10 +80,8 @@ class KMeans(object): @classmethod def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - # cache serialized data to avoid objects over head in JVM - jcached = _to_java_object_rdd(rdd.map(_convert_to_vector), cache=True) - model = callMLlibFunc("trainKMeansModel", jcached, k, maxIterations, runs, - initializationMode) + model = callMLlibFunc("trainKMeansModel", rdd.map(_convert_to_vector), k, maxIterations, + runs, initializationMode) centers = callJavaFunc(rdd.context, model.clusterCenters) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index c6149fe391ec8..33c49e2399908 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -54,15 +54,13 @@ def _new_smart_decode(obj): # this will call the MLlib version of pythonToJava() -def _to_java_object_rdd(rdd, cache=False): +def _to_java_object_rdd(rdd): """ Return an JavaRDD of Object by unpickling It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. """ rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) - if cache: - rdd.cache() return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 2bcbf2aaf8e3e..97ec74eda0b71 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -19,7 +19,7 @@ from pyspark import SparkContext from pyspark.rdd import RDD -from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, _to_java_object_rdd +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc __all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] @@ -110,7 +110,7 @@ def _prepare(cls, ratings): ratings = ratings.map(lambda x: Rating(*x)) else: raise ValueError("rating should be RDD of Rating or tuple/list") - return _to_java_object_rdd(ratings, True) + return ratings @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, nonnegative=False, diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index f4f5e615fadc3..210060140fd91 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,7 +18,7 @@ import numpy as np from numpy import array -from pyspark.mllib.common import callMLlibFunc, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', @@ -129,8 +129,7 @@ def _regression_train_wrapper(train_func, modelClass, data, initial_weights): if not isinstance(first, LabeledPoint): raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) initial_weights = initial_weights or [0.0] * len(data.first().features) - weights, intercept = train_func(_to_java_object_rdd(data, cache=True), - _convert_to_vector(initial_weights)) + weights, intercept = train_func(data, _convert_to_vector(initial_weights)) return modelClass(weights, intercept) From b5d17ef10e2509d9886c660945449a89750c8116 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 21 Nov 2014 18:15:07 -0800 Subject: [PATCH 10/18] [SPARK-4431][MLlib] Implement efficient foreachActive for dense and sparse vector Previously, we were using Breeze's activeIterator to access the non-zero elements in dense/sparse vector. Due to the overhead, we switched back to native `while loop` in #SPARK-4129. However, #SPARK-4129 requires de-reference the dv.values/sv.values in each access to the value, which is very expensive. Also, in MultivariateOnlineSummarizer, we're using Breeze's dense vector to store the partial stats, and this is very expensive compared with using primitive scala array. In this PR, efficient foreachActive is implemented to unify the code path for dense and sparse vector operation which makes codebase easier to maintain. Breeze dense vector is replaced by primitive array to reduce the overhead further. Benchmarking with mnist8m dataset on single JVM with first 200 samples loaded in memory, and repeating 5000 times. Before change: Sparse Vector - 30.02 Dense Vector - 38.27 With this PR: Sparse Vector - 6.29 Dense Vector - 11.72 Author: DB Tsai Closes #3288 from dbtsai/activeIterator and squashes the following commits: 844b0e6 [DB Tsai] formating 03dd693 [DB Tsai] futher performance tunning. 1907ae1 [DB Tsai] address feedback 98448bb [DB Tsai] Made the override final, and had a local copy of variables which made the accessing a single step operation. c0cbd5a [DB Tsai] fix a bug 6441f92 [DB Tsai] Finished SPARK-4431 --- .../apache/spark/mllib/linalg/Vectors.scala | 32 +++++ .../stat/MultivariateOnlineSummarizer.scala | 121 +++++++----------- .../spark/mllib/linalg/VectorsSuite.scala | 24 ++++ 3 files changed, 105 insertions(+), 72 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 60ab2aaa8f27a..c6d5fe5bc678c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -76,6 +76,15 @@ sealed trait Vector extends Serializable { def copy: Vector = { throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") } + + /** + * Applies a function `f` to all the active elements of dense and sparse vector. + * + * @param f the function takes two parameters where the first parameter is the index of + * the vector with type `Int`, and the second parameter is the corresponding value + * with type `Double`. + */ + private[spark] def foreachActive(f: (Int, Double) => Unit) } /** @@ -273,6 +282,17 @@ class DenseVector(val values: Array[Double]) extends Vector { override def copy: DenseVector = { new DenseVector(values.clone()) } + + private[spark] override def foreachActive(f: (Int, Double) => Unit) = { + var i = 0 + val localValuesSize = values.size + val localValues = values + + while (i < localValuesSize) { + f(i, localValues(i)) + i += 1 + } + } } /** @@ -309,4 +329,16 @@ class SparseVector( } private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) + + private[spark] override def foreachActive(f: (Int, Double) => Unit) = { + var i = 0 + val localValuesSize = values.size + val localIndices = indices + val localValues = values + + while (i < localValuesSize) { + f(localIndices(i), localValues(i)) + i += 1 + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 654479ac2dd4f..fcc2a148791bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.stat -import breeze.linalg.{DenseVector => BDV} - import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: @@ -40,37 +38,14 @@ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with Serializable { private var n = 0 - private var currMean: BDV[Double] = _ - private var currM2n: BDV[Double] = _ - private var currM2: BDV[Double] = _ - private var currL1: BDV[Double] = _ + private var currMean: Array[Double] = _ + private var currM2n: Array[Double] = _ + private var currM2: Array[Double] = _ + private var currL1: Array[Double] = _ private var totalCnt: Long = 0 - private var nnz: BDV[Double] = _ - private var currMax: BDV[Double] = _ - private var currMin: BDV[Double] = _ - - /** - * Adds input value to position i. - */ - private[this] def add(i: Int, value: Double) = { - if (value != 0.0) { - if (currMax(i) < value) { - currMax(i) = value - } - if (currMin(i) > value) { - currMin(i) = value - } - - val prevMean = currMean(i) - val diff = value - prevMean - currMean(i) = prevMean + diff / (nnz(i) + 1.0) - currM2n(i) += (value - currMean(i)) * diff - currM2(i) += value * value - currL1(i) += math.abs(value) - - nnz(i) += 1.0 - } - } + private var nnz: Array[Double] = _ + private var currMax: Array[Double] = _ + private var currMin: Array[Double] = _ /** * Add a new sample to this summarizer, and update the statistical summary. @@ -83,33 +58,36 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S require(sample.size > 0, s"Vector should have dimension larger than zero.") n = sample.size - currMean = BDV.zeros[Double](n) - currM2n = BDV.zeros[Double](n) - currM2 = BDV.zeros[Double](n) - currL1 = BDV.zeros[Double](n) - nnz = BDV.zeros[Double](n) - currMax = BDV.fill(n)(Double.MinValue) - currMin = BDV.fill(n)(Double.MaxValue) + currMean = Array.ofDim[Double](n) + currM2n = Array.ofDim[Double](n) + currM2 = Array.ofDim[Double](n) + currL1 = Array.ofDim[Double](n) + nnz = Array.ofDim[Double](n) + currMax = Array.fill[Double](n)(Double.MinValue) + currMin = Array.fill[Double](n)(Double.MaxValue) } require(n == sample.size, s"Dimensions mismatch when adding new sample." + s" Expecting $n but got ${sample.size}.") - sample match { - case dv: DenseVector => { - var j = 0 - while (j < dv.size) { - add(j, dv.values(j)) - j += 1 + sample.foreachActive { (index, value) => + if (value != 0.0) { + if (currMax(index) < value) { + currMax(index) = value } - } - case sv: SparseVector => - var j = 0 - while (j < sv.indices.size) { - add(sv.indices(j), sv.values(j)) - j += 1 + if (currMin(index) > value) { + currMin(index) = value } - case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + + val prevMean = currMean(index) + val diff = value - prevMean + currMean(index) = prevMean + diff / (nnz(index) + 1.0) + currM2n(index) += (value - currMean(index)) * diff + currM2(index) += value * value + currL1(index) += math.abs(value) + + nnz(index) += 1.0 + } } totalCnt += 1 @@ -152,14 +130,14 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S } } else if (totalCnt == 0 && other.totalCnt != 0) { this.n = other.n - this.currMean = other.currMean.copy - this.currM2n = other.currM2n.copy - this.currM2 = other.currM2.copy - this.currL1 = other.currL1.copy + this.currMean = other.currMean.clone + this.currM2n = other.currM2n.clone + this.currM2 = other.currM2.clone + this.currL1 = other.currL1.clone this.totalCnt = other.totalCnt - this.nnz = other.nnz.copy - this.currMax = other.currMax.copy - this.currMin = other.currMin.copy + this.nnz = other.nnz.clone + this.currMax = other.currMax.clone + this.currMin = other.currMin.clone } this } @@ -167,19 +145,19 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S override def mean: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - val realMean = BDV.zeros[Double](n) + val realMean = Array.ofDim[Double](n) var i = 0 while (i < n) { realMean(i) = currMean(i) * (nnz(i) / totalCnt) i += 1 } - Vectors.fromBreeze(realMean) + Vectors.dense(realMean) } override def variance: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - val realVariance = BDV.zeros[Double](n) + val realVariance = Array.ofDim[Double](n) val denominator = totalCnt - 1.0 @@ -194,8 +172,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S i += 1 } } - - Vectors.fromBreeze(realVariance) + Vectors.dense(realVariance) } override def count: Long = totalCnt @@ -203,7 +180,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S override def numNonzeros: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - Vectors.fromBreeze(nnz) + Vectors.dense(nnz) } override def max: Vector = { @@ -214,7 +191,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 i += 1 } - Vectors.fromBreeze(currMax) + Vectors.dense(currMax) } override def min: Vector = { @@ -225,25 +202,25 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 i += 1 } - Vectors.fromBreeze(currMin) + Vectors.dense(currMin) } override def normL2: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - val realMagnitude = BDV.zeros[Double](n) + val realMagnitude = Array.ofDim[Double](n) var i = 0 while (i < currM2.size) { realMagnitude(i) = math.sqrt(currM2(i)) i += 1 } - - Vectors.fromBreeze(realMagnitude) + Vectors.dense(realMagnitude) } override def normL1: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - Vectors.fromBreeze(currL1) + + Vectors.dense(currL1) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 59cd85eab27d0..9492f604af4d5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -173,4 +173,28 @@ class VectorsSuite extends FunSuite { val v = Vectors.fromBreeze(x(::, 0)) assert(v.size === x.rows) } + + test("foreachActive") { + val dv = Vectors.dense(0.0, 1.2, 3.1, 0.0) + val sv = Vectors.sparse(4, Seq((1, 1.2), (2, 3.1), (3, 0.0))) + + val dvMap = scala.collection.mutable.Map[Int, Double]() + dv.foreachActive { (index, value) => + dvMap.put(index, value) + } + assert(dvMap.size === 4) + assert(dvMap.get(0) === Some(0.0)) + assert(dvMap.get(1) === Some(1.2)) + assert(dvMap.get(2) === Some(3.1)) + assert(dvMap.get(3) === Some(0.0)) + + val svMap = scala.collection.mutable.Map[Int, Double]() + sv.foreachActive { (index, value) => + svMap.put(index, value) + } + assert(svMap.size === 3) + assert(svMap.get(1) === Some(1.2)) + assert(svMap.get(2) === Some(3.1)) + assert(svMap.get(3) === Some(0.0)) + } } From 9b2a3c6126e4fe8485e506f8a56a26cb72509a5f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 22 Nov 2014 14:05:38 -0800 Subject: [PATCH 11/18] [SPARK-4377] Fixed serialization issue by switching to akka provided serializer. ... - there is no way around this for deserializing actorRef(s). Author: Prashant Sharma Closes #3402 from ScrapCodes/SPARK-4377/troubleDeserializing and squashes the following commits: 77233fd [Prashant Sharma] Style fixes 9b35c6e [Prashant Sharma] Scalastyle fixes 29880da [Prashant Sharma] [SPARK-4377] Fixed serialization issue by switching to akka provided serializer - there is no way around this for deserializing actorRef(s). --- .../master/FileSystemPersistenceEngine.scala | 26 ++++++++++--------- .../apache/spark/deploy/master/Master.scala | 12 ++++++--- .../deploy/master/RecoveryModeFactory.scala | 17 ++++++------ .../master/ZooKeeperPersistenceEngine.scala | 22 ++++++++-------- 4 files changed, 42 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 6ff2aa5244847..36a2e2c6a6349 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -18,12 +18,13 @@ package org.apache.spark.deploy.master import java.io._ -import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import akka.serialization.Serialization import org.apache.spark.Logging -import org.apache.spark.serializer.Serializer -import scala.reflect.ClassTag /** * Stores data in a single on-disk directory with one file per application and worker. @@ -34,10 +35,9 @@ import scala.reflect.ClassTag */ private[spark] class FileSystemPersistenceEngine( val dir: String, - val serialization: Serializer) + val serialization: Serialization) extends PersistenceEngine with Logging { - val serializer = serialization.newInstance() new File(dir).mkdir() override def persist(name: String, obj: Object): Unit = { @@ -56,17 +56,17 @@ private[spark] class FileSystemPersistenceEngine( private def serializeIntoFile(file: File, value: AnyRef) { val created = file.createNewFile() if (!created) { throw new IllegalStateException("Could not create file: " + file) } - - val out = serializer.serializeStream(new FileOutputStream(file)) + val serializer = serialization.findSerializerFor(value) + val serialized = serializer.toBinary(value) + val out = new FileOutputStream(file) try { - out.writeObject(value) + out.write(serialized) } finally { out.close() } - } - def deserializeFromFile[T](file: File): T = { + private def deserializeFromFile[T](file: File)(implicit m: ClassTag[T]): T = { val fileData = new Array[Byte](file.length().asInstanceOf[Int]) val dis = new DataInputStream(new FileInputStream(file)) try { @@ -74,7 +74,9 @@ private[spark] class FileSystemPersistenceEngine( } finally { dis.close() } - - serializer.deserializeStream(dis).readObject() + val clazz = m.runtimeClass.asInstanceOf[Class[T]] + val serializer = serialization.serializerFor(clazz) + serializer.fromBinary(fileData).asInstanceOf[T] } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 021454e25804c..7b32c505def9b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -30,6 +30,7 @@ import scala.util.Random import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import akka.serialization.Serialization import akka.serialization.SerializationExtension import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} @@ -132,15 +133,18 @@ private[spark] class Master( val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - val zkFactory = new ZooKeeperRecoveryModeFactory(conf) + val zkFactory = + new ZooKeeperRecoveryModeFactory(conf, SerializationExtension(context.system)) (zkFactory.createPersistenceEngine(), zkFactory.createLeaderElectionAgent(this)) case "FILESYSTEM" => - val fsFactory = new FileSystemRecoveryModeFactory(conf) + val fsFactory = + new FileSystemRecoveryModeFactory(conf, SerializationExtension(context.system)) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) - val factory = clazz.getConstructor(conf.getClass) - .newInstance(conf).asInstanceOf[StandaloneRecoveryModeFactory] + val factory = clazz.getConstructor(conf.getClass, Serialization.getClass) + .newInstance(conf, SerializationExtension(context.system)) + .asInstanceOf[StandaloneRecoveryModeFactory] (factory.createPersistenceEngine(), factory.createLeaderElectionAgent(this)) case _ => (new BlackHolePersistenceEngine(), new MonarchyLeaderAgent(this)) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index d9d36c1ed5f9f..1096eb0368357 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -17,9 +17,10 @@ package org.apache.spark.deploy.master +import akka.serialization.Serialization + import org.apache.spark.{Logging, SparkConf} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.serializer.JavaSerializer /** * ::DeveloperApi:: @@ -29,7 +30,7 @@ import org.apache.spark.serializer.JavaSerializer * */ @DeveloperApi -abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { +abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serialization) { /** * PersistenceEngine defines how the persistent data(Information about worker, driver etc..) @@ -48,21 +49,21 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual * recovery is made by restoring from filesystem. */ -private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf) - extends StandaloneRecoveryModeFactory(conf) with Logging { +private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) + extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") def createPersistenceEngine() = { logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, new JavaSerializer(conf)) + new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) } def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) } -private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf) - extends StandaloneRecoveryModeFactory(conf) { - def createPersistenceEngine() = new ZooKeeperPersistenceEngine(new JavaSerializer(conf), conf) +private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) + extends StandaloneRecoveryModeFactory(conf, serializer) { + def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) def createLeaderElectionAgent(master: LeaderElectable) = new ZooKeeperLeaderElectionAgent(master, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 96c2139eb02f0..e11ac031fb9c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,27 +17,24 @@ package org.apache.spark.deploy.master +import akka.serialization.Serialization + import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.serializer.Serializer -import java.nio.ByteBuffer -import scala.reflect.ClassTag - -private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, conf: SparkConf) +private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) extends PersistenceEngine with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - val serializer = serialization.newInstance() - SparkCuratorUtil.mkdir(zk, WORKING_DIR) @@ -59,14 +56,17 @@ private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, c } private def serializeIntoFile(path: String, value: AnyRef) { - val serialized = serializer.serialize(value) - zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized.array()) + val serializer = serialization.findSerializerFor(value) + val serialized = serializer.toBinary(value) + zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String): Option[T] = { + def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) + val clazz = m.runtimeClass.asInstanceOf[Class[T]] + val serializer = serialization.serializerFor(clazz) try { - Some(serializer.deserialize(ByteBuffer.wrap(fileData))) + Some(serializer.fromBinary(fileData).asInstanceOf[T]) } catch { case e: Exception => { logWarning("Exception while reading persisted file, deleting", e) From 29372b63185a4a170178b6ec2362d7112f389852 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 24 Nov 2014 13:28:48 -0600 Subject: [PATCH 12/18] SPARK-4457. Document how to build for Hadoop versions greater than 2.4 Author: Sandy Ryza Closes #3322 from sryza/sandy-spark-4457 and squashes the following commits: 5e72b77 [Sandy Ryza] Feedback 0cf05c1 [Sandy Ryza] Caveat be8084b [Sandy Ryza] SPARK-4457. Document how to build for Hadoop versions greater than 2.4 --- docs/building-spark.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index bb18414092aae..fee6a8440634c 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -92,8 +92,11 @@ mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package # Apache Hadoop 2.3.X mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package -# Apache Hadoop 2.4.X -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package +# Apache Hadoop 2.4.X or 2.5.X +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + +Versions of Hadoop after 2.5.X may or may not work with the -Phadoop-2.4 profile (they were +released after this version of Spark). # Different versions of HDFS and YARN. mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package From a6d7b61f92dc7c1f9632cecb232afa8040ab2b4d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 24 Nov 2014 12:43:45 -0800 Subject: [PATCH 13/18] [SPARK-4479][SQL] Avoids unnecessary defensive copies when sort based shuffle is on This PR is a workaround for SPARK-4479. Two changes are introduced: when merge sort is bypassed in `ExternalSorter`, 1. also bypass RDD elements buffering as buffering is the reason that `MutableRow` backed row objects must be copied, and 2. avoids defensive copies in `Exchange` operator [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3422) Author: Cheng Lian Closes #3422 from liancheng/avoids-defensive-copies and squashes the following commits: 591f2e9 [Cheng Lian] Passes all shuffle suites 0c3c91e [Cheng Lian] Fixes shuffle write metrics when merge sort is bypassed ed5df3c [Cheng Lian] Fixes styling changes f75089b [Cheng Lian] Avoids unnecessary defensive copies when sort based shuffle is on --- .../util/collection/ExternalSorter.scala | 23 ++++++++++++++++--- .../scala/org/apache/spark/ShuffleSuite.scala | 12 +++++----- .../apache/spark/sql/execution/Exchange.scala | 16 ++++++++++++- 3 files changed, 41 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index c617ff5c51d04..15bda1c9cc29c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -205,6 +205,13 @@ private[spark] class ExternalSorter[K, V, C]( map.changeValue((getPartition(kv._1), kv._1), update) maybeSpillCollection(usingMap = true) } + } else if (bypassMergeSort) { + // SPARK-4479: Also bypass buffering if merge sort is bypassed to avoid defensive copies + if (records.hasNext) { + spillToPartitionFiles(records.map { kv => + ((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) + }) + } } else { // Stick values into our buffer while (records.hasNext) { @@ -336,6 +343,10 @@ private[spark] class ExternalSorter[K, V, C]( * @param collection whichever collection we're using (map or buffer) */ private def spillToPartitionFiles(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { + spillToPartitionFiles(collection.iterator) + } + + private def spillToPartitionFiles(iterator: Iterator[((Int, K), C)]): Unit = { assert(bypassMergeSort) // Create our file writers if we haven't done so yet @@ -350,9 +361,9 @@ private[spark] class ExternalSorter[K, V, C]( } } - val it = collection.iterator // No need to sort stuff, just write each element out - while (it.hasNext) { - val elem = it.next() + // No need to sort stuff, just write each element out + while (iterator.hasNext) { + val elem = iterator.next() val partitionId = elem._1._1 val key = elem._1._2 val value = elem._2 @@ -748,6 +759,12 @@ private[spark] class ExternalSorter[K, V, C]( context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled context.taskMetrics.diskBytesSpilled += diskBytesSpilled + context.taskMetrics.shuffleWriteMetrics.filter(_ => bypassMergeSort).foreach { m => + if (curWriteMetrics != null) { + m.shuffleBytesWritten += curWriteMetrics.shuffleBytesWritten + m.shuffleWriteTime += curWriteMetrics.shuffleWriteTime + } + } lengths } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index cda942e15a704..85e5f9ab444b3 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -95,14 +95,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - // 10 partitions from 4 keys - val NUM_BLOCKS = 10 + // 201 partitions (greater than "spark.shuffle.sort.bypassMergeThreshold") from 4 keys + val NUM_BLOCKS = 201 val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS)) .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -122,13 +122,13 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - // 10 partitions from 4 keys - val NUM_BLOCKS = 10 + // 201 partitions (greater than "spark.shuffle.sort.bypassMergeThreshold") from 4 keys + val NUM_BLOCKS = 201 val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer should create zero-sized blocks - val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index cff7a012691dc..d7c811ca89022 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -41,11 +41,21 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una /** We must copy rows when sort based shuffle is on */ protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + private val bypassMergeThreshold = + child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + override def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. - val rdd = if (sortBasedShuffleOn) { + // This is a workaround for SPARK-4479. When: + // 1. sort based shuffle is on, and + // 2. the partition number is under the merge threshold, and + // 3. no ordering is required + // we can avoid the defensive copies to improve performance. In the long run, we probably + // want to include information in shuffle dependencies to indicate whether elements in the + // source RDD should be copied. + val rdd = if (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) { child.execute().mapPartitions { iter => val hashExpressions = newMutableProjection(expressions, child.output)() iter.map(r => (hashExpressions(r).copy(), r.copy())) @@ -82,6 +92,10 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => + // SPARK-4479: Can't turn off defensive copy as what we do for `HashPartitioning`, since + // operators like `TakeOrdered` may require an ordering within the partition, and currently + // `SinglePartition` doesn't include ordering information. + // TODO Add `SingleOrderedPartition` for operators like `TakeOrdered` val rdd = if (sortBasedShuffleOn) { child.execute().mapPartitions { iter => iter.map(r => (null, r.copy())) } } else { From d5834f0732b586731034a7df5402c25454770fc5 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Mon, 24 Nov 2014 12:45:07 -0800 Subject: [PATCH 14/18] [SQL] Fix comment in HiveShim This file is for Hive 0.13.1 I think. Author: Daniel Darabos Closes #3432 from darabos/patch-2 and squashes the following commits: 4fd22ed [Daniel Darabos] Fix comment. This file is for Hive 0.13.1. --- .../scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 3c7f62af450d9..99c1987158581 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} /** - * A compatibility layer for interacting with Hive version 0.12.0. + * A compatibility layer for interacting with Hive version 0.13.1. */ private[thriftserver] object HiveThriftServerShim { val version = "0.13.1" From b384119304617459592b7ba435368dd6fcc3273e Mon Sep 17 00:00:00 2001 From: scwf Date: Mon, 24 Nov 2014 12:49:08 -0800 Subject: [PATCH 15/18] [SQL] Fix path in HiveFromSpark It require us to run ```HiveFromSpark``` in specified dir because ```HiveFromSpark``` use relative path, this leads to ```run-example``` error(http://apache-spark-developers-list.1001551.n3.nabble.com/src-main-resources-kv1-txt-not-found-in-example-of-HiveFromSpark-td9100.html). Author: scwf Closes #3415 from scwf/HiveFromSpark and squashes the following commits: ed3d6c9 [scwf] revert no need change b00e20c [scwf] fix path usring spark_home dbd321b [scwf] fix path in hivefromspark --- .../org/apache/spark/examples/sql/hive/HiveFromSpark.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 0c52ef8ed96ac..227acc117502d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -27,6 +27,7 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sc = new SparkContext(sparkConf) + val path = s"${System.getenv("SPARK_HOME")}/examples/src/main/resources/kv1.txt" // A local hive context creates an instance of the Hive Metastore in process, storing // the warehouse data in the current directory. This location can be overridden by @@ -35,7 +36,7 @@ object HiveFromSpark { import hiveContext._ sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") + sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") From dd1c9cb36cde8202cede8014b5641ae8a0197812 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 24 Nov 2014 12:54:37 -0800 Subject: [PATCH 16/18] [SPARK-4487][SQL] Fix attribute reference resolution error when using ORDER BY. When we use ORDER BY clause, at first, attributes referenced by projection are resolved (1). And then, attributes referenced at ORDER BY clause are resolved (2). But when resolving attributes referenced at ORDER BY clause, the resolution result generated in (1) is discarded so for example, following query fails. SELECT c1 + c2 FROM mytable ORDER BY c1; The query above fails because when resolving the attribute reference 'c1', the resolution result of 'c2' is discarded. Author: Kousuke Saruta Closes #3363 from sarutak/SPARK-4487 and squashes the following commits: fd314f3 [Kousuke Saruta] Fixed attribute resolution logic in Analyzer 6e60c20 [Kousuke Saruta] Fixed conflicts cb5b7e9 [Kousuke Saruta] Added test case for SPARK-4487 282d529 [Kousuke Saruta] Fixed attributes reference resolution error b6123e6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into concat-feature 317b7fb [Kousuke Saruta] WIP --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d3b4cf8e34242..facbd8b975f10 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -179,7 +179,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { // Add missing attributes and then project them away after the sort. - Project(projectList, + Project(projectList.map(_.toAttribute), Sort(ordering, Project(projectList ++ missingInProject, child))) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0a96831c76f57..84ee3051eb682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -974,6 +974,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { dropTempTable("data") } + test("SPARK-4432 Fix attribute reference resolution error when using ORDER BY") { + checkAnswer( + sql("SELECT a + b FROM testData2 ORDER BY a"), + Seq(2, 3, 3 ,4 ,4 ,5).map(Seq(_)) + ) + } + test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) From 4a90276ab22d6989dffb2ee2d8118d9253365646 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 24 Nov 2014 13:18:14 -0800 Subject: [PATCH 17/18] [SPARK-4145] Web UI job pages MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR adds two new pages to the Spark Web UI: - A jobs overview page, which shows details on running / completed / failed jobs. - A job details page, which displays information on an individual job's stages. The jobs overview page is now the default UI homepage; the old homepage is still accessible at `/stages`. ### Screenshots #### New UI homepage ![image](https://cloud.githubusercontent.com/assets/50748/5119035/fd0a69e6-701f-11e4-89cb-db7e9705714f.png) #### Job details page (This is effectively a per-job version of the stages page that can be extended later with other things, such as DAG visualizations) ![image](https://cloud.githubusercontent.com/assets/50748/5134910/50b340d4-70c7-11e4-88e1-6b73237ea7c8.png) ### Key changes in this PR - Rename `JobProgressPage` to `AllStagesPage` - Expose `StageInfo` objects in the ``SparkListenerJobStart` event; add backwards-compatibility tests to JsonProtocol. - Add additional data structures to `JobProgressListener` to map from stages to jobs. - Add several fields to `JobUIData`. I also added ~150 lines of Selenium tests as I uncovered UI issues while developing this patch. ### Limitations If a job contains stages that aren't run, then its overall job progress bar may be an underestimate of the total job progress; in other words, a completed job may appear to have a progress bar that's not at 100%. If stages or tasks fail, then the progress bar will not go backwards to reflect the true amount of remaining work. Author: Josh Rosen Closes #3009 from JoshRosen/job-page and squashes the following commits: eb05e90 [Josh Rosen] Disable kill button in completed stages tables. f00c851 [Josh Rosen] Fix JsonProtocol compatibility b89c258 [Josh Rosen] More JSON protocol backwards-compatibility fixes. ff804cd [Josh Rosen] Don't write "Stage Ids" field in JobStartEvent JSON. 6f17f3f [Josh Rosen] Only store StageInfos in SparkListenerJobStart event. 2bbf41a [Josh Rosen] Update job progress bar to reflect skipped tasks/stages. 61c265a [Josh Rosen] Add “skipped stages” table; only display non-empty tables. 1f45d44 [Josh Rosen] Incorporate a bunch of minor review feedback. 0b77e3e [Josh Rosen] More bug fixes for phantom stages. 034aa8d [Josh Rosen] Use `.max()` to find result stage for job. eebdc2c [Josh Rosen] Don’t display pending stages for completed jobs. 67080ba [Josh Rosen] Ensure that "phantom stages" don't cause memory leaks. 7d10b97 [Josh Rosen] Merge remote-tracking branch 'apache/master' into job-page d69c775 [Josh Rosen] Fix table sorting on all jobs page. 5eb39dc [Josh Rosen] Add pending stages table to job page. f2a15da [Josh Rosen] Add status field to job details page. 171b53c [Josh Rosen] Move `startTime` to the start of SparkContext. e2f2c43 [Josh Rosen] Fix sorting of stages in job details page. 8955f4c [Josh Rosen] Display information for pending stages on jobs page. 8ab6c28 [Josh Rosen] Compute numTasks from job start stage infos. 5884f91 [Josh Rosen] Add StageInfos to SparkListenerJobStart event. 79793cd [Josh Rosen] Track indices of completed stage to avoid overcounting when failures occur. d62ea7b [Josh Rosen] Add failing Selenium test for stage overcounting issue. 1145c60 [Josh Rosen] Display text instead of progress bar for stages. 3d0a007 [Josh Rosen] Merge remote-tracking branch 'origin/master' into job-page 8a2351b [Josh Rosen] Add help tooltip to Spark Jobs page. b7bf30e [Josh Rosen] Add stages progress bar; fix bug where active stages show as completed. 4846ce4 [Josh Rosen] Hide "(Job Group") if no jobs were submitted in job groups. 4d58e55 [Josh Rosen] Change label to "Tasks (for all stages)" 85e9c85 [Josh Rosen] Extract startTime into separate variable. 1cf4987 [Josh Rosen] Fix broken kill links; add Selenium test to avoid future regressions. 56701fa [Josh Rosen] Move last stage name / description logic out of markup. a475ea1 [Josh Rosen] Add progress bars to jobs page. 45343b8 [Josh Rosen] More comments 4b206fb [Josh Rosen] Merge remote-tracking branch 'origin/master' into job-page bfce2b9 [Josh Rosen] Address review comments, except for progress bar. 4487dcb [Josh Rosen] [SPARK-4145] Web UI job pages 2568a6c [Josh Rosen] Rename JobProgressPage to AllStagesPage: --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 7 +- .../spark/scheduler/SparkListener.scala | 11 +- .../scala/org/apache/spark/ui/SparkUI.scala | 13 +- .../scala/org/apache/spark/ui/UIUtils.scala | 27 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 151 ++++++++++ ...ProgressPage.scala => AllStagesPage.scala} | 13 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/JobPage.scala | 177 +++++++++++ .../spark/ui/jobs/JobProgressListener.scala | 99 ++++++- .../org/apache/spark/ui/jobs/JobsTab.scala | 32 ++ .../org/apache/spark/ui/jobs/PoolPage.scala | 7 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/ui/jobs/StageTable.scala | 43 +-- .../{JobProgressTab.scala => StagesTab.scala} | 10 +- .../org/apache/spark/ui/jobs/UIData.scala | 21 +- .../org/apache/spark/util/JsonProtocol.scala | 23 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 201 ++++++++++++- .../ui/jobs/JobProgressListenerSuite.scala | 8 +- .../apache/spark/util/JsonProtocolSuite.scala | 276 +++++++++++++++++- 21 files changed, 1054 insertions(+), 75 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressPage.scala => AllStagesPage.scala} (87%) create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressTab.scala => StagesTab.scala} (83%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 586c1ccaca72b..9b0d5be7a7ab2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,6 +83,8 @@ class SparkContext(config: SparkConf) extends Logging { // contains a map from hostname to a list of input format splits on the host. private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + val startTime = System.currentTimeMillis() + /** * Create a SparkContext that loads settings from system properties (for instance, when * launching with ./bin/spark-submit). @@ -269,8 +271,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - val startTime = System.currentTimeMillis() - // Add each JAR given through the constructor if (jars != null) { jars.foreach(addJar) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 22449517d100f..b1222af662e9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -751,14 +751,15 @@ class DAGScheduler( localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job activeJobs += job finalStage.resultOfJob = Some(job) - listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, - properties)) + val stageIds = jobIdToStageIds(jobId).toArray + val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) + listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, properties)) submitStage(finalStage) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86afe3bd5265f..b62b0c1312693 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -56,8 +56,15 @@ case class SparkListenerTaskEnd( extends SparkListenerEvent @DeveloperApi -case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) - extends SparkListenerEvent +case class SparkListenerJobStart( + jobId: Int, + stageInfos: Seq[StageInfo], + properties: Properties = null) + extends SparkListenerEvent { + // Note: this is here for backwards-compatibility with older versions of this event which + // only stored stageIds and not StageInfos: + val stageIds: Seq[Int] = stageInfos.map(_.stageId) +} @DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 049938f827291..176907dffa46a 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobProgressListener, JobProgressTab} +import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** @@ -43,17 +43,20 @@ private[spark] class SparkUI private ( extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + /** Initialize all components of the server. */ def initialize() { - val jobProgressTab = new JobProgressTab(this) - attachTab(jobProgressTab) + attachTab(new JobsTab(this)) + val stagesTab = new StagesTab(this) + attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) // If the UI is live, then serve sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 7bc1e24d58711..0c418beaf7581 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -169,7 +169,8 @@ private[spark] object UIUtils extends Logging { title: String, content: => Seq[Node], activeTab: SparkUITab, - refreshInterval: Option[Int] = None): Seq[Node] = { + refreshInterval: Option[Int] = None, + helpText: Option[String] = None): Seq[Node] = { val appName = activeTab.appName val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." @@ -178,6 +179,9 @@ private[spark] object UIUtils extends Logging { {tab.name} } + val helpButton: Seq[Node] = helpText.map { helpText => + (?) + }.getOrElse(Seq.empty) @@ -201,6 +205,7 @@ private[spark] object UIUtils extends Logging {

2LlTpFZjUU+oq0<(*;ra(v0M*;u zffvwPT-!b&zws~s5Q|>3eD=Y+7f;?MqN7%)0EF0}v_lkmpI?M_?rhafM2*^&vhu2ajIw;r@^$dV zu=3_KggfNT_KrHa>84&BgPT`jl<_9M+rJIoWJTo7HESAF0HnN0ex!gH_%Fj|Re2LU zN#3Ll3-Zn7@aFh<9rzPB@8L@9iK!uZ?uDZeG?jxltCTm}!G~SY?iw3g!12B**|ebv z#}BGy^TtLMlr7=QFSWlmZm5+fo;nL#TMN)S!?oLd@x}a*H$cMZvpD=RnK^>5{P_Xh z|5z78N3Gl8mvNva#-Z|GG(Y>Cb_resj!mGT5EKr%TOH`4VIDbw(_bWc3)1y5FH1({ zHK)ZW0H!J|#=!+I@*~JrLmGZw89aDjqq(Nk{|G0Fs|c|Pd=s_dQTLjcpg)iPRN~^3 z6HolP9D3x-(zE+!+5X`F0>s1K4n7LmF?dirjJFRJ{V~Vrt$evgsp`94hTeEg%BF^; zY18dehL04~FgH#ivLlXpN5?y2i!+q9}f)r@JO^cieq}(c|P|E z)?uXc^4asbUy$DmhYS>=R6;;Am06G)2xhq4Cfi%Aj(0vT*;)#ipg9G^=avRAcVbUU z4Z2*rYSNWMgDQtHv5+m7k%?*j@O$y%sD4Nt8XCtQHiPwb3Txpggh`9ovo~Npte4My zW)=1e!!VW)&8%!2zSP#^%W8|Xw>M#5ut#2eZb(jz!0g66Fh(3i4&$(V3OZY5vI-xp zD2T=#M!;DX`-24>@yGFc?8D2;=5f);q}uD+vULOOYSCU+aEY;yLcgNg<2-xPcg(n7 zK+j}mggu}81^Joak<1^$>L-#IO_%ml#UoIjub@B@2*N9tXa0oqez+f+8 YszJ}^ zenEaFcqD?$^*;?a`MjcF%8=p9^fFD+9Tjqhos}-_1sf}al&m}sdoMgsygV>Zyj2M4 z1=%^dAu?G%$YbRQA%!VT^LYCC07ESr%T$)PxKjTOANlw)ydy4&{}TNZa$6-YzqA)NY|blhcjM^LgAd#ZL&?yXs2RXmo52^TIcOl6Hbx7M+tDrz?bT2# zs~Ini)S%)^Vc2tm1g0IqVOs2XI`ip^tZHP`;>Fy)rs~v!P1^MIA{ONZ7$_fBEs%kM zY1C6CAH@qd&%slBL+wo}srh4+x0}MW97n<=U3O5Ae!W0^P-g8}t zv_lJot^@ueY(5ZX1GGZSxVgya^3NXdEeT zQmEv}2CwjoGDq-oyvdhPYD@8@l19&|>A+1s1;*5n8ii)o8?O(c-dWh9o5H5H97l0i z;YOf_z?-Nn5-J4VgglqLNn38!*Z@yW&w~FJr2j$_$_sAltIXrg&JG+ahG4k2r(TaT zP?HlaMk8{n)!5y|J`TvtTqSsOR64pKV6Whp-Ke7orybLbzb%F1)8l*=@i^r0BnQd0 zRwt<|pa0opZ?^R{|JMaH?iX~{x@hQ9SQJV*jvQ88gJac;si4U5h3x5%XvDh;i@cSJ zS5{;&SIVERkNMd;^VvSY32}U)^R`*oo=W|9tPt;pL*gewR!CNQA#D7jIq;L9e|4o? zeC2=0(I>yAv4e--l!>$ZrE$|OQigrPO7v8^YxN&o`qR{T8SD$Kx$aXkdG<{ic__yI@rYG$Px_MRG{-Q3tv zj`a*M75G@xFFDwn z%2hPrNFU1MINxj4b_h4!`lQiSj9!5h>8$#k&Y#2%Go;hgKnn7sfX0PNJxrf2?OzW3WH|+LiJxNd zLQ{&CO6S-1BB!@Aot_3#kRJs!E>!Aahv$A8PV#w0!BjjkC0c6L#t}Msp}Mem>%!}C z$=afDv95+*M0t_{?@I8~A~iDd*2tJ>g*Ga(VbcEBcM1RFj8a*rH3kY@`0zzhr zUsrHj-rs!#*TtWNM#dr7qq#~Ps9Gy`SVlq@ORJR_P;SXdu|*WOpRD{DR9DZ zCT`@ozDIwmdT93R$b6-`UE+F40CQS_U zMt;WdgpR8V{#CvHvk%_?&@FnHqfBUILxSVLTz>vS!P2Gkvn~_z1pF#ogd_MKwsg-M z35JT`I&NDG#=-frM_-+}+w3z*-}J@c2irSOYVUE6{^6mewj*FJlv-=Y2!Ch6%k zAcsUX9M9`2cqo5WaYHD%gE%vBo<_h$|d~}O>lHPLU-ukevO}oM%4O^TVU{bv!-eL z!3(q<#lQdhU;YTfsH3=bV;;xwI^`FB{yyBQw;n>B)p!}i7ZnIqDsb~1#76k5kXK*Z zFV8*qI*f%M#>?S?{I9?Avv{e6Mng5!j&V*s2fizbaB*F$@vb#pU}6T^Tf zggWyu;>ed@YAWf-187)|(l1px6vsZ%^O|-lS?y&B8eO(9oj%w?927cizzl4X7NBIl<^NUS?|`6myru?IV1V?(aX1 zFHEqpHJO1X)U<5f(k%O+eZ}KVb5xy2d#T;Q*7}!RzO36qt=P)$8!w!`aj7Kq@ zc}x5sa7Kir5R`iKLA@Ak7|~EUg*gq4rrOS1YYu}qGq@6*!jZM>H{&ZNcr&68zIO{(l~25<5x9PPi&)2JyVOdRXlPxOtv$(uQkL7+tYO&iuX zK@b+k3bO^LF?dO0d&3Q#e)soZ$5-45@FuLL%)vGkgh+q>7l-BU zyVuB_^L^6YS*L<63dJ0Hj<@`P#|DQk$jInCcoQ}^AoM10o<1|F+E?UF&IY}!pqZ!z z?InsrZm^)iLO-a*#iO^a;LTZVzDGtbLcqRGnwm)$Ct@PVD7p{h__vNr;TH z7r;j*9Wz#5+lvV66-jU5<@;0W|Mx?GOd!DRybQedh|Elms*N8Kq6{Cu22UQA<~JUe zjd%Wx5~W0cAJy1>VLokL{}I8R7IOBPKatad51a># zQy3R`25n*3>+p2ej3;8|l=@FnP0vQuw*6U>m!o!|r z?b=?wq3*Fq*20tp*4=X%*b=RV77_$Ez<2b)goWQ1XPDzmqixSoEP$KyV8c{|T65U* z<1&y*+%9$(GXxE(K?sZH^fth)TUKEYn~^X71_GJb3uP$7il>PzOU+rCw zk7R3cgs;_vR$3w&nm$BXG@>c9yx;UskB`vo$SY_s3OHXelZUU?@xN04BSa(uD+{JH zvkjVf2&7S&fEeZXJdE%#eK?(Q5cZje=^kep@atumUn66`Ab&`|Y|mlSrz?g!uhi3% ze!4uVH5Q@Z&hqm=yk;6z-SkpTbvi>|IVcH6DwMe`@IPh7Nvdxa#I+^a#)Tq^De;9;=sT#ZuvS5Z6esCfi0UI+c)CH7lNkp z0W9J$u9)LBwV1l_Qn&zNP<4HoYDcKyMBL^_wRAiF2C(W3K3|r{V03&DFTu+?j4>T% zd~wEN$&w2=RyToiXU|Q`p~K^_MY9BZJk&OUN;V8Yo;^3INBgMeob9L$FY;lq0sxIj z&f}=j4Dd8FJv8VmHI8cXI)fOfi>MhT3UMifyGpN?XcC1aQNBI&$DDq|*43HcD;HUtE$b$pC`Ef!wj2)dlP8iKY4Mf_L>c4PSP98rZM~@to z?b}x4*j~HxW_x=ZzVP;;Ow5}cAJmQ-7#PA0b|)j=+`45QzLYkC4~C=+Hs5mOO$e8r zH>rV0&82!A3#i4BxiWn9%v<-hWp?2JF9fKC@lz8Dc&DSL>oC<=fNcV6+v2qw^Q0lsBhf+k>}jRlECF5hQ9&*SMyL4&>Q0!F~K=0(`yaXeXhE4@;F%N74R3EkH|=peOr^4IDJ@K|Tnv0QZMYG0;glSE_&>=tKk;kOG{c888of?AANTIR-SN+l5;4CzZ9^sDORjREg}IzIV) z8G@;RokS@cG#D0+IAGD@5}7U;beQflb26MR%rqw-(>$!?(~T?#v47>_fM+%@pHCP$le|-2MA70&dEg_5qj|jXlmtJ zQ&1`Uwy5DbX?*`NOvUPR~R#v6N}EyhK&5?_#@3Qd_A_QKQ*;f*4%y)h!gBXiK? zn3Lh*S$T6W+JK`$y}fnN1jAP&EXK5vLk%n%Yp3v!<+UaF;k3j*9!spP;U)*Sku^Z` z1YfL1a13h!Hi6jdcZ&Y0=~c0k{L$$WZHcB!oo~^CW#-FdSqp2=5Oklh$*|>$f0^cU z9%DUh6@HlbpW0AU6XQ4<3qztOk7G=f%dTs0RL!BG!2$XDSH27(@G*G~yxH2?A@_dz zbFy)BK5rg9ykGXc`MUhO-~5-dapNj^;Qm|X$AA1Y&}Li*Z7Ckxqj1OtDx_-`+E+93 z$Rp3m|{y>pP*UKMn@lk1}IQvPIa0aNeXXsZJcR<)s9)*`z|N-Z4nMVqhK^4R%Ucb_TAaGcXZAJp-m-%snM zy>3u+q)sF86CQ1E$1xd~O$0R?pF?&~D+MnC>g7GRL1Avb2v{kfX&7hD<{H3{CO^dM zglOqrmOh>J!(#coa`{wyOZ^8D-#-0W1ZtorLF)i^y>jl%W?YW(WgH!^ zgdIoIaQ51kYt4HXPy9e&Rt7DpvL@_HaRsf^0k2pz%ci6mADFJ&-HpBTYS@pe!upD} z6yszXnp8)RUX=gzpC1JsX5`7I2j$1UxEg!mEcUg>ftWVQRdvX^_1)6k(f}jygZOAP zkIRYx785x?%(}B_)iA=Ngp0WvIectb`Y%??wb$Pcjj0|jlfttufukq z?eI2erc(a{iUa+KD5r6R#YDiU~mRG_4 zRAmLOFP2GsTbn|D`)a2jZ-9mx9G~HfWDT^4)L0^NU_C>A;wX$ih7jbjA8Jf7FBeoj z>K8LHaxtTM(J!8PtcV&Ey*Q><3!4i)-OUPX=Z;pXuUo_hqfPptHByg-y8^hWt(W$?|)Gv;sDzW#We`OVDPyj z2V9Q=_qh5Oh29DJTTP|;&(?w~YyKw>j*bk0Kk)|vF*O>?t8kMX#u>GvjvqauyqS%7 z6E^{Y7pYZ8j#z-k6M1$2zBi$fco4jq!_mkXA!87D<*kR3ggqw7DCi{{%VJ=lRZERQGtf;60PqqR^jqDJQ*~A4%ZoFA0{EVVb3Sir3+Ox~{aK zhxt6tU=V7eFW!!*ZTYyiHlRPDEn1(32?K4jCPu!tHi0X?Fj*@_Mdg=%z(VA{~Py?jPvpnLb8wq~Ris=>8 zoVlGgF?@r;Upm100Meg(DY|uc{)C+P_J5RtHy)Cn8$XXbXH3xUeDDMJpGUa#KRLka z)Qa&FKQR>eMiV`3G|7Kr28(y!N|TQ6lSWpXK8#lA;!7!H2A#t`oeN+NNBeLL zue@v)uLHO_uc8)O!Q6ZBXgx1CX~LSj27EPuy$C**!caLs-V#3fOPtV1`XoLWWoF8uh1H0)xiL`9v5vAR6h9GyW72#}dw=P# zZSXk?t<`xe^d#K_?+bl%>6rW;=RdK;)zSaZxuV`>2&TNFj+|m}JT0IycoE)-D4oe1 zVJqcFdSbFq6V65=U@Zf#Aip-*GZ$DaFT)c2h4(vZI zfAQsS%jZ9Tzx?!1eL;5b-k}D;CHn?AXaxCj#Vyr~z*tKZnBjjbZr`J(5{)A33kZ0x z(sxZ<2=GjHq&=oNXma$Q&*<|kjW6QJ4KLP(#^r+W8Q2!scJ&GV414G|pH<<4s}cv>E?y}UID`9l$3zx*RW zI=?|nz67C-UKM@9@hbW314?9}oMAARqXzU?z>-(X zGzgaXwP+6ecwF>TR^EX>+KBCwjT_tKm;d2?^5pkkk>{R&75ikq>$bsN?|`?@2b zDbCE6InD|BN~C{Y`_S5eX>G3k#QjM`YOvV&OAeN+xBuy$#?O~2n2HNnD2)n7`*O%# zDgR38tTh2!Cq+Dm$x+*s3|k`;zbW`WX@!Z(W)Z?m6uD$D1utTYL>u$UuatkKbOKAj zF7@B=k)(MtoQHcpr0`p!e>{0d=&|v?h)oAyd^y@^=)9-555kohdGwKoaBS=?9C69v zmMUJYF7oViC)IY+Z03wijE$;UykDo6>w7;|1qu#LPQ)f zN*|6i@`1!>O=cV)F2|+dMX(E~7J#jkf2DMS!HX6K2YGXDj#_`@&DGiu^5*#H8QHRF z5xm(3W7`$-;!CG6))z3&&#A39YBIj?{Id{b9>Q1?9FOgj2Oju@Y}>j)d9xf}MkP0c z@pln*QD{7BzHjp=Y^Cfwcnw3q^v?6a(h)qE_n^Mpm_sBqVFmc_>-bN#C9jRZ zU+BB}QDPKGZ_?Ay2!Z%BpCJp=r_&cy#zpw@u+xIM;&5SpgTY+IU4DzwFU4%0jzwlu zhd>bM^0Cx^;@-!BA}H_Y?a;ppTHM{+KPK1ScS`nt`>$jPcvi@mo2qIWq^WDQ@~e%7 znC2z=$GG5!gr(|sY2S6HoPYXj(!6Gu)RbWXS=PPH@gFp%NOSv30yZXDnJ28Ix0*+! zbQE!(Q>cK0@Un9P@@Rg|Z9i6XA|esA(HX;IX@l)8NMzO^p&<{}23}Hg`yIE)#OPT$ zbmTm2oGsz@Spl#fs@=PbzQtp8TLagjOw1RQ`uco>S z_9h`{+qngHv@YuYyb0?o7f#xSt*L~t3;UKO7*n6a9L^_{lPiaJ0LhjMD;nKMpDe57OWvxtWOD6u^5E z2}#4K%jg<1Fd@{hn-Zo+84`ItUjaZQg#3jKj5&(MwVq^o@3;G&M>7}v6oN1HzeNA{ zX8cpwGd?x~LxoKcF10}Y7+MIYPUAZHhp~W+;e2@wZry0d%U>2(-)10;!ZD)SDyZ9z zK?4Ej%O8JSZ=>qzYQq3r!nS-@Q0T+yb#k6^NTMX zld0)?Xc66pqkr45!1W^kAow1d7&*Qy^Oz8M%6M=df+^l+Qj0G#ddn7~eZD}ZqK;o> zreSAldU`=MBdVdv#5n-3{FXn-o&f@n@$n_Ky1ZJ(E2)_@C2cKbD$rWDzExkO7hxYt z>rOR*k9hZmk;odI5e8tOnRr;UxYZTg{ke6eAR!(1?Cj`GV{{bt+PlTWZu81(=Ns-70I1h*Xid%%8-h|_Enm=7W z;5{M`GaQgBZvX&507*naRC$~h$op|nF?gH*`8+;0hD}?-oVK8`;2G+@&s8)9F`h;xVx*bR3`j8*O>xeg*-}`JrV+yCpn>@njX!ic(bv8V3 z+-D#xu7+?ngIhHh7OPSJ%DhR^RDn0Muvw8?8dBb*an^O~+f-n@fEzW_Ut?J%+d|&t z5kK0s8k?9`K^1wkzTS9~mjcy6BTIP`IE^-vNz!ph@Gs%2t7+_YR!*Nhjk(aY)hsz# zFd0r1Mv!guA;lCGjv(@bvH$UYyYGD`cFhp7`%Lk5U+2XlxJaj`HIPUedl4Wsa~xIf zti093xI$dJab{a1iqi&pe2|AeV038}7N1OKC7Pr^-dA6P?4{T1EcHK4-!y&F>1q1Y z0CVq6KP=}?9h2b$FXGMw2$-?QY1?s=)HHMlT402buyASkq@xKPKD_kje75$s2PFf| z*)vc7wQRoo=dc~)F>l9La(w6GYo+mzdrg%cIR2dn6fk~#zBQh}{1+I<_?cWBXr_5% zwqmUDeGJJ$3BC_yE#7k*W3CLJZHe2LhMf=1@_GIm?3F+9$=l`Fk>m2cCtiY3YEtT} zr=fWdje2}!4P9iRE5I2(@*4s^$YHIe84lQc<*l}o9ma>47HB!w$eO+;ma0=63a5A;JAmD5W}cmp3V0@7{+B=^aa6SFvdF^hG2 z9rirU5KK|48<)J`pN|Q)&2XEKI*+rqJ2Krjn#BF`-1IH&((u23`V+p@W-$1icGkbR zjoy|I!uV%jz5nTcC!a4vFvaXn(BLcm5H>m*yks~^$cs$Yn;%ZC_PWOFMl8-2t@|3~ zaD(q(oYZFq#Cx3PXS*S|J`}EuXZbp4 z9O2^%UJBr~%Aut8Tcv-+6nY_tkrsU0~T)q7LzyC4WxM?lYsan ziHQup4D(1s$gJ^2ehBzOv;Gkm{+Evn#ujYK5vg%Z3@*7A+KVsIm(1Wf%}=BR`0-0E zn?m?`BuW;D5<;ZUE74qwIuiK-NgwIcG0EdJBK6}%-7ob&|D{ja0vOCAuOo~24GHcm zdK2k62%*X=>SXtQ|3FSOG|JhRpTuz_Oe1qs(Day+O5Bo`E|a8ZOxytULrVWe95w6t z$bE9)E59qNK5{RFdc9J?+z0vs z+GMlT!mf~y-%^V`YZYv+_Tht?*<%cl=)nQA$Uiuok;6wPp;cCgy-X|kIG(rja80`V z8p_gFF#WB2`R$F`&UP1Z-dFdTB^kb&#=myY=~bk!9<{PyimH~;Ky<_<;|xh;(mZ5& z4e;}GF9QukVLo%I5z%kO%&cXPfxdCC3xxl=^nA)UWfM}>8E;A9Gj*7 zm*^i2n$(DS563^hNObq$h!Cj3<2bxwi`UODE=*xTZNm|?IvE*>l3Hl5RG{J2czLEU zi$>9@-Ga}!s#!vVqkOR~=KPB;ufAqjYt{Z&0=j8hX+0>+G={xp9`-;8GY1{Lk^x-LndUs4+EaPNVm85KxgfEAjHx z*;NmlQqxF>(6y=!fA!MQ)vdgF=-__jom6{ITeq%*vGYFT%^WnPmcVCAJYEZ*T2rhZ z$6U#q7cRi=lk#Q*gt9%*qQv>QWqA|dyh%rt$FP4}ys1|}qhjSvXiiZh@=`8vfMvlz zqQJKS^F}S~J4L*yd=f*^@#cdMby0rBu}&W5kzO7RByZC3sBi^3LE##;zR0(W5V{p} zHg?HxHjW7oU{M}IXMzjuAX@3X2@Z|%ADZz`T;Pv5$#ZGbq-y1sP!%FiFZ{&Q&6Hnq#vdwx-B zJJw78OW%_7uRbLi+-`c!5B-YNG_MleS3s66(I3n}p+I{4S3pC)rDuz@ZP+fSpZ&V* z{MC)*Mnfw9I+kAlh%Kui5S>jN#fTxhEp$@hg|MJ?LH@`41 zo%{J=LW!Z_p>R@AoS*_041F@6&sLsoFz2wmwlq$du@+j-?H%p7t*;##M6G(uF13ef zhrHdB#!t2ngY5C=EIe~#G}>BUxUj0r2ExG|CQRC-@6DSi9mjimEak! z6jKQp@!-mbhAi*$`1kfE8dDHlSullGlOMgU2EjXJegWe}gU0h&4r5{ejeWu-a@={t z>C^e3qi>G!j?P3dB(LyMBfz+X=b!_;QvZQ%;3AL%h;zCo=!3Xr>0=ZOgoM38tm?z@n5@jrFX(x1e(A|#k(j|zxD%%*@WMJTH{N(FUfi1S zlDk%(eDYD)uz3ns&)bQj}wFYc`;sdipHvBxU5L zkK7=eH)G?|-i3SvSi@kuW(j{h?x*7oZ+IAP`jHO8nL21MaZFcUX1gdM1A ztb){f;tS_l7*uS7ohKfXSMUTAv@1f90&o@Bz4FFCt(EI+G(Q=PsGJUFd*Uzi@n2gj zjlV;{g*AI716;x|dONhJ2!;Z-$c13cZ>$lKsFN@5ytQm*8Y<54Z@Os@`q64dL%GuY z2Xp`ziid#IfSL%!mkcm72^R{H4p+qdhfN+e5Zl@+)QIQ|gi_>7{?XBH8hNw7e+I_M zKZ?99vSv-6JpI%Y%AcRU=VsZtb0d!9ZG{p3I^|8toxu1zj{Z{f3EDj>NkI^uHr||6 z-fVC0magtzXca-bYVmA>H(7JwP4W-zzTt>04S}!7n=FbV%9}Mg*lVj&e!Ku9=)J38 znD}k*ChM!gtv@6h%O!b}q*Qv+SDI>!z6X)#`LDUj_>jE02rWqRCN-scd+PIfGr%au zn>p|!d2==cyBL@&_w3m%tNVJjQpH6WP(BvjkDA2*+y{I8{3&Mg8EIj|ER3OF}~y9`)fJ-+Ea4ium7#=`r`j46+Jto9Cr8=KA1a(U!uR` zocq7->+X`bzWTc|b;}88v8=&5K#kB#Ij4mC`Qy8=q*Ib9R;G{#J&*a!WqAsr6_2qJ z4xgos-UiR+1jL3(UUTc1C=RV~)*&3PgJlZaOM6M3I3aNVXYQ37ZoEPM=nsAmm)UTy zJ0{&-)w1WB7X5JPZ6K7Z!R4S8C1s=1+f9uruIFtSBh;{_RyH=T_-k%}ttsG>$85RY zR^vPLk_ssEd&u>-1vlhos~cgupb2pC5s7plYSmr6^tUk=_aZr3;;~~L{UDD2m?r`$ zQ{IY#DPs(5tCUH7M5Gwcu^N(@^->%8R2zysavSG$unQJ8-?Hb0wQa~3{L5Qh2s3uc zCx_!c2TG~`F+FThInjYb50>b!1L|rY|GYt@rmR-7m9XQ4#iOp43qvQg7;5C=#VHwt zR>sf}wHmPCluaRg8n$m>lzDa<0;$z{^*n7SRg~kHkZMWM1`;$eRP)Kk5f>P~6b=k# zaGTbw-k#Od!`tl`kGwf9(zJU+%?ZA61`egP0h0?gGU5da$(P|^N=NO-Py=Ed#*SxZ zc@=Jed#UnXHMcKetb3_E-yc%2SOvE}Jl z@#0U-4c;)gh%*VRx_cl9Fk__Nm(u)a^EL56M_d}llLShJ4ZWI~`NNoSpVYB%nBbHc zVIZ)O2SYN8zdP%neLbWQrX1QhIHUsF*DG&U;wan2fobK#SMN-5CjhooyAxm zfFN`dLaHwC=NjXIO7JdhGU+7*%A0(_AWk^uuu;S~BX3^lpO#y1?Sl4W4{S%UUm4(n z+Eq#3#3NncO=>R%S((LO#xMI+S62n?%thF%gC-ch9LM-y0RZuhEPBN`8y{R0M4%{} zGah+!*c7Y4Ssw@qn5CerhR1|Ca!7xN8vDkh1Ym!Z%aJ$d@d5{#zN}u|jW6O&NcO_+ z$1sYANCW1z0p;Ns4y%k|IiB3N_WH|~<-JnIKih3>kK6A!V7#}JK3y`-^X2=FCxhou z-OtZk__T6;m!`$aOSu~;Iefj=1>(!puy5c7R6={{5Au$QH8DTVOj z+$?EN?q5!M1)TV~G(4-rT4OEs9vcXMTE4xBsu1+r+-GVWzMS^Qxvgym!D26`Jo8`5 zttgnvXL@f=3K2TWz>LxE&BKPMbOU_Tt}l5=WVvGMqv>+-bAJ7CyoGFl*yP}8K4|PY zzkFeP&}(Fgx}!w%GKAn^`qqBNmHHo#55tAwN0F5Haom>ZZ%Db4#y^jrl$Sx90xxGA z|Md;!($lw2HgCNRH`1My=UzA?Z|)z&`EME!--~@7)VZ@~aRXRZy1Sd@j{l#%_l&aZ zIMM{8y!XBeRj4X>???!scOoTGlqgMmbxUe9J8fp>$Nt*=HGO*jtUI&QvwddIXm@(l z-KRCv-D)|CGNeSxP$WeX009srKzLLE1(ZQ~@Ams5ZoF&WD+78`l|a4BnjAk>t?Vil;cWWnbP=*-NVT)H?c-QDx@yI<hXe0 z?;^CU7SE#$Uw)m}7{u`#d|e@4JRrAv$8pKC?4YHg5Vru%q0TYfFj%fyV6-tuI$8AgH;Kz2!f&DwM;o}!f+I@p2Bh>Gc zyh)+d#Kes9=7sa4^5rje;ATzO6v+lW<;}Awlg^u%I#eLVF^kc=3TGpoeMQ>p>Iz^Z zYf+C64#AEcztEFq2+eaE(8N>lsGc)iG>(N7Aw=>&!}NnZe9$=n@oJL>$;O+s|Hj=j z-e)OwQW#oPf1FCr-{f5$|08dXDQ}AG*s%*YCpLNI@Xe9kSvP&U57PW^_!w@{_A%nU z!H77+Mu4yN<`aVNg#rQ@m2%eCUN#ktfc10yB_JBdyqw!Sj?W&ZzZ#b;zfzevrOfU1 zn4b>M=N(H=FTZ?^Nq9;6R!UzUdV2elIC7y?( z4#J^N=V7bC#x&zY9O7Z7#a$9BAFnpQIUM&{HX4q@N`K-}Zo|>wdD!ta3an0kRLqCU z(C>Zui_moLl|TQV{%2e=G=&?V$E2g9RBe>b^KzZ^YTmUNu1h+0;p?7NBhUs|1IZVv zk({x1QV2|uVPUQdVeU_`H-%XMt|j~cM2Az1>YOGntud#01zO#C(%zOQ&;9Zh85}t) z2M_Jm3rhHY;bVg3yl>o(#>L>K9s)eImn?VKCvOE_3a|!lrT^n4{fQ0YING@9(?@Uo zJH6@tDg;wLYTW0b1aR&Zs z*O=bW$208NIOfMOotk!{K?f0=p9n3;?&%g4f4w87>$n4!=N=bi3>SW4I4rQxSrmC!&B$w=+nhvg?Sl|r(mvIoBzW0@;1>1 zvQ|n3-`DvssGLVr$ea9nNZuSCx(FL@$CNiaJ8+}TL>`U?KzuY=2HwQ2nPY{xRG?9L zGdsHnWvP)!-sE=2#sPgFug;sfxr@r1qcG&HxA0|+AV2w(JRRZ9Nqpgj))udQwz2GO zso668c%7C+vUoYb6pZZlTpd?o8hMj;-ONr?9E6s35RP<^#iNtMJ{r$(SPYd1iTX+b zUZfB^6K}@LSt0(;<0W<)xRW=>(X2Bwc~V|k3+=QL))Zi7p4aO>V?&{iK3T?~obPq$ zYbpwQT#zy0F(2!+{9)Mfi2FX)-y8n7H6__5BmgOjv-^y*DG+o7q-1ff;{RZ;#=M+I zt&EmP{IDS4($)#uk4>n3%s5No#F}b$n1!1$o0oPhme4UnXNYI0bLYb#}*QzXY#klX!a{`B$t!R~)*wl-`#B;7}z zmGQo_Qe3?mZR0ob84o*+B;DzD-_z(@x}~rkYzzkDUQ_MIW88NxqMz9jCeysuWG;{Q zx|!Q*XHG8}LQlpl58unV?=V9w;`MW1T2Wbyu~sLKKK^SmHhe|C{>`W4OTV)Y`)tGq z1-(=V=WT`nm|DiP$r!C;7V8)<|CpSj)--PRh53%l-IF^0t1I(l0|aIr9VMz&9bLl= zn*h?{EFO7j3ik7qWbp%!_eYS0F?6IW3Wv`;8BmU+{T=H|WcT)Q)PGa{+u#19bZ*=% zfBJ8~3|zpEJOYu?5dte&s>M9se9Qg*^?ybZBDj0sxKG+RUX8{~r9I8)OV8o)=3DOf zgZ^~PJZ2SwDeqwSIfw#+BBLo11Z>aiO=+b2K(S`E1)qq-Q`2LT^=WxjNZ)c>*qc^2 z9{}nL2{&98ml+?&EA*GgKZ%dxU`?XplAEra3s4M&U(pg2b)$D6j_do=#J zf#b{7mMuG^|Heri?U|HEK3)q=fFcJ1pt59WcvOaAjC4_okdKDaY4e5)4>Dunp2Olcf-Bv# zbMhr0g~MZU)?%yq}^i;JX2(o(S= z=MnJPhx)=zR)OOTrhX((q`9!k#q5SZZWzYLuxXg&RyhwFnmPyrsM;KEt=8s08)_fv z`5(QXjRBMoBMn=&Y?rP}i}KvFWAd3t+hL!{cyk)Uk&)3XXi$|Zf7aDjsV$|2x$78e z&YK)}U<@5>IxsOfA(#nxlg7Zwn?*EOx>$;wc;m{zn>>D3j2k9-3|NoF5v-484(t^i z`-(T?4G&K%pHIME7I`x_hd_)s(m(|8W7VX^MN%Vasn|%I;;YXlIQD9zyy-mYzG4>e z<}5B#Fy4eSoZbzL0HQhUop0jR#sjv zwfJj*A>yj?3fwdX;PW^}%EE94%X~g3;U`I5NpO^+E2zP7rJo7D)9rqcNj?6oR6f}g z);Cs?ze#$WZuj32`e{f8h%CZ}(|bWb--B~Xa3%fpjN}HwAs>lJ`iWOI^6;{V&5!$@iU!FdpGVr9?Wf&_$a!oo7H7w0YNSWTuu}hdzq_w>dgtIFr>GdW3&J)Q#yDNFzIVF+bAd>4!v?8oYLRQlpOqbt zLc45|m)U6bD+*4XZug_~GfWE79pj>Dp2z%Ts=*9}Wd<`^-ox}$s5=RWu{k8fC0?JE zps>ZtuLPfU#9b0&*ja-8QyK+dx4vC^x~JvDk!cynG0C}L9L3AOji)He0K!AAze-tBR%Zx z826K@A0GNs1g8a7A(-O0arvANg!tlbBg>m7Q$u>3)gNzGOS8?Nr!3zwKCstYL5S6Y<}9-ua}F>JLFvXd3p6j zw-lTnz@m|j7s*LHUxkgRyX5d4yRpF1t`xK?7Wne(=gir9{%1knfW;3U4?WN!XIsbR zJ5T&XI@)Vw*UsAk_a;=BDa^{oBD5e~mq&EboJF{*1~wpglO6X3Y3!O75zGrVLvbOa zV2T2+F=$^+K+9tqf*!t5r-T>)n*t1Q;T#ggGEVbE@&!T0r>2}BPxLt!X1Wb0&8Z)i zB4w-&g{bxMs1P+suEA(~S2u1&oG8RgFJ7n`TOc%mp=iw&j8Cn|=RB*(=&o)U%`A`9R@LkWFLp`(8~w_gyq)j0*L!5|zQf?n{kTz&Hq~HPN_mr7iR3Yk zu?REak+I}4^5(|QG3CwDl49At>ke!ZM-{g7=GE>|Xl5-yh{PKv3zy+d0M)wahmPZx zqmpBif~jKMf;WK;IR#VX%>o>A;1NN0sWl&Xu9i1RA*+t^pGIkdLDsbB^AfxnmSbka zMtKuocn1b1<>IAL%r7w5x=^XSSq`B`P($D=DjE)}^H%T}R?e^^;bBYnHgKQ$ENuCm z*TeSlxxmstDEyylkVt@gEjstDWt={X_c-SBu+h&L!~w)27ME)S1M<@G<8rC%3JlUu zs!ha_!XgM~%OQx!g+{;>K2YFe2nIZDwH4uv$=0o#W&gGvQdx`-BzR-<9&z9Hg0}zF zx-f7wE;YDVx%ZdjyEHwDaOB77c0Wm9TK&M#;~Y%1Ej-TsTZQLzyYF?S@LPfYYzU=( z{_3kzTUjX^pt0$<)Gmdi!*%~w@TEra3{;ga|NI;Hq&1COcnf6xoxgz>YHAeQxLOKc z9ycSnw4ki2O*TI8`?3Hb(|F$n={o*%neDqM*QhmB+#r>0o20y<9b*i~lrT@t!(L8y zAv98ZFG=>?ENn3Cz@?@0L7%)k{SQ49xpDM+(hu$R2F(4{J08J!#&`|fQfh~{!~RlAankrq5`?&u03-4^+NgXcUtArWeCr+$E0&Z znQ9L6X!~1r+~4w#wkhcru;HAYUXYtNXXHxvID}Aha^}pK`lrv{lxsaxa-(lXMn`Zf zH$D*Y2I&pZlKTDM#n{I`y+JzC2kCePZJ~lGrs87{5a2OaT=K2j|C8VoYDDuELR`AX z@6SPx!bj8c7-30Ct{m7`D>XHf^22A|gn5AlId=4vtZQw;U!zv*eeGlT!`lBL^~u%@ z+EUw_Y4P#+;iW%GXZXAd!BjZSIU912S*$l$yrB$yBsT{n7cg=LoQK1Wlg4UVrk`8g zNTtJYT+G*f12VbK{F!kH@&^DN${PJ^{r{^p{<8Y7DitmdOl?eE0i3ID0+~ z!AlS9K+R!6%2Hv}3|@F|!e$P^lt8tYbAiqW8DKhW^^;ZNndS0vY>%7cIUEhtKfF*z z=YbU1Ncl+6@@6I}y?Pdm7LMfzAABtNOshr0AI(S{X;^s>8YTUMlQ@=G0&TN0^h+t) ze5GmJkucxsb)OZi&41oT+8WyC-g%zlJ_n=snPp^3a!^EaW8C?gr_j;xwx)%7ysn_) zm}hxy{3gpYt+=EF+Bwxy-_Qx(9Kj1WzO*WDE=u>+3E90TUz(cgAvoe!b>~g`#F;LN z2Obek?LzWq4qm{%`JHYETXK~*3k&hZJi?oUH(?73u1~!ErU;`QCkI&WuZtKY->?!uE)cABMvm z(QuT0qXGE>S_07O`TQ4+2?$~}kgklh0`V;`W$7p0RhZ*KxJ^h?W&R@JBv$*AV zeO)d1n@9fHDgpc;ycABiLO+jc&Q6U=->K(heBioNHMdIhq0c}tMVp!46C9agIHbu! zd~t`YNfvWUB)5K>RDOCd)E_5gsPD2&-n=B^z2{`I_aa{EakL4-tlXj!DJZFyV%T{s ztm~BAENF+RvpT#c_uq;BhXE@YzI0S>y!Nup4RlJ`hC4COm}aK;RnUFmIDd!wU*SVT zs;F@b6bq;1=IIxt8G&8#=rH8mk;}iBZaZys7uCpYFIYJ0|m9D z=i#%5m&(sPQRg+gpQ(H_e>|Mb?c7d3be5|a!l~loLR8 z4LP<(K3W$|J%w92)FcEx{BWPpdt(ZMsY$HwvoO3qA!D#d$>*NyQ_wD*!3UxR%IDRM z+{plqmDd5YZkV<5B9?zq##^K>P^_Aj62s#1Nx+DP_*hkp+q4VvxmTZ& zzP_6HfE}*F~IA&C*|wYt)=XN=SBex~x7OENZ@+v#7r1hnOrOJrJfQxV_1ge5JHypn#;zT0uD*Csjr#tGcScSA0mgAwGM9%waPgSJBr3>7wDk?Fz& zKm!XDO4+6(>}d~Suy8OSdHeQid<`j-pZ(%G?Boo{U;gFO@|nll0!-HxugbEDuDrG6Dx``UQB)e3Gy(J_G_q-GKQ~Or(PS9a~DQs z>#mQXObG`CsKW@?_Z=QVZCN#0~ z<-Yq`RNLv6c$0(H$2<5fH^Q4d#;3dq0rCRuT*Y`(%V|M|LI#RHM0pcgqwE~zP1YZQ zvEbN`c~A<`Dai3mmOU5Y-Z!Tso7aS?6(`sFL~gJ*us zNmD-k$xq6bjt(h6r4&Z7BdEdSWi$#;IG7^s5Spy7yIbzszh6$AIU_%P{ugrY>^XVh z@F8icsbSW@dCOT_daV1&w)S}Uy>9w+QJVQox;>81N#}ICpM*lOApj4d?jSc3SDvy>0rjj?eNaCU|rnKJ>B`uK%o>eiTsc5(qLE zv+|_0exsB@7?ZVtxAgg0XjJi5QwU>l8%us6?d?EI3Uw#pd%f-_+xsoik9tTOwaGcA z%_cz5VcPEqzl;4(oJ*@)rFPRU={fqev>bX2OA5>@U}AkM^aEKP^F9W=t!%vT$;J$Q zb4forCYkPWL6L};Oijl*o+JL`kcrdkoYCPlSebZPt3mB1p>1pH>f}?Oc~q`n{U&bO zy)N6fR6r=Ry1JOlcnw1ndu4q<9EDI6I=y;)2p>0R|9^YZt9`)&Cbzk#`ZYiE$t3(}`!DUZ{L z*iSO`gVO&_eEN9tvEz-C^qtlZEB&$V(T5cUQ?@9X;K4eZ<>FoI3zz%MXB&&98YC1*W^|;wVG`zVbk8g12?44H~Xoj%R>?K8IK^mRG`-!6wy)j$-NRS`exc zU%grd?TiWO?VZ%cw6HK6$JKIBC%#@Q42PG&k5(B}MQ*n_#@a@wd3+^jGQS?NoxH80 z5QE|xrD6W0nw{kU1LX-r7Zaa=qCl1ihZtD~#gFi#l?G+?hypgDw2Rf-HwBwPvvS+! zR%pnqgDP~M8g;h~eEzU60Y zXU?BipY{;PqxX=9y*zPT3gP&HqSS&%e201q3TR-5-Z0ejfe854JLc+Tte z={T2poIY!8SUW@SAOFX34^T7YfVh(2K54-SVBgtDU~cAc82*G*d~JCC*fDwTg%@S( zrcH9!0oVpAE|xsNO65kQWP-;yznQfbn!)$pcAK;|HObRI{i!_h%ro+dyYH4Z978il zXmsu`6}EJ1Y41RtnY#r0l{(X=7KRZtvc0DA$|MY*t zCf764wDn#iJD_{D^y|Jwn~MZln+1R(p7)0P%;!C&g@e3a-qSMYx3z%n6sCJzSgFHe zO$KlI{9(!Tki&96nIn!bZ|Q4jtd|>?aj_7-@lt!3X7p8X=O`yZ_%qs&N6Xo+)6lk> zg@AqpA2f%CX7$+KwHvsx7Z>Q1mcbSzhV1lov79|O1rRy_jS+c(} z!{Kp2esHS9hb$H?FVEACv?v4*(w=qA#dfSvcFb$lJc?Yw}S zWWPM~vrG7@(}#NUaL#>9Uv@wH>GgVJ7hilAb^RosOXI2=wxBB3<8?8923NW-V6o22 z$G224#sXekc||<;vHC(BW>yHXT9oaA4FU>qk>!i;6uw5~10L=2IK@Hrsk-$!NYpA& zULT2{(t;;T@S;{f(r2(3kHEO}=`%xe>hvfKQqIf2`ZtHAWnDXrKvE-vSfXtlcRWSI zYy59b`%#|%DF7jVY}m9JU(#WS9LE95O7rE!v7h0XU$Gvw%g*Md0^m;{Z#Kg8H~GV% zMBb!On7nzdXB=9MSLE2My;4|^ha-ZT820K+Gc zaFpkx;SVnVJAL#yGMpy&jZDtXn#61#rtb~|^~mdQ;0WGvIdt%LxpVJ67`>j;7db*Q z*kJ*Sv%*PmtSc7+hz-q+Qt;4&^8KIuL|!;?QmSg}q#SF~A~a?=lEQK9;Sjz-_Fs=T z-)RiXdp`G>?-uFzqUw8Dr`+I~xWY0nOF!z4(&u6Nbi~`^4B9H_M}^K;?>_y8mH&Dx zyj}W3!IsmH8mvjQO;QhDamo)eMcR*f-**3};3os5*!r9)Mz{5v=X@vnpTdNy4f~{F z%U-$h@(-nX`#n&#@fxE3^7Iq4=)Y)5?i|2O=yiP06C&QtvtV9kR+$k^o3?FEFu0v?FD5WL5fftLSZ(@i> z(xEtSker%?P-=QnE_RLTN6eRxP|JQC_75jz^Tu-QrLo_BfJghYRnTm^r zgRGQ4Sw5sSN^>-vglIK}<>KjYef+0m<}s@fOo8Qq2M3Mwgond{6i3_{%>A$#Nsz&D z7<}d?%2itV95#S8mNPwlON~zi8oA8CN^aZ~9_w5ZUcBMg=wIvq_;?M+a5(mS%y9JD zxOM-dHvajwFfR{=KcP;}qb$_cpqZnb9H_nPEqpvG6t+j$Miz?xlJ98%v97idM*+*^ zW4AZqdiq&;@_U!n=wnNBiPSX|s}V%wV{2Q3Pe}e8LSPtY42B@nW8xoex2s3A((f)f)fR8u0ex zV?*oWj~@?;WT%ZC+QAwcoRaRYF=zvH;|lt9Qc_VN>)SU%W2zBtPK^w@6xIlP+tX(o zXa#}KRMd~SwfQe-AGDpeNvaLw?e}4t|9R6Gdw33pc%3)%u#qHh(!LaJG{naW9#eEP z96ItQKSq)_Im&K7&;SG9v+}KPUzAOoD}=mRi?Lr>P2<9Z;$zU-H4QG3JqHlcGXBHy zI6RD-GQpcg*tF`A-3V`HBb`EnR4E3$n+PPRl1CP3jQz~nA=S8QZY+jiE+6#gsCFZ( zO~LTW13x2gYFc^;@+MF94C6RsH(mgqf9a}>PZr7MZEfJq4bt4wia8N=YHiN%hGhD9 z!uUg^@hL0x`poySxir5<%P{N>WA%No_#afT0ldSAt~$h_gRO1MXSB~!^wnvTFmLdP zVBg@7yzuhNvSs52xns{>Xh~txs~)5s9BPd_gx$E=j*)?&FKVJZ$x)!CNK$ z)ddd-|F!g*6A*d{vazq0%@*yk$&QubkRDQX7re^&p`n%YW-z2hxyruK}Ky~ zhC`a8;WYX|x30D92)K8q{}ojD1Kh&7<2~wswnq-^iXQl+9R1tBkp45zOYP=+Bxm0C ze%21wj?2DKAO!aF3gv8ZCkO1<;&Op@fZX-bNKj+b&UIL7(t_^G6h8x z5^>D(HeB50fPHB;E)OY!h7~l4@^-?Q`K(;)y&;>o6v`tH7s}3ULpa$mFV$uCF&0~R z*}k;|$L8_j^UZPn*cuR>=5Yw;;ENG?`Bofxhnh~!4TUm?Q_o4Hrcq!U|I}|UO^ocLy zGUNS0>wn_uVPO4q1%FfcAsFPVe9a)<2i1b?&H@dCZ9Z2 z%^r5NsQ_OeV+}9kQMPYG>Fv5n) z8vSSuG{;*1>)=3JVB$bye5A&|ewmrTOW-tKUU7^E>q2%GzEZ3G8rqj~263Nl5e?fT zwUnsoQC5}*1I;CJ$Dw9<;>pYM^pCH~mzuZeF~6#6+(3t~J8}Cpgco`0dMBKE#4%5 zk~i7Ihi-3{fBx1*`Q8t%$iHgc9`I&?`fryvxfh^dio97=l!r|D-)BD>0Hp@f_4Z|LCpKTWnhffyEw$VJ1Ode+`x5EFJr4>#D zm;njuV0^e%GBR41ucP$oCeab6uUy74j|Mpm`%rn*Xu^w~uLUdbH5E+i$JS`8pEYdl z+#vn9x$n}|tJ2lqFP#nbvH;%l`kCKxG9dA`Fx&~xt04WJhjAXJPg-w{ek;$CVfiRN zUYGgI7jI9C34d?3{*~x=$mC#agLjX9YXPs(d~0^EGl)y|MLaK`-f8;Zjs8zCtIjh^ zjmLMh|8rn}tDV~jptqA&fWUG4LGwgj4?Sx1;WE9QI5-U5_ zMHrbz084vXq#i~GF2FQWIw=^)R6YtTS^rA;z2?k5r#60YEU&1+hsZJMKF_0SGg5`4 zdAWJmXF^a@KzoeXH`5q7^XBH_W9bxyQdu%RL%|fZq+$1UVgmM~CWdhdM73<*REb-5 z=cOQz`|~NRkAMoT|LW>|sj24ew{{93GgcN$J6P79%jJrvu}k@_ps)aII&5XZ>W7Y5 zUTj21h^Sf}!%4>^OzBd9kI7Y4c{nCtfZK-0<;s<-*e^_~rdKZPP<~YUKWyJjt&waU zJ}1XtGLN@SW_Z^0oT1&lzvG`+tSp#{zGb8)jd(*~ssT}$d~}EbD(KfHNVNkeCO&u* z#PfSu`sO^Z0kEeO^wdVcCu6Nl@yt}3wHj^?KMh|kW_2;>3wsD3Y2FT+ca47Q;mrNH z*8jf>^n0vCnsR6nX1F<*O#g$ z1&eTjrq-14djK&t*J))66JP|gEu3$+7wGNVD^+lG{=$e1LTjWBnqE!y#j1%#EigKL zu|@?1hyO80?up@#XXfYeKnkS#`zB%RaZ3K_pU*%g8-lakPI>r|&&#GQJEf?k%;qFQ zB;d9AFV(=>96__Nd43h=e|~l4S43(}&Er_p+vQDapsZ^xk@oe)($QX&;?1JMOj`&W z>9MiRDDauH6*ts+JP+fRLzfp2oO)wO#_@%>xw%-s4o7)2mznUNMvYTI*tu|oi6mSP zZ@ft(ujEZ?HFZJLaGW+az@LqEu(ws1lbTQXZI;3-H?pe0f*-<*;g4tKO?>$!fA;oH z=uJlC&8s~yM7*$G_8oXoKJlr~Lnzf~Wvv-da&7+;?*DwB$QF(}ox&iraX+4FPYy?Y zjParI3AuFXvK-#GPbzqH5`rlm4+H||VIOxkQkW90KU3J^vY}G2xwRF?r?1MJXU|GY zb+zOJb_g#%91V9s4?q%6-%qkutCs`AGFX12nW+>ccpRUTE=Ip55>6DJ;pThSw@ZII zJ)~{*^c!xZi`?vZx^L~|G?N?;n?Qvzakm`((LRZm(zalg>Vl40W&gij`rn=YcUsJ6 z9ZtLV&3o1V5S2*lU7wTVU;T6GI{pLMaL*THVSMZs#=ngPwln{nFN`wpD<3c3iH!F+ z=Jc?qTj}t3rTiQzRL1ok11Bb?8jY)V1MgIpS*JHtX#Ty5gJ&f zI9<`I!mEr3Xv?y#y@fqYpU!$<>2&7tyqWTQ9Q{l#4u3WMAC76ldn@=x@rc*|*5JMK z^hYsFhBNIdV|jF9B^zK=l9}kz8`hX?sdQ@yE2#hi7#>dg>D`ckAK*fqQZ5p-ZU`CO z@HM=dpLjYAsq&G4)9Uaw{~G;k{l7N;l}nU&s5*-khXNhGsPV;eL9YxZBSu%Q2xCTg zEEWXA@~ck{v?TKJRi(5PUqg<{r@V;DW%+CWi2w%x^wEISZ6nT?&r8aG- zP}^3@lpM*(!+4^-ByX}3=2JnF@#f6TyqxSF!dKOK7=DFu3&5cER}pwqH9Wza#uw2R z0TBGxKSdltKEqK%iuUBlpXAM(xHa^`r70O1FTioeb+GxiLE1YuK*&zpQZ~k{WB?!n zl>E7rV~XR&JPymSB|f$n^Zkj!iGZ`R(K+mc!~agV`-*!&fYOCE#eGJ5IHc8ECJVX@=jb=x0_K_+}q109OjRxtJ|N zQ;c|<(8vj7<`BNs8`7_1RV%bKIUg9(WVf{g_Bc~KQ@822Y(n#juSD@&B=#0Q5in8(tcDWw1kTwyqd+uLA;=390ILz?187j z!uV($2#B>0Q4Mk65y2XKxZ#n1vbs)B1e|c}AK#~HQnZN^dD9+|mmh$*9~m5)Q=6&X zJ@fduLu2f-*z?1%Irap;w~m(Qu;AZ<-k^&^x!9{zu=Gd<8qG69SpD9t&yshbH%8{P`o$3E@+&DH?|3_u~2aTE0TTw8@ zQa%B=AB|I}@pjYWF6=`dds&T3Hgp>N1P~t}?x&YeAe3VIU@)$boTzS%er>;S9HjM6 zqCK_W5x3U=Y?zf@`uL5vbrOP)d-v1JCpzTa82_lEu&5CG1Y?qcjAWQP`d}5Nv!f-mt1ttBLjV z<#AwO3OD?XOIP<8zO0Pu%jBuo`l0nvDjPSJ;_FMkE?``Y2~h=0I5J41kOf%$c(iYx zU*MUFAJYR{#EW@>YFm|+=HYAos2n>sfSXN*arN#fY-H7AbC554cGu`j@Oo%~(MFF7 zt8jdfxak>v;7J-(pa;Oyc!{3D!aIZ`g1nLNBH;e?7hSmh3@-!On<0$aBwzl+e=Qvw zHpAXliM=ckZMgTP|Fu=D{?sbYVF$Oy{~UxtJ6Mw}`)~8Dqa=v#zy?hn~JJ@LE z(eA`F{-z-OnTBzGY~E758S>In^>`PPH_@rmcZ0l{k1wyaDc)2CCJK!_vxWlL@HB)U zZV)LzVEwc=(c3p6*I+>XTF-=>J2wL3t5b3uhSi&!iou`j!DGgod}$$+5nMCPFT90C z+?3*?0Z5hr?m!X0qP(e_SL7pa^7vjk1W-I0_}Xh1<@jsHp9l8W;mB8!?ATtd@Yl7J zz;G&fS#Jlzab#}HSt0(jO}R;40IpNuO}F*dh53a1`Si1w^=3lJ-UQykAu z%F;RB6sXSI_OkEK9FYz)8fW2TdB$d8tTi<@$)%nie1zzh=E^FJj?8Wh!W)Wj%D{j5 z7?fH!o`>=Bbn3!+xr`%$Q+&z9{qOhOb+CM?d8x~eKUum?t{@j{w|^)rb-`65aG5at9hk(2^{E=M{u!DS(?cVtA49zP*h zZ`_bs8j#1o9JK8;1RR~~*2%s-d!!cHy0o5_*)g89#M2p*!VP@90GJoh7QZ}p6t>wW z(RCEoQT~MwQ)rV79qqEUqeIH_x&CwIwe~R>z{(zNEFf&c5<*CKq~Fnrwy)=O`uUu~ zF@W!R+_&~*eI66{b>4u$d-D2e8SgzWLtW=(W*oXzQ4FUfe%d6}IUflzD)b0-94g@saD z+ameRTcv5!LA=akE{*g*>whcsXHCya-PU_#;OwjN=J&oLdw=hL!T!$-Yunf}J}~;i zI{3_fwYDUE?56~An#beJx0aw_B6QJk80#UXvg}g1JZ-h`9gg{+Kw+EDI>|K-lg}PD z_=bnaF`dp~+7o|y(GdPidk1f^i{^JYo;Se527(G|u4AtHtH1mT%s|}4+Sw;PSMz0P z00OA0IXy|BC;=w-pL@2IOEdPUwKe(JgHl*zdtQg`KG#E?LYqYRl(q+Eoh=xNr7VCZePWQjnY`9o>wfKc4Ekrz>#$+0wJ#Dq{y*1u0 z{h4sFxD^Fc#$xOwHoGHkVaCx>5~>pr{FI9XH5;Fu9uIpS#4F0Va~KI-*2g`X&& zKKTXs1e07^`U?1_Y;*=f zsWBXfo5C?T{6S-40tU8VjLZ$04f`A%3L(LZ1LrDVroV;Bg!GP%E!wO4Vsfy zeW3>qoZ@KzCI@S?9oNgk*Pj`!qZ==+Rh7dkkm5eOydn?uOiMLhvMJ~)FE2!ULFfge z>wL*3-mH^!PvEN&?c?+T-yR5~y1OT3da42{+66FzxeG798>Ja927IB>`H4&vv`N~0 zg?Z28*jyf_Pq)&1ot7UJajsjN|E7*od2<|EP2@}E&0_K% zuCrF@2H4h>Jk234+<6ymDsC~qPUc}{r~1<0G6KSsxlHz@$CI6tg{U(PR8 zu&XgVG9?udN>RXEf-koD;Oj!n37il(Cy+1sH$E||_+RgZpc{P2qX^SeKOGN_CMDW=V8|G;SU1;a}FVXK9^|w#D1*(oLM+!MKvs( zwrRXTy8yKA>RbmyH?2Ei6r1;gFoC&wBwAZfrt>j6{B z)zK08`nSK0wGKABIyt*{{jSrBgCKG(;)9 z=1tYF_2DP$MahBGp?BRz+4zYsO6|r&G6$hxIIf7N5$aiS<83yLA< ze*X7z&;8!AnBt7%+ScBl1NEgB34b zA2kmzoWCsR-n<~!F+YxtjbZJY#YdV+71~r)6=FWYAi$Vmy%aWapUMHSV{5sTV=a8; z)gfK?SX0#T$Q}2kG_}EF^Ev2qnzrEHU?s;KML0G{+}E{M;R6nY-Sdrd`JzCuG=`6@ z`Oy5r(d>~qtl_Y0S%Ovoo}`o-eiM^9a{fvcJ_e1;i4*75<|>6-{rwX-oiHrFeBn6u z5M%P@>DOgs=o)CA0_>AG-dKUP{qxXhuEfWx5=E1^cndPZ!XMoJPmXWL*nCnDr^uc@ z4C9};XLhUGrpD=3rXjc(>50iIY&?eIjHV-j05~GxZ1OT+pM!>W_{qi2nby-4$Z|i# zpU(YcSQD4|S))JNovD5g+n)=MH96*`}qU zl19Yen&VfIUf5N-IW!}qcnO}vOQFsU1c>1kvjr0jH_iD^d@U}?)fZ77we#z5=WtBo0$yIvL%XU2XWq+j z?tbeQynx~eQyw-4#Ag97$Au6q(eX&e)U+8w9~+;=3-`PnJKnFGycb@)fyFjQc$0{3 z2C>LCHJ0m3HMKPIXuJ|H&-%iR7iATu;W$$lglUsgg@9cphwivXIyyJVqmTVM7GG$1 zRa9bMk0TXm6z3^LWThXr6`lLZuqH0^6STFLcfNO@lfxSSdn0+{zQ6NdX=xe8Z4thn z(w+kaKU3gW+@QHkH!qh&0Zm^&d2<>^*I-j=mAuI{J_|fHGDM-ym^|^s6%|gsdh(j` zW?5;Ww5=bGn%F%wj$QI#iX-SMX`O-B5 z{#D)tk>pJZsfTC}4##XM7(4a)Am5AS`4?{BOKJ`@vv6|}c0l}^T31^N9?O$jT(m&V zDGIGgGxqr=OoImIdB^yubY)>r3Z* z_kD=QGA;AqLpvz%eDD6P@IU7*owGSG5{@s&I=AVItW~Dl3gmO&>!Psa+O=y^0fA9H z?5Zq6P!+lmw!ak9r_=RB;ahH>x6Ef;axSxS)#%^bw{M505F7?u01og-58+xqYW_^& zPV`w+GyqMYXP$cw+OHSow*3cWAIg_N7?F)U+!3gN7rr*H71!_-2I&(|JtY@m)cuZq z`(^jGt@w~qMCmLlA_6D_KMSztAbdD|>5_c!hfm?WDvCi?mEIu4;0 zHIxbNJlgTwPyJAN?6Z$OCfhpNC7)wi>(dHxa~5=@fa4k1^LjGC?F*m%EZV*TFS;T9 z)=zB1pnmhL%|O>lw8cL`DAnB5EdTNgzavc$O0oU2SQngXPC?*B!Q*qsj>*@*^-Z~c z_zrpK@F6MGMo;#SSCHA~c%yr}{m*e>aE#Wp{?9Ftf%7lP=_kJ;W4+y+fS?uCAUV}d zQqj0c@=GyBq4_m4F)A}dH)N*oiVXI2%gu|g%Paro&*jh`{W~e|JS>Z&!-}hqM{h^> z6_228i4WT=4}zvy+_rl5pZ_oEKK2~SXG_hx^^#TAAf*lKC9k+tj}uN0_Q-TE=HIRh za_zMvGI-^zv>tv;I`8|v;FzXTh`wV5MsL#hv7V$Kv$ZT1l*`7?|BrI&AOANw4*}W6 z`@RTR_=rcaOE`x^-@;BD{b-l~mUQ{eIZJw6G@L1p=^-q_q>IBhJQ(NrFi$j` zeD*Z=nNyT@h2B0gAQ zeBjiaqRg%?!Csha-qtPn5Q-1B$4(CG+DAO1ZW@BAYdw?Dr~y6 zDK3Vo4y>yiHk4t1Um#WWTkz3#5_4N0juOKB1=2@GaOwhDVrBR^WxH&kiHT==Hb*XX z*UIa!FUUXs!xiAW0KD^{5jF~(2jLIP0EDs6{q*|~sxM&wuqdye0T06H`^RvbDv$HI zV2Y&Ext|PwnEO9U%Tn}ty!+8I48Pa(Tb=RW<%)^v&k!VZUdGKD>ERpn!hPl%y&IPR~}Z*9+Zpj+$z zweiozEgv=%_%g<`>_fw25Uz~kh!2+&Zbf+WM|A4LYYL4vQ-kvY)kCf4a;_dOqd zPxm;DV41b~&zmC6_jYvu!#4j@c$tTnyByq>NrBWbcyj<7IGD=8`CIZPUjn%nuaZBJ zmD+>bwpBqxWkOy*J(c3keS2%LF~qS~@Ft<@3LoK3^2O=1LpTC8id%F>WNd^sYs7dH zT8R|?Rzny>yKm%86#|1lY0rwhNv$XT72`%8@-l@|6ab$&Go-vpdlTzhOOS!gcZ{ zXu_xAd>GX(051C*DxsxSg}-VTSKkG#t179jZvd|sp{;EmVLjZ-koD->vju!SS#12* z^D@?Xby@z66mYoiud4j}#Q((6|Kcs+?Zv%@wnC81ht$Zjc10)8p@ByrT&d-trs0zm zJ)W07VOn`_8;jE<{aLxK3|{E<&c zCv0G)UeNg=C3x*}TFu8RK|!JX_$SXmb6`Y%>+#2>1;+++IG8wa!8oz{Kz9z}ceml_ z>*qcv-}wIbVHT?`50)=L2FADW`LxbpR z|NM1%o_@_fH&Q15kojp)Q{3<2es|i-$gTNVaU=BLC&9U&Sr7EwZPRbU~Pg z^q@MWFJ7P3>c76uz1{w|F__+-tb0BOg0#_Va{9afS;qRVYP)sb{TXRH_?YCC)yV>c z;@W*^BEH0DLvT3RcTq09_+9Ba^0drD`|Z`Q{d@V?zxfZKyOI+X(6{!q#$lL^6wqx! zeIBQM9SHv4eDeR1u2+641!YyzcK7FG!|jjYBAP;Nt3^s7U|TQ`+IBM|H{{$ezJ+?9 zlF8v=x$@jMA;8CF5s&ka8NwDv!K!T#mkS0(;T$cI+OkbFX3_x_eKh6kxqhI)0qJ`_5Wv zYn_m9J$Y64vD{xrT`#oxzy9s3*z3>ght&PMD>46KU(IzI{m=aD=1dw?Jzea!<5LU+OdXWPYz9~Drj%z!{^6leoUHyFl=CORz}9MrLlHGhMRB# z0Ya=2)LBsuI_6Hu)W{|94elY#n~{6(Etcknd^~4EQ)(2rWXXx+Z$MC0D2+{ZQF4|E z$NSIwnm(QG2N9-kF1<&?K4*@2di?3_!wZmzc*z#Fnvo;Ool2(4@8gx5lyYwPJgs&3tUK`Mj-td}@;} z^Rv|WH|-t4%kLxvNE0|VHzxC&`DVyzg^S`XIP`BR1Vg*`)480$ zjrhV+1fdmg?jz2mnu{y^L|pmO zN`V^Zm3b_p1rSsf^HwpPn5Y4k3j^bYIzC@~sZTXTx=}x0innjBl$|@Ppki!C07=;x z1WbcCHrNeKkjY8D9GRo02Mt8Gv=qwiw>7{(Z#G_zMQR%kNFKCUhKFGH0h4dK()i2gI`=h`W(|Ip?|<@!8b-z#=XKG8gX7ZIJ1Xt%EKc4u?F2rp$^nmjZ1i^Q zs=>`5+@Q`vdx^Yx4MO9mpX!ECY8XPGQD}}dL-<=D zhYr@O<|4n=s!%J+n`kf|eI;*d6QO=yh`{!u5b90Xq@wl`d6T@h4|p}gb`&)tS&kXW zo3t^-3)HUF^!;or4&d$$Z&VW4_@Lyw7 z6Xu>G*|d41G&Qv-uNFfKfoW0qLFxar@dP9tH{zQdpHUhzg)_p7Y|b3E`$_om@o!@U zSY)aSaUAqa^TF?7%WqRP4WCconC7VLfff=!66oL{L@kcIL2<*yXx8K)tHNP+&F8`} z=I}TR`!C_p`I_g?>1{(;i&&3dRojdA9y%l&AXJIlWHMNU$NC9c@qqB=TF3CHt3 z_27fD4oB&5Gdy$Q&&nk6)#ENmZo@IZZJRdBn-|X`+z2~RkiKFbNYal%E9J0DMf+^i zSPP&2*|X3lZjpNq+$MDpN*S4~ODksOF_#M*OHuB(p;`4;|Mjc#!=L|LKKsA}g6}Ya zGx{B0`mE)UZWU8Zm}F3h>Q zP*~PO?z4O}96niWa=(_u*UP$Z@ZwK~mJ$CxJRGUqw~CQ zMCVIS2z<_K^{_L=TluK1%#T0pqNt1!Atc_)DnKLAhQ1Vvufv=rvfDt_f< zSrCfl!7RoY>V*w8@KHfQmQ*9Iu>rPkflt@9DQI3o3u|~3b1^?o5$bAPnAbz@6GAi` zK3VMh)&C)7j-C5q%IZU5Sr55S7}0R}WU;HIKP;Dy6?dsiI~_>)f%8oV!#)ww&*ah< zYa5WB*C#&r86So36OjAK`jYS+j{7C+(cd~uG9o=8+f7=|so8x~Ls`hW5c^$T5&S5c}kQ==)Fnqp8 zZa-M48WbTfMLA>XFc(XHwW+Twlt&&}FP$9~@}nPjL3MXP{@4HgxEd7Sg%?mBx8a9e z3WIoj?)aJ`_qfnfvx4@ja`1w!qY4XIE}nU= zoough_`1Ru>~b6>+P|+xZ`}LVw=dz!-BEew*&e)9*WjyCquh626SPmTh~w)NZ9~zZ zBegNyQ6p>r9%woNPA}}zG&OeO<-JlKd;HfSe5%roI=5LmDPRue7kGXl&c(T+;^Jax zi%_czU!k#y)42$G`W1wS4^7)4YG8eH0D*d}VRr`0;t) z{ac;?c)QiPH{Vd+oSMY(*1iE8YweaDJE8H1uZ`h)$C_7m$yvdt3|LQ-#fz4=<^x&qvhd;g!T87Z7 zity$PjHkLIZ|;E>Bj*_&pDiq^k?lL0<4l4nqht)cWU@2ucnE zzc(PTdjqzyZrid2!sk#R?D)_RCW-)tLt+92S!@kD9}8ZGwGSxQ)VI?AL$D`x^UY%_ z2&mh-Pn!0BLKa}&fLn`jY$*8159-5*wRMLdmywIde*!;FX_;cKlw@I>b3!61YNNcM9!ACqwTC??T+4`wbIr-0jC384( zdhV&O%I@F!3)RkBssCBOws(Mt)5u5N*Ffl*lT{#1J0F&*v3}`!^OwS7gcCLEq@=b{ z%33$e47BUBaMarR!#mIWBmf+Y5ZlMHO}tfDPZ;*xp-(tWPr~yYo@OwNUfVM;N62%T z^IL4R{&;=q_&V;Z!^iM;9GzB&;k8HWw>X|$=)K;Lb^nT9(h%*JbFk@%gQW87pI@f} zr-}-G=&_GT3mAuVof}H9XU>%e?rwyZ{~$g9jT+$KmwitAj4%9hKu)|iBu_kfSsLmJ z+$+W5?AG|nl2FT!&`IGgn1V7a$0eGLp+)u&}CopTMmEb4wNs_Qq`DA*sd=kH8eaZ64 z{A={1MV&Vh^nQ0gt-VqU*joQ{EUq-Zc#eIB#%H^);LA)8Ues_?2Gr5}`mp%TQ+?bw zfrPethj%}G0m*?j1B5ep!98{iNAm_}^yWL>oK*tNB`!o%^SyFq9M{0}sLq%))|cQ# zwFttjGJPqfv2p!^gOx82YQZ!%o(};PjXvh)hXaGt1OI%J#p#F(Ck-c8L#u<@7qnF~ zIy#F*aTYe7CINp`wr+)84ZJw(*B#)@BY+A6Kh*x9G36$uAxB@FD=O=ywy;@R*SABH zsyeNGlP#1SpKJVYeCh45()j#Up8u#VMH@2Lu3ZH`UdQ-{%^Tdx*V{V`p$o@p_)?b~ zcgsDKHwqv~YHlfoa5Z0k`O9AA&91I7H7H&Hflmp>Jq1%eutPD7BempBUNX>7S0XL2 z@wOQPtr{2|C+|>8kUXyZxdLx0$D)2}WRW+?U!1bZn-mORzCt@%**Ic2D!X@0;Qby) zy&#;Ubg3AcmJ8VEl0PZfV%bJ~=`Dq^)uEwT*k>uj5!*It18=TxU$0HD;`j<0%Ex1} z-Bv1}Oiz~oRp9?5&3xtdZG-SouAg z0xE_BK^3aQqrOgSsbqbQivP6&#!aGRYu)?K|hM!#BE(yd#=f+ z9(qt3ph3mD=x&An^=+*<-n&hXz_@!`ZH<(`IE>TcZ~#&F3MY`~4{!vq2>4lRTa8ir zy?&4L=K`$T)#YV!v8!8lZ`q974mE{8a4t}A3f%Wp%R`vz-r@dVgwSp5+Bq5Oy$(_r zpm}sBw7sd9YA^ewKiOYl94<&{!cCqvjWTepTSl&4!leqs*snlHh`Ei<+tGb1t4RuW z0pp8WO4}a$11YY75LAUyOpms2qW_VUhcfF9JuZW99+4|Ye=dDzUz6cWN2F@wosvzM z!Ep7y@b+W8$7wBq<#BxWFvTc2&;~Cqub0k)kHZGsq|8o@0apmq@S(&scMX=q@jCcy zbf!8D&kP4hF*)K=b*6A${{7O5I&9)#VfR^&g)`R|sd|+`ugiVHW!T{-eNS5|f4o|! zpIZ2{xP6zG>{M45V|^Qh5XxXr;^Loyo3qe(nncZ6s-5pNdhNXYqXZvxJ35NxayNyW znH?|b;Vsfvundok&O?|rq#tLuY%IfiOa6{i=TO+p*e>h(&rH~AqX|L*wdO+{@tLEr z3}Nmk=vcO@f**-!Zz@|q`mSrDrgjcZ{vp3+!!y=+oRHv0ZYT``Pwvl)^h2*N86xzF)cs+>2=j^c>RqbY;vj1uqZXPJlmctU;OQDQjgobsKLZ*;`Id? ziz^qVJiIK^<_A|mznw}v4!JoWRXH_m3cgmZ&7{?)RKL30#=7LE%e_>dU0W5+Es0~$G zmZ!W~3OiENlHyHz+_aWLP(;C0K6r$@sJBxF92n(I*2v**HCvds^N=@b)ZBS<1Zw^V z58k0ds+W#T%H%{BgvTPE`%F8o#x9oM`t?qHeWmg9Mg6MFOAl&*-*woKd-ADka`4bS za^C}=!2A)~pK!i(pLj8>_$glY;^O?cw*L9tW#h@mV%R=5hxt(RcJ*iXJ>Gqd=Kyf< zj6h%T0*YB&Izg>9+78hSK2Vq^;xrmRsbmEXfL=+ki6>lj%owRJjTYwTn^2ISHdUBk zeFW!mgq6nrTW}n^wi@=Ycq^O2jKXIDt<5}y`8Wt0OxY0FmO*1y?Me|xCPeVjdNnQ` z9&hW*g$9`#H7AZi@s!Uh0W$~r-gx5;IdJ=J($>5V^8wy413^f#zGdl$rp;a)%ln)E z_BFY&cdx7m<576*P0Q+t{3MLBm%~7PC$y?e+?JVs%daSU(C7zl>({l(QQUZ__$@)d zriCRn#H>mBt-~}0!C>tO`tz3PCv0dt=gf~uL1_uJFY6?$5Eqm{=z}j*OjY;>Grhkw z89zBVvQH%Ec@%4A0$L-uNFm6c#K*^-m*ug)^5#})*!wV!?@cl~=9Z(MM-A)uJP4uG zQD`Ds{9=qv81N@t3I~HsDf|%da)g1B z=EcI%xG>qf&V5#5b9F#Om^BCy4$~Z-hduxD>FK#KhFIsS;d#C8ry&@@jcK0G{dj(- zdmh?_vM?iu4|Yg>&4T=O7foB3NsV!6AoX0Ek)QvfU)Hx3%ce~@K8KICP;S$fAVi)Z zD8-)oq5B$Cfc4TV1F2~tUZYJH*L!EM_Rpwh($4J_a>v0M?7?|7-!!q3WvD?&r=ln{ zBB!XNV;ny`%wiv2g%4oVs-_^U9B|tXl|cJ@TE6+jHLT&e(%uHmC}>D!W3RuV14oz9 zJZ77UM;GTo!!-5^xpL2g)i4Esv6rHh?dZKo3_R=zEuWsA`-9Q{J`PCFhkg7nFPLJV zfk>Y)+_#OI6OSB7@Hj0Jl=HBsS!1o7-|KPT%45(4EuXB9(ah@|<6$rFc(`wqt>KZ3 zvu7Wm<~tnsZX{>R`sjwW^+j{zLcm@WV{4-ZYY0((A6=ELOYg=Eg2FZElEe$kNJ(x#>xH#g`KFbgfzCZ-<9Tz_OpT>dVcursKSK;^% z&(x1YTMGMXTyHy%8wYFfB2)sSyKl-Ov_WX=21_t*^eR+=78lfF7?#9dRd}^Lj<(~= zR2yv4;CN93jNkGrIB-eMUxtq&=V-+IHU3}Yf35|~&42tdO1o^AE}X|T(br^bvX7j+4V{VXuMmUZj@L~ho<%y zr+hl&8}ls&2?P&K7sW79o#SbFkriW(Uy-G zpYl;tprNJNwBn$E;|{;2p-~>b?|!K*qsB~-DS=-)LBx%DnGr8_5L6fq@wU(`ngd5= zFi%Wl9dB)f?KG+>t9GF#I+)N;Xc$>8tBc{*=4QO~mC98dUu=fvRRIKJRye>P9_Oab zn>T|G@$yJKgCv$u;hsW}utR!$IPii2(lrAWh(!p}!jqkZ7s;0M>t{Xg zB4H1@zXJW)=;ki*A>r^j-Qd~xsnvV)%?_yo?n=jU3=Q_u{IZW*-ol` zNoq^8d1vr(<`*ya!-idv+dF=kKn!t#PC_5fx&U##h?gRdcf4)WLvfa(yR+^9qcrH9@3ytZjU4TS46EFL+| ziAr?5r*I)cNLN_;6NZDw6fPlrlp(-W{=~!A|zr7XVnJU;BKm|82_psE+?B ze39$Jtv`SFH(!;rXHGz~sYf<+mdNjaX^Z>_Hkf|#e7{^ce*<4DU=SIbR4x)JCSx$8 zb1t<+AgOQ^X(H-GImuy_1Iy#|G`hnKYZ=) z@RjHcjuW1R04`TPc92IXP&a^HfwARL7+!wq7beCKIX!LX#s{4QUH-7`$Z?D1rPM2)hTp(ZkolWwK z7y6VxuUzhf=F+T`?WvFg{e0;oP@6f!PduGmr7up%>AZRPjt1q;85|Xuf#8@I5=42E zJjvsFe7VQ}1aAVqR)HU;1&&daIcQQ4r@RTis+VcG{!I--t%4-vaDdPc(iG!my6@ z?Q+-t{n7{4+_C+2tjP`ng|&uc3yA#8&-8IIG&!4p2KWSwwcR_o+5<0%lgn-EnP z$^1H*Xp)s>+!9W~YbD6Qdb|$iw?y-`d@>@)Zyl*QqCbl`K9!f1kNISs!p@=@0}6GV z9?cxkhA`((j99HN7g}UJum4mAuAH+8RrBIU%W5veXlSe|~XDZr5xC)<~C(Y7PlNcL1a zu^i)5_1ZfbV;E<==r~!d|2m(#pPt_<)CvI2;L)dJIgc}#aU|nUA^HZ*AcXC}=XeRn zJ;W~{)hb&>{?*Y>?Kx^JZ`;xgP5vTiM%ClP@l8F(&#L%w=9ObNVY8_K<}oIu8AsY1 zaB0W_HK#EqC{SWIv@{pWMra^ygUJeNC@m$JB20ehX=%!rwzfjlSAuaw0hhNdbKJV) zo}s8XONQ`amqyrITZ^=wvuDTkDE~&hbJW+`>4;g-%cJ^q$~Fj2s9>@A~f@PjefMAw+Y(A+ls!n4}6AF$EDcXh+FIbU_g3Xczb=w#y>az)38%>^2AYG zL;E6b-#dlP_keunQ*Btpig3kjIn-*$}a|Uo=8qS zJ8yC*QRu`EBDv7QD8yG1ZN^AjvN_q(Tof6tFSQZygq(OZ7||+Gkf$zmk;9&_4_D*% zL$jj35%#VMc>~}CG*ZT}P)^I?JMWe4J9o?3(?7=-+#xyj`Y`4QY@UHYR&FacnT2xS zgRm!6-6RKYI}A1CN-U_swh{H%d=k!QY5YWdnJ>UEnO~6)#*k(q{Uk6b;Ni9TFK9rt zjW$g%2tG>lKgS8}?cC_ODo_9LN#)J_oa^%GPc}R;XHjXL|_@i4H+iViq>i zXn=e<-ekKaImX&j`O=fns8s*QZRzlF$pCm$zm@_gI*!{EZw6zNa)cn+0OHrdc|=ku zMH(rX(hCMqeguY4uiwBi6!2yNY*@|U@`dcIQD}pW$l<&0m0hFzA=cavqrridjb>!SicfQFuszkfWlk zOm=SFD(hQtV;-ce#MS#Vs7XDKW1jODFJf)$h3%H1h+xVqc3<>(Auq2m0QhB63ZEbx zW>4Wx;Y*rhIj4b%!E+Xm-yxnFzTqlJ2MH&xFHraY_8+^;yZVwh(uZ zOP041JjyqReQFU*X_S>!${+sz{kW<6ikx}#8rIGQ{o_R-{R1=dpZ>qovK1Ope^$8_ zLLS;vgY(#s*QB!&0(0)n6b6js)_wl6S|>+f+Fmb0fXzm?g3)CZta0X zlWRS-uXu4$9(ZU5rx$JxO<}&umV551!=5@1cLz|@%lpjgARqCxLQn53@ZkxETD7ND zh)YY{dB3}#-kwI2_jK4|;d4|HEPaD>VM8`ad#!Q;r{fS-$t3uj8oB zAU55t_B+2jlS>e z?{mK|tMa|~zrPy|Qltow{{#B3dR19jSy`1;^|Gq6IMLJYg_B%73QKsVLm1!Zv6Y_r z=I3Idr6-y{;h0WS`yU^`ftkng_}V`>iJ$;zVomHH8?yaUwpI`$J+3rNQV11={diu<7&&pLZnEKjgD!uiAka2Bnf`b}okkUfTX(ago$|en>@^x$i@`#)ND4TV4i`XGGiOKLdZqK|VcYf|Z z98su}A0+Z+cxX^wef6g}x-^GWOV}B;8Akvb4*z3FR;I_4F!RkWoOL z5o9i40nFdkALFY>Kv43AR?e{yjwY#9nT+MSvh$HyUzl~2Mj`pnFX8c5GAD1<_K77Hcwx>IrvJD0ZY z*p3SbN}aOF5Lw3F&A|dmZGF9zW89*$&6Nf882N;w-_n^sh(|=UF$1^^c~;g`OMX!i zqHJz%1Q0w&m35&)63Esmf0!EiaS_>TtDTG*ks;KYJ}Za0ES&m%f#|S(`PZN-3|`{_ z6T~ATR(`9~MSM&baAyWS?8my_myzyslD`bQcVh#Rni!Eq*yfxW8-iWVNm;^?#3kI2 zmV0?7La6OH!2dhw;DEu*8=Elmug#3y(KrzML zhfr9iCdOS_gSYIiqx?*m0Uilc#9HrwvifH}R%Z1p6a)Pq-_x-YJX!@on`2<`ME%5N zI0#_=C`^!W70-d4ZgYg4sUDzt9G|2LGF8d(#F8F0Wt5HWdqrtIjwenL&c$AlM zMQDdxHrL}g8+Q9+te?dtAyiJ0ceK-b4ma-c@{ysTIVkCIX$cfQ$`;K(2N(U|D1Hm( z4_FMQ!ihVVtjb8EJ+LofCVtV2zK_R-5U7K)im^3`A7B8LuE>y_)Y1x2}Rg3<3=_Ep2UVmBtyvpCW>jt|EY zsfJPV6~EP1nV~<9FOU#F6eEy9KA01{1$>B7?D@x=`q*Lc8-#o#SgYBn%lW94yzeYcZJ7KG4 z5_WrDe(A@ubN4Mc!`~ox;{$hYc1|At{1-Jn`-~#k(V#7sx@K@-fI)DQ=^K(OoKH+1P@|UaldVkPnkh-sHpG zf|N}@Y}Y(06cswIV$%3H|K{fxweAX_%;v*nLH<6O2KqSGE#qll&SY~CO5+O`Csa16 zlc4RWSr~bJ;rVA|-z|r*g4-x}-}4aiTTmHY!MR}KD1tJXwxmopbvFYFVoK2L_|Mwl z^YP7pbPOcVt@CpkIwAJrZ9&epK<|RY98`J|2Ey|b1u}vU>CFud&h8Tvuwk;Y>8|$p zIL0~%evq-;9-c!6U)|6V`Vq$dc`2|K?BVIe6dU>xg#4Km(VC055LT6KEgx2koZK%H8&wt_BD8$WwepkCT&P;4CI>tCcI|1#;%JWikAHGdo_e-Ro__Xgq!I{6men^}0x&dhV(k6gO(j z<+wiwL*psfe${?d`%;SyT}%OVZmy;BZF8+A%5W;ASU&P>2FLGLP_A3|SF2foLMVQT zCun2v6F<2;H=m2e@fqpFO}#DcdvFoU2B)Z3_UVr&D)cKONFc)O7<}x(&TRoVo zv0*gu{(S1+N?^2y<#BvWA9F(jGvOVk5W;-sel+DLmbISiP)vcXpqwN{8$uMJHXuY= z#TfbKW`212f*a?EEDgElhCqiw)5A|Ucq0CI{u%ixdNvOK`0`jwBv#6kVRzNY++7H> zQi=?~@OU?#zvW|Y20g$-RB19CDUZ*M5Sl}_{9dr5Jc{_^{LkPwxafGO>{R9}XMKHI zUHDh6cMZ$dzcP%5W@m8?@eEe{=2BDA*ieY`$ z6_o|BvxG?#nQhE9*H?u(CS!ypKHk2?hv&_CDnbV_0BLgm;LlL0yy9gJ+kL6fvei{g zw-krL@g?eakqiL_jIf+s^UOMNOiaz=ZvnS|@YW~Z^06r67pHONz8i*@apf)us)zAd zKJ=@t6X2m=Y1J6_lhGcLrREwXCeofmhG{GHZ5K2~VIux)`**w{ANXYO+rl)KFOI|3 z7_2U=IWw)O+4Luh9Fw%H z_<`seD#tgcXr`?S%BI?56CBBn=FPO840ZH{Kd1%DCMKpPn_Lk!+2j#~Q5-cG8d;IE z7rLR8#vRdk8n7G6_)WxAmQhLPnuui#7jc@4cbWEPs(+;yg|ar!ck^5Mtq$nwgR-ih z1;_LAEWh^Ps0-hdI|7UdPfr+V5n2Kl>>8mY;1)xM2D(CBd>|32O|hRpIiga}vMKunj9mfQ7j!BIpFGlUMz`euiRaU_n8 z_p9rI(to3cK#YPTj1v!ib-grd$hBz=jr|ykSg^htDGpr*c^Y6eGC{2Tl(AeaMva_* zRn9!|7c$X*1qR9Ir3kn5m1A*AiYswz-d0Ig)v1B-k zT!zRGr4xkE4IU%j2P)s4p9=zxAtcKcfUN(Z?O(BJodKO9K!^sC2>Ojl%#QRS%;y*Q zEj?>N{f-`DNPmbpf=}~d9tgHSpCF@;NJkXBIc)q!aF1&Ve0L$lL;tn#rN=2zK;~Tj zMkw~VgpS2o{;R=X3%|o;c`{eaHw+7N%k;VY9V`{NEGxqbps21WPr_DIm0a!`m-823 zU#c6&Nzs2`ijGTBmS6~dR^C22raO#^Fel(#p$bRn@}cl7fQ-=uzzl4rzVaG(sKOXI zjsiv>&M$^B{tn}K;5clua(5MP!wwdX0ZhA-6BE3wf_&r-OLsX@0TfJYK0b;L7nzxo z^l|Q>g^Q3u7&b?iYif2GW;jwXX|O1J_t(PkIW8kW8vZlBu9jzdoFR8Ef0>7c#?xCI zT|TS9UkiVj%6yjZS~)s6`*r!>?Csx531Ql!WNE|OMYN71I7GP(WsCjC(S`^P%v7Z=;0 zz=Dxjl|}yGkD^Fd!w}>@V%>7IfiPhuAp`V9Mch06^9*1kSX^{;HQvMCca z#52_UZ=m*XbzrWiP}?5cpSA&W8L6~;m+vei;*(IYY^d7+`zdE+0XuaHtn7%haf#pt zuP)zL#m7X9^iS09;g`|BkHMbR#KeTdVX#UxUr2Q51b+P*Gom~|lt3oKhkq`PbkP|5 zv3F0%BM&|(JKNhuC4}2;K#z?x(7dX>MRB*ly9`02qGZ2@f4XClcolcVg?BlHk2V;5 zr=2O450=$<0n4(rN1}j>9i~J}&BYl{qwKBx;DU`wTB@S<43W#Xzf*N}NB{}iEeV)Jfk+I5%IV>eWyvV>`Saq9-|`hT10a#0|Xk4 z42t@ZUR_A7o_k z2%;j(zq)eaC8!+pdjM0e@znc2W7U%U+y4ji zS16G9%)vu4>Ene-`RLWQ+Nn z>)KN|zh$VkG4j#;p1rP%LAYTdOw|7~!~6D6ylXwzR7|liiB95dG7$|UV$(*datiOR z5wNNH;#jBTT`d@iTZ1a5vceN2j^A~1fbjSfp0s*=E^P$AZ2` zzK*ro0bg2PX)la($B>xkGQOmzu~Uk+rS!_$;)E38Bbqn=72^y&TRE?*o`=mE-q6Gk z0ESsxr_0A=A>03XW@qJJ+r`Jx|8aLj2@E-x(g<#`lg%F747qD3Zui29opbF?Lq*$lvwKrtM}N!}fa0W*v57B_Q`J_{ehQ!|o=0m~h3E zvZ=C+xM5r*z#A^hOR*Y@TlzkzZ1Mwa0mlF-oAbb1;$<`GWiye_U6zcSL%s_to0M36 zR8A?Fu03j~g%Kb-{{P(OKb$-8F;`zSKm)Zdw4P8W`VgLepH8b3LEJ&qjE|1@pqT0z z8j?2HICAjLK_?tOQNJOuwxNK`D-9?kEieD*#qKOBrgRa)=pkezDL)8po*2*^g|)1F z@vN+ux-Q8(Cr`-3_uePFS~g13;z=juEWpKZ=4bKdhpRTDNRPrG!15(b@Z_jcYI1T) zcL;~XrvU}fLD|-bMy4_c0mcwVn@3=H%ymf8f1`2cHRx5c?sJq zU>#YC-{wyl^DE#Zz1D}A-`ZGw{>2#@dFEV2Z5(tj@n6!L#&e{%FvS$Zfw+|q0#DkB zf@$dBkLlf;QA!xc#Op0T%~;Ih)WFrNlbFY@=&gLYFekx_cRDtqUhk@v%U5tw%0Nn9 z!A-qX-m~TJvqdx2~N?>toQMPX@lXmQuDy_(umbPNK_pU}ceP%+Qey%@l z*A&?@mx4kFWBX%Ibjcl1WF@h)sRFyMsQ9MQaw;mTs}jp$cZTYA`jZrJvpS)D=1 zH5F5=bpEjeFqbLKPgfs!O!3Wk^Q)@?0ATC$g2>jWUf&=c{;;EA66xz-B#2%v2JD|f&M zFeB0)Z?^WINZ=y?w^uc9?7)$rHhKGs>N|Y9!qj;!~lad zW~iJ-9cf!?9;=dXzdH>7l3s&b>hTAChtJ2d(|o;vt8FQ+ULvx$cUIndYZy0z6yfHs z609`gs(JL1N!Y8(&!>GH+^&PI`YEhDmzDXcjVEB#SX6b z&yVYFt&LbsO{yXc@S&br(MM%dMF~GVKCl3r7nIF$90#CmQrXqkUa7K4WjJNi-2e)O zF=TTd_IEffb42;N0)avOdB08Mv`f-9>Po43Gr4;W>N-e|@EfK6Nutel)YX6+cmCse z4*Imgkl5H`1A@7Xj`}?~H|bRtB{8nl%E{BGr0t%2(4NqsSZS!cpz=I?;1HL>5Xe7K z{Hvu0t%HL@(%svG@g=1@2Q6tlzl?HeyJDV1%*Xs;{y9(_zx)0(L>CMB}yGNTVOw%7e`EJEWR2EO)fb=10C6{PoG!bVw`z>>)0tp9S%c;peNydDlm$ zIZWnZR`9_;*?UH&U{8u|ulc||vhj{zmR#%>u}o_j}s+=<15qRQaYQt}S!x*QBIcoWQj^=kRGg!U+Tnyq|w)ZM7Oqk$h z9}RkJba^EY@>z_FQK~S8F2Q!)jGV$QrwN=6KzD{TJfbJ^^z;2XKUjn<$XP64mE*)g zOM9s_U>?Dn@Hoyt|KmZ;BPcvP9OJno`iUpI0h1@&FsG=%0u+6%aF?1FR zY#Lz)CqOIv4p-el$)acje_E3WLY)Vwl~?nN*wKozoj67NnI%$ERDoN2H(=omM`k_0 ztp1qWr?LJ+)>uiM>&j}a&GG9-D1S(Artg}HDXl$J>Ia?pXf~17#Pr%2e3(*^@*FCs`_U$Jyu^pBdU+TpzS4mtuTdnpg_(95@QDvnGIf!FI za~Ry7eWpt`VYP;XS^`UYCaV;-NE6W}<9=MkA_myE-Wi0k<{3HPIR?cP48P*0Ej=j! zz(stlav;_lI55CY;cs|!R*&#Ka4)WfM^~Wo9M4As*(iOMcuyfO4(MYzE8m0Frn5MU zf8=ljj*8G&a<1z;kTI_*=L!_*R$C)P66q_fzfsJQ!e!h4&E5WQ^!`6TKLLZ@wesM@ zkI36^{=M|}^vjQ5?v+h#rMQ`=Qa4=Co(vUKl+FEn>SP*9Bg*FT@ji}l>Jbr~RGMs_JwJ-0dHFEP%oS1<2TW9*{Tj+9kN%B}F399GZlOZ>7and7 zWs?|99vyoArpe|!Rk=fJU~pPxb1RM^@|Ho$CWV|IeLN16pI@l>-0eed%vW>~ z8CCqqG-EzNnbt^_A$`7Zqu0OnsW-*^hwX=)POErd=j@YN3Ffjs%ukn(#iI>Yz?l7x zj&^zDt+!+nyLHN7b4ue4P7|#1V-?M)weUw^Xpp>rbPVk*4|WvmwEb&3rZ;*#Jj>?n z4SL+5)8!FC7!Voe&)fGl)YnNVY*~ec2;urqJB)|pHZV?70(?NI zX`-1rKYy3zr*UiFpnjxl2IP^YWk4i97w^OLh2+A<&--sk!}ia)5X;6~50~ad;!+Ef zGCu~z8uG2&(2Sk8b!ZP6#if37( zJDALGiX+E0!;*>7#aM6ZsEtBrhG4wVGC7LO?_ARB;tiuK55xQuqc=YOTyOaclQ27h z6Agd!%~xRWX#jKLc_@VPW&iF{EDF!*xOT^p8qAq-X##dIb@$B5lh5|c@wZ1|I>R0B z6Kx5^Vo<}F!>^I*Hl;cy!_xt zy>bPY2K^8Ja*x!)fO;v8;bGd0erZJ>f1*duU6_(HXD47#Jt@EOYi&}ABZS=b%*ALn z{p)==zbCgBXj%k$zw_R-oOyp-o__L*Jo@FYOI!P9d~>)<0X{4BAFwaho^I6gU#qUK zt(fvDGX#SLN1RUB=(33*%mgn(aM6lvaz~U-d_$MW*w6w=)fVb-5?&!KOy3+ufoKEN zb}1;dCpg%#K>UngxIIz7g{|RwbU|{KMZ=UP;&!xV%dcDp7h@dD*8ff4{yCZL*t!L~ z6;e2+_p5*riyCQ9EWojVYQruYJJ({hj%EO$-*MtDNzDjPn=LQA6{Kg zIESM*n9DH|?U+FMz|nvseEHbXd#0;P4)59p!_)2q3lP`$cwCB; zDI)k40W$CE>ys<6%e0}sUO#qJ@#N_<7msUuS!&vMNJ(XbETcUJPlVpIu;2wABbH(2H#Zt) z{?+QA@c6TKPtfpTWSEFE4FWsYK}E9*`WD~EC^Ypd@IW<`?6XiN&Fay`dl;BCW7J zRf3(#J8;C03av}9RXKu%T<(bC!*pymJy+*2PZk-0F?1@Nm>2uY5!i;jf@6diE>0>9 zgTo6j#6Bx4%}J@P=9qvYJ4Oa2R?iwgY$wcl9=o-sF?YWVqwd@ZHafZjlLd{~1>OcF z9p?j_p9A(YQ2zm|l#zJz?hRl5fbh_BO~sT}zE1ogRNiG$PF^|qx=E}nKVgj@=BKNV zdCc{}X^z1n&~V!TpOGJ6qGGA>4D3RHb9LCO{eXa)|)k|{g!A2YhEWuT?37vFu(&law<1Q_3MV^Q2j5`u{R{pd7pS1(F zXY&W`&OGK?qmQ}~nfNgMUou%NTej_nY)4odUYNwKJe4Y+RTbD# zg2(JcQ8p<9CYvN-H4;r`^j041icCR1RW@U;HtyU!R;;+1%Vw z8OSCTV0u&$n&{U(vwyPlL2K8~4r4j{oRGuQ8wcApIZl z=wGzrU4@{c{2a<$t-G*)C74dKy-R-$Y^xnUcmO*De{e-81 z@ditmPb@o!7=gI+07i&UtAZ;4Qrux0OTP&DXlrlBhb4~u;A^rRJ1yJ|Z>!|D3cZFu z7xsu=dE*~s*RI`CSzN5aS`;G^87kd;kNB-jE)c&Fct7YX$$_Oyp0)7L~!W13xdnByqf&%W!$iYq$9HXO+X4;j4!pan>5pYdzZHsO` z?E7^-|6N(eB3llRZ|dWSH9eK(9nb?9;Q zpPBJV$4ZqDWr(heb$*j8zA=q+9Yk8DKD`SIV}jhkeAcy%2P9Jco>-TUqk-5n=?QT% zK2CoaYxEO_`0O!%K#|59hb+G`*qW@b-v9-75iSUsm;e3$_g&T$Qt>$7qWtCGT!xL<0(teP zL(+j`eGlGW4`owA_U^8f|NNJoP&7@slLSxzanY8G#OE(8Vo`cTji5hpe}Qyfo|1q2 z=V#RA1tzb*J_Ng#x#)jpk$00ka8IMO!DxC3jsY&i-Uk<^S@^L0#s}wOJxXoN!sMh} z?p?+a>y&)wJEt)(DV9SAkI3QM56Z6H+i<*}>J6)RbBQCp{WxA*SiCj>eUsGx&DZ{2 zeXjOqc^kZF^9FE_3&lOkIDe+l@DYi8c4SO$rINeUF-FYR-2jmef&+*{tK|8TV7Em z+ji{6%5FZk%|0cIFlzkn=?Uyy5Z#%aZwSD3B7~CYY zFe3?R+405o6{K+ytHtve-KKHweFBCPyRikiuBJ#@HkRPX6Rrp6wYT^XXD#vLmnie{ zm}VXW_nZ!SLAwV~w*7l%XXXFr-v8xerL-bZ4cXkIw`V^8+;cKGJPL!Qle(KIACuAf zMzm$L)6r4xGQ{x{_&Ld=8$M|QrZ-E3rVzahngz7oD%L4+#5Y1-a%=cKmlAW^3{D1I>TP`t$}D z@*Z)qp-@3B%mAawsS9YSu92dWB6;=gxAD8Q6>$uU6TCL{hW{ksbL1nG{d3#=Ubrn7ii{;dX3sSmkr&PgDnK07KRT?bg z$V(1(>RiMk(YcEk<@aIVE1^f+*-p~wjnj zgObcYman-Crhf&y+b;j)+t@924*lwi)b2bWCAC{67rR}r$uC3X-L>k=XpDg?RTk%7 z^w$wZx?Go?7Ufh5ZrmAMC{gw%&@4P&%2HhtoaZwRuVb3kQapVwv z8{~_4_%My}obTqR%O@VMb?w6y;3lbmqkFybtru^2%q5%D@7Sd*@vQUgVNE%{GJxaX z^|XIYyvEP^iYaE|a*NOs7ESRQ!RE4eVedm1k0YAm+wG(FKDwm zFs4^O(s*{G&+_ra!zWw+U=nz71GImxiX6Q4C_XYd0MuY$>69lP|1P%W4$BXJ)Fo{# zTtO5$itIKXvdN|!l^52NJ(Nu={^^3xp*YlkPcqt33bOYx3ALPs>Ah-7T#+b_ea~DpLp77Hkp0)geoKztdYe>(2HZOrq*Z#Y+}?Ha zXGq{0PR3iF^m>1^%z+=UR&O4YE0Cqvcpu~ayrLdjY~Xbnu`eSnEC&;>RsR7Yip~+M zb|tZxMS3(nj$bAlwqSR5sic-C<sq5;+us#44KT4`D4E z7tXOY3^BiR0f9o99Ug^`70R^{;&k~t{6-Ut;6iK!*Y?F6ePJ{fe!6_(t_!`^ucdYQ zyOJ0+VY)&rPL-9FNK{&t(%erjpLT$u_` zlxb^w3ECva>Fs4|*5b^$Mmc+K63VD4IrbKA@P%RXlQ5S4_K8vX(bHF@xiNvUIDws6 zHL_=Kz5Mt83T`< zd3QpddahT#^5r(&-F5dJHEMT?3aVVlEa&qk^UKiaazT0;N~x=Zi_+Ph!n~$ix~|OO z*kC(s;BJ@S|D#`*nz|~Tvm2jX4LPonTc6hYx0b+ihoyBTwlcdn7wf_h@(Hzn(z4dG zzG5mWq}m$;Rm_bu!gc&8wiisDV8b7sXj>QwBb)e`OZ=MF@`;PG;zh0=ztTg00PpYx zzXs>sxJHj-mX?7(d<>VRBVG$*yyfFvP0L`$;mMXC9VvK|t$){|t*w6=+CS?wS(xOZ z1FV#6!4aOMY{zQb*hnuFJH4>kw1^cG7$V1RjdJXmsKIS~i6o5^dqt56i4zr-8BU^U z7@78Hc$97w+mQQUPbMb^AI&AWNvxz4AJ1jFDnol5lemed0!9_K;}9XhPyV&tLCWy<4s>P*s44ZqmNg5CZQ0U$H#OD44xNZr$<5uMHP9< zGv>D31f57=2N!zG*_kOAmYl-m9ur4on3aFWW~OVPwy1^f@$C@Qag1SvWdxcA&j7N6`1llJ3FmMeFFp3nuqI$L9`al#wIIo zcKmm97yWiOez7C<{L)TUJ3}ww+Q|aYy9^~~eJql#-Tt)(5btY*xrL7*F*l3}e3^M6 z3V|*7FnZ{&JLUPGz9!%K;S=(?qj$)yyLU+*KD=}E%1MWUwDB>QY1MDUW7WSGH(dVc z`RAo?WJn&q?|#|8b*tp5l8d9BPw&t&^}e-67Jp;V>eme|8&${yNI!A`3e-+Uc6WxXVK-F{Ryx3tQc)9?ACHK5dy8n{H2 zhcwl7^|E~Fk|e>Kdy(!(*j>4Re&OyTw@WF?4?&LQ8ottIOW!m;cE9^~_z>JG_uPKF zlxRj)AP?*FHnx&4^vC(2bxONX?!{pnwBnJid5R-Mn3z4%{o3UwjO^ z-R7Vm`VTUA`90Zq;9)7;ICLL$6Ea_PeDDv?WR^V1EqdKwYP%#$>GLtc;v|R<6ABvMgf+9;{nQ8|Bp3Pkej9bEAQ9%Szj@=O2uhU66tNU0Yn-C<1H-6S9_n}E#1So z2)@8?9Y!qPx(E~OctynN^ZiQDLnv)*ni2dN^hPN(ICEq4SlVmi%a%XN(R6=|pV|6H z*J(T+AV}&7D|0X2dIQ`GI}@ z!i2o^ayJIb9QkklV81lOwp0l|))}uKt{lbuQ;0#E8>A*M;T^`u%ZmTt$;!WLui5_3 zHx|gLwFzrGD#O7`%oCIEpSAzvEt=bR?Ssv!U2^B$_rV6-B^X2Pl=J7_m#e+q^47`k zVDdJvvRMyhEM;?vZ8;)4q>!@7T~xy`TuNo`6qL8`of(xEUh0Kx?tyH!V^Z0GrC;w}h!ZJKtY2A0@!UyLOr#|F0>p zYykGyyvN*Fd$qQHO?*}thI1{~*OAuQDli$*Xjt0MW8tA0yY{~F`OnKs$KI45KK+!u zcKjH|iG#9d^A;&f;>JdN__IwD-pEluKCG5;v}JH&LVo=Gt8)CU<5;9+ME#~zb`kx{wl z@L}1~iXFM!@xuiWJ^;+cj++$TyDM1#O?ws6n*Ilx^qg{%i)Xsr| zSU>Qn1Vg@)u!VH&)CqYSimW}@VfE#Q9+X1xQw#6QAMhelZsi&H!gBI2{-lLjyty9O zc+bmKm`9Mt%EtWp$NFJk#=Z;3@R$^Xmdg5ODXOlAatyb!L1{KKd{riCqjPK& z`nepbY}hF65B*c@u(}IMDa_4?+}J=j^Hs}D*|GA+;nQ#+y}_BQNFzUKxESvFO0Op& zjuPP!u54_A*$db? ztHLq9L}EpE8S!*LLwzCaPw}E5k;70ZaY3rPcLsCk1vz_u61%4Q)IUYvFJ4> zHMLck2S7pw-PskzqW*xTo^?dXFm>Vh^uZ1k=IJt8 z^;f}(!O#I0y+PF}dg6D65!d0qdwNt(zB{64=Zi5|_G4!Tm1BMV=oT@T)&4v^lW>$Dl}*SEWs^1~x?qFt(&bs)=(i{jJp$R>x(}1gB9{hoP=g!NI`C0DKK>3A zVKXY5rI1a20IO_b)LmBFaJU!&w%sT*_z&Ffs34E;O+qn!`SPT^b8-a72S#+Ymb;-i zp{D%O5V}qUv@ovC&Ei;LC(`r}q~y?%D%_sdfXQ$;wIL|`0lIuB*Yv7l7r2Q(iR;Hpq3q%9SHYwC;?27S*y&V+iwLM3R$5Uo*VcRr z!bJjt@*uBv+-&!IzwuAxhc7%Q-+BBoD1NIYiFRCGS|Y_*2%3gsh)X+tFu*=FIR#nk zkU##N-`)>K}W8as*xOfrcZ?$~>{s*KDFWj$i{g$Ey$!~6D10r{haf#b^` zl!s;H1D7|3awjJ!48j60;yoD}7h{jZ3m6`mr|AGann$!Azon(iCl|-|Dry^L0Xvme zs8EX%WWWTB`lIg_t}fy5hXu9jmUhX-Q7J8h&x^?zpPz$Wo#f&21{SWi zeJA3eFlAzv)#s_@M!72Mn`8otrSe+bqK6$_IZ&3hKlBGu(6kGN&>xq%!5$gwy(Cx9 zo>F{7k%zvYG*mTil_r9<&)WDFbAyOvDXQdt<^ zt-KBo1K_iE#aHjQ@>m+W`m6(Qw7cAyUs(Qp*p1qi5+@e?8i#;%EUYH{tP;PE$ zScLK?UoLbmLV=WrMX5%_C*@mz{XN;XeKQnNHMp#(3^2v=D_?ySwxx#T``?8^6pKYu zll?e(;1&h)637P^C*1qsow9q+4%o%qg2nA4P=KXi3gC<``kp_3PWs-$t)X)h+UIbL z<*k7$%iOV8f}@H%<>*}y0 zkCoDuDtt@lPzX<-hX{#5Q|1(t^;$VyCR}E+EaNL0>t52-Uw&R zYgm}xCU=TIE|@BKMB!2>I_9Ck>;D+-j!p-kmNu4O27J6co{nt!eSKu>AC2m>+5TDe zoHW*Aa$AYXe+tGkxlNY~y){rw(f$(+EpmPzeVp-e*p$L?ojKg{wTxA18d~hUGzq&^ zJjz#y-B^cU&+4##OquHP2A0UBNZDefYTLY7#b3>8c_JdVVlZ zj4xuPW)@1VJT>;**wm~$wZeM;Y}P;P)b*>`@(1Hfe7xXb!>-Z~I_7n~r{mwpG5(RB z0_=RK#nGMG+6_?bEJES5h*i!!OhCCRSs~T1{}z``9-C2nZxPw#?R}SVyH4lj6pW3o zU`G~Y^R~nKp=0&mfC=KxUEAg57vF{~#?R z!tq~x}$0`?NY?gT>>k!o6h2e=-Se)U`;h&nUml@8ex#ChH>b@Vj=vVDC40 zGMk{D=`I!CBM+mD1alc3OR6v$?r{+>;XNrX0mhOVd@%i^yYG^FZ@pD6_V&sd*oqn) zhAlztidy1PHKZYVR5H=_*{x7y?bxZuF@Lhh_mne?cs;MjqkSDUwNlR_r+yJhK( z(t7|3o4p9b{w!p{NJkQB)(1gBq3HFrs|)z*!i4Fh1t^U0m_< z8!s#$-d_9p2Of|GD7O;aF@4SQyE-u(B+s))wzaq6<9EB{VdoD9otSwn|x*m3{{%WC7Z zCAX*mwx&|B16CqSQ@E{nuN&ncTHBGhK_D_co}4X&g%I zxw!>25&s7x<`+AsFbBwmSqii@%n>$hfQ#~!!rtT}CIr0vhe!Mpu;<9r1WD|y;s!z< ztt&zsEkr*>TUIpXaqjFnxdNqq_Z930T-u=XiMp)HC)7=!#p7m^Yk(( zz8afapwQZezYfd~s?f%9Q!sXLxq8Pu0sq+dSZjdh7Jl>)a9GzYi}eZd`bSbtuQoj3 zyP4{rtt;N1ZuIs~n&Z#9iYZNCLS!y$ihivzLa@^Lc-P|AfL8%U+CWgvZ}@;(anr@) zIvm&NG$0PIf!F$p(Z^TKFvlQL002M$Nkl5VU%V48D>bEe`X=xNi48O�iP|u zYdhKc_YFMWp0#bVjcMb$Y1%(0{O_H5TY7uDB!Qzs6%|}np^d3L{h-8{>ADF0O6A<% zeE`!?CzV;;m2~#~F??*6Nqa}VG+^+l#%e1kRpbTxf&GULV<6ioKY0AVNn@Br*pQ+K(nb`oou9-u;3uE!!4bYWdE?lWv~1cbU;FxY znH(R4QfoopJ2!)Y5rZSjp@SEKd%+Y3a0!rB|#srkgo;2A+Wd^c|9fVwoq+DD$ zN1GC5xFxd*ve^nH5O!igHf$cSsjUN(!TaRZS6{~DazbUZy{!yaLT_~VAfFSEP0HyE zE;djl6GWTWQd2%Dn_aLS@%B5@z`sLfa~elv7O{)q{3YDVhY9ln6wGv2a3mlH6J7GM z5I1*nN5-%G>K9cuo0?lOIfiToPfR}f%Br#8czb0!hS_6&b7S=xBFeUZWm%k`QGVH2 z7ck$LOy1d?&|C(2KXI(~SUp;G5sZGWQsAk;@wNkPo8&-88;+wTWEr$A;3J72I(blD zCa?=;g&*gOeiso{s7R{lJZ`Oage4#Yi5$igz-7qKUqU+I9&z| zz+-8=cV!_Z6_ClJxGnGK&K(ZdYUKw#8Te4HV#ri61sXSHhVdlCldeSR8{+X9gETyr z!+d!+-+|5T`bUUN`avw@Z>{n(Nu)U34k@{?UsdR`kI+L%|3nhFIbmk~=K~LM1&{HA zX+4kWrA?-S9Lm0zf3QL0>Cy{WYo2^3n>5VFGc4`#xCOg1R|GfbS!3{}wE9PN>CUQR z9Q7;Se5=%LJq!|XdnoWO@+%kyb@JH82nRpsB#tmR2+N})jk`Pn0&96tG*PjJ2gKQU z;Q|l@h2skJW64|?t)5I*l;z=LJP*e-3t$4FL&M}Q zHiR?yXxgaX;l^AYz9+G3=>g1Tu{g90Bj(4>%WJQl#k{vu7lN2>c5WF8=oF6W zIa3yeP^5B))b1U{vI#d`w=|bw(Y6fZd_p#Nl;|znm$7i%-7|sRrE~JgFWe7BeVaW0 z?8|zb??3+E|DWu=WtSW{v=0}sY=wfXQvSui{9TCcERN}4mFbCIEKH^FO;7`c78bLy zL$sg(WBGhPj_+fk3T^k6J;kznXRF+K=SK9?v|}|Z^Qn2Af3SS)vrEvgC7>9iax5=z zR2FB)p!|i>I5!D*1KRln{&<1Pq;iL2fhuPq?0Uo>p*&vf+LN{5u->>Tos)x|-ibPO zbMX;ibxyvS-_>eZ7ayd35|2HZ_50!L-*8%+;Jf+FwYE;z;w{XDglYJ0Z9Rk!!!#a0 z$`iieF~9N0@-^2%UHf-^#S}Y^*rORE++rMKT?NHJ*Lby*<`OLGk72Z)(`Zpn0p4@U zwB3OQl-L04`Hr_3z8LEaHZ8qR1t;SnrCdQXisRTb5@ZBeN^=Pi^~W$~%kSzsTmP;l z-LUPS+ih_z?DPyKM^F-Bcf|BGu5X`RKA0KJ2*5k1~VQX zD8v@{`o>nQRBu7LCKz;vQBe@>DGOX=cO8$(y17ge^=IYZ@q>IIYrq?8^L#gdw*MzL zz#fcs;6>W_#W0QKk84k}xHw=IcA#>xy2uF{Ws`Q(X!Bt`*<^i;@fe+yJK6L`)-4OR z-tux>M9|cNmC`!g2Gk>zO^7#+0=bBkYPsi{f5Oo0l8B^+B}D(+~3VB!N0lW882!Oc=TckP92Hi8b=yz*^BGw|hi zd|(KvH5v2#V<_8MBqrpa1_wWy8n6yJ#IzQCkTCFAN^=Pf+vwGG1@} zJ4!7t%j1(QKWT|PWC6dL&Bd#F@k^Iatmq7JX|z5G;Ziu4@BU$Z;X{Y$VI$u?Se;+N zPC!(=L(EX~)8%7q@@X8e3FhK88h%5Wk$@QnG_H#tpn%sT4!=wN5tLsGc{d zgsyu6j|Bu#K@1YuaX1Bvbqb!_flM-;g%RHHm>UhF-(ZbL@$_+c?F&GI^$F~xEbMyl zY59?vx;}x0Me_>iy+-*t?(ufMW$a+8!7gd+M~2NNn5e++I2z|qrEshcJMl(f({&y@ zr^aD7lDF;J!y|w=y^A?}4t6MWkr&5Uaom3xb7$Irq+Qm2>}HzBu0_&Q2;i+-+kkIe zcQBWh7Gr#0)cm;cR9#aA#eX4eW`t9lIF2Mr+*#N!^!06O>uZr@XjGip#2{v9(IZ zNBWTe1ZeGpJ-0<&gs84c;<#TC=KkctGRilPZx5E*YT0psit17vXDlkMl-Bk#x5)~M zDQgoBDr`f1x4OO&>Ywx&tyZ#l`?Yptu9ks6{#h?Q!P3&Qv09sFr1|0MD}D$cr$-~# z_*q{u<&j6zTL$J*qM}-TRuJQ?qHNNfHdzc3lW4+ocyPrvr4LDwdZf>^x+4|46f5;CMHJdhdV4%r|4Vi?z`0^l* zPo(6Lhg)#PY_YVqz!+;G2cCQkY`7teR~jP?H#cy{6d&Fo!W~LQ7-%%i)%76wTQtu7dLtEb}-_`poy*{A7L{XSoz`mci%sUV}dg>InyK? zo43gS^1uC_Y-nu4jegBoRl)}f@}`kfuB3B;lLkxKD{0S!cOD1KL%9kt=ojH=16OW> z2bpE-A50?C@nrjdwi?F=^5aGs|C*aypqQG%miZz%cVSZg&wuld?7yW(4(_kTjd#2O zvq1lNJdVm_$|g>ss2s8kb5Qo?KnBWS^Ck}yOKtNkpl|<$P*%|x`j@`+W$et_qO#f1 zp2W>Q<**Nfi83aKJWgP;NyXSSevD!tc`x9l$#MS zc?JzsXmf04OCufxfleL?#9s1EkA`i7bq--k%-+L;e^~lwYwQ z&sF2D^76X-Kk{Bv8kQJxcD}4TbqVSG>Jk*Mn$nZ1eGLM_xhDK*znYE0r{i;d6R`QV zK)hQ0M68b@gHLlIV&@Ow@oF?Ed;qC$4u6CmEdl!jhcBZ%4!g$$=jTxVfMOO6<@&6Z zU%w85`plO9ldOL)U9t8bzS8F<*h>%tyli?`allWPPuz`$(I3aB%~N4xBz8rY*LbQ^RQ!u1!KViSl)=d^x`w} z{`(iPyJ{Bm^?bSeuFcZeP==#;#qyo+o`5NhajZ4vN_$(4eC<~bN_Y1N<`GwM8B)LQ zv@XXbO06wrh}$ARez{j(c=;+6PuRr@(*(EdZIF&F+vL}Nt6k2Went9vy5#S^_Y*95 z@isz{yYIlweRnj;D?hyo#oDY~yfmSU*Q$9^F7sy@JGk%-(^`RJ!#m`+ey<63<*HHc zqV)O#q}7Qgjn#+7eS-CWv$TKZ-5NjZE2dZ_CPh}_I5Jj@NjKtHF?`HrTy)|R)T#$? z4hL`luu>!V0^GjmP;%fma&(}wlLUR0PZS${OSrN@bQTGFz{XM9EY)V9OauzO8e%$($>Bef4F6*5<9c7%R@gV z!1Lf^1!>t-*{$hgZI!M))|UBBz(@j1W-jBh^6z=1{KAj%lUz3tj*q2u zLRo;bN)?r*s;tsB&nolnpF0Z^*rMOk+J;FVRznMGa9jY#dm)=0?d4D!Wn$a$s@TDS5ZKOQ{BYS*tajnCWKLD-iA z6VmdiTKGi$VJLkry>r8u5I&y268ghAs+kVb|+gT65#+ zEHBHS@dX%*YjJ^rEKTDQ64;5u(Y{J3m-g&w#xeRbT&N(JYtQ58TqVX}>|}yr@ow00 zc@wD+A?-a@zg#qHJY2F362#x`Oay;&A~oZH7;tX)CH@Ok|KA^ z4s&!~ibO>dFMM3U5x(BuQCx~LtO~JYGGDfDtHI?8IFg!NkmnH#vdWUgSFzim@ylhDc?O@fSVrF1Nb_ACw4aj@% z4*&;_y@GMOch&%|5H@6|U@LPJySc}8$5?u+S%H!Za|sY!RxyI_80^N<1!`qOTAfGh zBF=xtS(r~uR}2vEw(R8G>Dwg(bl`GaV9=m zyV2RDBm8>tgt3ua!9f7tZ8E~>sGsj?PYi#?;?}ez`WR5bMPgD1zIR9MLu(_4`~yCx-vyYX3AiynD|9+zPT214W6v z`T9@gIIiygtG_vc!Esb7%J}h{ha+*tQeRi3#?fnQVTS>$6f>~l&~tT4PMjE)JMJo$ z?K`&XO+-PtBjrzE(7XMPdvG*wNM3pQ$I{*PmVEQCPT|A%7TLC~N_S_mdGhwXH;?zr z)6aIx#B`%nRBe(!{n!7qUT$a?>eSs<=-jg0|5tHtka8lgJ^JDqsjGE zo>=?SI<+u@o1ZQp*0qJXkej0YE8RwK)U~p#RlfD&iHkh(&xe)&tx!m{wr|n)-*vfD z{^9ZO%Zo2vmcRPzvr<|-t0rlhv8$1WPg|NxU@$!iC2%g3Sngtiw_%k0&~5u+KVl~g zr{XrgRqDS0lhFeQ598LHHpu1+Fo1pqviTEv=z&d8948bncQ}p1-rJjR4al?4ci|f7 zW>sSScYpF{syO2jUMjhxWsdqkZ27fKDJ$LFfwIrXHC@*aXaGX8?Vr2}D%iMW>_Oc` zqw(UH=;GtQ(e|x=0vK)s=6lF@$NvbJ2iI)jr^|g3TqI9bW>%_*lzmZFXzGTf6(X%O5RklovrUJ$;|n@<+=Y^=Hek zEh7-mkG=kFyO1(IlrmCbBI4vbkT_3Z=s3w=JDzw92UD5f(ybRC=Hz_W18=yQA_E?g zY;(boZhiu;1|JDifY^`aWiF$0Fc+tN*4gQV^k44AF}MOLEiZuW)+T6~%4ORYD4uY{ zzHeY&-aR!gBO@s}c6}s zz>^mhSUBG}3VU*>AYUa4XFm*MN{cgpl>bqG20S7^?)o=c>1N~~3A>(n!bG3-@DcWU z${)oU_phs%iVm#y@f!_OgvMp7T}qsV5x^wS+-Mm5>lQQvfcPVLi7)=FmY0Pgtxigb zL!a~v;^kwxEsWsiR~!tmvS!OqZn*k3p4fZTjrr5LkK~BAFoFgzR$r_~1t8=akJUA! z&2KK!YTX&WSO)R>&DQ@1YyZ5eoyNg)a5lT8wL|tFyiM)I%uNr#?i5yI=Ekv8YDxxh zZoRjkD;i5`cV=<1Tz2g_4BIjXpqSc?Rh}}}!D(Hi9<9Duk64M}v7LMFe^`#ao-c17 z$F}4*$FS-$DXnd#dYpp;;rnN2WNM}q1{sgy29O;v_FRVJNw_nKI1FF5{wX8@gR|}5 zeSB&gAy+;{{XdVqHEwLhQCVDjk8^swd24W>AGRC@u`PZME0rTS8Vh@C!&8t=*hrzh zHQbD|j2o%8Y&$5s_TCCxPWv!Pq+%D$;bVQhAOC3ogK~Dy{f|Itd>$*a-MC#$zbunABE65ytz>oM;;Qn0UJWT>FFoLdi(`35Y|e z`OQVzto(y_pFsadzO&ZcZ&+g*j=1$?E!NbuYAsB$X}f^Wmo4_H5vKehJoB|Y`EI!A zy2n0FTw7xkQbXbHbzOLehj|!2xDmh0-?v9b0nlh-3iH$DLp+WKeGU9R+TkXs&x20n zRU^XenHND_p2Qf(&(sdyS&zTj@_)SbuSc`lM$w>L8xLEj zwK29!-`wcMWA$gDnXTNxxnF}a*Dx3ibz?-Su110u69ffH$8UJdWn#xA3t0~yDGXHe z(@d0xRbDkR3hn?c4@R4xu6roc1vt{TR9GY>Sdbl`?32;qerd%Gc!dd^M#vw={CQrQ z8?Xbqyhi46gzxBKxUd1)H#7%h(~ktK z0v;vgLw{jm4$Np2Lcz8H3)^+Fv7rjPu5lD`Zc#?Cql(7|Q~1Uh8^evj!?=AF8Z@v# zni^rVs=f+G2AlMTZyM>ahj`1%aEhapN;KNn$_IQY7Ui%wra*vIaQ*}#7UR*}c=hiI zOQSO^b$eaHhzJvmyz^jk$| z%fA}G9nZ*dI{G~rh+_T|?#KE) zbCqYlPo!V_I`3+n)=#$oyAk_;w*BVjP11y0^>*()fa{{UsyHrJySt$z8j({c-o_+! z917lxxLNNiw)Vr|IqI_*d+v|iej9Gxxedyx{a6uovbkFQ^DJO-ak1R{z{7I>Y%6w6 zz9oHwopR>=OVV+P#@9txPUq%Ju*JVdZaZ?f?AW~*M+B=Nn*~If))%fO{|e;9*U55Z z{IIz|cKmk?(!MUrA5`ckHUH5%0WH?|YCYlyLHeh3!B3YD!!1k$m@={>$^79_JtA^F zx`VfV-r;k+h_o-_XI}<4c%pudh~P8a>iQHBctxc+U0l2^vNSF;M-R)#$6PDx$60=s z*Zpzp11Q37h|f~_wN2UhrFE4p|BX=pChJZjuJ!AM#uM%8ll_8A6PG~+GqEFx`RVeB zyG)>AmJfj-6fZ0KSB9%Qd82LuJA~?B^Odj-b%naKY2!o*b|Wpo^u+>>(xvqB5+3`b(eP>P zV(PnEAZ19?*FU6}74>xYp+6BUCU3#mPMfg_EN1cYA#Sk>ANGYdCM$bLK8dI80AmJ& z{9ou}JlCRW?+W-lP0yro-@pVeb(w*U%#@oKAWc~*?P4Z% zF^=X2>|vHHe>IGvH$xd!0J~R(P)u!X!o^6nRVWK?EOyst1{7&p5_l8TzjblS&@_5| zaTvxY?lOUfv7q)u{2x*I4fAU5x{4_sSV1Kx62V~U{S4#d27AN@oM5@Ew4pyeEJBVJ z6!5<6me;lLM^Yo5&)YX{rNhDi7@r)hPH%`Y13p`RR86-2t^FD=uGKybmaWSC=DM)3 zy=lAG(aD8f5B^B%8@~M~aWtn8D>q48L%X=RAiMVM#{e>i6_|OM!?ndrP$E$#^Ks>C zHO{J+msjDo4-7ikc@>q#X#Zm6C+<7$z8||ecHrpXi`e~f6}E5&b%5Nmvs{`tc3`XV z4%xnQ5A07>>-OTT{HH7O$;+^}UZuax_J5z+{*OD9Dl4j?cq)-r$R_1z5!OPfu$qI6 z>V*K<&(ED!HMP~MJSr+GRofAiO>H}nU6WG{@IQp>*t}i-=wJMaTsU`H&Yyc1is^pH z`m(fbt%huFk*4N$RghIeDMjPu>(;!7Pm6Y_cxQ5+lHrO&WAtRVM z^YbLf;jOIZGMA|TqbUETwsxs)X@MO_+$>A%pRw{=ow*v!mj9Eif7XG?4*k)#&iCti zEWNoDE*|d+gv|W5QbeR){#L^umq}|c0e(ws?mGD5c^DpZGw4+=5c&0S!|<3J#g|`@ z2fL9KFjihKvxD>UpZ?-8oK~2Y+YYQiA+sWTwl8A*PhjlLQTwgjpj26fTkgv9vAA1= z8~0!s9f-+K;$KEzF+R3{g=HRvTab58PXlfNx9?TUum6)@#v(A4T?Lr$!;UPJLOjk# z8&j*5CEpuXFTm*s@_fCAJDL(Wl31AJWh#}jaU;GHP7L7tf%a=txWRV}cC60G$j~r$ zhfRSl*rWySrp8*S#ty6E3fxkN8@)OI;6t0V9qlFZOJ6>UzdK;l7B>jyK#>R96NTlH z2OF6=c`$;7$J&IxSiPIez^Gq=*8Z`2GB;XZE|s<2>tG7A_IaC|f$s)w|7&vyJL|3$ z8N{*R#w&PjQeiG@O(zQ0y2znOP1>Z@Cb9uu?@{!?@A(_x3ML=x;a8f%rXM3W#A_K_ zURLk%1O{hrEZ)j(Znpf{`p>q12+11q&lME%rx54P`P7(P!R2jRy0unUZLuPPfk%5< z-+vfi&DEs7f1&sm0KTE25d&3?OifPW7LZ=Kbg5GZ#D@Ahy{TjKww>6nT!k~{F8`nh z%gVp2iCF*U0wce%cU0lDT!8fmk-G{4as5FQP<$;)qTg7xxQtp6LY z{HzH!Fv>su{;@V^;t`EAm{na%XfA=G{#6qNfkuP5jEMRJK=08$3ACBq6V}4wu&cit zZQi9rWr79&4CQBCbCVEvIG5r^>w>~6>{8Cbg+x7=6XRy=X&mQkE>9<4t6-J0-I#PPvzC}B5v7kErFe=MY+;D?{+C+k(oB85?Jiw&Ze*cnWbhw z$6I-6Cu$VCueig?9u}8>(MMT$xY~7p0=n8a5{thhK>w3dN0Bl`j|{%$JA_U zUXhUY^0FlCXBMGsNoDDZ8p(3lNeBNp$0>&m%tRq>_Aaf%T&EP}E(E+&kuZ<79;<7* zdNkEz>Khu+YP9#z)?>bG`**_hLFAtwu)4XH;~1YbS@#@G3v*x& z&iVB=u-v5FdiV}p4GTpV?}tTFUK^jlE-YSc&uz=mGOr;*In>YM@QSJy#AW!ci2o2~zkrv8neA7%gd|FicU zfR+_S+BI*&n>RVb33(xh)v2zouGr_?o_o)QOnYgT z+QyxZ<=6Y}pdg=nU%6@D(LnnyS7gfZ0 z-QS$sbwt1GMt_e;`^`vLTkId|_%ol7g}2^uhn#c%`LgK1RkH8AX`YxKrYcTafQ>nt z%B04+m8Yx_uhDweaIGLjQ5ha;EEX7tx6^`rgy7Hd+W6ksKUaS_Z{{q0#6|b>pPL>kpvlFZ6bG>2sHKV+x;q+7dS2 z$LPY*p*vHvW=)V|o;hFU?=V?D_oX}K)<1Q~x#wJ@&z5?E?04Wk`Y9}ZWQKlPtFtrQ z*wYt5_f7h|s>S+id-&zO`|j_QmHO*>d4vBdChK_oBKKJRYT}rVF12NreiLH8Zn)iM z!*I*LR)5Lw?j?8WbEh7W70dOLT)OGrsrMKQ7vhuNcz)${dr#02Y}u$fJoeZ+`NQuQ zE8k{0=+OQ3hnaSgJ?9*18&pXjXaWPM5pg$)aVH#(@lUpF4K~R3tBSq&#FRQ89%uH$ zE{h9jZX_>Kgy?WQFw`p3ZXA^sH;w`xFXk>b9eef*6$%jQy1r2#Ga>0442b+@PCWDU zBTdQ0{t|yV;;3Wg2>r@fY~~X`W1bO~{O^7J+l}ou*8dp&ZxQ^Db(cO2-CEOpjgeD? z)eTrbEA?MrZU=k)fB7q3qfvntq16?thEJKQgtGqQTE`l6_qb2!(5kfnvWq? zPkMt1h-L#Yho5rYBSeSe&VY@l&WT;pDdA)B=npjy3B=3wKls<(nvNhdNT3oA{;v}} z){KGkI}L-;ssgHKDgIHLrt4|%<6oD^`@Sa}@*LS`^AeFI*SK1Ggr3~&Fik@qbK?iH za4YIE)slca1YMia9F2H0T8+g|dv-SGA+%2%=?T9c;=1dnkjTJ!387ww2NfbH_Lul? zulc|4-h0XGXPzQ=tUNo`LvIjXmIyH>$Llgsf9Pi&5^p_*1xpBtIbNH=1cgbgi+J-f zeb=WtAN3|gZ;q22*Wn!(%$3fr33AG*FVe5RJtV)o{x@>R;+3-Go;7m+1M6%fY3`f} z`Y9tkm&rGluo2av_t;n&@iTq+a>GXbMZ3H1S|txYv|iTf!^g*TELT}q>!+pG={FcA z$c$ME!L?JRlvv`iEy?;g^2zyMLWLst<49tTfm>XYMThw3z-N zl5Th4D~YfLqBhx>Zru=lR5!@FWwL&&V)K>>vgirxWrE-tnyP>`aHA2{aqCQnVH$rb z*!O&+-Gu#XKlRFC`;93$8Tt?^Y8y-XMvylcNLXW`WaCwe1t=1U`0lF?;w~&Q-f##) z!;U8Ku(Rnah!VJxpwn`DvJM$y226f60F< zj8Ljsxp+O6`Y-Dr7f`ID3Flg0*8kuW1@s9#Rjz+p#4qO`S7R?|&?` zA--;mTLhBczxk~jH=mp#hCSma+OVmqa{PWRjkViDq>wh^t!@>4L&-Ns!U)wV$wbz1V{WH3>iS@&p z35!Hn12J4>AW1?|*IYnx2`U44{E%>?DS2Dx^YMra8zno$k#iV_60FC&BOXp8Z*-K$P}ss1p@5_#~!NA(x2Z`5DTU9G=rz0Bsquis7AXG~3> zHrBr4H&&nR)Dh-Fj5g_}l>b6sxBiOX7W?(P_3Kv3jW_&O=IHaO_F6PmfBa}i9WzNb z=r3+}Z(6HncIzjk@k@dFoG5*s6*jh(E?sSLgXjrLpQgW#H*02x%$|vdkmJ+V^dF$_ zHo(*^n>(aSznVCE&TQS7!Y8k>(IvXMv_Y=9?jgDNKK*s!)jF?!I}nd4fW7#IZ_%#+ zVxtOvK*N}oD>v(FncQ*bGTCqc`7(LxB>e%U4H|>)Fe9c`e)<>#i38$PUyxXh*YJx! zbhQQ_0tpHe52w)<_mc7rko~4hi$nX3DN9IInGp6^U;+$+X_TGT4QUoS+jI)1(*uOH zz&G5eH@FV7W3X#MEQ-@UrzJ|x&j3Z(O#fnk^lynj>Y?Pn<$)DED=(ZhtDmes&3UE% z)mdRZ;<7|o)<1QP6__U!a>bT-U*FwtM#!=c&hvg^(VgmNrzJ{~5r=Vjn%mZVOrdhi z`X3!t)_>%KW<|I$4H*L9jk%lWPsqT)pyRXrWm*!B=K41q^*~!QmtjK(Z$=5N%Y30& z=Ib!TfbnXdZw4ueel-Xdt$osP7Um^H3@O97mDi@s#{0s+fgx_O-=@IO&}dkIE-dPg z(P)I6oB9{~OZ+V-m?4%wNEl{}rB6K6^Jiq{Kk5>7iuzBDl>{2_*hgOwVn~LQx8?XK z%{hEl+#~o5_4&rAY@=m!eR{dp7+05E;#R(a^JlV$i)|3t{Xb5W#fkR^7!N3`U#|N``RANfJ`*#YQN&C5F1Qg zonz(DgYneZE&8KFYvum?{w!Bqu~t8^bhrJI?~K_qwQHknTK|~Lnzm8)-D{dG)L+2E zx{FP!N&2ZQJcNAi>)LbX?Vy`e zlXNp{y#7Mq27SPLxBl|4K3{5`Zc6E<6UI-{=U2^NFkfw*rgM(9PfKx=jq(Wb?5Qz$ z?q0B$aT#s#Pu1j69qRn2|8v-WW6HRhmI3lYpI@AFsVEY#X2nuTF6+~NiZy;cA%EM# zX37LuxsNP0EuYHzr(SY*;7DH?g(m@) z_1_&-)_=>6hC^BZ!|ha7PpmDH2U z@lX%rxi%&KdH#8xA-_gd{-c)T*H&Pix283Ly463ldd-1Knlb3XIqPRmb$&`qV<7{r znLO|DBj<|zwKd850}UhEt_itt@CR!uHK8|k7q`8bxqWVoKMhC5>3+0 zqzCjzgQm-Ex8AA0xVK!EFJG>CrJGV6Sm*GyWImW&&vi1q@M)%LlgG-GDdT0Iy{FnQ z`7V}CvVQ$4xqI=w`c=Hm`hfEBGDkP2AP1kC>h9iX&#juEn_19dijJSA4_wD{t$^dJ zd)N${)TPgag;CmIW3D@Jyzqeau1QmL+(hLcYdZO-rgTRtG;BN&`fzb)%q=UX{Ya#hIZ7KX8LN?(%#m z9Yx^`loiKgNF1Btq*3>vp8c&Mkw~HOVlT$IR1s&u@cYad=~Db2wRW#s8DCEZ`oYJ$ z4Z#+Og9NG$^V|IZ@6@j~1CamlzLv+* zzdkJ2{O)(sF?Y30(ubdC730McD6RqY@&-O?1qyNY$M+_D|EQDMd&;~HT~yPftyy6V z7;G~T_axc)rUL_!b`W~en;_#beIY;NrwvZxNQd6nnxA7Sua*7SUFy^gndhGPOnngg z207#@2g=G7E9I6y-X!BXR!ZmiwfdF0Wx7$bPE}s1Vsx`fpZ(N1ajb4IjniM#)r~FP z6k4!gqJAB2lU#ShV{-n5OZ9b+{OXDaO?7-a%ibF52G>qIj?ss$cgTVrr|7Tbjgv)- zrs>9#KFz6Xyj<|hyY(4Yt7XaE>t(M!=gCgH>@928>QkZctANonDpFmrSwB@ZapF{& zH9uY(Hj3ef0O6jH~DeE?9V5u;cYjjSipofzIO4n%tKOy z!neY+Y;<#N6I{`_8BW~0eH@CM<1N8hUOLYD0t^P^Hzo}$5#*-fP2`9EAgit~ZMA?X zPs2h~>{qA2harF#w=8~)B_RA4Vlk5A|HJQ}C13u+r{w&LuayH2I&{0UpEY3tE%{&K zKY;cB;rG2=ZoTzpx$~ZtzJ8ST->zrX8;P*?!b7~)0is!K0>nA5#qs(I8ahI&t=Fve za{Y5BmFs`)jodb@|CY6S0Q}8)XUOu0mdHu3e3_-A<(>8rdUo0fMIE-^I0!jSfMKi; zpjrcrCs-Qacvt5m51_Zke(HvP>JAuNhSuk+^IJc34E843#eVZ|2qxdG*8LfMPNc+t z*yq2+RdfETYp#*Eeei?&YrIFxPW#NaT0$S|HNXBbyc~=7aICpHf4i?@A(xm(HlLo6 zX7hyiF_?p`l`ibhA!RXVy{6NmR7=suH7eNFYEH9zEA7WKm!?jcqF;IImPNbmDr;7+ zQ8U-;SK3zS=G1EawYrCOV``0V20g0JYwDJX6E^9lREJE|*HnEb)I|N&y;(Clb#rNx z?7VQ3(jK#4!`q6)$sl=1zD^UpgLRCr{E(PU$DJ^5quHIG;2L{S#Ez$@C;r;!Kb#M|D4%VWY=Bi=+_6Q>1Nz`{VL-M?bT;u=?0Z@X)fRa?%FbiTE)T{tH14l`fK`{ zZs>rH+M3Cu&9Ps3i~Sxk`X12zqnsP~VZDWS!ZjZbBHo|t zKko@le!2d~dmQ*9T^rk9|B@e_*T?hdZwMieZ{q5?-Y{3FgI4?vYb|8selO1*n~yqR zO@eM=ZL+?^@ATB4cxg`+l9N>4w@}|?t8+NL*iZkI_A%`@uKok6G-Y$#g>(pChpC|5a&zf4ja+Pdccb{z8 zv_i&rJgA>uS}l`1m+PmMHp+zY8&z>^^r)^Xuxs)b*>jIcvS^oO`pKyA`V~QJT8)!~ zp1iZnnbjpv+J6UqF!^G6_|et+pN@~}rqEc~Vg5YbsOr>T%$us4Qk&&3_iTVVW;?zr z7~bv~EkEEFP{(%YX4KfpGFvy5c9=KC*7W3Oj2)w&rkbn|KA$errthFScj~Y3Z8BRa z%QmL;(_fIW^Um}1zG19>6>*I`y7UH_&^b-!&fibxn543>L6sVP2WSn7!uM2|c!)>R z%SZV5;T3<+3D=J}V6*1>C#%eE+Kd6j+t}YsHfilQrc4Vx5QdS-tx9l998z*_Rm~Fq z0#W=`CLL3eR}<))6#ixx;yn+%AaKr23_!Muyn5o;Cw)}G@QEKAr>FUNH#kVE{|_PN zG>5R?2*v(9<~cqJ;?LA#+zBlMU@L75DdAU^aKgTjo+ba)5%?GC4_>PHv(&$NgbRwV zZ`<4Yhpg~LWc#$Ai-y*QEC#+|kh_NpX8`mk^WY2EexM7oi>S=?_ zn6rl*cY;3PTz~x!&#_vx`);~HHAZGm-zXPedXIkPZnb`8bCb%)uj1*pq^+YpThmNq z-AdEVtqJ;!swpyahHgyNm|(~B>67#|SvQwDHNVEGS-P2)we=e|%Y#cd=)>CYlt&-k zXrI6ur$6zBjkE(aUP6It9}CrUU5l6FU>a|ZRHltrS{xrzfai8ot~Bam{Mmda zU*%Nfn+%`O5(oU@X2RrQF$n{3;kS#C=>*5SE`YIRXe0+(T_2lA`_g=lSM)3PBVy)W z^Iz~e2#h=7aEL#11vx>Sc~Ii-wZwRp{MV(t)PGt3;`OGi|5y{s_0KMMdeFsTJH7rR z2kgGeFa30gT}qvVBX5P>RmK$f8?Lp+o9-EWH16lvs`5M!)4d=!eGJ-k)q6jiPo?qN z2Eq0QB}@QYc>jnQ>-^TQRI|Bhf5lqUphBS`0T%mf_Zl7z^E>t}$ZJM_M6O;9=b zdS6fSC>OBrHH=ZIS<_d&@+9 zuG93Hi?nZ@tX_7vEL(Ab{N-NVn9^U+>(UP)ZPJY?7$5C~@;x*QH-;#JPdAOxuky{D zg=b7n*Uh7<6r{>5-LRT9bERi2k_-@w?PJk*G_-I%g4 zsHOvkWCWVcra)9Wo^PBFN z(7OGQ{COYny8gv}zfzR=hkP2w`QJR_HK%pafZCP&?(EH&^=ZFXFc8JKXoYdS zN7GZ=l)_xLIRye*5AwXGltus8a-1|mldUI-yNZ*qw{Xbq*Y$C-{`%d-E`3n>6#aF( z?%DRdCj81Cbi=cxHgDdnpMb)%qx5|27Jb%~eo{%lkGHv_OS<(t_3QK#Ra-WD9L;WZ zPPd-prT;1pvQs@kHa=Blor>!ec8+}m8#6|K^>6N6eSrCn_RK0g7m5w*@boO)y1}+ZH_v?CU*EY&RxDpHtJmObh5qY&+kN~slfA|e3vUCR z|Gn6lQk66j^bnw7R&-Dn%?d|P0Ii`T48gfA3NVg0&H<-!?fM~uyiuOcMR}wFHf!~Z zx$CQ!s!U(QX;8E^5Q_ckR`rGXVLOh0NG`kXxkrrR`Bd^hucuP~DM8w;Z|(Y#WZT>N zhgBxGImgXkW&JPLzi>Z+`pEAEaGtkv{r6geAJF`IH8l4>zSgM{S?r(_G|S_}dIvqb zm9#a%Y2jWIgR_>)02y*f!#Qz4w0b{j)~Sa&KSuc2eEx*j`P=!rfxkiDVtxv}~wDm-W$zgib;hL!MhW^%F{X;loZ7^y_||`a+wa4@<{;O??cN&a1xyh&~-NUO%}sew^}P zZhhea>Qkobr?O^D(+8dFSM~IHWqAG+X7AExM@^qLS-$brKUB;>m=%D%L z)7hLnV9uB*Ms%h;KJ|E-KAHWLMOf#j?*Q2nrWDGef;Im_Bl;NQiANcf8=&zkPIK%0 zZRLeIFc(_go~^atr8Pgc-I&Uq2!+sc8qVvA*86#rK?o07Q0OTL$y?2H4RJ!6LxzP! z2iExO@{KR!=h$G53eZk&K0zFUsGX)GdIMhU_ki`)Ka3M$JwEEL@W#S7YWP6pUaU1b%E$O(!D|em>1zdL(}Y7V4 zx1=&4!n2?{ z#&&3~%#rSm3*;G3n`gf$xO&Zc+eE{syY|_4k?gSJ0^OHV-Yxp+Esg2I1yf|rN|m9T zT$?sxt**9lx@omw{#4mz=P9NO(x6>DO+tx(bYwe!G5_O$CP>?jDMZBzkPt$MV+OVi zXMGbkhE@k<{8os{oELhI*T-`xA#Osd3l%PI*+Pb*&gDZfBvMomcNvfdsLceU-%pGE z?u!@`b6@gL@|pXKZY4~<4GslchC*&7KH44UGLc|LY6Pa3ZL@aSIa(|mxwI6CRRGH;e?iVsQv8vTEXLqGgMcKC^jPX zdxduLc@IIz=Sih!@@QV@q*RZq6yVl0+D+I_zch2PAO0@!&##n?891Et|L{W&kv(=@ zBzLSlyP`O2fe@WJ9ygJ~4L6@3mAFiAFb%#^ijH^}tK6PUMyO#lEu07*naRAk{U z3-yyzlXX*SoV{o4?9yM|o3VrJf55)7ZtXf*qtC$_r<-6CCr*&r^LCP{)91)om8lP; zvaj^*vg>XZt7)_JnOU0}H>Spp>yWuK^;h~9Es~CL`p|Rz^}l*6@%MUiKX@)~RsPo< zo1M4an6e4=e~1V%mN+1q6}A>zwU|?p$NyonPbFc1zT7qpH{8a$IF}!a9o;BG+qkfS zMyQPUItLH9sgFaPbJGJ5t)0FN{KftlbNH{sUvoX=L~rt`pzwgUte*}3l1sQzjFN_E%!gjWxr1h*8AMSh1Wv3bOb)D1+5D} zYYMP26i)+gG%$cI>&H!^ueH9|u*=WD?sXUCLq^tI$69S>`n2n>>(g+rsdG^$OiR3_ ze#QP0e5Ybat^sxRlj*SD}y;>LS>T&yl8|~obHSB z`5Q_VPx;~c6J=74Lmv+tyn*U8KR4aNx&lf&ee3+mR?bW5K4Uw1$!?A}EnGQ0R;7WU z6xiBN_KzReCF5r8Ew6g*>-DYB27AU6Z1l|~e0Q!x4tUBs_Y2-v>9eD-DYT_~llJLr zvp&m;YH8otakF&8XoB*MlUehgAT#IgChz^{eU(SQ%7=$fsoo}k{0x(YPfTsuqMJwGWz+gGc+^?DA8-s<*u=*E_enV>q%w()jNP?>2A$2#=!CI0XO8oWZE;=G>E%vbca zYHc^BY+!9BUbG1tFRlnux_K}>dn0dJLoh9YDOLm?Xj`bXkBQJAzqV0if|M2o5;tfT z#4ejQ%p2-X=ke*YTn&Q)plDZ5wRMEBK|k#;_Lum_{6H;)oVAQ%-V>Jm_X1>HX(6IE zOa1#g)3CmTT;l)Q+v3JG&lb3_V3qYRt>dxQLaJ#`S^wQV<@)EjP_F-}Tq;;tPQvo6 zVC(OHa9@P=%+{>RB;ll%j6Rd!>^cYUD_GFV!FS^()%!7zHL?Zh;*}=|c3UVDd4N7# ze|!LOUW-gD#!@cvD9@)2r^e`bHXzg2xwB)jzfz3!BOx5mm_<)#{^WgkiGRp9rd@!; zKmW5@2qUwUae(@@ux>#h z4%nnoPr|ys^&yS@l4@22Tgn`K;wf87~$ zzX{W4O^xYN9)5yJH-_|+N_dd^mM!`~bLBT*+5N1Jv1e8(MAOYF=ssrb40}|$eq+y* z1+P9o-fp4EI*^n8tGq_*G~3Ot>bfB(xI${>guHW{X4HjVTKP zF4hQ9tBZAAcrDKM21;r1ja04QhlA*?H#Y~iXoSip9v2emi-n{|ebao%H(Ns}Pi-|C zpf=N|*x$yV9y>e2ZSZAqaf!b=0{^&CQ1g#_E;tLWYnQ*4ugcGSwF@C~*k!aGZxg2Z zO8&!la6BX7zQvk{R_ed3|KH5&MtQ#VVsH~&CgvfuZ6}`JA})mQSqk6YCA0ccHanE z#?%Zat!=DobNgwsopK0ax8uRUV!yhq#Gi`=a~%+EH06J)fnb~EKB8Z%QKggSoXn5r zHG}#_BlI!#W2#BY21MifkdLq6VXdJ4HeP8xjb}WmMVdY7$vN8VGu-Xhy`V9jvSsW9 zGnY}K?SQrqloJ9D-k$BpSNiY?tIn4Gc0XcsN;j#nF=a9hG(Ge@pATVdaK^Y3+V~Rx zmhqqm9X3C<-I&t5Nq4-igmk>l7(Ox=rf64~AxIqGE^tAdExJ^jG&^S%2iPD_Kh9@2 z4ECFnI-;g;J@LZ271s)gMM1=AFnO_myN>_z70YFnzUI!`Q6Jo=zcOJT8MSPTercKC zCI4-aYUv-XG}a_M%33GPY3A`*n06LrLx%B|`XABj-$P56>d*IXkOe#K;sMrUo(pMV z_U^L&`?a~`f2sdrzy9G?3hT7OH7hiIcR1Vc_rB2@HCwGssG>Ht6){@0W~=syQKk0Y z606jvX01@QN^OcFM(s^ed+(VLBqaIf{rO$Le{)^AlGk&d*E#n&=RWtz9Nf&w_{6M! zbk@D0^I3Q!)R?plhHQM2KmgffLAkpz(m&Kt<*M9#l|2rc=|irgZNoVP%qlB&i|eJF zcK)@hv4sTJ0#U5&<$-N~AQC%38+oTs^cVy5< zpNP+tYETb%<@Y?8YdA!dQcDwgw>Idk>A-W3*(@HXzcD_NuKo&Wn!d)(F#x%v0c{%i ziaSky5I1g=rgD(=+ZXq-*tdTU#nK+`eczUGxV(n}Z`6f)QExwEr5$~L$HBiGaz!qz z>RjjCHgS?GlA+#M4$g(&i5{wx1g&7hQa#$j_Qv?&2%cl%$)8c>JI)f zu$%XheSrJO<@59c>o_`ht&J!Gn35*xT8mcx*e-z0bU_b5x8(okrX-|*_YrT zq>aAN{CoQ5R+-sYqRWUrVEk~%19__%yJqwg6_F0SnooC7$e6j|ms6iT&q4L99UfRY zC7Wu+a6gfi#NL;6onArV=am1s{viH;Dr+WE56*!*UUc6vyd@0-WC2s-!aZAG&3*g z(qPfbdoR9IFET=pIC-|3*mNx(O=}bKCx6~#1Mq-7R8r#gCTJE|)q--A7lr@Y zs6#H*2KW8#|L$ZC{TgeBDaGxPM{lEzs&J~lI9t9l(vtTYZ#rU}n&-N6BwgT^qdM^q zsm(K=CLe|;73IM#8y1%R-FADjd(IO>7rm00QOe};B>om3 zxuL#v3mmaz4{jBo@E5$(?+oe*ajgYjlXhsHeEiJ2sbjp6boI*p>x~-+0w76<%I$eD z*ZntGXO^fCZZAnF`>?zMaUWF+uV0rx4rv=LmMIv&lQ(^vxVT`20jH2a7zvgsOA{6I z<>rXjyYr~0%#Gd9Cv?!uChbOCCU9~6U|j;k9Fo(bcMU+U@+NtOddq(w#~ym3OqUi)%yY2 z*L;~Z|C0Ru**WC!`5>Ftqlh+7nl2!@JxR)ky!_a;*yUKt>Dwg!)&uy!4MBIHjzr*} zRTww4OY`UVAn07)Ijx%{K+T^EXm{twe^eG7KUUe}(d|oj2;DVV8L4$G%#@BBo1Zcw zGEOu8YG=ZI1adG-iFfRXxF6ZYk}^EFKl)8Fvm#C4B1*8%e3qP`&L#(d^F7O0_Z3`l zVD?~A{6kP3Bxn#fnj<~j1Hn{OnzT=W$9xwW_MdspR=&lVE<~8(YG?KAEZyud zS4iKWR;U?LZr?_-WHn2)zHT_V`zr9y;k*^t$F6{t)+z$$#)tct z^QTYRE|s}~@INN)^mC^Q2Uk3bfa22mn8rnoA7dFO;a^_ZP&CYayfzkpV-BS&Got}PTCaTD%(xo4MNjhdwzRjdBb~cZUwW(gsA#+%x|mqiNTQ z@z|onoOWCTvZ{(srv1@PkR_ewtwK*$LXR}=>tsUBabKFqUt|KQknK;^jyLe|(R&P^ zJ|*14s8;aa6JZUVrCF&wTQ`K=_aHgdcVsOn($Zi|(hm`%oh{}~X;%jKIopPGl(|i` zt_9g=7TMW{v$0M4F{2i|-G)k2zkp_Avi{BVE}Dk-5%pt|jaaQmJ->e#I+`S(ei zwv_X~f=4;bQle38!0cm0j!?@i>$?dh{N$l5Fnc{n#*47&xPX1HO=hgB`F zv7T2JQ_DYrOnHs6Ub};Yx^gcP{delOQ6j-#1&wR!O=+h&|9*(UxhF#CbyVN^b&R6B_c> zQ2CjP&yBCumXO6KKOKMmqk+1l%s9;r1u%dfGk`oknTKGIo>%Qx_o|iy7ClbPWw$zs zx3jqNIYmNN&fk+N^c(AbdjPrqAqK3yv^SC8L(%-y>#^jOv2`+_<184Ww>hAQ{z=R@ zeDx(5_8A8wu)l>++IDxDc|rG>X}1>d-mNs(qe8Y=sEqfw-pWbuaMGZIwL2AP9#1>5Jve?a9cjQuzDm@d&equCOpm`S2tmo-{ zgX)nWj0`kvf7Nf^83JYvp*~$T4B^XkH`{J;{ZU^rM}Z^l8vq=ABZy;g=miH_J8dM@ z4!j%&M4afN(`){$iX2#;pLp~ygN_dZL=lj@l|Aro4k%q5dstt0(REY!ez-{mk^ui@ z1ahq&2cvsD_w=FoO$z;_c&WNv+P-&qHwanWIMpfEZQs!|%a`tC)t!Sn?lqs#p^eX) zwOD_ZV9myLE!Z-yuN1IJB0*YXw?n&IW+0r1NyjbRQOU_ZN&eQT1)sM6JKO|-Gv#ku zcn23(P=o;pWnD>~zPy@05Se9MO`?f7yLs+<2y{j#bg$Obh~Fvil!{!edo>h2ny7gf z$l`HX#5<_jd=+%u%AUNoMD3B7yYMZ?aztn&)FI*g_ucaT-9GugYGMcRv6y~vx%rm_;q&hanNe2L4{GyxG7tP;%!YXWRUwZ@Jb!XBO-_O4>&bjeN1{3 z2bi)7etj!sucFAkgxPE%;m2&}pSKyB4~_)<3A>)%fhokzdS*TP@>2Rq%l=R-|{QClrBV5tuk`pk&xh2_KB(u)r@$JL11G%^>%=aa+* z9a_r-7Csp@y5v#~ovvBa4|;HJFL}bWiY78B3l-|cD z_q9UOuX%x6U*ojY&U2Gap2bdw6APHj3>IHNupsdMC(on%H^b^c@4!WEKK-A*&De#C z?U`!-{5Hha=JpC~$zp>B7b!QurTz;kiH)N_E{DEe)Qwg(#r3cH2p+dN(7f z==+mT#seL%d1Vkj3@<7#DRbIgAn-d3VQuXG+*Jo)p$B2{dexytN$KOzfDiby*vpU( z&OXSc5KivV3C?fBr~({i!OaV+a)Aa37{Qrmk9QkpA9w-{im6;~OLcSjg9J^#gJw=7 z9(@ekwK>ckIo_ul3wBN4w4_SDb1E?_SjKqR9_v)atZi)xNVXbfAb3=4F3a9}h;Cli z`MFULws9W49zxO*&#D`4MU`!~Cx1e^_QB~tT>O5ZX$MwpdMv8;>Dzq&He3wdh<+8W z&K~fh16!ePw~+SAb0LOwCQ9P@%Ub`|Z$`Z)7{YIb^sgu$-erwY)b_O_dg0U$?TaFJ zeg`fP4qa0d!%LKuB;I(gbO1A#SWAVfXk4gv8X8Ogfi^%0$Qs_LUcZ77MW^Z2U{rk* zzeM$R$D2~^98Mg3+ik!r31XdQasd3fe)i!vvaJ%bxQqVv0m&?jaj8x@hhR559$C;V z;TNaA9J}t5MSLfwQQKj%Z^6<)Zvf7W-hOTwuqD{Lj-W~Hy73+Ey74bg@2_n0g?_W4 z1mhS_OK9ac!1gl~DlWCBJS_Y+GJ0+gA&+6?0XLnAtm+QcrBqr5Kdj4A}>f4tF_QT%; zrnjM!)rzYtpWr-LHRA9#B>mSFQ{rhw$It8P)^*7H<<`c!x!kz{bIgK|kx{0iNR579BAY-v znjiE#j!Wbs)PeAYL#(w}Lj}Jp0~@iHP=7_{ZmYFaJSevj6jTb=%BlbBxwQQe zU{G*$R=i~cTiuve@PVXX8=Y5g!RDASR!EsYNSLx)@XF;?>0ZuS{q*Arnt!p6l7sQW**c$RjRB~VSHr8y1J;-_jXibB7W-ncV9j#Zl z`XC|cRbQ4odv`=Us_q=yKd@v$Jptfm0__j$(&DyhmbtH1JU_|%;)H4s$2BtsCPf0u zZ;x2)2iHjf5B8}vV(<*y8CBwJ+~-i8P+7|FgFlQ7#TnmzG#63HOOec7;K`kJSc;EB zGBCa~%%%vhAj+P5&#W9jQT!%qxQH@W7T~S(lPTY!(=*^>u5B-N2{viK*~X2{2KI>J z4)Fr1S2ni%u#4!oY=rGW@{MR+{{+8PJZM2MA%4&&M z>A**g0*NoSKg7lu*6~*7?JJWv@onFBL{8AGjCzFPOS`x)I7`wp(sxuM#o&{)!XJ<& zr;bCH>fBfM7QfXj&tp^3A^JlV zZ!Q=ljfR~-NFr31Z01l`G<((7LF_vOWDf!Ld(%cQ`hos2MJhX)t$mc07-@vExqZB> zlK1{{c5P@C6|b)c$r~0AB?&~-iH7(Knvk8+5sm^y;DEBGo zS}I1suqutA4c#zQ@;58dqprVDB>8R-P+N8Ka*%6CTn+f9v9vuy!M}6~jHG#wVTOEI z_DsihPg`3oc3v<-rr|11&OR9DG}ZQ{B3Hi1S!Ig6JdUaS$6ZXf z*D1cGOXEd_F*-Mjf$=;D>CXLz4`@BfAj_J(34cU6U@nBi2%MUMt@SQC3@*H-^y@X{GwXy&?h6(l&wBWnJ53?QRC+w{7A5a z1rCI&9v{#h$5tgU4)pw9!QzhQHEvF>zFXTtLfXB7kJT>JOmIVJx4VwyViR*PnvUI3 z&mJEmaTLj&jee>X_k}p*p%RSYHWt)jnRXjJ99@EKIUzP=)(PR%rKOam+9hNEIEv=W zpiLo#3=mM3ZyaqByubI6P?Cu;x+Elc(|-5(S`5h8#h}EaxL_mq4RAZu+56Ss?Z!no zT~#Lh2!j7pLXc*pq5t4QWL0_HYQ3ev^v;pWTj<&dDrm5aw?h@{-7EdRx5sbM*cse@ zfpklD5|#m#=;oA2kVH^BG3ag!z-hn{nu{n3lvanDFXm%!+WCk_jhPdBYSSEVUh?TY zM@dOxoX%aFUC3?EPelGSkYB*Ck`iH4rPx6Eq5iI)0*YN2D*GP~wsR z;>Luv|nHOWUjwv5_q@jf+xry95DiYf{|usy8^a#k9fs|NA91#Ajzbq~9x z!O=$)RW$gL8Du~CEw1eR*b8&h8?ZZ&)PYMNEQhsDz!V$atva{8>9{l{nH!cRoU$3 zpQx~Qv4-Sbafu{l3EVy^2}pM@JrJ@8?VF8-L_X)_dz2&6o?@w z9=7BMZ9C@zq*RB{k7xg-5Bce@kl&Zg#u)nm;`e$Pbhd4{3;bkbYx*ZBlSdrRB_dwW1V@i9;{3>UtOht6!|3Sx~7$68X82Q{gEJO0PKN zXaU;dxDaS{Q=epLQT+b$vrdNh6T_5CXhH(duKs<}Ex|~+B5h`?L^JhzZ0on_mk~@dR^kCnKR-eke{`BXN9B@whBJ8V%|0|h;7(T(~`(O3Sdg`R>@KC zrB!h@Ls>HEl{hSE%0Ww0D(vPx4-4+~48ZXM$<>Wnro{{| zgLXbHw(-3TV6*KnZ!^-|-TL}u&~F@7j!)XD*qwa9F}R6coG!#8Odc6Yw>>m`O6GCz z6(Fu?CZKG%rsEpIv9<>(6S+0Xv-3Xy=t~vb*gbsz#Gl~w<4N15tHC45+GEry^8NwE zobUkEoT!4w5^Xh4K>%iJGv?Pp(%d9RHPhYOJh$cyJi8SyYd(go^kS$-Dhp z(9H-_+M8&rk$zJ7V3j{SLs<1^*wgn%M&O({&qB2*eHs<-UH>d$YQ9T1KltfN!>hB3 z^lJl7@fFX51#PF@8bwb)eaO_q=vmc3edwy2xqxac3^dpX z!!#zcEM2aG(9*KVMjQtn(uk*R2C4bJ|J~{Xh|j0~5d1Kesng*`OMm+*{3<&ldVQ6E zuo7h+71dx9Xy72T{+IRbe?yCMkvh-Dd!K1zFK%(`elTGgMR$-Dv;0{Ld`dIYM4$vH^Wfi~Uz}t1jHxVtX;yEH%gzVqi8pPszkd)H7OepahI?2%fn&!JI;% z-L|2e??(TL>d9G!>ylR8dS5l;ZZzXmXJ^aNp~aRaLi;`o%T7`RAFT>PZM_ic99cAh ztq#6fMC{esO_Ik^%UhO=FAC*fk%sG9DZmGe1``-IfC+6O8x)}Aa~MWQfe(Hq7b*}X zjQfpT3GC@h2uxy2T_*Y@-Q4O@qFy?uIpHWcn+~<+5U6^Tf3^5xT{Hmk{_8Wl*E-B8 z*N!2x8$S#xzMY)!>>Dk(94CB#lOT`1-n`7GQcGl!*7-1=ukh{Iq9TdcxDaMM)Knf+ z^sOw$@ug@rdDkjs^;Dg>C5S3&^1EGK-Tn4t8d+zQ#?+4*1{QtCW^VDqd!b(n-gsrw3`=KT00<!6@vno6-Iw~<4-S{ zAzM+~a@GTMMRwSa8a1Yh>uG$h&yb>_DA*fF%1H@Jb>^ZgCSvI(le^BbSj1JoL?`vr zbCEB+N55h6sDjd=NF5CNYh!i?mjrHH?p(7gzw2(2BBJK@)*kpJ6?ZaYy^#$hmBms~ zSWcr1W~>{rT|UZ%QAoAl{K&WGZi2$h@5GyZ7dl;$aj1x;``Nfj2omY=`3QJ8a#2FN z!dB@!oiMi_F?0_Cwq(0%eG`z3xOeTch-!jDg1K-ToN zDWKOQ{wvUdhL+z$ZOhadFyXG!m6u?j83X(bb8p?%$#18HPr<6JjQTq4wncX~BDan| z%5T3g-hi6l4<>|6m6lJ7I=?Rbz#9HS{=CQoK_JI)Z|XtiRn&lbR8gYb>%`|L&HjpS zQ-+0qYmF&fEtO$|R?hRUXR0+=dk3UpDD4(_AIqqU^AKhlh?)M}beWBx`Z$H)ZrW=K z^VerMk(P$9e?;&Xd)WSyjyVskoHg^2Kaf1F&NuMcG!c<12_|KG{&UITs?nH%)HM8V zuXk74LT*>#ea9dFl;UHEh}ir3DWu0apY&@x_?F2L1zwi9_1yBu{&@d3rYYU%$PWka zes*sYK=jv2n#Wme|<2q54FH zm4oPbE-{?Z!Omj!-;+W|;Pad3tztf=S?o>>b59EhMGip-nh=|ZRmez=9e+l8!jQu_ z|Ha^K^GO{4K(tzn?3Ce(=ugOA0*Ux?l+_IFKU{x~Yg(@#T!2|QN%;;C1NWQi`%C)| zbO4oPeo)6Ah>F+cDGDQF)iWE|Cv%D43D~PzCMNwVS7iXPV83HL?|&>oR)g{k z(7u=feN3R6MKzZ8N7WkEB!`g@er;>*;DKx{K(-}e>7?v|1A@KQ>mB21)ENF%rN+Fg z=W?!E)`@`IBYVVnDKsDP;}7KQ93VSl#C^u0!nXSPH8ktskYzJ=qWNL7H?(l*%~&j{ zS~#h(M6)5Go5$#nuib6De`H*-v&P41!T&w8oR{P8z-{C14_-}>zBEfRZKY7*_#itO zCq!Lb=@cQ-u9R@c{}2pt1WSQb>6cU13+5lnSW)hHhL9Jsp~J_X}mjk4%8bZr)8%Q_kBOhdgZL&0Snj zQycnJCv7v|r@%a5Uc~lRa8A{NNpa*jIuV9}4cirg1%t~L8e#3PLt1Bp9kvUi^{5{) ze@?)h1cHjJ2p#!4S$|1VwsqtuWFwQ5DK1_7F1d0W*hG^85=rw@XmJ7}nlw@${7LC7 z17v5#^Cm3=Khye;SEn>%6F%QStD&O|D*q$u$2OGGj@#Wc@2* zuZ1v7|ATPyp!kmZd?psn-^=vm<>(;cj*NQR^>5xa(j5isksFsmh#d0-Mek`7X0(2t zKYyZBA{-GkdM&f2Xn9&gN$Sn2P!OQW$aj~XCZ2#sbUEd!^OqQ|{WVAQjPgCF7`kvB zW~U*}#4_#fHeZt@a+|81pP0ThzR<1@Ew5@>tb4E32Hs8e#z5BY>ZqeMjKuo&fn|GF zKma+DD^X%+s`A>Ya)+g!GrKuncIJ!nv4_bk#a+%w;Vlm~v0tz5*!a?A)1*D`E-DYCf|2Uhr| zp8Trg^;XNnb0+Nh+WJCJ_8F1Lb7h*Bu23ciW|4Rj88{Bv@jlAxfYjewQk z)sK`A5yHT^@qYc@1e~^8hN3)7PYK?saYk@Z061Sg{d4a4ar@3Jg;gs0ZJmezFZkA- zcVaTE3$J<#@nMrOk%^4u(fV!V+T9ad83oS$NNelb?Ou-&g39!w;K?v=oY$+b!FNVJ z-=iA0Nj1SUjEgB`%L^z%rG>bvuNJAKfQk5iUq$p zVYb*`Q%f^N#Xom-a=EMY#LFfr$L^mDu186x8{7-F*7l_ViTe~hDBEDKB zEj<`6l7Fy~B>hcwnSS*yn~Ja+*-uYqFMt1CoK+vdN6Hdl_|j*lB&g=$^+hfwM>G-#tv9dRcD$Chxn>!AV3UlHD=ugC zi1x>L6uU62vvfBq>V%*DW@cJS-al97X0cPqzvda1Pr}@uGd_@j+;aiG`0G-OWb0a{ zyNvg}iJG$C0h@acX>-!u(j;-g^;+PWTEN2(J z)kt%7%2km0l1lxDhtjS41FU|i4s5ukDZF#wTnV=YJXLJd`61c0FnCx@ifuZDt=rBz zctP;o$R9uUV*^mcSH+zH!K5SbgJ)~FPj`H`cfYAJepgb+B#D0=jfyvdi>?qA`fun; z#|eL$B1jch&VFM){^FH6xn?^h9bXnxvl9WOOpwQpgw=wB(*#UK_1FEURJ^_5t5;`b zC|&p(@KArLod!Y`e7lOoCJk?p&JL$i%m3S9(Sd_nFO&lY&+|F2zw1MOJwI(Wu*74a z0nF8E#ka5PXY%ez1fpw4yd9tGrp1VJvr>t>1h_mFA#xcRB z9!fNc#S$M-$VWyAe4-8X=Q>JIU{oHZqyO+r@s#^+T^R8t43m7k5RFm`!z9(w|Fu<} zVSmHMsNU_^U?F~b&W^ZW(j4)2115xmOdSG+01#ZMs@z`odtAzQc52ZKW6#E`^2Os+ z(=`fzvb(BZIQog0pOX|U;zVVQX!P%Jz9jyeiM*?!sIpnXrsY@s660N^I6rL!o*CUD z4Jr4Fem>H90tP%vj-a28(ZT3*O6;2~hh(`k;$=_WV+# zBw{{cOt}tPODth-tJ>VYja$ui4>yObfAai08D!?~o*535P|XMmhD^4jf*1 zTFQ$)ke>jd02%V)O#T&p9V-a81Tem!a>TjugVVb4$RjGpp=^&{TMdLRQICb;Y#@<`j9ar~zK7@auPU^pizTM~l;C6>~%2fr*HyXxd{sGU- z_>>e%oDI8BBEyd*t%uFocFNH5)3_>i4afO)COD9u`hDb+O4GF{gF2C-f|WJ0-JbLU zXe5D1x1`LV(-syrN{t{T>0(PC1ia33W{9?W%l4WT%smpKjk8UD`IPo^v_4zT%2KJ! zcYkN?+qo(W1&)rrxCu%0Qq_{-^EWCw;=u$*{esB}M(lJ+j}k{h&|mdX!)UyII{@GM z3~Y;nU}=6`SI?L-9yGa*E@WMCg87xN)ZLGNHCeUQ*3`@NDUM3IGvGyi+Gz4?iSF zSyxP2-H_EoHlq@z@A%dLYZ7Jc7!)s#&lMl44%U9&s?T{1<9XxWpY(w7kkD;{_0IlC5O zExp{h+XBPgD-$Jcg|SIzaOUW|D0F-j4i1}3S+{Zf5{udKLg&%eo;5$nxZah`+$2uH z0fJ9+NViho($5~9?LK!)ziK}}wOyvVy7u$Y9%}$Fdw;=Btt<7f3Z0uYgHd1A%jR$G zFz1I4s|oBdfa^n0=4`j#uc}uTKf=}XCY-F-j_uiOEUUGmI}3>m4>uPkL1RgRNZ`aP zuly7fT!Tn-(;(4Jb8W(HX}57MYjHK{gu5o+6aa!?0n;R2{YrpVX#>c^VyBHi4IVkIJ@MU$-k>tXviq&j8?!F<>& z(7!vl>j{-alxBM+nj4k=O~SuH<_0i}?n&Gl^=)x6XMbUo?r|b#zQ0Tcep~ljVQHyl z2{}K-jPuJ^L{NS%x|r09D)vM+1fbI{@Z^xjGOn`rEXJQ7!h^4l;759oLGgs zagXnhh?VKh0xRn3ux7%>3)Bdt+gL8!!n@&4zw!JcB)KXerKjEzKADxeaTCe@?you|L=6+mLc0xyZ#&JwR+`K z4^$sI1RjQ2tVx^NS%GwZbRbi{EBhy}*l-OhzAQQlZAZPJI@0xWVAbkS?Xk`o{nWw& zDxvaMVR+9(HDK8ILs2+fLO~oubSF_*rB{27U^zp{OIbTHjq0BRZpoy6Bg>}(kaVr| z_|Rz6X`_5JPx5&lWgUiW=aLmya#`vi(VK+}m0d}j|14{i=yk^{ZTCN@N=YLzyrVYi zRmIB7d%im7Q2HgRK3FU9IdAn&sO(AmXkyFJ2MqdlaA%9Z(Sx*dm#yBsg|+mrT0rxA zZo6}_&PjE=haRd_86t9SW6YLsCm?)FZEcVUtsQT%D4Ax(?;)N=lIBUQB_JLM?e;R> zhCxLE6e~MF|DA%F+TnAN$w?vCd80kR(7(FD*UNP#0M_SA=a63tx3YJM_hOsA`K@}n zRhWr>1J5Iv$2=}m8E7GEoX&mY9^@HlKT!M%+MfM;*6zQvIGk(HQM!Q46PejNZR8Ds z{Bw|V;`7fnCRYDQ_x!_~z2ItDvvO6SJ<_qAXZ{(=B1Q~MXTH$20cmy0Igd!~cX5F) zmfs0YQyr{-qI%xVN`t}j@zj(*+UtiAxF|Y1Kh;h@WAIQqLw8vvMl2n@ocB=Fzdy2~ z)`|%_?s%@9Vvw{9z=IP+K;M3nv*3cAeUj{dgh8kyq2grbwryyDYv0mw(ybd^a1|g_ zcFfw`$tFP{bK3a*+KA)8z5F5yL3Tzy_Py0xU1a!M^{)C_!|XX-%Lv)A$&Sx!f`X4{ zQ-*bO{{%LQz2r{aNGy&i5asdgb?NH<(iJBiSThhk`;?~0OkhY6I)l>iN=Z9D=e6k6 zY>70uS04FSEILA39U}KU`i`%!P8x<<=4n-s z9mY0ZPi){(wCtD?W-~`Oogwc`BEM<-3dwL}hw%f{&=W4tk@HjrxOoS?KrqTCa;XzH z!LV$Iw8EeL3Res2B|3Q@q_)`y;STOp`!qM280lHuwmn3r!@qC$$I2M-w`P~NI|Ed_ zD&O-^l1e?b>kcgUqz3FZM?EXl8VTS;0-P^GX|3`}->}*dGgn2BYtzbbPM8UE_0PMV zZ@%cAZ{uNjoX*l5<3|?!eG&3cne^p!t#yO-@(+JSy@5K?X^~5I&rU+L)9|MJ3o^P} zowDZMr=2_SZUPtjQc=cc@-PmGZVp|(uxgSCSHuvo(Kr$1-Q($@;)~u0yMI7fP52a8gIfKCE)uR{>$_|d zTN**(UEzgGNdlKM&yz<1#xl;k2PuH^%XJ7vyvg7PuKc!>Z><_=kHkWIBqUQxB*8l z4%A)nCu*$=e9fc9m>XR#<%QCx2CB$Cf4k|4|4N+G-lpVE@*7(Z>MJ@^Eb7R|RUI&az%5S5U zc5w)G_T*w31-(Hwyr9IutdAM9gyp?soD^~;T#v!XuDrdICR>D?oF2Jgg=!kd(-q~L}ncph^ zkIJ%hWOe{LnX5h@&!A@pIEGxlQNNbKcOPKFC=1^e+*2u-A+ST@9{4gv-4H`=MluOL z_o!LMWCgp0e6~IdVE-M}9DQwQqe!#?UBq1@w*Oa^bDTjl0@O#yV~Nl|UhExaL3@?m zIhieDJk-wNE3dvb)~=W&;^9OmapfmdYBxLVua1>^!!~f+&cJbqj#s8ZONB(Pb? zSV=S)O4wiJ?~9d(cM>&k$g1>S8^p76t$9gHdsM35a1~f->xrNR>-tG z5`o+6aM;QlMLom81)gPEcq+eMRqm^5yUncoV1TXgn?ZZ{X!I~S(TVm~i5z>Q zup^nfDa)Y9=5gc&g{G1`!sTSPJ;VNFHf9T3OQZ{0Tn*$3Q-x{ z9$h7{Ma#Md@Aa<{P|Xn+aw)T4 zXE!VXqH+Y&3u?YmM5;8o5XeZdzF!<5>vr+ny$f^Wk&!Upejt?!DpY0|pcd^eKVg*oeL5?Ed_%YlZ-b=-VZBXWZViga434cwKY)y~ z4s%Cm*lO?;n+Wr{$})1C=Znfr5uk2zaf))RVzQPFk)5nbV)ZL4S2@j|PuUsa>|{Vh{Sj#-<^0)kJidss^9WS}#hPn0xeoFpaalicBi|*9TTva<*HA}*MCURs_g^FVf z6}w+G|3-OVuvR+kwf;&SHb-+Ux}y?eT#9Inh40Vfe^b>UiImMVi`9QZzuYwjCpwH{ zWaJPGQVN-0U@BOD&2uM6w6|bAJ3i9)m6GR<$Utv^$TmxLoqp74f^0Wn?>pfyHAbDA zI|U4;B1o3~OFwV#0T#=2EFCEg59~B!c=RqN#27orjW}DDR;#^k_`BlYmyVX}(+?8L z-?9-TMD`l1W8nEQAmbfjT<~yxG7NA?DNd|rWy`y-+5lEJ$DD^JK}>Ii zSsxRQyO#~*j>9{4fd>RWpLUWBUOHGb67RYy01k(@Z``LGySqQ)i`8WhIMp7IYW=qT z>1uwSGtWvRClTuF#%fMt??ucv_i56>FY>vt(EESuq*TejF_DTLj-d3YdPmrJ9yXeZ zwMFp{p%-iLsA=sgX7L~)^(J{`3A+Iq7Lu0K%uz004jcw}cdT3Bq7Ua#)DW~d>5NF$ zk+xe%lDF@^u3mJ;TySDNjQZCVdKYba0IGg_Ii_E2M2JrBt#&deOaYDvjB|tNlzW|; z&2c*5UZ?k6I_QFRW!F;J4d@%df0=j72RdO;VY!NT9tJndSKH`Se!$n9pXgXk4*3fS&^5i*-e;H3MHKxb8JG+68q$*p75n=5fSgmIGFFMvnne$Mf_2f^ly*ck z_&bk(xqT5@G_2k=>*U#BnU?^TT}M6;Q$+5E%k2i%q)RK|0TKyWnLVIglg!6e7EnO! z$RIfH^dv|4%QnQKk{Dg>V*LB9Yx}|PM_~Z}-`M^c(HbbYY)?gd@xbWZe#f>iVPd1m zliVe?2_e?mFA?XaGYs$e1M31Wuv=Ta@=hi(f7C#-S3zmrae0SllOTV#$eh0A-v4mU zzsuVzIb}A9pe~wnHAME1N1#CI8)54&saqcA)QhfK9x{bARdU~65BJjTIw&nY;Mr@| zA9SHe&nJ*O$_~=3F4d4B=X9KK?z8E}2!VU(mMVgA zmobkENvE`<1v%1>;0(Mg82ujBHz_>nQ`EF#H0hUC1-VuZO$y$K`9DJxrsPi~$xZO- zaU{x<=+B$B*J-wWW$1+Z7;jQ2ci(28i+$T(;=+vU&mcX^I;{-)@w>lcY_1}gx;gls zC^F(7|^tLBA*fLOhVYU5l_4##8;qB!7xr z8CTpkInU<0%(xZZB;(?Bsap(D2gltSnhS(pLT%WV$5~Men-eV$gi3!`dg5*Rk=@Fb zIBWkLZ<6_FPLh35$?$kS#z&{GjeuamwbC5M## z7&W4NJl9%WR3u28&2~KnvBdpz-hb%w=;~j}&Mh}C75`PRw$sIj9;4~BPYr6W=?~1E za%7lvRsw-2GENqmg3W91#|}Su1R+tP)Dlt~_+$VpH{vtk zf!J;^r`gCNmf4N?sl#G^11e^D3GwQNzcZ7n!n6G>uc37t>#!FKvm7k5RmrRPouu!^ zzQ_b;=J|dZ58hLqJXyB_rvG;gN7br(i6mK@IzOidQdN_X6a7%MA!mHgjf_NeXKgqw z(bdY@XZH`CXmb)Z__i%ROqJez{?CT8>{i%7?I-Of6nwj1Z^rg7>A%F0qYOP zGOd`W1B1V$?`Xf}Y^lZb!lt+d;vYLd+)zWkOT6h{2odbRAAaEP=C&2ww^gP_^^`?) z^!Vi5ORRBqAkwU^>9 zTQ--}cSS_luVM0j)+aE`=%bQ57tEA5EJ7!;I1sL6+HQsVoFy75ewW64)f!8bZh8O> zNJiv~y>1!vA+;Vj$5q(p31!{}9s*PPTNGG^(KNVL-B51OG|p#_^)mxV@m~Ljgt}B% z&=@W$ zsebK9jH{=NCmosS>XR;(y$^1Gy-XAJ9dw8-Az?}KiX^g0v+`GinzAj4vSVXbZNLA& z764XGVlcORr<|RGn5yQh{KcDSRy<@A>YoQi%hI_2%1MrWP}0K3J16$vCp@w-Js+Jx zn2XZw!JA`-ap@|v@|@S8j2pL`M)c#GEc2~Brhucc#qFqboT{Osr|u_@^;^_4mxHuD z^}Q@+T>`Hq*#XVpo;i3jj1L)K+WLbE^#LVfXC9gml>bI{{$lo0-k1Knztx?90vG~jN?OU*`Oa*O z6Te;8$mKq$g3v3sja5@4kb2Qg%eqhkLndbaGW>(1O7O8=oPfV*)Fzp-XKtLx`;;B8 za*g6%qY%Z5ln=qQ1Yji&LcfulcoI2dcXtgoPco)4Tdh7PPvqa7^(Fl2yUIPhS++kp zT!Q%E#OZg;&@$0JkfJ1s&1u!h*_gHPLA*eRa^~Y}h7I#*#zXnO_ifXq@FP^;<3w-0Oh7!s3%u!X_Y^U95&-;|?pW zMkiZpbcW(n{LF`*42#U!zapC9=h*dk7!7&)ov=6v=-3ZzFc^G?p-Y$guw}fB*Bfe6 znN&V98eFX!*EPEc>!GH2zus-3-CuHE&R4PEQK?-8kflV)%kfo5bhH9AA6O$h@^5EU7E8+2MF+sfo;dN%5KH&UgD`k zTOaG~X5&rRAv07TrGhqcquW=`B_RL}V@YLEC`w)x#hSQG8KSJon4j|}CyafvZ za%rBnTDD5Nd=433 z6c6q8t{2hiX;i?34D=9sp$2ep_5ojA?KMJHa%DZ?G#w(m>_L#S=&j2Nn1~VX zx~FyByJQJ-Lt(cMtxk`|d8W!YTnDz`w|H%5fFSB%uS@l9n#^E>jQ!GLMAKpSr)M5E z_D@0(K-d#l>j6%XdL&|LE~OTl5=_0ujkTnxT>NlC&7C?6up<)c_?r?%OhQU0m1p^k%_vkipwDBMN``?Afv*PJ12cqr2$AD?C&|9F zHf{Sa8IMS)iWeEv!=42 zEx{Lyy0yo?tZLo(Vr`3a7{=j+7RLy-B=G;Ox^^wb+6!}daPUPW1Q~=l1zop}90+CkvhVkbZ?<5^A@w-1!o~s>sG%+2RzZd0@QcOaBe%iRCHO!i zK6j>5HR#wDKuEOJ%o37rwW^ab^#QCB^}#yCu8_#C6KiYWqZ7gnDgovK*k8IR;%;5A z$i06@pK1&0@AHbhC-nPDZVzE=+U%ES>>&Y53{|UMGU=zPZeCBi;Tp>Iw|VXJ`Jg!b zVw>uvV(kyj_Vr+!gIZG8RM;)trUfGnOf##x%~-mGtY&gcTXN8>h7_$GLIMu=_L8OZ z0)H&IuZwI6Kwpn%zp!XSvz;d2AgBL+X4q6=t(m6?-Wv2k*to7aC%`qmwnMh6dvZye ztCb^*p5xZl+VoT+3v=V?LkzsWw17Uc(kK>vBK$Kye*UzY17C+3B*rA;kfwX(JpcN| z+#h}XAC@|7NzcLO5MT8yK61Tdm40RXj}RWM_Rsn8?^@y{xd~SuBAriSBr-YUB-uDr z)~s}t4=P!e*>0x>ycwl9MM*RAmnHJvABAtSilyoytQet$&KuWqnawy2?ZloL?(y=jie|vwvkKbQ=>~Zhw z*5llB&v`zd=bV7o_&=5wQ3(G6T^#5nxnz&9sT|~8B@qy;)f=-DUjUQU2A_u2um3SO zVIyJiQL*loc0s(v{ripVTLk10%gtvY2GNRCKo;Or@F!?*SQ|kD4=7_mS8VkORJR*K zS@ucmCV!b)n!6;FFa;$C>_yNFhQdhuDSx*A9@v# zh24m`9^ASayXBtkSn*QMAK!rmEgu-tNFGGk1HxyBQ{>NavRe?b9|FKI<4Kcukc!RF z1M`D2bj>u^vJaE@-<3Xmsi**O0Wzl!*ubQ3$!INOV4xU9y-Fi4D6o*$w;<{ele!fU z=Prl8f*!e9$jyQL@%dc|tG7E89Q%Ylq-s1YM;OxXUnh&|CWNu=o{}=vgXO0>G`kUD zgg*qB-%GKl=Ng#@OEO8&%Z(5+oKn96*s#vjV>i+He0j==*`bn zThRJx1JWy3z2RhK>}C2YR>-)NG+6-cc7VonNZ^zA85Mx**$GX1msn}v6<%iPHjuT# zN-ysGj0?k!fB%1~-~$+$8++(GZpEPzKTW{WY~(c_^4Lp{vX8RlHTn+{wp9;-i5zLu z>^6%YmpH01Jzm}TODXa0nTMw<-z&oD7HW^6v(HHM!#f=fe?t;>3Gnit0sk5Kf%D2? zpxwqT8$9@*ot0@$`)SJPRvh%c3(^_RZ?RkbKVG8J0?dx=hrJ}{(d= zydW`Si~8y(vrzC0euLpFce*1>Xk>)1<#i;0C#VR0Pxs9EFaLD-^r0a3Cm;Ts80^z~QRAySualDmeZWb=n;;9FX!f@a`B)=%eyT4m69fMknuTfdsIlg6*5 z9I=UDm*|@o-T~0Ry%k$9T3+&|vP9m~VKM7Q(9b^_ID0C*k#mUtZgPnv*}wvB#r>E>LK7w(E@62Y`byM&UROKklXt!mB0ycU$6a z)+ESh6jHbcJ|w&yp`QaEyR4vj^P14C3b=8M{oPNAx+y}p^?4At<;2nFn zlE0}eSl?2&ojFyv9Z6v3-pnTaDrx)jk-Od_LiRUZ*_yKF;+A09vj?^#hC_xz`L+rV zj%M|z&9x5q|1Hv7{PY(j6E@fI^z_vA^c>x9&oP8wK4m=tlgA~q=_P9U=GC@83@r6z z@>uL+jS&2m56IM2Wd>y6`acJ~94Es>IxGL^uO!*5& zZpIJK80(KnZ+t+nmU66T{@m(+7|r|e=Cdhbob0Ps{-m4WVPS!eXUe%=W&Q6?B7&%N zGLmfqJ8rbDZ^;V50AYbnPUSO*?y}E;JH1iB+iu7sR=W?}Uu`R9L#o(aem!q{OSY^S zCFlA^DbG|X*jH;^hDFco+gse;!na@9q)H95!oj5k#SKE0%Umi&nTA@%bP&kP;t!~g zkwj@6VDm?fr=byE5}H(+88NQHnQkyy851z>>I?0ih9)t2FhwJ#S{FW>2C16v)C{iE2v-R`IMSS2aB!z>;%lOUF9-Y+9madTxKG(o{ao^njOw6Mtx$hpD@~^ z(Xa1KTE7Dz@d{qC_zLQFvj+UJMWXzhhrQa6>G>{mji=7ngX~#OY_?|*Rp-vNGCM#3 zf#og9^+?CAKm?L>MrgTts$2C;v0A8VVrC2QrXc4SbXS06S#qEFQ-#OB*2E!kK%vt| z;nm;=Ol&mM$H@TuB$f@zvZtB0+r^vq~fQDl=7>St97*`E%|_t(-7PWiFHp_ z^*|R3An5<`98!xO6%yR?6z4*(MzYwfk1SKitQg6xe#+M1#+BpW@Pm;d$d)x zm2}!C#v&2l8N?3h9yHGoj;xCoH|;ogjLEJ1(*LRZo=o=y*fqHsgz0y;U&x&=c*sJY z!MpGV{Bky}zaS1=DB^oC4QMh-zaAwMuOkauGuB)Q0y<)tZlXP=S`BS7wTp8)7-tuH%z2z{YB4_Cr z_Vj6_RMqEr=N;@BliH=Ii@H2RwDMQq1@1{kz4NnuH6yB(T#3q+wNejxt#4EY7C&FE zhai`0=x;rzmOn*m8N6q;TR!@mo^Pr7H(f+%BgAI$Bh9j-@=$YikCtz|02kRhWPO_l zzoE)n;KR|?edoG>tDkKW3@${lPN&ulFQH9)9yWb1&4KH?^hqq@MU@ThHK| z17+|v4bJL7AKg_##xZVu<8dCR=r&!yKI|0z2dSdvHf^i^Cg8g~Gpn>Qq0f3pr_~IX zMbKJaRGkWK+@06zd9sOs@BVS#RszmK0>{{AO43W_pY%zc_+(p(w!VeO^YQSV!JBmJ zX>N4uiXKwbHJ;Tn)|O&e%-`;A1Id10>}Td}(&lXdhnMhifdlr$2rnA5pd`n(-^3v!%A6R-0Ndnh^wJ*_~f0TP?AupW3GUq_U+w_=+u(Tv|=_3t23b_zOBYD z4FY2qFxr#cK)pK1Mo2MuvdHf(({}F_jKWblC%$87GC3wo4A5+O~+hf{d%N1!` zpL?@?RzU7oA6Q+)deow*!!YKFR?Um`-n zehEEy8tUSJz2JI+f-n8BUZ&=AW{H1%3Ypp7V3&?O~#Lh}7Y92eu9wY#8$ zeAMmeyA5CNP6+{l7Zy#);0hA*-N&T3z)0rlx=$hYyJ?t3QK zdeV~Mywj}3{a#c#8tB!_F7dDT(8NM`%ScM_Yv$$CodnP=5+KAbM6r#0Zl zdrc1mb^wlvFgYZQ8vsyB`!rzT)#S2Jm@6 z4jYm=nUb!%8S&qvwN@Lp3snDimIO;a*E3dS3fx|?k-qsQZ*@~LnbHdqkYNe}7#}4x z&Mqn)G12qTwMB9r{7L89xuUPEtEo=~B6U{SFhow4Frt45H9E_j6D`W0nKPw;U_Ap=$2)G&CB z$Bw?lYa1$if;fWE151}+6|p}bc#2a3F#Qn|&6guDocHI%GJvFkDGd{+Uz{Q@%Djpf+ z(LX;+n00Yd!^{$5#oI^$zJHNKy)03psQJWWNUOJtq3yJ8V1NfOc3oh2pkXk?6uc~ z2AprGc+GN4Ouy3^Y*DPL`koa;t;5n@VBgh}HW1gg&%?nGnAd)V)v2XhNni9`JU`D; zUz}^E+st$@D=$Z0tpr}jzSx{Kw=y^0sXo??hJ?}$RAce;+e7kJlxirX&U}1v8ORL; z?16Zx_i~V8oMeFc4cGPFt`x2~KxQ4{K8G1R*cKPkXWTMsyxKZlSw!_J=)KKsdIQ?* z81K*4n*H1^v;PbxD^iGe;fn-2;zE^pCox_Ys+m6xwNOz!hEm>TMWvu{0vd zbmAxkY~j$N8>y}9!w&0B*|V;W7e1?{7iz&4k5De}YfUSU2cG5p{XuGy4tqmp6Y^}*_w53A;v9Kj&)AB{ zoqs956?v4)eMTemfOcV5`-cZ7mZH`|qJG?s{MYVY$-`Iv-?`v%z#|=XXV_U~1!Cs1 zf6F`_%jdnqwv=|r+u^-99T$B}vBl1Akxk0-Z3q?o1iF-9DhwdCY5C>`6hN>s>W>oP zW?2XQ95$kwwS0}U6KO8y3^Y`BgH#sF)Td1|(g1ydKx2^!)0XW>Z+l~q$!YH+elbT^ z@<-4>@N)({w47Yo6Rk z?)2m_jPFvTGtoIFh9|dEobt+3(feuXhlt|aA}aUKZ@HpuD?P=>)1u9DRi@zB^VxAX zq|@vm(>24N3N*}oxb+|xIXqH7DxrB)?#X-k$2Unv0?WVS?*qB^l&K&UVt=G6cn+yc zI=*u_vp#!u^sij>%vjwq0+YOH;O3~U%=F>4eq{u>Pu_rZo9m3;YRlXsry~FzJqi&8 z|DA2w`&ze`OOkp93eqxzxjI#Q9ZIk&}YkIdQLSYb=D{t37pr!@E3&{r@c;+ULR=smn3I}=XUhSukv zmYM~(T{r%-sp27MvB-H*ddc7kl$8m($xBshar{FVH}boNy<|n@)ZML<3GQBdi=Ozm zZn5^Z9LrPQi^u2XFRICnZv9AjL_WBoHr3K$cl$Z-y&O!;$Jq$1`&v8H`lgg|J^$qz zzHoIR1p5(%FdAZl z&OU}{vTdR6tQOYx@8}gssZU38QFv5R^vC}fc8JjsWHU@qE0Lb8%t}Ibl1By38Itq8%-j!V_Vc@Q zN@L9L2CGG&a$Dj(uotf)o`zDBYf1IIOZBUOP8(N3kNlkwZCF&ezjY_klkl$+6#K!F zWa{?iLylI^^y*iHp7l7w&#zcM7rWGUTqI1mzO#a5$0QNT9WS}zcNGK@*O^ef;?*3ptOF$-FIx;J{lyp0!)2G(jU zn=NRBGFZnrhhZa@i!m@U&W6~FnE`-W^XaoRGtVsXswr?9U=MoL><+9Eo-&{F^;RFP zY-vR*(y&^MB;2jXIJ?q$f;DXI=1x3Yg~21v02^XMSpXNZfwrY?c#=(4@=u| z-$T6*?$YT}$#02yPW#9?N-!IR)v>ZaJI^C;IEYaR?GoTAeWOb9Gr7oP*Im{2rZm3s zXVnpc3AN&xVZE_W9S38JFvl()<6BVD>$f+!kQwnyZ^>KBcH`T|s>ZmZNvKu*#^{NluB*)#~eT z)V9c&7ttz>$m;PgBGR()uH-mfAW zCK2(dAsx#h9VeLH@>}=$2G&v8SsH^++J;t@Wa|RZjLw&67q0@)Topc&Tz2X{w1HkR0n|JS z1eR9C6MVIlE_2!{pOshDI!|BC&;b|!5F3wgYd=Ta2eJ@2mzLp?i}L)t0t4S&5UGS0 z3$KE%55(al`1WyVNNEhPf6B`q&D)Uvl0iDxiC&>X?>or?#MFY7F1C@Vy(WQpH=OHb z!%ooxD|f8dk59x4#ch$Q1}g+Q3z&*`)s^&h;?o4JGsNcf6EsehHV83(?4?X3s!C|y z8=3RLIIs}5F)7d)zeFzi4u_o_W!FWDKXs4h`J20`9D7_>tG@2~XG{!_C#~3IY3S|D zXu!W^%y2N0uth{t+(#wD?y>XwTxl_X4pTTQuryc73w(9UJ#QHeUy``3yOwPi*t=sL z;$R2Xg;!5onO*lLe{9qwq^sl+7}tBhjr_=UP#T9lb7RD)C;v=q36t;HA8xIb`85MVgq2^b#BxA>k7O+G;3T} zPFf{U6Yqa)D2+NVPTVi@x!?8Ia^&(U^0&xuW?mnD5PiCJwfN`7r)^QUVU5fFP7)@T ztn)wM-76E9CyZ`gnn8*+sxnIjv|uOWdEj}PnVIu6@_K~9qh;vD-hM3wCD3}UXV`|c zbo=RBGMFx8RVO3p#J@X!hizN=fhWC}`$f4ADSyLv-f{?;iuJFo#8>je@fJCEH|-j-^;kX~t)i8W%Uz=ul@P`>Pv-QhffD)q~$cOtn6>F-l{NF>o0(_A@H zASw+-iFvGKrGjk)o>?_M?2UeqA{;(QgR^y|T|vQI_IR7d6r6+qfdcl0Bt4JNEyn|7 z%p~;#K5lmZxA#FGZN^I8koTsnP^6`1&!(!uX%{mix$GfJ^@B}W?Rh6L0Ixz>Uvp#{ zcj^UKkZCStQua|&%34ysA&`2MXfL{VpOQYR+t@0%EN4#$IKa<5Z|i7#d;0P^_~0nuSo8YZ7A)j=u5HcK_=u1Jhh&cBnlC^Ohn6J zi^Dhdhti%!aBu0@KanT-x^p}Bf`BI{)^x!$*oL35cC9QJeF2nBF#jFwgbC+`Mk-IWU(}(3m$hKmhv5IWY*?)Se2C<0b`0Cul9tfobCg% zZjs$rOu$ZY#w+I^7T?v&``k8(Xa;U5AeL6^s1GZbjkN;32+tFE?H0o?q3w^)Zv?X^ zrq@gRZOueWKmX@xl$h~(RFzF&5tf6(=@XOUe_%DwzJ$1>`Flb;3#GFzixdBES~?If=Xrbw__-d1Kc23(=HSb(GKG(f&p} z#qhda#+jW~i>l!<>w*+A-d9UxC%a=)n-C2@L_EhqN)XuI7o#gQ@sc>PH1v`-B*3Xu zJk6Q*0_1xeRo1x#&$ZT2J~GFFQW zy#~(*I|+f8iI-<-3YAm7xNF0>vJR~|r>r_Qg%1z=p|Dp?%I^e+qh9S1P$^o{v=iiz z)BM!6e2Svm9jEs(?1wLDE1IP;+O1{5o?Tqoy7MvHIWSU+T)&XEAjDA}-1R#iJv+oQn3SX z7cRW46mNpBB3hL0#430!zyj_}(oyqyy}YxadnbO6GxuhA5NbaC$7z;0u(QRvy1ZfQ z!EMV^(ec{9nUu*jrq6T8f~>xV8`|dU=-6=R37&nQAWrL|sA?&AVE$bA{_-<=g6y%? z0J+h}qYS;zpsplf-@M6nEe@g=V<)2WeFfE#dG5x`Oxr&1T$opFhVJzc4)(qWZ{IFrq{FiC*lcSyKru}oZRvXNLu1EUjfjjQ9u|zV?#Xz(`OU0jW;(bTM zm4>!Ff$fJl|A>C6a(*sF;vBt`D^?^3hFuCo#*X)<=o#v4=K7mS+c3?^3-^Lhw;AzH zUS|do_QcSHcw4;0pf^_WK{JhiTIxb5I8zEY_7VueV?8gNegfO(M~z0EOP>y`|4W($ z7w>Pi2IkebEBIM6X*lA#?KcVRFh*C^XmeWXSPFcs0>e5|X!V*tYF)FfiECn{X3(U8CM*VaFR+>?J>ehYaX&Mdsm zy$O0AC86*vDvz&FDKxD?rkb0V=0VeemP1|NCYHYy!$DmuzU63G!;Cma-Fb2S~#If z!*zkTF8MpVX@HmWFPT*nW^NwH3MK7{oQ{Z?fOLc*VWiRPjs8KaFXF|5E%vvl?$>Zq zwa69qNl{NQp5;;Ae4nGClDiXcI3!Y%V47$7Eh3r#v3q1xpePNq$}}xBt+q9L<$qy= znWsnyY!j{8>qjyrvS8k$)jEWx6S3Y-V=;fl&{Y1UG`+D$Ku)uApLap0=cVV$^PtJT zMja?q779|aH}7CO?_9F4pWKQ2Sa>$?E&Wq%oL(B(5vOX>JweQRwe6t-T?VWCnK*US zg5D-)02A zuXCM{uWzPE->uV}qZ%)CwUG5SCQE4~O`7M=Ng=jWWE@{ta3JDobR+9{jGKjGwII7H zJDpg!_m7lv{Wl&h!=hlAoj-th7hq!fVLu+H>_ybLZpl-j_U*uhMb$#%Wc|?VhL_1FXDF#G575fLpBu>EUjX=Kd=|w zwfnkg*ponG*fW5>W2m9Hq#d_?^8O^$FY8r4#TeL?A+{-CI`wPr;+r@j=I(}Y`Fn1C zm^ZOTV{(q$U++pG0FI=f5;CfQ@~Dj+ z4yFWoj7Hte5!CO7s3%DaQ~|r}VCJd#!QCc;y(+l0zGM_jl=WVI2fo=R9Q?BA+{!+% zTB|5im)<)Q^G-1ff_on-WD?mxk@dTs3kRA#bTq7TzqH#^6OiG%(b{%sdI5_XV*9q0 z2!nj!RQ70@p$E3TNwJqE5!Z&x(gLR>$fK|zYL3T%8d5k~aSdS;z9B0#c+PA7g6Tts zv7F-$7H)aMwT>mNo>k7PhjSuOGkfcqJp7U9r{6$z+WsM#dLzJVD=d|A>Pc;_>%=E& zDz`QZ^1F1Ekz%}l>K?-~w=o!n23iLz%Ysf8BEC$Wg+`|`Ay&IdM#_ezv-@@p^Kqtc zZW)f{Yh-4fW!-K01%=W7?Q-QPW{ZvVd)*QX0=T3-;vFI$q#D}hW_(>g*>i~p+E8(p zSYsi;swU-VetnYABl|^2T%8z{?qOu&t1Foe^$qP(b=P-Fc}q%pPhsG~mm5clvl!atfv&?>PC0(j%KyDt?9eq<%82BJOnLVxK~PbIQ-40Jtl9SpRu`xbl$yr-=1v}_5cr*D@XEOrMJ4Io zH~nI99~yo0+2?4*dl40Ao8uJ|WVO3&(2RQ}8z%OP{n0`iMiCKJBYFOFamwYUn^sfO zn#3Rdf~odH%LRz0Kd0My_5D3PdT1|=)j>h7tC0hiPji=$s>!hZc5!WiXlnCr!h0fL ziD<8B)io%sX{_mMUQr7MZ&t~tHnry_y{`Ol-O;Nd=wQM#bHRVqZdlt4YrPzp_)dHF z!(s!;^FqaqBAQ&$wt>IaOxuu%t@oh0>$WP8~w zMmy$>$RQzC%7+--zcR>_{@bp3k{o(FI zR+jE3a96ROv27c+M`L!jX^~D|gc9~r6yZGT|KOWMZ|0(#X zxD&d%Lpi3mpUB-i>p%q7H$)X4Y?w=Zu0)u{z|8#eoenfE7sN407|ba|9Dq z+RuSB(Qx`w`Hw>Al)l#aGeAkcad^s?cNpCIH6lFXZ25KP6?QUl?q{N#gKFkHo-aZc zgaW>nq5D;^Unw%n;+y*ZXM3X%AabC2!DdTk>%47c_!2Qyz8MQ-yAQMB@!gK!(cw&i zbKsMIi!KW!By;;M)g3IeqVaJohe&T62g;lR*bx?lfRA)rk{W;{%oWT9koGp!j)V9z zKO7yv13dgB`_qF=jYMB0cN=%gX4=t*egRrbc=~Z@^QK1{P<-Ukdo3A$H^|F|q^P#H zl``HqQifX0&-&zgLri-Oo%Y2SLFYe7jhxbDUo^QkDCaK zJ=o1#&D!P%2#v*?`qxRz{>K2F&ekKa47*X}P=#NFauJFS)gFx(*(+Vi;ZGjv& zl3b3(!**EqCv}&f1)=DPS}TqW|6DY+*UgRXhz~~*=JZ1U)TL2$rqNuL!Dr@mvCWwP zWU$l!EV&JLQ=u+A?D2$T7UudA1n#U77)DHB)SIUGHwk|I$;N6S5llEXP0p0RgvN&O zI4C>B^?EejAlUH_N98e`5_j075Bgc1eM0gG4D4I&_; z8a;_&zh=H1Xwja!;vVSccIp$v_V3?9oNWer1QTddOD09-=dBL~QY!V#%K@YFDK+)$ zJErAX!xuFkge*D@?>cs3e`%CkCD5Wr}4)tE|@TIvf`uttuU}s zWqnHo!#?WNe_kD7sFl(v?qhX7_68I3F>=*bWK%rumyG!RG@7Ro9y70avP?QLyF5FU z(8?}>&yTJDMs{(4pkuts%d_xvrWbaa*F`^6Y1-_A{Fq*lxUM%8Y}IXiN}S9c7vam= zoKg!_N;-v42H$hV9MsU z;!ztdw(xjm(2048AF$NC{$24Rl_hPp7Cal-d6XX?D0cR9^qL-m8tvRRbjOy&(NbHz znJnCgBo-clb_BtvAVem5lU`uZ|1hRTJwRz|1HF~|4PoZ@kXMMwD?2(*NOj&$vZVGD zG@1U3~dwsW~AN~?$ro4rQ{-ueRV(Kg7m#geB437=75sb zLP@I%^*jfj%|m+i1o)ps9km@nf5kzHQIEHB*?63qS^`d5aNPB&i@0-vrvcL`6SkL2 zd|NT4ZI)R2T-cBU>)+Ig7`Bt#q+A{NwmYldR@z{u73DQ&BlkxsX>sV8gZ z8{-!!tD{u9`T3@2ltx_AxR=DQIN?zWlHhwU)$ON<&IxLLy1F7+K8Y-Qh`C>$&zjSS z7iB(ei>jTUH4n5ueb~t|Y$;#$?N&nCYjPv!a0)akxX+1CNxH958CTu0%HZHII8z;~580|WQQrgY$AHFiQnRl-F75&wa z;2XS$xW3-{sU+R?Bmb7?X;YljFmyzkb*NYTI8Jh{;`I!aULf{M?DR7U9Q6cX zUy?JjoWCVrpAgIXW%NJ*-;>lO%fHAWU(Yf~o|D;*Ui@1u{Pb@?1qy?ogB8j(Ls3gM z3KZ!rtBaYrxC>`UoAk8qr)Q_Zhr54Ns;{1?GfaRnh{f8-SY{KbP`; zGn}{KXn&s0#+5IJ+TfNW|NQzTuF_u2Q$=^G!)q6)^1CctWX&0 zyPdz&u6sSyIAf%?nPHpD-pCcY=zZs3%N7%QGt1>0b)P~|Rw#_F$3*><@AWLNw~mcM z(zB0M-Ult)K*#QtMU~j+@7=K5kVa zD-X}B4d$J$$@0er@E}z3?=&OL%`7O3?fOCK7a*xr z)8Gq5$PcRC7qLM#SUlw5!wC;ON!YL7ds2rnqTd#{Pfjj2PQltko%u{?=H;u>08%7K z{NmG>>e=G*yUR{rKD*@dRU`g1w|8C_Ph3qx`{H0&Wyo`(uR_{hq&MQzQJosie`eY~ zG6NhofYUPD^nF0_qn)6m#u?f+|K2I+_W1}9_QcGDINLvip&f{u0e-kz6;)4oQ$Xd9 zoFxRDqsKi*fG9sKpJVGJK+0dA$t$LZLWe+6PioI}WxE3^CZj)8^DLcy7O####%u0O z%ov(-CrzMncpBXEc?Q+Bs%N3dBBi0SM^JG>AOX+ghJpvXr4i{TWsReQg-)eeH(S@I zO&K;#&o@176E(HtRQNYeZj!W=qz8hPWqXp5Vq9sBf~r5fE7k zsOwHO?vpNXDdARJQ1HllL4~h5uA?hC5qnFa&B)0mux*tE55m~rtCnS2jb^ZK*)1#I zH_13*x51!bkhp)bbfT(NKmLC8`NI@-*m=!wD`sOP1{fs*Vl;nc-uwb*Np){Utr&bG zaQ(;01_9nEGi^+Y#_SOaCG~$_5HFsH<`3vn3h7|pxqH=HKA~I2CieMTYDy>O?Jo`S zr3P!&uQLbSx0l+yN1fH}&DIp7G)96mgkSTguESLwI#0SEtY@^U+g66g>)j!@w{WDS z`#ZNw_*cR_vqI}}x^~pJo}885m;}7r6VmfKSZ3>GdExXqyubuFzkr^L&ZaE~!cs^_ zq?e?yJ0+m?VA_m?tUImMX+s6JYS&mPl0W$J{u#dHVbRUH_F*N5i8V7_v01>%o* zM0S<9S0}@cd7JVQ8>T#Due!5g-+wjX!e>QfQq#dVSZeQncPwGDN&Ck|db&x_Y~lfZ zzRUbZYyaz#TmdA=Gq|z)rNHvM=P04Tz!0YVeNlI5DlyPO4SN~&#{LWM=0e^w>hM_T zx3gx@%3VNhl%p&!0d;8l80XG7qyXooZMmXBnfu?YI1V2CPla4wO>4<=<2=8mc)9IV@(}Ky0vLt&ZyC(8>DdXXajO zdnUyG*_}zm4OTO$$<61NGr?&Hm11X$B0nqkwta$@WYaVEz9oJ0BJ-f!b}E$p@ZTz0Xy|QsaMs!ba?3A? z@a_r@8(~4iXNIM7KWJ^qI;Vtwi`Aygn)y3fx|fdQR&%DRJ24*%HWrgC0#88h$Svch z<>FJW0*6r!czw+Ki~#I@;!YuN?FC*|HC{waeg66OGk)G+OK$y>SC8#3dJ@PFXY;Zxz*ir zrHcM*C!+59WbQ(fBdDHupC9AIzx_qE?D@CO(9*|v1!RGzvdHaNyu?Dpr#0lIhf(1n z702LL*%>((z2$%#YssBCaFB(9>TR6`2hVE1%3BWChhnAQcU#Yk%R5T{xxW8lKgxIq z#0~NZ2;J`h-am&Bz`V7{p3a%in0mURFFU%2USc$2g1*T6zb5n)rnxP}_{Z&ZiD`_N zy^v%I4_ULLeo6mpLadG}r6!mM6t67I$@lJxi&46fQY($8es3tfZ|6ZNPtC)nWB()M z)lbILexDC=Od^6sin0bVUncpV94MA-fB7AHb_-8_!G76IH5^BPb)MFI!z7nBpy|f! zmZ%C=Ebg~|F(OPDSA20V%+H=8`EAtY z9>ajOR30+Wt%vDtPjc}ckKjDX=CzAiK}WI!4Yacyl%x!DRr)B4uh?EKb!Z{$lTY zM82L6cq@q#)mt74`*i;HnqJ2)t1bN#s@9_BY9c|y)#K~Jm?ZyIUUxNa?t2&OZN6x^ zE9MZSU%YYn*!P{(0P68c!Pm;vHGl3Y=FT(u;3(!nQJ zP*98;9C{J8-{!Anh?in)6i$*1-X{IaO07+Ck^Vs;!qtBNDj7|J7*@7X_XM;cdzQYa zMw*SB=4C-1CjT>nryhC`$^Gv-pJSg2k|!DACS5Go5Sx>Casib44eAHAu>EW@#1IY;F%N>X_gcFYxvEDUk1#_d4i-)- zzW!+i%18#zgUXJ+@5L%q~0DLu_UoTt52j^gqT9bfqUZu6_hHerA<3=m*Vh-X;}|60g3;?q zb^&SprbDBB>3va`%l=wNcR>SFn6jjzRHHIe{}^Rq2==IZTM?;=ak$7P>@moC*mw$h zm~B30*vh`^R}A{zQe2oRUq2F5s0ec+Q(e!G-g~$%+}b*hr_77Mu7N|)vA+>jp9rkf z5EI%mV?n53lXQb*yd_57I#CtZQdXX{x%d5%PNfifS7$tP>v#TE?5ufWvvNDXLq%i$ zki;o(nQ;cC5(`;RzH^@^WWa6|cmuzP5PK8R%jrw#hJmXs3xr)t42xZxTQO90Jk5J2 zA}6Xtaz};G^}Debm+o>P<@ZvTrg@uY^Q1-By++h)+z+-({O$eYB^7A;(C%t519E(0 zV|8)WzM?ZZ zsDAg;PT*1Pi$?HrL?Q*WtooZ=kqCSF2roP}SQT52`deB4VwBJM1VwfN^AkT^M zEpe(b^wDye^-ADn3bm%hPl| zgqKeeY;%2!P5tA%Ll0_^Oi>i+piU-N#(LF4UkQrUA42HE({7`JFz4cDf&oEN0_Df^ zO7?v|$LQ_ThYSnEN^1c{V5?29M$m?iwR3R6_%6}7t&E>e9J0SnUy6b5J^MVrt-LE< z;zmSulx#VGjM0g;uk$6oR#LG)Oiwi2dmCuyI(s1A&(h(Y3IqGCxGgm>Z)IF6=~`lL z7mIL&=G@V)w(Y@x>Y_Av$s#i@f4OV*MtY7TmJ5{kIJo5+-Ax3~C5t*z6XkYUKBM^P z!~y;N+SZ`95(Ll8O*I}w#Q8fN2t1=;ECiQ$h(PfB&ND5fFnLXi*1~s+ucIs4em*gK zZ*tj1)Z5RV<-$fGa}mD3cpaITB+L}m{^{sxT^xAX8<1^ooP%=KltyH#&tAp`Qp{gV> zGI|6-WWcEO-ZRPFG1)+8Q9<0869%u6;po2Df-0VZ;xeoM(_Wu0N`imwq?dRsG5}4R zK$$}Y2MK6-^h?q8IKGzd`k*BRi%nX7ZOVq%!il{?a67|L0QSr5phmMf z2=!4ZGgoyn#YPUX+v9hK#V7lY7(?n2FTYvHUEkyvp}d|1I#Rph1$o`1rE@uT{a=(u z0vwcb8K)MsX}@R9KdSi_IMed)n|ug)%+yx0&6jt!s`bmmjhZpf9=GZ6`%?C0xH$FI ze=(B2Pw43|h;wON3!kusEfpxQ07qXOp{pS;Y=&H(L`sQC<~LP5gQg(&<}l34?CK?@IAj$Ju1t1j%tWor~v9HuCc9;$gVo zcc%n^N#9$QoK=h-Ub#c;#TuMRJR4}k-p!20(7P^1=>7B5W#u<-i2oBJg(Yb027`Y# zE>7_q)8)~t>rtk>q>rTJ!|QkbUf!ZP zw@4D-S)}Yy3Zc`aX-_iwdBfxHSsBzvmEz`UAlzl(ek+f&`o_M~W1tac5qC}|#Upb| zll>ax(NIx)@v0Vci`G_2ML&UNvP7>@bnvbf}5myPV-hHQV48j~wH zkH0G<>t;8EamlH3w@@voo6_;!res|ua<33TJuQ-EdUNYQF5*I)<7}IlneHHB)vtf& z3*B~s)ZM3}MXrxHEmTRo7Vg>ACw-53$n@?z`H@0|P29xHg8;r>_ZQN6!HxPq$H2RF zK_;GS2qa7Bq(aZ9-qmm))|fzYNzJ40f}}H%1?O2hW)nhMJ8yElEUohJg)NmQtBY6o zDw19)W@L_??9r{r;ROHHYm9{M)sP4hPE->SUgekuIFKk=J|%HxB!Z zo85g&u}IZ;pNKWzXEFRto)YtDE-sQAA8t2se-R`0Sge=$VczgKePVg}n&=>~h5S1| zR{4RZ^ofe0HfYVt_qcI(qXvhGkgPllSIv`9!Oz97+8dUmO#WcW3NDJ#;q^ zf9*XJY@a;AbdyvK4lHjSzE&0CF$F!3QyJLo#dCueitn=l3 zW({irH+SrP{jOc0>G$_yT$-Uo2IS?2ZqF=6i~V=0Jh-I@u#@*LkYVBG#gHm#=;PLd zYakhB9;nSr8TtA&+kvONV+EExw@Tk)PQ^&2K<*!(`fI75-mfAS2qog-oL}*Qo4!Yk zJ`<56SyOE4rcoWvETiXw?~1biy>-fuy@H!>z%B`KzPQh)TBfBMSOl3P8DX`HKgCkE z)7-+=6}DfJea=b+$bY~#r9>`-nb2&MnB`z&4Z}5x4d_MJ8UliN+*WFKs%gfz-Tk;e zv_Gt-&G~&ZwOhy!30xY7?9a+-ds-we*!92pk7wpa`)f&;2xF5w`Jpidm zjQcVpti$9YKMa4^`NcRLp6TWg(5_ugE5c3dt@Z2HOjxft**KV&{Py zeGnQeD<5vj>X7fw8Q&W3+OsyWQhgo!)ske4OgBbx?lD&=@3Ql(R*>QA{|b)y?PdO* zqKjm;9aD2^-152qz0xSj%$k5ok#Z!k(p2>;JZ;m%*A14SEY^D>;MFr0JfWz&q!)pT zIR0oaloE!|FClM}!CR#jfXQTMva$<*Vn0RY&G*iwv#weHB`G^iRfK|*177Qc%}P<0 z$0bS}E?syaaIN!TYp=BJl%cvXea-N^!UwG>rT|8*eH%vGgy1&)>i@Anml26DVS zOoMUu4x3(}%=CA=zU>vQDI;oOu;9|HpzwIo6lE~BJ+vJkBn4^$`TshV;E4{H90uhrp( z&wEq;Z4d;Dq1}i~kX+5$fb3YbRaW`E?s{Yx0-NB?7CHa}*ZW zRy?-=dB9gLwka|0Jg||ovEu%bdz)!_W5AK=v+8Z))L3JdwShtbTA@Jb1l_UWyMBqf z@eKx_xu43rf0UV>f}EZlZtf_XR?zX&(gRUjZH|+d;(YL$K${_h|C)Y26ug%+xc$j1 zxnPo0*XIou?9nJ0uTsv@aeiN^iv*v6w%`#PupwV5U-#r|q(pXVV@9|TPW;aj)3>Lc3Y`~%d91=$S z7pVC#jj9vF-bdB&7;lZn{g(Bh_M1P_Y{szQt zKQ?aubvgQx9f{tN(7Ii3?t`_CaScSF@fT|?Z~7VG%UL4(*)=90GOkQ+h3Wqk?NeX$z5Zdp)3PB<+7~4ucCe~gVQBJZzF*C;ld;_ z%dK6s%Wu6jzJ{{X^?sa^aKj2g0Y|{rDO*Kdgig+8p7LU0ySnRq9 zIq!|X#Y(nDB_hmehG0Wte>n+(iBJL>M&}m~=FNV8YPG6Y@pn4x3>5i0cJVh|ydt?vs-MhV0>1YApS zsBgZEPJJzI{k;GCQp`XHdsfr8^hoKE>)^{U0+``HAT9uT79rJW@vfMTEcJg0Wsmh4 ziQBjOQzS*gGiEYFWZApGu|NG&+I`VvLrdUKpqEP;F!NzK)*xBvCm090f?@7aCRRXi z(@i~o@d(bZEV4%K{{9E*J)AdH!n=eR!=8`B7$j(~V&rgnK7#-*Lq#=D1buFo7KSR2 zaZdldxXU>;h};TlLIc}=`7a#8kadJr%$No}E$z_4%a}^Zp+=QiTs~KvqrhlpoUJ>y zpzSX5B>R}V@Ain-VECiI>Z$|*8|9D;>ORPcj?{L9f~AT*cF5m%!xz*O`l-0&g8E!m z*0wSv4*NQSc^g`GJ#k7EblO`1RUbTz-KX}$f-YqDzF`XBU;aHh9*5*|G^RI4U@D0{ zIR3}-v*P&4=rAtTk?3aDvi>{v(qjOtyTjsnLMNtR{k6MY*2R+yo0K_f%8e5N8QY9` z%93!eyyCpr?~|LEQ6_pCIG)pCkt*@ij9vI7yV+x}i$kiD37*eTfcaPd7*G>7YRxM! zuv5ANAIEX32w}3$D-0*_TM-+n2v#vMlSwD);0Nl(^(FFYKYRCzmg$%;+BB}m2| zk0BQ98=fp=xG9^&i-du2omLBhgT!JZ;jRwHsT{i(eM#dEe{B?^@_d+Q-OOvn6XIX_ ztpqXNYU$@-^N93RyWTG!GA?ZfaxK-@L0^A6aowHN$**Kd1LJfaCI7r0$0^4V>##CY zuRpC(uw$-!-w^Q>o8yrFA^H|O_-3Q&&aq(rnr=^hXdQI#wg&z|^9g1*d+P4DO^Hqa zVIsZMn)l#6`$1zaz9RU9j}0Y$OfrqL%{{Qsx|e1FykT&7Y^@Tu698U}sg2xoW3^qj z!fw_Ri-{Lmk!E~AmFB7vh5ip72MBQY3#Dd~%z?K%NhI^uUt%}3wgCeaa7SYBom=p$yI~IlT zT9X*qz!5){^~ZXY#rrO}YE7)@+|BK%LsQUd$iYKX7;d9@%mwq3RW%p=HsO{_r#3_F z6lCPZX-L=ee;Mia7#Wc7zNnQVkT&3-?t$b=C}9c2Hj>7Q5ZeaycHoX81`%{WIeStl zT5XN{kHsj>EdDdhEHEmGo0Q;wT;S5J#a}^J(VoBSq<9WaPf90a?Wu8XvW9%xpNsl& zvu+Q6^{>l4B;R}TaIpr>Jg${s%e-@#(Mo;ZAm2}1ACR7EyX08bwBI4+o=f$0;OCN2od!qF56Lsju`Fhqjj-8M6JBJOv@_F}fUmI>NmNoR>o@#t4Nk4uR@JWs$=Yb^8 zmtL!!=x%_C6!&>Pz5aQ=R4ON*Fe?9;Z+f}i$j`ODJ_h00*aSIfwDEYLKhsZkWpHng z>u9)!ahtS~fjwT@10GPkF*emO&#UsS`0@$Gm+U_V08Xziz~ncmH6XA7DH1N*k9(56qwr zj!m{cN8=CX-R-gBUOo~262gJFZ41`N#Y!G07N2v(B;$M9{X*$K^E`*;yr7bLab2Bx)~cL z6^yA>5l!yGM-R)*c{#LdO$`aAim;j0k4ih!Gn0o~-(-kux@Oi2Wk%X2h&MkQKELz@ z=mGe*AGG&{3mBxu)ArK|evyChoTzah83)9?P~)2?q33pTX~>P4G@vS=gI$YHxh&e* zZB?6Hl~V&#DCLj*JBku~OfKl3!dfW0JSk~1mdq=h62Nr{|toRzrxWjNEZ>uA9meZ*V%|Xy^ zG{t>&i#Og$(#5T#{<3N3`IO*h#g5-VphJxlrh##c@MSaNu~zpX6G?yJ7x+NSWxZL+ zw64x4n(X%~`1pr7SyQNjx9-$MR>zq&ZA1}k7IPiKD2srvb|veMVOtre@ytM zTD2I351SlGy6@Vwz(ev)+AK8I^C%q3@G*FH5H^hCF{O0QSZXMnA-37~TlNn;f)%&? z;V9s@2}gvR(LRBpPJaf6q#sB$oAJAV0cU^ZR{vWfq}b{DhKEoXkFeEu$|+_ukaF*= zJ6?_i{_x816U;rsd{D=ctoAIQkHwgnEj(XNhoGx z{2*5>U+N_%pYs9w;LUDtYB}rp65fj!cPZLP0|JU-jrn`aMdFFoe>F2PmZa$Dw&w&d zjbLkJ%Rj8A5-M~T=B_3UnC>64jX!C=;T_EQiLwY9d}jCiHM`2mMqCTQ?~A@M{F_wv zV`?^7E(OH4C&LfLc64@Q$P^zyoU+60%3VnM!K??&S0*mk#hpjCOz#pQ?guX7pZVn4 zH2tdn#|+Apthi`kF1E>}##?rbH5(CS_^HV2YX-OnTUle%yg3c0Fyq$rT9f50!S2V4 zA97ZV|5}f@RxVL<)}e>4aP;-x(iz;SfaQbBUXry`(e;4Ei^qtlfP4B&KX4)XNqFz- zOi3IXk0TK%(U+X63U~EMI_34nuqQmwj_WrmKcmO1NFtYJtz;y7lkw!#K4JfRkWFKm zsKpN}i+1;i(@zrW7fdZbD#Ra(WU4%}tv@36zrTk6>wIfq{3>ikLr{Nx^o?<4-Fg4s zT`ohKy9PwVYw>NMQR_tql6ncy(GJN(1fGYn%*XZd;gKk%KW zn;EF!MC%h`?j1I2d&>$P2#KKMqnGsjjNS;nIFH8Puessj^d@fa4iuRNe{3?8sF1{N zeAf9#VSTpRM)cIMMKQ}Bgdqq2f1;$FS?1gX{>ltmQU)m z7QcvRrBOOnR&pM7E8S_WnxBIiYesb5f0+0u7Zp+k@qhB%oM&hZ&Tf6E7l+>f*6Ow##a)$l5Pk&p zV{3D~{!+1msY7-TJ01>eeD&2j=n{@yK_?EtM&Cl3Xg?b>h4;Jszl9=mv9g`b)HfvDtv@U;j(a$&(7BXSvC{KP`LGk~6a8cAH zOmJc?PWVC>f@`(F1?vl3x#L6+Sj@KKOX zMy-iBaVK%_1FYgg{s#1-M9sz6vy2La%lB4nV#JkwO$lkc1h$+Kn{|4&eJvA+BT5&1 z3W@T{Ll(4@AFz;P2Lb3C!bz7&d)n9^C-4gvEIc+swMYN{6_{S(fPcEz!=5Y`c@Cp4 zW$I`~1yyz@iCh5~+jRa@Sp8<{w@!=u54GRW%It`=zt>`z;A5#Ls+{U8txBHRrT#WX7soH{s3&@&uWoLuf!&G@8q#?9pJBng)CKGx`6Pm zUc2uxGwBb?wx|L>zP%0f-fanwONyN{)fbt9@~^sWKsHHCQIjIS{BGAB_rGpV6eEY> zjDhm&SB={5!he@s{ww!4uUdQh9*oY@TrzrP{;SzfQl!VjIcLKigfcr}j;j72Cri$J z{&f4v{~oVHs%udULa$EOJ>Cc){~PV1 z2ZJ9yu%8n}qJ_~4FkZCW|LGuj$;yU4VMF6G(OC>oFMSvbOtR;d#Gh6@PSV;; zP0vf!iGwW6Vt~9h`$AxspkjHw;Nou` z`Syy!9^@h~!(EC_|9rhS#2p*MJf=|0^EI{D5P8I&PrjCKaS*CxQgrYkc9KDe>z_xA z5-XR+a}<^@O>-|tlk|c8ZO)*pLWpMGqzGdw)i&#XjL6qRHcD5$Klx5xjbZoOv*OQN zDfL3Hx;1=ek~qSx%3t8nYta%1YEOiPQ*rFmL&Q!xkVnz&$25>!iq6I;f{x26)%t_# zcD80F)w)73RbpGy`qFLF{(N?T&-RaE8fiyZxk6VI0zw?W@EaB|gBS7eCSSS_e_7%{ zvfet`6!^gF*0r=Ut0cpQt zNvQ@#(DAkaEf*EbtLAWqeSR8Y}b5u(z&WTO&DfKR&GxbgZ+#Of9&&-HO}=K z_8=`K?RH9rw)G2ap;Xz2(~H`m7U9olRvqolpF%s3p&i0<(2tJYeE7st7YgEk7p5l* z@$)zIl!2-j%g@f<_1qRaR-s@gxny3wHiLF)H^)i0lD&<#w$-4)0k&;ez?J~mT=s5F zTxc{vpNj0nOMLhJR241#)^Fd4YPjf6xJT6)lnbLQ`R*qiEcFRqPGujWp~YS3r7Sbp z=(DM+yKnWs^d*~zP%XkSc+F*~dAqQkwi7r;LHh29+$bjKKbO&|d&B&?PgT_oBb$0C z-&;^_Bp&pA61bzLVqkUm2>iM~A&;ki$oTTDh^+deHLXv%lY*l)fLQzO?Q{nuV`R$# zg#2cYQ}uI%jS*`{Lu+vMn}s=3-)G`A7Lb3k(fZP(AAs|SmDkl-zTda6Tj50ED)Btc zDU3nL!2qe7Bvrd#TapJ(|ILjAXnc(t07iK7D;7W4?Mrq>A&@4}!Hg%cRL8d$PVBG1 znMI(ihFGE7Eu*N(fNlO`M}Wh2?B_tT5U~n=udx}sY1YFZQTrqS|xugOP0bb%e*?FlbY6Z>{dgl&q~nxi~12aGk3qNBw&~P zd`UTjPue>rUn4eU?%Ph@Ac&uae4?DTz|Yas`<|fgH&*BHgdPXSO^jIO0xkcPVtm*= z@!K$ps=M82Ozg((E-l`(>YA=&o%ou|;Ff85>b%<@It(_#(mHP-rg2io4>GM)xZSU= zU`JOXiF*1b%DfDIr7pLm@u4;Q7QTB-fsH4A;@e*uzB;%(da#S$D-R8*iY^}*d#+`oiPWeE$Befr%1}M{Yw6DGQ z2epRleA}a~w)lW=!?^kle9kbY*mPg(?tI;pKu5SOnRKHUxLNmCWh7fw0(W4QDff#3 zaKDzbc(kS$C+VAT*l~yDIS4e1VBAN<({o^xSM|fq2S%|9stq{nmG4gC|R$JrEQ&*nCr|EmM^yl^( z5bpd?Ym*#-81Q){Q=Vpb;(a^0tnmljT)P+4Q==^SAJaL3G8{G zEl|jw*VvE3%MYkOTyR%4c##+jU?cHQRmbQo*Ewu*t|k5w_*yUUXTsKemsW4vNnX3A zmD*%NOv{2f(v}vTJw45zk?fBSb@>&a2Zzn2#8PIh9>~$s*><*3x&>U9iRSe*64Rh? zB;S5^;z%DCieS43UOK4Aem`VliQ&zq7xU7PM~sWhmOUTdczhHPE}oM^t|O7^**3#c zO}ORur0kC}qSo@(NLpUZk?6I|gd_a9%TZWd`7n(=qc~oDX zAxPx4^O+nPpMkhw^B!-}BQ|nUA!-HEglC*D^a_tD-+C;vugk1^!|%5p zRW9q@(d0CYkhHX&%<}@SQ}%V!GVl!XKClA{u3;X5CvZBSg^>0Ek?bl?XcxvU6JW4$ zvG&@FBfzd>300z>{Pr>&j1?v1cw$mX#mqAokn1l!X^hq*9RYT z?&lT!wnO{;tbups54m_T2AIw~L7zyvk2v0{!K1oQ<}Tr_h;^H$rmcixjSyD-(~J#_ zzw0vyx?D7FlxlTxW$GOZnyt--lk~3)Jj!JOMk{|SYw~}8McZ7|jv;F0torAAJKwhA zM;9laIGp_O3=SuD=1_oeE@tyaeuVLl`A!hx6tEUdU1LcWz!v&qYnkN5i_VX*AqA7~ zxCG2?egy6Q2zQOLJb>sZ-|5cz8o!1|B`BH{P0HrWmHh4UgJ>As0Ez7RDNW6*)rX}s z=NDc{4J~=*TjZ4O5*RrGPgqMF`V%X0`aEp{dDyRx9P!@ z!7^g0^OhTJ<=4Q)nD_E>EfDGA-UqwzoKNlYKceUbAZVODV*IN_i!&THkvZyJelPMu z;B>TM&As^@h83C|8c>;%oCB3{o=_Gt_9P7y`A#{H=R5Ar)ck*M&$Z8rr4jjKIQbTa zNu1++BMjNSA{J(SZ*^I>&}{m9rsMq(amCGwB%7m&`=r^Uhrue@iJ)3&lPQ{3hNJ2-ep1f&=U0a^Z8km=_K|KWn72fI8E?5< z<$?U|Rp&5kt9rztlQoetz-)%K)Hv&fmAwf$l5g=m1->Wa4_T`=9=A|3qHfHtM@jCU zjp?7m^1!oxxEj+|^x?W&HKgtl}AfaxKI28olJw!;ANjZN15G{D6LMIv=8W@K~p&}QcJ zXtbDQmZap>_}gkg+vsV=L8;LD!DzWBlav>bvE>o9VmGHk-;$2~yr~l=tAlS-F`f*C zjNtGF;{#ntRuAc+Gf_-I|3N^rSZ=bt3Dal7d$_T6Or;w|&Rn%bC0T*Sim7Z=@jqrC zdOf=60*S~|+Q3ixYmYw<^3$_u9ypQu9XHgVE*l_0B#Gj0GvvaL`P)s4MTChScYBIv$RQ~EgTUiYs#kKn^_FF@4uRXY}c;U$x@Qy!vL1p4le^=@7U> z73BFoJo9ZDV#mZVZNgg${j7t1nBs#){#lubD%N(TKq2GaH%Xo*)Xnt(lpHFGV660P zCkL`6Fjwbb?|*$LPtxU$8@beyp)xkyw)J;U^m2_ymT`;;?J1-!qk;XLwGjJZB`ho) zGrcBywOGbz6zzY~W>no_`)tuBze_tzl8#s&c01cEdfE3I)I#%scT^Rn1JUYJ4QZ+7 z&I0MK*nVtD^%BNi_X`l4V2pJt1SZJmIdwjFD{6l{@_Lmwzwos7B88y=rkjXY^qwB) zua@Y8drgVWf2;y!?& zH#%)KZy=6&PG|V^)c(zkePkP!=u35hF$oYcZ#o{dIYciQgQEgXc_Q%Q7 z*yrTBcD|~AV@O`Cvvp0aL5sSWK)~z|0h0xfH@T3zPN`r5pyrz3^`Luu8>RudTo|0Z zLd24LZ1j(_pQQIhq#&pasdM{#6rXDZbV>*?)M%(fevO4?^tK4Y5Jmcfq}e%B3w<-2PX3fg-(6~}!r{KZvZM~7KyK;&(?DWLSwGi?=s?x` zaGHVhsxE58X_$BNK~1Nw#uI+sG{biYt$QX5_SD`lH*b4IwsAd67OaMd1visukA5Aj z2V9ist@^BPdEvNMxOmUnASK4k^`k9RXuU3)uA?I%=oJ4G<07B)!Lo^)ub#=H%_|zs z3x4*QANn%oc(B7CO{m#3nuB5OE^O_Iw2t==cy}`Og*RdTqd*d_8mg>jV;LwWSyojF zI(vdwFiYf`<7MIdFcXUT)L1ihmam0*k=yW#Y15v38-gv`tM+2>lfep0=jDf2QazD^ z1wdlG^A5$K;_25QsblZ)Tv4#kXU;b+b@eKxMX8nEm8PbEPj|gzC$9c%dwD1LPW zl?o7i-D=A;Z{AR>2sHduUK)-O^{_H|@T0DmU+&MqqSq1;h22%!yP;2S9-G*_r=zZo z-Kd(Kq0lTbR?lmRAiZ4B{zT8IO?M`M-B_&mk)~|92=J%({St7u>g`%^Ja6dn?#Y6a z*B<|~z-YZ#Hn#a2P5QRu5y71YIzx^(ai5Qf#jXwi`AP(LEP!`XKbKf2H!1?%Ms!0d zNxNV5M+C5>EXES-7gwM0V_@0n8n;p3Fxy@;uhcplbfAe(*_4|qH}f!e-FL+B8K7;! z51U@S8Bj;!sB3oK7NL8r9)OuKW2k=3TxYv8eDQ7lFT4X)WwZbuUO4FH_yX`t<^v+l z;;_Ss2iLNHJvK=*p|9m5{Z>L*Bvf(RAJO}&g->(^gX{e%_K1H`=4c07s8Lj$6v_%$ z&Z_I)hq3#*!j%@!lCP^N^JR@*wJXgvG(XY@j(NW5X$~r~2<^Y!$EoLH!41IrfSc=@ z8pGkEh|Bg0HzRw%4#WjLX&z}8dc4}F1bfH#W}$&NIuahm6}bB@O)y9EB{+zu6a7dDukoL=5v+R}J&&7wD;jc{sgvSRz9(JDK9+o%SckT_;_w zCkN2+;m+|DarfdypW}-C4cs<-h`|>(kCRv#*~UMssy4!LPpMS{`&F zfx*)kakD1krE|l#?R<*ojJ_Y~zJ&mzJV3=Igy!N+KfD?PLI3Q1l&_%P&}7A7sxEeU zE1Q))TLX)v*s%N{?`O$_wI7aRcUxsmYh$3oWre5u>*UveS@--o{51`^>NA`~%|PlqN_{?iKsK>UP)I@lh{$tHqTxT6La~CakUV~Z+j!~a z$#*#6I5?U=CWz58BqCFVP_%*w!-PX*a6h(q3e)-wim{MgDbHUxu*+@-v6Jw}WXzx} z`OR_dNz2KahL^E>^~X;w^Ys36wXmbxn$^zsoT8Six{(R-&R!-o!)dg+c&T_W`J%mr zSF6WSLmpkg->CDRgBkoOBOd{l3#e# zYTR`@V8An*@Ylt9*F{E5zJgrEB`ePI z@i*v*Of9{6geJfz4d`~qH}!E_1Yo|d_|M12?gMRsqG|nk{swrA-ff+);g(vv?z3U$;)-+76 zK=^;jh%{^|WzR2rJvBYYg}vE4z|}ShKaJ`t2jHsi$TC-NsLmqH-xIC-4&<2(OrH9F z0l^-TRMd`C4c!f$`${O0o1@ z70@}&aLf%{&~Yf`i?r%3D`~wc1alq1cs1%(Hn3#kazye~*}RZ{{1vT^w>k%~b0Ss) zw%>-Px&oaV;@E6ZvhHzvYJxP8kjK_)4Qp55KTc*uJCJYNu23Fvl{^aBCIg3YFVO7p zJJwvWmh}p1F##^T8EGhlZSpV45BI)=7}ui_0axNQ(@`(y;5QpI=Qnz=p53BPeAfIi zyGoqBm)7ZRBXP_`G#RVs>gDe_$vAXyg~SCu2&s~L&|C-aAfipzUHdz?(){z1o~4;9 z9lXm2N|W0{kUIcftXFa+Oj?wHve3h%K~&*TUO2Kfcc{2i;v^RIci|Kr7~o+X!@xD_ z`rXBZ9t~jbpqqtv%H{QaeDixef!pq^NJxaaG3+;U|1ddU00%8pguj0;3W;;E(0h|q zg}h4e?(d-JQ&b$0jv?zCn|~9OSYP`8eO9xEms2?u(_pe)h_&r2-I|P2andt*fEmAp+RZ+lPOP0Lt z8hI7piK*RCGO+D=+L{3Hj~{(uEQyDp*`}b@%mC;n^a0ioc;CE(=>zB?tFOo6paqH4lzNoCdzfud!EX5WK_@ zHPd0}fGSw?^|i5WoOFeMAqtvL2V$&&7ZpNV~JA{3f`tNVlmSB2CbKsbO8q~Dh0 zXTTr(yu|S?)D(IGef)BH(3*bxRop>zhC1{>GN{4Vea6hMS3Qr28thR?v7UD?pU-R>dU8CYXCv3Yv!fAV@f#AK*g6O#S2&@w=`*w(`HriAa~I3CJ~gGP15^oz-z)pJl?7HMm3ht0TresGw=G zw0GX(w3c|;n9kuI*COUyc1?=?ot)uS)bjv_1$^k|(?vG7A3rns_dXov&&FLZy81Qx z2L$V?Ji}J!#zw_pTQ@{e)UI7Hx?_u}BYZ!Pmt z)W(IJr!kGJW&D41@00SElSrk%a|Reu>-2*!C|5%JmSGad^1k*X*>-2JKZ;bz!`1D| zu5$0QGM+y55#9l*1kn(=TST97{=G&TWNMdJzmtk=l1nq~zx;zBp02slfTUGqc3q&yXcrx9H+~MoADVqZlup0lH!G*0O6PGj~>uJ z9s(>O=TtBT%r4}w)VJmqqfdxnI6q%y?36RKHLm#S&P(74JII3Sfk*WAqY#dh17=_v zk%&-hd~Zof$(5-%{&Rq9qLfolwqu`Mhb@IBle#)^$yRAysVve{X&%=E8oVI%mN08* zsQOVzDJfQvmfiW{Bg)p;pR=Qo?{t|rKWTO#x=SAk${Pdg9uO>n!kg@3^UwkcsYn`w z^;`VfciM}U0V`=&4iB<^q0hwCAd#p2hpZyO!1&860`=!a(!hw#9N)hlZ{I2G$Wt^y zIgf!$nI2Z!`0gmZs+UP>dNYldcY66QjZCV8ZsYexW_up;J2Hck254Ox4kO3MJms%_3m_04%0hWRXvw$%bWf7mP7L;BtbN-%CYh;8U1wVh|uji+7 zu;w|A+)ClvxyyD&*NQ1w^NL*J+0!wfX7`%z-;7Hk8cSipyn-OGMd6GKZpwdKmPB=+ z8~(_W^umE5KQrUul`7O10^h4x9DSGUa+K$ikndKN=+$>A%rt3qEW^{Y`$7jUD<3RB zCqG_?h4F`i zryNEO7>y|Dn2kJTdY*AOGHcNa^JC;)U;cfXDHyHEPp&bGhX&0F_bx{T&G$~r1ezfQ zu33wOp<@_Xh!6K=Vn0QtLV>RW(~$p2u*5cG9@pU>edDsJ0E7LbIbzf&VO;mAt-zq7 zF&|lmI^6hQ^AHA#zHgdpi`adNG1L;myR5ayaeMW?GxQ{xm90 z*L{DC&-xt^^7LV}%D$!zGX8t#)i&4HJ-Y&>&TF|v)Xw*o8Q}RU@Z09-x=USjU4s37 zGMh%i2Hpv)iEFd|_qO5wM=q*QV4I@|``nD;hC<@-e6N@ozpGzK^DrccNf(i#IlLL~ z=%&9ucBiMF=9OKQSszLVF0-kZ(~!xkzoh#;I4Qs{&)&tGyaZ=P-0A8);pkWi`M@y~ zc<8QHRuAqkM~`NXY{os~Bfq?FE1UzMaLgOm_Px(DD`45`*RS7TT?5*Ml=HWw`)xvl zA1_Nj01QYQ;*%c@S3Hb3${AECpiCyDp4N}btxnVr7C16Jnms_2(9+Y6KRm4s38&V* z9piVUow+AV=Zn)a2&0=HVsu~ivbzFuAYvVROLE8ynDBI6T z@_rv{iqD8m)>TyKTYd~<(D~PuAOI(7k7>@(F>r6*-oSB@rStr`azR#@NY&C{8tsui z*c7+~t$Hb|#1OflTZH|%eU-xyLbLtMY*NJTDK%$tO07igbCPwL>3;5_s=7 znLSS>5NaS`n%CY>0tWvHY`>Y0P&^)E9iiHg4R`y=t19Zz8#k?&B9Ye zQA=M(b{@Gz{Q8@E@ZwQ_3~@$&{&bmo`W~{FK-?~S+b?uf3UIdx5!<{Yu^=1nJ7sKX zPHPXi9#no!2M#HE;=PG|UhpIiNGQH(Sf-$59fPZ7K4oWc58fum))#|Y;{P?NtR;5v z7*O61dp9wnIS|atb`t8)tF((?L>$CFX>;8@GB!pv5?|hus8rjPzp#*T5b($LgFo>- zz?P)=yvs@mPxyZ>0BfWg@#p99s2+HN(;RW*lA>hf+ym)C-|~2u&o07+Oa`e^msFgC$q{)-vX0m`NqQD!+a~DwqeZG1ICSkexFD+RtkPz zi=+V$Ut(@oq$~$W{filJ+J@?#RO9tTC)fRc?XcNZ$eG$T$&R}mT|qAKkgkLj?SlS$8+hc@vO4R}NnmHn9)h3C><_ia=rS7l6N>21-ZfsnYzKS*=a)Wt$ek``?1&cZ+5)S z*Ca6szn8XQW2V1RKWzI^uh_b#&s+isruLmK)wbQGY8ou&3Wj}VYNRtTze$K)f_kWH zR7llFc7iYS!I2`I+ifhq*Y`Xmez7Y_m*a2oK=yDthPNPc4E;YH=x@vqD)~qt;>K|d zf@;PsuF1Fr;vepK3_dO2RnO6#@wxo!1Hh?g$+R`<>t9kh-pOFIce{2SKZ^ePORqnd z`i^F*ssM<=L9O~`UzOZ_CVEOaZD6UUQy{(tt2>KFM_6F){dgG@7`KN!WppoG+3IKP zy2=@-gf((qi9X3ArNIvnqp*&*vlhbN|)<*W%QKWmdOLX^SMlAIWM2G3&n7e(r#l- zZsR%|G5DBw@oN{b@zQWu8NkHK^-hwy4^Q{8g(&OrztD;!3)u4 zLDcv(TV08)B{%vegGA%84I8#p`+B53L(Q*`36}1Y;a86fl$uR>%5YCnedE-MlDUZb z&n*uWvR!=`>-RXV1Vuoac^7zXs*@xFL*3yE`gA+#?=;m4qXIJt4R@nszs4fJ9rrG6 zadFy$yW1fV{$TpU406KF#p1R_d^B7AlSwo8o!>9eY-~XW8^ErpMRIQQK37@pdk)>S zrr%Rus1rKy(gg+(KNkKkg2X)wi^ICyqF5-ji=%WO2nvf!W4A!PLZaejSIy<42=C4z2EX00ffK17If9Y2m8%ko|k>~DP@emaA!6gOtwBjfM#^!|xD#N$?04 z%Sz2K3XUpjpkWS0n3?(|TF{5kMBD!mYd-BPM91HS^ZdI1*X2U2N6x8Y@xilhuW5XV zl#aSL(!MEQ?_bP48YVIcmaEL+ZJx9ByigW}MHbULG>@|c;Ee|u)XM$D3Usjq%J%gt zxEl^?K2kO`*tpT+nY6ruC8_PC&f1;kZ~VQGe@JlFT&FGZb)KKP=9SpzSYXlMOUw0` z^c`KjY>mrR_x9Hl#hKa2*c}a*a=+;>Wa2xYxhiI4Egk`(HCIQFtNx4;Z$k&0IC}j2 z468(Sh2@SO6-%`pLVarR_{%3q$r<~%HojK(xmZgy?xa5hNWIy7@3og>8lHB#HWy&q z{~f+3y@GhQk#CfP$M$T0m{OG2fs~1U@aYk!!ZU2U=>G(b-P^MQ-mdkk6${5N@(R2h zI@&*9U+TMf%`O11lw&;Xj!JGRwg-nN(;KMzvWrliZRrD*wNRD}GIeolbKu!6G!U<3 z_7~SRpB1i^CcImo+Z&>XHkxd_k8G1h5bc4YMT@w+2NW*za*!O6`BPI zrOI}V0?W}#*~G5`g!8Qw9-?>B`=46KSBv(eS_H}qfdOST7x|AhSm>Ig2DOnIzojOJ z$*CQMTfylJOz)@gEa1EPqw#T0}oCaVYi@>&K; zGukGL@IMSxW@GbAJpNc#--v~z83Od(OQF$ss(e#u6X;pC=iI<6kG21{#^GX2eu9;X z(Wo4UrvR-RN;&ok*GXW`+B^80^gKa>Y+o9tES8J$_HvKo#-fkc+Dxq8*V}pFpkbq~*EQ|Z1Xi;?XTQPDIhj%&TIp24#y`PUEmr=VB77UI!HddV( zA)JpNY~jCvR1z7#u>#8!yJRv-teNf?i`zNpVmC=02KvmtUwqZi^WFRTZ|QEZ=I~Eq z6ZbljWa14hiqmgmhhjPxF^)tN_va$9F;4^EUQ%4ZGXk1nBX{Hd@yWn`LmWfLN!&zh z_SEC+i$*B)?c<9wX;e!vE5C95fyMKUc(0d!RqP}XVQ?G^Qm%EkLkE{c*#60fgW2MXD^}J<=z8u8bkK*Ed)8SdauJ#iV_Lg)#}u%esBPrMD^RpKD7KJW;;Jr z`8epu8$3@`h^j69#Qux(z8LSt1qaFN#K=}oNd-tPoDKa9a;}H7ESO#~oxVUyu%wjR z7-ndJ883!<#SUx2$5QSsrCz!-yXxB@`@i}i^Mqo{z8zoWHnTzgsA- zQ>}NKHy7`}3aoXTUmJ{ie(MPgu-gK;-si4EIRJmCJ9GaD$98kt)94K0SMx$bJvR?V zZcYPPNt*C?v>p5Qa^e*GVxHKTgs&cwKs&GI-k!+2&}RraCanZqj5fKf2`w}Klf)Qi z<>CvE00H&_P%-?JFWTe=A1>>gQ03Ezv@ z^#5M(HNQQ1wYI$Uk*|QHIop-V<1Om!O3qU#7+4RwiCuGRu5tc;UGMj1-|J{uiYRD= zCHs0^X;h>byXLGgxvOi{dg?!n`zkBg?g7VDQRecS58y15CXp5`+UJ{HOIX%QA*Ph4 zU_Y+QUk{}WCU1VRAxXi?_DY|G!g)cQ*W--=F*bS{3PM-lJ6aoH#m|tHNr$!}G~rPo zunmt=M8}uDl1IXQ*hfB{r)j8q5pR6+nt&G7(@X)jtMeMptBpazaBNDU$Yw0hUPE&6 z;~yYFabMVuDkpIG+{uWUj{)o1diZ; z_z)O=fwgaXEOgwd4nj_J#v2UP8V#TlT9@JAL=_3YE%7Ul*BzBHJFQv{#L447HJ4plvmBzK;NE$EM}Ud^eKtPlGb#Qxca-vzIV#i z_do=24`&PngH}pL4~A{;9UdcuQG2*W3(P1DgO$w)Zvm}G zltL%;Au5M$&Z!H-tQMuq8E>K6Jh4aCh47}Xzsco0i$ID-8?tvO{8~l0QzV8?#NvH@ zngx|M3+#gJ49kTMjV%~{1pnY;!5DjXWH+eHfD5C*ypVqwSHKOva(&1E={(ztS;rohTBmqOww-feNozK35dxK3 z>HXcy@sEAcHn!pvV5T?=1vjUIag@GOIAY7m|s2J2=*5(%GEe7JJvxS5lbnJkS>IwRjc_)6IF*W?#|o= z*DsH~`Okc__M=*mkw}olW<$c9rZ&=$56NXQo_L$T(nqA@I23|qkx$Iua{g3qUqh{n zGU}1tkHCf(jI~qG^3GWMP#6_wK(p7ID0|!#{+%`hoGO|3lK?#|@P#yGvC_LjId5NB z3|X(ui%&pLTgRW73vcuBX*^BfSw`Fxc>pT7W<=H`dHq@8KhE zG1_zRy_2+~>>83=!y^GEG+%&_V#ZVt`I2SZ!r!(R5B zY&oxY`MjS02+6z9F_H48zqtnmCrsriUj>W2LR+9Yo=FyiaFSqOZ@MxQPQ4HR^an#A z^KSSig=~nrOx%`IUiLe{qrU)j@!fcPrYeT&BRd6#m`*{3=*3*WAQ6j0m6ITht=93c zHdd=h{Vh}@vZilDOIu(xYGOEq?P|RKdnERn@5LD3Wr|3zD?sB@Q=AzB3oY1odF>@7 zv-D1=Dq<~WNra-YPHIxUh*cV$-}=OAtpENfRsuJUKuIVrbc|?1@JxvRa*&qbVIZ1< z$z_y7+8NvaCb48Ho0d-*ki@aD&if9-DU#M&|ZFW-iptWl$ zWOD#EqYn~)xw1u;uNB!eus!dSU7MgQX*!-S~6a7eO^lqy6#RP6}h!e5r!=RqH7M0VHUxh9da603bG~gsKjrB&!du zQbnhKp*a-jD<+!aVT1mYaO6NuyfQ;`xb)*$Zkx5`)nE=C|K6UTU6n-lqNYPOLKWLVKd0y@n`rVVq zlDcZiTm*g%InKB3BKjD;-+P%8LGRCwVo`T0-+xnK{fa7W=gTiI>^>p_-H}J?U1b^z zH80GCCXhjI*|sxl*Y&(eHIz^KKm2CKwTXT}4N!rHEBOp)j$qnfL$cf(ZsqF$yL zeA8*YPH*4ok~U0no9}&*n;Z8L=9n}$pSYu?$5X6tN@3gic*D__wY8+lO0o3KQ z8VM>5{Fs!gtN8S4YF;2B25icuT5vq&e3U6I=e7MB2X;TOZqkszJ$z1piv)yRzO@B{ z!q%qEADvU8o@T&8|3)?`fE*#FD?NFdQ84vYD$=d>W|ERHFZL;v`^Y7gE?3Y*v9Q@v z7J2r`2-|9=&F$}Su#!Uj2I^Y(;q^CR{1hQB{ypBVPM9@jw4jRGJT!e@BEm`P%Of5Q zMlkr9ftBgF6aDXNC?&YVj-R!D{OfO}nieq@s}AxHtSu&3g|&~jVqz1<-$mcr{fKF; zRi={7jvJzo&*Gfz2|R+9WFPC4?dxWnED-tiEJiHbv=Kk zOb2aOX=0>rM$o|N(-o#L(M8_d8q*xU+p_umhw2iet)vFKrd~nd%o9;wN5b_*W`w+p&~9EOVj&mxZIG zaRoXq*plo4Hf-GjGRS5JY#I%TYLuFU*gj|%UyMeBC?ipu1P%xFfzwp@%N@*x3M=kJ z8;8&h7x99!@ed7OU$zwiW!rmCEYDV^EF&b>E+LFDVEspjnWCuU)-!yfLg*?>#~^Cx zd!@&{Ry0^Q3sWI)(U)5y-2l1QHvps*E0IrZpD%xN5(PkWskmd6(}!ri$!LWE!A%_# zB81M_)e7^Wj8Aq1!l3iPp?C!Pfp-1ice~LNb{%WTLeBHU#5!uC42@drVl*Ml6 ztj9-7Kft2@E9km!dC{0D8p*8KngiOLZ$=f4q9R+}?7ZLC{usf) z^&&2x&@q_g$)-p$vZN4J`dA**L|9K8che=K(_m#lDUK)0;rDM<=hq^z97YSs&gECA zMKs(=-NR{2v5m*wYUdT30~egmf#tJ{CWz_TBcpWBJ$<&3N`$H0gJ}-%qEnX4F>TZK zb3?Nt4A_j85VS546gYb%o z&nS9$!aF84{wm4bQfcaO!(CTKUbUv#nP` zP0=?_%*8u7qO0Kzl1iYVED^BziKAba19X}8i#?sw#H;C!>T1O8eW~B#`wI~nw2R5r zm;%1}$|5|GftY7!R9EA427zG(%^v7D_$IITC`~B=#7p=&iQf}L?>~&BEeGk&-Pvd# z{ISWc_p*yS$$89Dg&qzq>P9GouJeB0E*Z)Ouht!bV}1czjKG#&(Wg|N2xIp(0n z)Y+@ziS;bMF-3J2`7b~g!Pn-GPmF@{?LIxV)4MiYzGLegbyiky3S?>`(rHLoiqgw{ z3}@>Bc|M~QesAX-D~6+Tp!#U>03Qa|_rg6gau@d0PCzIV!;E+o%XLrvo`J8aCv}y; zTDN((rr^lu4Ewy!U#;C24}6iRiu!RnIhM7?fCBAC?`zOe1)}4NdF^uEe_4uCtrf7B zjrYGQ;1l5Mtzc7o7k#|%fYAP#OxkPnC?u1{8(G3OyagT>C3jW*(jF@5=X}O z)^u4(tMlDhyaECA9n~w>r7O_TIm26H3id4%f!C$+eSM-AFXE>acND)|e$)={hJ1Kh zxC^emSJ5YL@kPUCd1pD{rJGfyY8E!~0U; z&<%>B?2E#L)EKXZV>sukqn0Gd2?{j$rv_w6_a}Wce({K|{nvKXH7`Ittf^^OP~2&< zTES$DSUY!R?Y6LeIIe50*ifrR)=~FnO(U8={m^Eoqxn?U`%#FONKVV9Oaf$GH}>R9 z;Ip?BdEq9b3Prm)VBV^-q%cZ=zVLbDn1q5S9#!sk#7Lj%{Y1cal5@qjene$pYoJQH zrS#5lW6SXLe$lVeO(K#bGt7-4Sxpwr?SJ&nIIAz*DwXoDT)%PY>^uQzxq;Nz&rOkc zW5vTd#F<#1Rq;AYu<6q5)>Bpv`~q(L6^Q(-$ZQw!vz}uSDtpG$^Izl*M-AZ8pbNYy zk>r^BT!Q?L0V`0HdsjWI7wVB)WF2sFif4;~1L!k7T#VIR0)r{ND4YcY-4IHDl9$a- zeI~#r@#Mu?0t#6wR~Cvrq4)*ym93B21h~S#hVJo-IYJhV$fsW72>thS z4%j_{5kA$mAwwp&bGJGBx^U!|XtKb&orQ2t%+(#&R?#>Og$&}PSV{4`MW>te{~L!dJ@tTLI9(0>kO>tr z7ISD4*Zjp2?Gmf8urv7ZY4tLJS}e9)EsbS;4#}>7-&2z`>cD8e{qp~Nmfih1ZY8@E zkY_m8j=h@k&-&<=OdR)}5^c{9@kg)@MTurs?k~JOmy#V|sp>c~M)^QqME?s4 z=*>zA?tcikbv6KW#j!dI2NYSoW+Bn3y0!ZZqPq3_D&%YArZNZWAY#Pm3}Y+iX#+_3 z-?a{^E`&SOilRlmy8JJr5$B7rWY}^HfeAMf5UAKLaNd%&S3stA!#DVE%4oiMh+fS_ zy0H54X208)ApkeEKBJhGXdwESXtWLadDPP4ZUX4-k3DZEnMrIC+2NWCZw9C|0eMs5 zal>sQXuGCBkpoO-9g68}=qjT4WszYybzsZEv{Hi{?+a-ROZw2=?cBZc__Rj>Qd?sb z{T*)lFr3eHnT^xU`qa*NQ1?ylR2yv|`wChh)LRT4$0NMQO^J(PF0e7dGAFGpf@F`P z&v#~!6o>g-r`)Y3=CW#~twx`Bz83E^^{1E^!~1~GN72vo^^jDJ8~)Dw**=rmNi7`+}5;-6G#pyuxPRW{WCv21$0A8)$qgEm}c8bFZc5XLd z!8pA8{NIQ{-~TXlr@Gt2eB{^vJ*brhuxs|5T<3}AHb@{@`B1XO*}F6&;-4%meCDqH)D z$c|V2CsdD^;r0Pd@sGc{1=E;0Y%^{cY2|CHV&Cl~A)`vkr4YnBE)-JkJa+BIGdgTc zL2H}ubuMOeoJ$XMj1CiWa zRX1KeygVCaDPg$Ug+yygXx+<}#uYZg>`ZcUo|Rr3oVQyZzP3VQ9r79YsGPPvdM|wo z6<#|V1XG1uLk)M1EFMR0L6oRI)7>AOsZ7yEsTWVC#BCCrCx1uCOf>SvreuSkj(%qO z<}j6S{r#4shYi<0(@F(X;Sttq5PRldYdKctxpx0Dgd>eG5|h|@}%r@ zofn(OSD~8Gxi`5n-^t&y))Hq{Ut&X3@u3fT8CB1Qvn}>CqPg<(qVfM8s-o!P0-Utr zPUF3@BdGlq1`rzoUh2`$!~wm{l)Y6%sQB6^`bp|RB#%o`r2m7PW5p(~$TiRD*6ns} z=xI}VstCfULOx9>{9Q;(C+-CC;kv2*l2HNb`TBG+*My~9%|znI$4;bC+cu>5;6;Xw zdC^T4%OkQri`;+IYT7EiaISAKr-J$@5LvbdW@p(#E&K7#^NmC$o8oS$J^Bi)NJK>& zA4RvHvbCSO)!5cv9KuZ(H+;De*QRDc3eUbg+)((^;GhjT(NAkp|BM3pF`K~`Qd)5W2P@^T6On0n4F@D*Uh-FAoJYCxc{6Ez`Dd; zX|jKQ`#XZLz_gXBut#i=!|H_Y?;bDVOcO(Z%Y7z6b;h#9SWtLq*{wCnyfWL*EWuxQ zh;!r+?+Js$j|(N7JidLtQGF?<=gj*bFTtVuHwlEAhjatvO`#%&r)%nVEDS>Ez5tOBn`o3UAJ5Be1s6xJf2AK*oZom(<+tF5+>+ce zc)&>pNO!9pmZNJxm^|-h;!r}54wZ1R^|ja%EFeD=p8x8N8J$|ZrG|!)EMI4ir zV71m56ub4mT-;cilZ2bi1EOC+;_R;#t14$PBwxTUa!-k+qvAtzyrAd3N+L-c-$otN zY6gbF2-{ubM1<$#pOD!xk1)4Nb_tio@P4hA2itqxBL+0o)-=D$+<^SUeBu4X;b1N# zLOL66)Modh{%yX(5zygd96J0Lo9*2S&RetTJ4)xl$%LKK75kY_`-9vZsQMd6M4BkM zAXV|J|K4jed@1LLft+v`%`N!1{>nv0=$?Fz@>BJ$XnWCrh+{|s`;NG@gj#K(0!W!0_*YX2TLy1pf3rpl-@>r^U`V3JBWxI zDi69(YWy=CHvj7=Z5#rRhl_XBvu3kx6DDfW?J*o06Wiqr+3h$zgQ2*~SP{-*%nS-a z0UQc{*ii|=!3e;tuMB-o&4OJ>=R%P>8E*C@Xe%?u>L)a-&$YyY>a6oh(^=|C+1K$U z*{0JC3Eq?XKt0b>Wv`Q@y-P(|Fr!c^hl^fbdY;cX* z-fhoLjHqr?b8OVMlg)uW4529f^cDG!IDPKUX}k-GN48}SZyAHu$Q$vB-mueO5V}wC zzH{55_QQa32me(x5tvo;7dZzcNf*6Q zQbj!y;M)zVaW=_Vkp+Io2%)tY<>zg2-mhAzPQdZ@h&yT8XVEfrkB*#57+gwF-6z$0 z{PSy0YA@?u225Y3X_okTr0=zeN2zG&ae)XF1ZzJf;TVyo>Y!aPglz;~e3NI;<(URNYN47icBIB~!jwbjGfKsC zmK(5~x}Cr9LO$fgW1Vd-y|$Y7-cT1MOw{wnRgW5_CgFDd`y3E;T;J^in6zP~s_s7IXk?e{{72)UjYCW_*=ZLOsRxwis68qJ8e!>d{Y*t7 zWpYymZ)&;pC4L?se?CX1mMtPd?|I#_I@#j7`()J92LA?J&-2svp{Epr-i4>qQGT)_ zk9GE_C$WI5bsx_l5f3EiK%nQJHeRQXa%>kqQPHWFo6-Xozn{4yQRH~3SZ$7C=dShe z?;Lg!GO^)Sf>tl@Yto{>j0;F+ee4bS?_Zo`jPCsadJ2xdE95ed4@O zeyW*L+srGDEZM#{AJ%_nvjux2EXgYI-c?)On;_`Dd46}n%^e7PSZp(cAV_#QQd>l0 zUf1PC>&@`hPK@Kd5Nyl1llBy>Qq_7-bI^tuHrq4)=b#R}eoit|Dk>?dm0%V64aDkg(!gs+NW1x~*it?Rq7VVVg9YEUkplg2kfLqIdA&i)LPV zd4haiqcKE|-_4Tw-Wpz_g~9#o?tFKpM;r!`UaI~cO;z+)fW+#2EVDO=?cn=iPtViz z6UP=WxyN)Xzr)|Yq4u*bBxfyWQ{pfnMFDsHHs~%ge5CKI%`WI6EG`cNxMB%6p(wp> z8Rv-_^?aIAwNQOllF)`d+<0tThdR)$nt(JD0Mf-K;C1~OOhF} z2KZ3?b0RtuuxPKfiG0xQC{wj;GZGcot0b(@bqCM*ja9zcb-Am{>A3N@4e~i?yPV@2 z8%7bqiPnQ|7bQeI4`o^(|3;W)_&+);Hkhs17qZD5tkf503YvxaULxw9rx=R|0ocL5 zhz)bk0HHcFeWpt`qGWU297 z#P$geV3Hj2h2XpGh}yvNLhWNCPDXoso#WsG3-RprUJEtQMOdH zU+Mc3F9+Wo26$|1+*J|4KjTO({*wJwbHqhoo>;98IH9iJDMRk7frJ$~85TY_y;dCV zy?*)ZSx%MU>3><<$<_`#E=~+R@7>LPvlS4dg#I$$g^)_ix_3`Smvtrb|D_tR058?w z(4Tuv1uIDzw4qN>oS@qU?)yE_ys2{bU{hYMYa_0NWm4-Mb zRuMTmrAsy+C$>*FYS6#Y!auT z5T{2apCCU!Lgnze1JCO(b81WaVlR9-`?SX&WpM$fu z)E{IP;5i-l9I-n{^Y8v}*}kK_<(-2Za21DSI|B9Zy=c62Ms+N&3K{RHsuy_lQQnpH z0h@>Q9?*_IxICM{s#5I`7%qnoXeiOtsj~}rAxy&*ap~~0%11%tvI)jhnuJjuT%nIL zm=?*Z0K715`w@wiC|GFm%>GK}v_+ETC;Bd^<6ZTELHZ-hu$LadY zbMRfqhxt_r_P5kZSu`D!bfI^Z^k01{&B)G&B*>;K9G2>WZEL4On(~kjIsWqrwuVFR z{hsY^aG{c0mz#i2GCgGhmbS3Y&A#hWy=lW-VqH%f{)RU4Syl&yX{`wI36v1p8KdF*=V$><2l*oj3|$$Z0|6h3o1Ki1-AiY34OTQ z?=LTgU#k$zEM>Cm?>=|ST?mWI>U~pg1mrV5JstGg6`mS3w?wHoUx#5k?p3!ZI@O%U0QA9iex&PoRo-w~=adJNvM5+>)~Dw$ey< zKGbr%^R|-^yO}soeNRd4eB&j~y=qb?FA??PU7Np;zP5vIzJ(^D=>PJK6Im` zSUZ8Tb6=EDVmB;i^F9x~6cN4@>zum)*ZnC%lQ%@NLQu{Rh8id*6gdQmPOP`}dK=g)zQOcD8 ze?XCQZRZ~Zl|%~IGundz>AmyMD8yn|WPqIBL+H*Zx`_Yl;fqxBg}C74Q&yI*jU%Na zl=2zZS-w|O5tom}1>00}p`7N2>Px4&V$M^Od36gnD|8V2I=d@`l^Lo&L71!>WA|uZ zOY#0`0^#k4A7Y^{zXJ(N1G8VNU`lEy8_md0Fwg^Sip9R0j%Lc%R3u))ku2qlg zH3Wi+$GhTs-P*<{S@W-U$eky`vh`x!nDeq<@K|!jkP90^0gC~Fn=RCYuScVtNBs77 zvl5prLMe`kM4%C-1HR94{`bR6Bp7u>_BXf8tp$*kuzE0W$M}@Mx9Wy&`60fKYFP>D zPJzL@UZV6@A3sI{8oML0kIExzhB%lDEIcqe^1)i0e`Qc^+f+l{y5LVgD(YvXblHs!~cqh-_&xeqxArU z=8;HmfcOGPDOx^JZXF?$BDaJt3#e5s&%gLCIfpLD0nHMFqwYAv`0ok%1ZpS${MoFv zjs5c(U^7ZTf{j}G90}r`TT;jpA&1jMMI?^^j^W4|=wt*o&Ej(ul>(D?i*(J-f5d02kjF(Br=6-HlZ*WSwVaq@`&k8tcWXu|^0@<4&IFJdOScHXn$ zCoF~Otn)r*vBlyLwXw!Go0?*||v2=I1KyIIDV&oEHQu1IW$YvA+ zAC+#Sr;JhI*PROJsG+#WK&&yWtm8E>+>5c#8v}IlV1dB|E!rqdQ@8dVgHBG@y{w&f zs+kr3izX;$Rp;kuD%V)#I9GLrhn_I(PqD+WpOPMn*1CKCD!lvN8I`bbm_g+b|B%!2 z*}m(Hy!G-X;rj0L{U~MT_&ThgAHEC0hP~gzpwD?+81Jf_N1R zQuFS=i`PshUR3KY%Jd&2A*9d$U$CHT8aX z(paWW`Te81NWIp?OKu^~0EW(Fl0DOkq9wjE5zvn=t1VO24;L+v{BP-iG~GnDu+eIK z!4T;K;qre_zy&#C-XB%BfqIJ)W21bJ!IxWj8^IGtv%RxNiLy3iy zgBI1P&T=!2^o@M59+;2W`RAHb;dvO^8VE?NUk(N#@&e`!z7)dKLr@KDuyK}-t9cu< z;>!7i9Djre{=sBkLAH?h?`%{<>H6QR)GksS?UZRCuytJ-t~4~s#hOgSvRjk3RVzF2 zUsHca>aJ%ijsaTR?1hq@g|1N^a{76zzeD(hJZiNsoal{%gINuH#i1)cAx3yo2d87e zt|=66Sq8{G&UuTIr`}2;6=|SN)BZ6b12oYu$2<(zJm3oQwEfAg2WGvN!%p|!5+H|X3x_VVLS*cnQce#u}gQ{W%pn+23tmn z+(FlTTM=gSkH7CB`_ROpIeKclyPQ|+} zk|j6Gby|NhbQPmF$6UAC){b43Tqc*MQT_UG+Eh@-b;*r@2fWX<0x(s7A)=XrXTKM1 zg0}wi5#{Rsl>k?kvZxRtl<-ok zf3*P+d8YP4jtic+^Uf1EVIQD(Vq8Z!h|Fqt&>H$a#K}+V-Xe!u5u=5C$7{Nug&iTs z^=R3qB*(C_y8DScn)%!SqTA<5@}9X0u}|K|D{*(7h84cVT80LP^mSkX_nunJ|I&XI za>!5_s|}tcT&i#)j4i^5P#A@6xYe+WKgQ!XZm+~4Bf!y2iOXX^JaXQHH{Q7o(Hhp*S!Fi{!*9EE7IZJ876iQ%hk?EUSZSm(wOh`J* zUI>McR$(q_xbT};vRp>1?KqrKaGvE#(C{KUO^MU~Nz z@GsbL;jl4V&v!2C@_fd$Fm(DJV+s&J-Q4#{W@P+g-43P)HqyQD!k<9{!+z@7U>!^d zPrY98oM#L&Z(CQL*WM8n#&_?RI&cXn8wM{+8U}D=%?oM>b@e|wj3RQ>Hh#NZrlm~U z`gJ4Z{WyA%$)L!f7)r?&Tl>k11NJ zTJttmw&wMRimd%L%b5Uv8&$=v(I@k)gB#AbGs{TQg9>luR7^!OIWH07SNxNrF(xx4 zU>WL{SM#ZA^{gP~#i(`b@MT%A8(DY~;=ZZ3hXPP(YZuR{d zndeH?;)x>xQySD!YM%fj3`aw9l4J&53_V-TK*_)_3UYw!SDRB7_xqdGpwbu@VBZvv zJouF@#y<3q@Ucr2(6$E|!()A_{lr%B{Y#uM4rcaDxgmrK&}K}XHEW}`_2F=KT|Mhy z^ih#n?{${dY<^Fu1?vJZTi;PIe$)YkR{k8; zuSE!Q?`_!>VB6E|!u5o{iO?M&975qG3Z>*7t7f$ii(pc+q#SS+DLcXz!4o24iXxBi zc(tlzEub5Xt-_NUDX@vhqKTS7?#HqG5u=zuBs~IAK;AGh|3{PEbfpUQox*IjEZ-3st)7#gE?OHz&%Vsrd2^Mqh`Xq*xb4w3-Y5d*_p^Q#xm!Bl(o zsNe=-xwsktKv?2%3RXQxs_#wiKJoj^Vs-1Ye*uJY4Sner*5J=SN*K$l8h$F;bR4CT zU!KY+t`pOLnRf}KPLial{t~*@iuO+P8Mwq9H*Umvw_80-;`@CF%$mLq29jkfHK`b7 zt#Ox->aHC;`mg8bE{T8)PS?EQ8+_byg!E2?lh0gdEWSYukSX78n125!G+qU@qXF&0 zxF9&U&kcds1rU5Lug*HoD$BvB?!2skqssoF1UQ}kDsf6wxBM1@)?XygPGkbqc)9uG z+N#dyfo|gqI;ugu{ zd-~jnan;_j%Xd!C#z}>xy?juG+BkHwD0N<|ZtiD%fT2&Dj8vzPFJJ~Rt+(B*%T&9P zvU&zl4Pz|mF+H5L6g2c;YZ+-XG%Vo`;XVJ(yz~N}WH;mg|E!RUni+8cayq{$k(uJ8 z*$7^aEZt@hA1u%=+|FL173rE~jMuH#tP_xsRNC+>K=?_U>r zrdt06R>qi~rkXBt&g5y{Q%zuiHqh2YBJ@FB1)<`BVd?u1CgNmlGoSISxaJ6dzgCo= zqBbd89Q#a;kd6TVaSk1nD6WCLF~uU){sWv*mGMOtIoZP3iCrl4CN`?O5b6d#_d01J zcGiky)zvykm%ke>dpJPBNXysaixD75_0A&e%%OB7C+8kXtEv}D!W5%8wg-A-Ydbgp z9pI0gWHW79jxay<5{OkFS#44`JoVg;uIqY~-l)2p^lM%j!eeNKg^XOM>9!C3WU8ce zonCN7jGTZ5o1S|F0=_ubL0G1p9FROh0Q8Z zMW-mW7!X5{liV&hDDS@l-9~F#3q#i++i8dL_a3Ah2wy@!16{$+u`^Ip+RPN|5vXj? zrS01h)yh*9BVsS;^P^H1vTul~iHQy0cWVA5Z29y>laPL?C=OlmnmUl|MZr3yJ!O}S z53^rqsA7W@OYtHr9EB-_}Wh+EePLFXe4Yn*&VgB90xKC@CU{cxz^D9m7i}$3#PdEhP-H<- z@2bnl_b9$Zmg^`t28|eSmk3@Fue^Zr^i$hI*|q zz6PoQZqR+>$DKaX5kzP|EDm*i65!*&r2C3;hDicBE4S@uFr8oa7*25?I@vP2iM({iGohI@;?ixFr^%qNDJz zRII2j}<95v@!nR%NEHgp%WH(#7s&fZ_X+cp|;|$~- za}w(2Ny0M%SbECvLQz?vbP++&YtTUldSER^x&ypaMjoo~)qgC_B!q^JSHY^)^Huc0 z2yI_g&hPF_peClV6{8{eM(inf;lGvgyaaZmz=`xI3WL}{z51g7g@$Gkl2dAYR`;VE z%j_6{g+s+g0N>ca(3tpjrgjh>W9SJqW|SE)HcA1)P(RY)Me9*siusJADwhAR<-hAf zPwnOAjtn(wbM2{{pqHQnr2?zRkMm{-)124(Z(rU%@o$lkR2K!+{^~l9dTafrHjY=^ zWB)0JWdke^^VOW)^7X;xq?*!E0P$+qg7@jY7r;H+R9!QVPPTj9NlWR?9**`%#~XtB z78c49`tr}v2+wmGl=k1$6_S#R7lE2$IT)1A+!%N*vwyR;2On<=Ki>ZV%v8UwCitG* zt*bu{T#1Kfx>OxIZL9Fp7q+oBCU!V3WYrC|9X-n_zsUhAX^l3yYD?jUD;G-7mY+kR zE&=>K+OLF(mb*=Y?iffdjZ!aPh`aI9_x#VuqL0qbC|*TilYz=4XbgWcD*9q~aT({W z%j~XIE=|yH{sZaY+qvx2zyhFMVVEaaE)3kgjhd9ywO{Z0zif?>B3z4nn*sI3;wqbW zK?Kz~0XDnbmbn~HRZ(BuzdNc{14=^ee%$;DBLRp!244?P!ONklHxpAo5F=oBAh*9Lv+KBbUx+1qHvF;ROn| zGtXWGHi`d|7j?LbVJ>LCtKF1bYl+#AgGt%xg$#d(0Xj-9D6;TY-Otx%3$-{^u${t~ zu;^B>1yI#1n=O!uk%EC6sYouo;UrZ1G0&HCTW+Gv5+4A8#{4Xb1IYuU(vqfVS%T69TzJCqJLVOZ;x<2-G7A zpu;>Kk(1@t>l_*Td{fInB}jhlV-C0Z&zGwXpw`XhBg;BtNSEKv628ZNeGU;{#xz4q zRL#=y_?<(o7G=#O)$;e82Cd)M=$}m=@GNWuk(L=m^&New>>{GKIfL4-@@kajr~R4K zfj2Vk(*tkiTf|=G`F$R8Md*bKV}(rWLgtF3WZ7=ci@AYjKuil(zl;AvK+ZiLOHWb} zIgw60t@7^`41gZ>q+jS5ZXpH&Id4uDD)sqI+2IOce+dP<5Dm~Q2AkLi=scAGx0gD zsnzpHLOM99^RC-6wLau&nO`W%eelo!7ojUC^4jLqpJpmj4&P%TWQzI=p7m-1-Jw&~ z#DY24ifEI(-YZTgw?}$CH0Iyqt^8xf{2xZg?>@B4)W;frPqkv!F#fOv!@Mu;t)65P z(MIps>_I|~SUQ%FlSP?Gu!w)AZRb+@WuNKolzOl5t>Et}y}o62z&0$7fs*6@EtSRiY2|-i6=kQcd!ZpQ9+Tn~3cu{d>lR ztl4cLlbqr9dVWEBZ)k9DQtqp8hyh|o-^<64eH}-!tsJ{wXuT5FuMH~Yf-h>!-YjaT zloWiGdeiX;6YngH!RVrNzEt*!;CQ47d>|%o7;} z1CokY-}SZrQ5atZz(>L3Xc^erTHVThA)t4biRUnhJ{vb{*PZ0RqryZs&OGr=X))7x+?8}8 zjxOEupX*|(S3>?2sM-w+HSfL1@voR^bVbqVDvJO&Mug`#?5z1Grfp#@wOg1uK>@n= zKO1A*pxm&TPz)k%?PpXzbW8A#>QZNTcqrY#dxo=h;xH}1d!Z)w%rwJ9fc9G`RToIJ zY1I|SYp4I({k1x|`*o@F#hiz%jqlQ&xG)C}<-O!A@n!MM>#Rblk>rx5*PyM^SRpSu z@_*W+7LjFyZZVbj*TIotB7q{?+MBt#8F%V~iyU9QlJd_xQEW~MCF798!vmLNkXJDI zH+_|RfK1mG%eg~X@35oM9vYhMBO&w!F&yEt;TW23a?+^*gU!Juq@I}G@8x*uJv;R6 z>mAoQ?`XPWZIy}^yw^*!yIIN{`=re9`|vnM5<7mbvARm+hQ0{fy=YXMu86=3jum+z zNu84G$3a~FAMZYW@^nfiMDc~FrLm#t!^|*xQb_9^Qh)S1W&r%nyT)eD-5{;A zKXb@Q0^d%MIljPq^=@Kj>r+>c0|{OZP-2&LQ?Lw=NuVsk1S!8;PHhV^DCZlF>0oO{ z+hC#TIZLY{r zNohp7ly0OY#uTI^mF|%4?hxsY(cPnK)W+EUx4-9j-t5h`W7qEcy3YHIPaxNDn$R!8NN-A34UOO(|1qW^~wbT{RP0{lX9{>?*cag3JR5 zEBd#A10*{I_Ac17z&Y&r=ocnDMb^!46{**70-B&C-z}6#xry$)?(=b#=a07`%LtHt zptGQLJl=xQ^ykN9)T4>AlBWS8I=~3OUQo*GhVL$!`eB<38z3(8G)IHS);yuIMoxB{ z?vraCe-J?gbuuRUWIvtKfsjR@m^g2T4*}}^$&z69ISrf3s@4y&de7#wZ)o**3>>#b z3SidUi>}74KdQ{;Lau(WEe+}}KDn*jbh;h-!BNm)y;kNh21e*b?dwVAyL~VIA5M-v zMQ@s!bct=zX+^6YvyYIi1r*3#w|VJ6Ej-JU zbRlwaoN&u*xo5Ft*KMz`^R5cj=P~v7%|+EK37CN&mg_FDgn>S)P})-sb-VK*UxVvK z^4+%3LOF}SYYlbQpw@Mxyo~*+B;(oLD1d3Uq8FVdD`Zden1A81G?t&KEvVI!wq1a4 zVun$6Xa>>fB7Lh)&iePm;zv3gzR3tu{x1e*ndS@js(%Ku#y@;m2aEDaNHeUpepJYv zeA*|-=7@Y!`SJJzb2Mh$X^ux>F|ezQbUEE~UG99uYxS1NV^sMWH`G^?I!L>lou=Nxl35z`*72nUmdvOht z7X+ci97;Bo;j^yQle49A@5GA`#R4%`sDYbuP+4ZFG6|$Bw0byuxW9C z&{gELWfwKyWW!RAe79t``)hI-us^i&V*tUuXYBUmm3p@Wizltt9;kr;KU!K*gze&Pl$Kw@K1pS5f2&@~K~@+X z>iA=?QJ)T;w3>8r!e{j~!V9>EiFSs-E2TXM*k+E~=Bbf0O==(hLP+Mym&F4N9rQ8d zl>BL54l$mgf-`E-p>ZXIz~s4EFI5lk!Ws47Vb|5F^r*ARP%Xzrpm0h!H7rBc59@VY2Vv*!Bq9<-@fN!NvAR!Fh|mZaqN z6+_k$zLnH?Wr$7v_J9o4QLQck9!;p@vsRK%t?CNln!uNh1F#3zhc6q9K7VsJG`3zy-kP1_V5(Je6-ZK{^CTDq~ zl%(NV*z&o*sUK;Z`6y{jJMLmukBmt;>h0?ny`n%w_*HhAB9DtfU08~QMXS$w8|pii z=YP`Gs(e2BX_Dx5ZROp6w$sfX8DVkvdfhV^sYO}gX*CqlgTXxxq5on~FtE9dRTx%v zOXmnFW}f9u)a%{e=t#B(In~eq<>V_44S)3}Uxr&mlxWY?KX2M1Q37Ans2z3A*L3k0 zR#MAKrt^G;8rat)#z9!Ei+fAAhm)q|{Y!sqSMcsGa;FZew>biAW|ALHX=?X}w8x0u-I zn1r-zF5&R-I5}*cr>?jD^zCfb8Y^R&dgS$R`IK)aTjrub%v%XqV5|hN!1;{cBsh@z z#0Oq48f4I3>ESJswW%e9FhI3+Oq#rVlX@V0qTTop(D)qN-OJ|_z{5h-e-IECScFj_ zy4_Qv`*_qab*5^Bhdznuu}fwW*ElDCACdz0O`30*LMZ29`seNs4l*iz;8A_jsEv_$RErff_^7w}GI^&(#J>bvAUQ-(yTbjye` zAd;+Fk|X_J=eqS1LTZUoBy;vdrI}AM+)+VSh_+x9UwcxfJ8&&(mZ9v00?0dj=NX*w zGq-j+!9GCsc`aylx9_cJNa;AMlMMJOY%hs5=O6gCpR4ai$UfgKJ2-qEaDklOf`)28 zo%QoLO9H+V?HIM^+Am1G06#Br9{aMR52&~P(&w|@8SVoTUeT()(Wno0^GMK5TRQV9 z4HtCL+^y4XMK8hFQ-2TI1#z9%ZrJZRLwR1`NAZxc^4A@? zDNqfr7>oX5Tz@bj(~dfB8DEmZ7M@nnI*+$Bj}nf>ve#p!_3^5{M}cRaUBVo>x{P|8 zyvg83^1_E%he^T@Ka~_>_E*Pd{j%L=gR`w=_xuSbvKKTa@6WsJAMxe4==x*CV>S3VF|7>e|g| zaP525q|LO1XmjdSN^wbGUwB*eAB1srOB2mVv~oHO?+RHC7YbblOv2AM<3JYW?Qi|} zP?rzD3U+1AgND#9wd!d>v9RSZSyPp-gYS^o1?8@-LzoYtiOg8_UqTK-|C#zeNqo!G zkaoEXP?e%-(qEi$p;gz#G>k5Ar?$1cdjX*L+8a-~3-D~Z4PD6o+hK*=cjahO$*v^! zQ*bFxQ<`d|ZXOt`b?zDoJ@7o*&V_NsrB-xIz5a`6C_jN_14d6#X_(C>_1}OqASl9H zp*>l+bERCw`S`@srKHNFzqjk|>~&LF!=l`b18j8Ybkty+Y_=PTf6}IpnLlyRueH%E zG}1Or_`C>FS7L=I-u_C!PD^5#ij%*m|92pDYVpieNZtWhz8a=u_^w1?hM);gOk?9qXMz1mvvN7r7v)oOaYuvt z+K@Eo){OF1?L+W|<+J}Sh4P1W{%hlz3lPNf7vV^U%EO-1)faX*%LHo#PyNmXZoc9= z?iyain4}U5Uzx8>JUaohwrB0Ly_bfGOiHYrLDuu(uIr)Nk=y<~vAUw92sY;JpJeUI*S~YML^Af8K2LsbviMU_4S$F~ zphItUic z;~*;qK4CzDEelWlbdWshZ(Ng~6kMLn_bX(4Bht@uUwtuFGQUE$mZPn21%Hr4?1;_z z5wz2%wSY1`So|uB?u#O~Sn#Mbi$Bx#D&+TFt@Y@kez{~H=I(lBW+9~1lCy2?^C?;1 zP_~3GP~DZs&6S^@e)>EVi3)C``MC$hT8M#B#cwN#IXZj9+zD+BLM z(|%HvDlSx|;8u%!;$~=9BlqkCGU&~FSfqgKOe%-$koFo}^`U#2>e<`8eF43|D0;7D zq;AzEhCIwrh_8t7zxMDx{OMat?Ti&W>?hB={U(vuZlYMfvE1p_&Lw(+>x~MNE?+qj zD;ixqfxJ(tXl{mA_+SI?MTn6R^UzZby#p3D!10^ggDet!oVIVca;EOVS0zd2a(+O_ z#$ays)srtbtC(<5xmE;X=nQZgBHMjR&F)L-d=Z{C;XL8ACn7WR@X=lh*<=s^*@vL!Q z{sY&I$cdAIGSWL`6CZ@Ce&ZXYes1i~LsO7|((zq`*os}vNbdEgkb!T$$7IX#+ZSUK zV5-aZHP98JGq<5X%D81W{k_n@pen7s$6+dhb;rYfI56eMZRYO8$^t#?N$A@YErBo- z%$MGC_QA}HJhTclP5Vsps6(Hgst}Z&+x`7qfaUq8HA>?w3R3^D@gxg>o>9z@x`yfV z|DBXSbY6&M;}69um=N!Rl>+O^RPgvVJoDV*^q_TmI@*pt@q9^3X(~KxT615m_pTV zHovb(G{l~Zo7k;XWGZ3rS@&NZG{jj7PITx=@EGL}k7B=>GKkr6s;40`{I`3uUBgzS z`H0$S z`+M9zSEX!-yzcU9kuzx88BXS81^G(N$Q3y@tZHZY`A4+8ihrrfVFrMLDY^H}GLl9+ zL8}6n+uz4{Z+F(l^tA`hI&QD*0kzB}nV)Wh(Lf!LHf%r*WTf4zn%p}44&S*c1+->X zKgXV9TZOKe@)5}lz?)S!FjsaF%+rv}k;{V$FGacSCy>Xew{79JlU}AnE~wY_fi2F= zUt+EFDf4oU<)iL)B}6(-cu+ALmlii1W5GM10<;2&*vH$_@b~y2|Fjhmq`a#p`$fOZ z(*dmbLn+&XYA)I|0bsXb35FgbDL@98mOs>&-j(2IF%*UPd701kq7c=&URj)YqO zGUJ7Y(Cx27GIvy_apPaTeS==^6WMOwd>143Jc6k(f~O)_5pZSoS5J;dByalY0_u?) z6C?CO4RndFf4MIH3m7|kB)xgTXf7Ejp*`68=7ITPpn4E_HqNE&@6HfWQ$-v%$1RRe z$mef14@kE=?bTy^hWKNn#6V0q!ROV#t};z0@Lfk^bU_`99LKMqi_NOYh|s29iPzQ_ zbNU8?2J%;6gaI{X;6*a1Vb&#uwO(rHt`8J7*)g}8NBPR1O$lRea_IGkOyF73^0lN? zf}vAW8p;hhCj5DYd>tRKqSH*Y5BHWjP_q8GODQ{k7~Xrc4sR^9e!xN3OMuT1*_+LC zLw}<^!2HJ zgsQ2fkLW*Kn-8I`8A(@H2`^=^JAX!+_dEdg1t&ZhvOC?$L$6yzXPA4jJ zUEL0C>c7tBjh?tIo`xm!+h>S_J1#9#0$)N@f0`mHGhLvi)r-EEf%s){sl(W>$z$!rW z;DA)LDw%xQB%Z&@3fx9iu)L2mE^LKo-fb@sjYc1rx{mw<_KI3Z=>Ixc^FY5dK0b>y ze*&BOC*ap|)fh5SXM^rQVtlA^y3BF0+O<@+4m|$@Ce+%-%P{f?vq*FtSubsWAoFRemrF+kuK_lyQ`$ zNNTv!@K?GYFw@tn_ton!{upb!DT*2@7qKwkM?im0bTPz{dls{fEO;gS9^3564J|SE z60r*r`2E8*ro%6{!tB~_IVzrubMX3iD_)5+JEZxgvut*LHqK~lx>$WY-lgX+4hBEX zc1#|Zan2<5`zY>!Vd#xYijB+1avbvo{4f#Hh`kPa_qbczM0kms@Ocs}t;{Y~)i&DS zr$yBH@xKmsr4VqZBxL@Tj?fv(PmUkyzetTu3UycUuXlZPc@{t%v>U~HRizGam~}+Z z;GOR^xp&Ps$nPA^ZJ1wc)IvS%C+(Lhbj{d$-m{W3=@%>go?VX7{oa(O_N*;c*l%ID zkh1&O*`rFHx_)Oq>;5sY>HrmDp9V^Q`BCASrX1#495wY?ds=fs>SSDOcijB-n8Yxi8j` zK2>842w&{Zo*;19i@5OWPCF1=EmhnPG)GeS69TVSl%uI*x6GqvzReF4w|$9A?eY@{ z+rG|p0mPFON+lX3F_v_@HeTO4k+L13F$zFoXfss1#?oeQ7qshG9GK~Aa_XS?thK?* zwC%skkM=*S4#FN8DsOp87`M0Q^_P{&;x&*X6%R5$pj$9y_Ca%;N4~Sn(SzSuGhXmn zkK`!YHlg;<_nAqD)|aml9*4LQwzEd@7;F2$(MG}ibcs4BIz7EU1ljE3-yAWW7P_C- zVsOu7QI3EAaD18v-?)yJdK^9c?fZK+xvWvyKQF6O4$fP6kz?K;-K0rFo8aoau@29Y zeVk=vV~K_u&yX)A#w~H1hl#X`aXj^zHoo#NuH)KdiEP=P=5%}H8Y|moQhy8Lwft-7 zIvb>*)P`50G<&Qtw9L#aA}gwNV$TqxNnS8$YMz;3Vr-C0Jt&^Re@>m*8DOcWOdGB1)69?D^7{dO7_|b3lVd&k zyN=x`G%+2yksKv-C~NF}^Z@i@Q{6`4Wm61ig?6~W7N{WOZSSMQ{7=4hZT%xY_w0Q0 z5{D3~)@&4u?@rR1KY-}3>)WGu`gzqtf8slX&l(--CxU;Lryu~#jJtf^ku>MQ^BM{L z40J{NS@fY+>~e%VQd+E2I^{@DSK18{)Kb6m*Kyp*nt(>xP3lyb;<53+bR(JIDIfO8 zy@#{P$)AmW>%gH3*+ZQ>?1cHq^%5@G?kk^Ow@Js~hw&qUct@pj&v_>tgf3^kD9+Ug zYl^rZVOug>Wv)(>5x5O)wp%}B<3)#kbKPPYQXIN$NF=`nr%^;ZN!$*s3H1$?J}}=d zck_LhSeI&j_Ov{S`{WVBxd*3@ z#PKk)Z~Gm|)`x#%rVJ4v5W5Xd4pyVbm{y+g^#3&r=JaoWx_@@+HrUNWDZ=IFLcdE$ zDPVVxAy;LS{dE?aKEbs8n|P@+0K(nO&nEYi&&Hlo@W=R*cj~HOXZ4+#LmzDqJ;dDG z)qK4HB(vjeasbwVR8t;D+J-5h7e6r@1g-p#h;0-rkPf6!g?bzU@^pD=rC|Dz(+<48 zRlj_mKS@{j!urJX5j{hsF8!wDCBxp`i+VCTX;`9X`+lXtlYY|zTNA9SanpvoLJQ(} zy+E^367gdl>apE{{P&FY#RvL~%{iDpbf_j>K`!UaW7tCl77FF;DXLD@44bklZT$0P zIqp80b|2Tkm2`73{9>>jd8&`p=GLGn@k6Y;@j}kw(nh!Sj$0Qd(AiC3$V^ho_Ph#E zpF78BJ#5d)I#njJEvYqjF#p#Gj<0{_SnB3i6_Mmwwr4$*ebQ9&W2wOFUEMnd;+-U- z!_-|)oy3Z4FQvUH^;L=KdTkDG%(q+1x_Dh4wz?jyLn4*L$8utcek}~TAfuQLy07?n zvn&pAtmSKlntMXwd)$3_+#kuOx;sez{CR{76OCiZ*BQ26(_ayq zUi_U~Rp(F!_)sos$WWd2G4~L^9h=y-unYr3BH<6rhm#ZqlbDnIvDo!Mu7tq)14iPZ z>!Fr7MyRlFsrab93H|NBj3Ya9X?yMb7gmwGed+Vr)}r%HBEZ+>crOzb8H2IDh`BTr z{H1yY)?%Q?gbnK@E!Hl42scu=$9tmR*UB2e4G5FT(<+H#nO1fJj!(0W4xAyv2l@}n zt2%)>aba4}bO=luoLjo|7?)Q>$d z*Ar9MX-*U6X#ii7@aw?Yd(MB<&=4Kcr+;?gK}UM}hBRVEM85RL8TKuA)Ky1m#a`k5 zGPf2qPMWmwOyM6#38k95+Fg9wA&LO`f0(_Ip(JW|+5qu}Ct2t}Ir;(R*(sL(E?P8y==1{{nDxReDU|j;4542 zapRRU=*gywYY%4w-rfzbTZqXA(FtQQ!b^&mp3|9Fi3znazEeXf)Arjj=$5Ivq<#t7 zbh06Nswu1UG0w zw`7DcrajxcqSY-I-kTkwT*Dub zrSW^N-z6*S=Xn(dS?jiVKJ>p^{Cp1wxQGxjSB9fkJ^k35A)?(;LkH%>eHqV^=GS?q z--YIEU~9^90O(eG9gMb8aqVH8rPtyC^#x--AFcb~nvB1;#YFJTo;7A;A5f2_e(EUx zTY!gEUZF+ynuV+1YUyJ)g%A)P6VTW41Zf4;NNDz-r;FG?lD(Hu@NvXjJC5a*k4TE# zBLmbe8K>nolu54V3UvNAO`|!#5BnZMYlj*Qealm%ax2xQ5z$a#Kxg+28{}71vYEM; z=yOIXI8y&Li|0Stm?*Uw&$gIP_AMaJQ}eA-Q&~5D*pam0mf6r0<(3T~*~`EtalHz9 zTZYb~jBlqxCiw~YguXV?IEzj3t|=I3)IE*^b$KRHsZqPXW8v(LY#T{Woz_THOD+#J}he>R#s5f5p<4J>W=RU-*o z+*cOFwYi>mi=;|zbDuS*_E=^BvDmw} z{NAfw&;--2h0iu>S?KtX!uNxgJd_&a`wswz5n(fSh0L00a;0s8AllwD~V z^SqQ}1*J5s99x90ljoCgPQ>i@%2DN|lH2U+V-)RBO4)36n07K6{^I4^o_~$qrB!%8 z_%{7W;2G#tbQ=~0R?P6<>l$u}+T|cCHm3>sDH5L!kZE?OUeYL%ojpEooKr`lX)GWR zW@OXVf0`O&q6~dhe%hI=5HZb+@rDy^ucqUDgykuppG-WA0sXG5A(Xa{@rwD>{w2i9 z*ZU41d`{=gCWABpjYqY1%Beov$1{*F|M2ndpD8DEzB>5U6*6}>9OGsS(LMqnHl0?} zTCOauULJkD-l$X>TxxRLx>%7oEJ*u;Lr2v2Q;!@IBC;Nt!zrw?Xx=IZJiP8leYq4t zejFBC`LX{Eyfc<)=)BK*=iJEsDp$3zPz~$0R}zJ;h&nrIMXXm3wmMJCr9Ec+8^Nnt zlB8>7U(5bGcZR`UhMvea+NP+0U!;)>75~~0gqpxR@-^%j+9thAzFvdJ3cXcBo*N^# zZ(XW)@Wck~?46m0xW?dbCqh-yZWqURz5gzrZ+1@vgd6VhZZ@oLQ9-(vw7iztlsU^9 zj>d!enWN_MEC%Fmn+H+AK^m%QWTW3G5Cmq@nJiWxEt*TB*{N=)Zop?7;pYRCrno%I zmjNFt;=4X+hL)qkj}aGUv)W)Wmztg7pR^}Uy5%BRaZ0bUayD#WAQgUZXuQ9}VN7k` zf)K{;=loTo!@Mnd3rE$NDA)%EVX%ogLjPfX_TMbmT{go{R~ys8I<8^YLaA6vg@wl{ zeiDjt(HF$gq!Je~Vuf9|Jm$R?de{o+o$>}EBBJ-@>6#&=g=Oes4|ae~YDcK;ct3u> zVA;BeuyHxZa-+AnhDj9+>D_lxX1sSaDk+Ge{EC!6c`OkBxE^&mx-`0^G-9|-?q(%r zZNnnx6XE=qyR;Kq4{~yETNbd}cW;rKiw=MR@3~(xN?U!e>g0GbJE(oOKFJdEh1yT( z!9Km}O{@k*d=MYYKxzeXd>=zQw}&Pnp|$qUnT@cK_zqf!#<0=~T0%s|E(dYt&~&E9 zOjKAo;sN;_;ZD?KU&gquj*LlTgR%Zax=A%5Y0JwOPcc6z~s3xP^(jW%v zOjY)c5k*5^1NXD#<``-NjjMPi0*fL@zv^~&P!cm zuZCKnJ@=^b_*c$0ZU$SuD`MK>oOHEQ0C{@n1!$yDK_)1KLa2B>>C{lFr6gEe9ETU_ zV+^NH-O%Qc>lK&uu!`g5aaO#6dAKQO$*x|{m9Vby3tG3;AZ!uKr8bd8|9L6SNw2w=ljXTC&z+b&s_z7sWfs&*-m~AqA-#p zMxI!QY;(&siEJiGfJD}iRRt<=q)$iN(gD=d;6zd_B?CK+WS?7X`ZNrpwRbC zD;?8MV>8@CskO$2zU&tj;4c2si4r?*d^wWmdW+~H1uIsVv<;4!Qc?krjh#Qh1UnPI zK4Z68etwUrNf62}l?L8JT2_VcJVl1and(Huf&&vfdmh}Uf+3qfPXJui=DQ1X@Zd{U`7sNhGZ|#whF2*35j>)byT7YF&#*%xPgSj4=>u09r)e|; zi*8!h*s$HqOh&TDwMHB9Yc6`Pe;`IK@bb_v3z;eXaz4K=0EE6z>+u6h!hn+o#(tHvF)Tqmzc&zNU zZf@Ro0_wxxBmO~aX_cgr571ukmoT|7?l^_uRwny~lgGN;&)Vj942NR5BPK*djA#U$ zo>+*xuS$8ZCwDTLoKm=y5M#RfzeIKOFSy44v(RpR(5FB>K}vbj8{reXTugDMm2I7Y z0mG<)8Wj2H2Y_ojPn^ZRgrKQOLqg-6M1^o@%0gsgnzLV2btbAkT?LQJ+th*XWlfL; z>*Hm}n&mt&gWVY)Y+GOb{B*s-d^BSO<8b{NuF;`-C+~mb@Lf`D-vx;t39&#M23l++ z6<7>Xo@}+gsxzPE;&tsOKaUtR{7EVP!3ZmM9HYD9Z%8qD&|TfVuzwBi#^ti^VWn8T zzgcwOXieDv!OgnRaU9CxFYc=P7C~Emlk44Qy$|=DkHlX3SWPc^BevMu3_0RxPC?Cy zhiBI$D8@~3`kv8Q=|yOWJhSzwDVd=k)6i(~&tLuAxdv%YEZQZ)!I(ze{|z*3{yFEx z4?u@;99X)TZB%HU+X$J`(R8r0nnnG>TflpmrTQziAT8J+z9!E5(~p1vjf#Gipmp{q z03VCq@;F^?o}5!`8g2> zS^k6!=wTb_e2Z2Vhw^aJMVL6x>DzgWWckT-H!cbz)O_7g$JxYx*V`3Jn5Em&qqKpH<)AGLURhiWU5B~b3OrfQe(!Llk z=~eTs)Ue|MHru;U+Wqq$Zic|5QES#-k_S|}e@i-!Xi8fX%08$r z;mrf!v_==1EWce&h<+@`0HcSa?SD;=8`uX!F9GL6kY@DZP(`1S>35zty9aooa21ke zgZ*|J-+`z|C1pJTTLgGBgRn6NvO5xF1CeR0J44XhDG!I6r?n#Y+-v$Xp()QStFa`~ zhW}c#fHn2=iLuSMnR}L)^vTx*NEAlf=Cf|m{h*g@zmlapON(`JXd zz{A!>|LUNP_em8$!AG;pQsvDUlk0ksvOYSAgE(f3LP=-4%LMxwCG7IX53&4TK*){s zd*x6Qc)!Kn&r!Tm2gxIq_68&1;e|JX1>hd%cV4E-2le)g21+0pf&WaEv%a1QHkq5F zv@)mGZh6%IFMgA$|Cw#Rgb6hgcK(U>AY!^N`E0ZhhI4*jbMkJF)4O@eRQAKQJ(pZrPpJo=}%NO)oRaK zCa+4vtgcS0nLmnd{ClMr<_yHwY;~MqCJgq2T6}3YI3(p{ttQ?zGCV)eluUt2N`-Gj z%j^CO)tn;hmgnC1Hy%~uk++mZ5^mjA{^l?eTd8(cyzEkXc= z66cm97TQu61D>>gWkH{b@8E7UI6;iTld` zV@GdKQWuXE?+UH3juFW*p|N93HGO{xQ*F|FVI;uwcS;X zcPTi7bY;74V^8Y;YXNMMo>j^6dkh}(>=)$5yN!w{j82K@&JLw}t+3spE{U8}QuyDU zh6iHfT{ps52P;2y4KEW5&Pka-E~iD0P#xGi=e@-S=hJ)tw5D=+Buocmv4`e&JfKSu zn9q@I2Kgkx0!9?s{+a8Z<}g(>p}sxImwqBq1va2dvX$CN_Eui0;!LltVd?NgO0&8D zc-}9L`#vt&LkzA}PH?o_uU5$mo+iXDf;V%5->ERotJZ0u&3ue#YvVU*Y&WI!MA?4& zrk#Jfq^f_Wgz_^^gBKhE4lg(=s*6=IH-RFr%Im zSy?oQZ#*t2{E>ihJ8FZk)pLjaA8aA}vsazu^iM zCg3iO*z1;Y(U$xIrjt{r{T!PKt8)(C`7-uf{o74ei6}Zm@?`Dy%9ktul{L$vEEl@>6oNbZBdnRDQ4p5Nj6nC85jU_S$re{+^VR zftGqYh-bjIaU)}*k1=dHwjoEF*VZdVfGt(g)mHFL&>3QS@HE6*{dEF16o=59Shi?r z6SRoAY!up_UG0b`DM4`%C5-RC=g%%c5l+J$s4FpVryBy>rU3_iujydv7o*!Mbdqq5HEuBC z^X}#tk#^K7B8$FFHTc~-8}8@>LD}E@4s#kadJ$rb#wQCk&Qprcm*IqdDAVRQ(^=Bh z!hcShCG%Ae5zI6i!#K-7eqob9^-+a>lm7m5D7-p<)F=5S9B?M@!gEPA-BpH05vXQn zK^#&GI5ex~5w?_+mDHJJSrYeZiT$#wIN}4y^xAD{=YP*OonM#|ZMC>m3&yQdmLl9r9)~Lm4Bn1uDyDCAjzt@E z(;Ht=+?K;*biU5T6v*yiLv=IaOSlg>wDQq0BCLL^!VQr4A@6t<>fjjAY!Cl}pY`)c z%8Mc4cx4wyUma%+`WQxsC~#Q2<`@HZyOE@x#`>czTX^#pFis|*RUW+(Tf*f5r4L|m z7+2*_jpzHQ;%h?i<7nEBLu&C4J}^V}UYHO`2qCh04a__Fo?KP^(TzL58Pj{Y>nDfJ z%tzyE{NC;Zr4v_SFDR&B{I%7Xu}o};IQY;-CbnC&;R1tX4MdJFG zlkw~?M?w1{u$Y*R{Qj~po%(6dm8;Fy`aQ0XDsWrq>B>FVW$i}v@jpUAh^MLk{8WO2?{o-)6FW^u+AkHsrbX)-rhac7_E!{6V?jMY1aSH9e02kh)?zcX@-?Yl)X z_^;JReDS#+&QG$+9E^g#j`b%u_Z`94PSFfeGKIf_mrQ4`G&}&PP~Hn=6})#_@cU8r z#ppu&x86s2k!uQ7{!%XkB9d6tjqv5Cd~wqmG8;#2#x%&0>SAG)A^A9fO_7xP?+ut& z$bcJnONY(xO!%PGg-qL}(1MFMuYRw~lVNh}93GK}VGbXvxu*M2wzLyBpa1h`R9~S# zm(3`A)ES(YV4|dBTGv$TyeJRNZ^-X>^#E@lZ)YVzKfhg^cTL`F9mlwK>3VMS96Dx! znOdY7*2zor*4aMvvZ$Awg5kZaLD{EHPYBX|^(0KI2)esITp086h&wJyJs*$8I zV#IcQeEjc=kkCiEJ=2O=%u)1o!uttO9G2wx?tH@^c5!8Eh7b!rmM##w;LY&Ui33QA zS^saMbe&&d*;zVpzOx31<+QrnCKgy1n%va2eqJGf&S`nh=X%=|Kdw?i`R_Toc8&yE zqPO&Uul7>6PSXijo3xoA0{sz2^7lj+bQ>2m`mO*RRBU&L=|*UEm{YkF1)S_cLjl-% zW6HL+-DUxzj2PE9r{$f0#iFy|6VPR*G4ab+iX5YG6|8Z3tO`wNWc@@2zv%L))_7KE z`OLI_?D04#$aNzmv&(5pt70O9lGn|#3hVUAqv~r^BO2qoqyA#Een01oZ=@D`Az<3N z&bO&ShCTagE&jz9UN-wnl9islVlT1#VoVn1uYQMApgy0&ah>WEQ@&mnwav`d-@IqI z!Rct4eO910SjdhoVvQUR`THfcO$VNHu&@mO=s_NRv zsS7Ca*jQ)CVA;lw4~|NQde3({M2_z8h>hnMe^S95gq8;td?yLWsvs;lW`RAqWs_f| zDF)^GjO%Wj>RF~Duo27~4rf{sd*kIyp_}1f`V1ZEuuH#p=)Ye|HXX*kwZ+JQOMd?+ z9QF5OR>d@uKuYb!SYsWDP3`#;nV5;y10k;1bsyp7<8lrKu^|5k|3GQySL24iQgDpVh%kG#4|8ezh$tB9>Y=a zrL_fnz~c){NzDFxwKB-TYkZH_=J0;9I7YPz5s<-Rqr>C^v|3+{!K&0r z_Zl0=d=1tBZ?rN1XDMozGkj~`Xaqq?PiD()XXtA8qiJldbMAY%#SZ9i?ED9R`|BF8 zWG62hmhHGNEWM&{h#%+a#-@dTk@fiKKLn_!45xf#?A`9dH#D#!_wfo<9cvX4&&LrP6u1iny2kP~aHX%!PrW8H@l5;{in1LuB!`YG zVPi4#9b*%s&L>X}q2*_$EX`k||5!-&aC|S`?S4*DlW5c-S@EGN`(`t92r6_uUI(yv z6rw)$xbU+|wOr0+PtXuO-`Hpu7ppxW<|hsI-UmCzAu_~jS2>G4W)=VVbx!lMpl z(A0{hqyEx?p(U;1NSR<~n%b7*t6>kuy_n1RA!sU5aVSAu?zg$a?)l&8Cx0vZC9gY) zO#BJNXdIf9Qb6raKnvUzHJ^xo<&D@GwCw}Eay$L#!S^;bKsw`U=?(D%8>?dR=N2sw z!gim%7=?y@=K#EB_Zi1<_ru^&N&<#QBa^ZxNlLeYf-^t44$DKF7JqpYSL;bf&j&-Se_t2la6Vp-^d`qAv(YD#r&Oe^m zR*7zo`7pgv8<)XS1i!{Tl4HQuVg+}YLIfIK&q@HY!{11&WYo-=k2aF^Lp-16{9=Qa zCVP0sEL0H}E}CtFW&esVs4S#9u%~@-Udg<7sUX`9nn;JfO%NJbPz_EzvDxK_2cnf4 z?k_(sc}$`JcxgdXeKluIQ00jCMx^2mJfhcKZ6>Y*a}KZ7G#1@|z^!sd^pdReE1{VR z>o?b<4^{`c43G6q{|?G!`FG_{@*^Y4(<2bOKwd;ceve_A)TMe z$1NehuEPQKAskssPa6du4&MCg5pGAOfUZNtu7kTrOxVbohJhg1GR+SKw!nvoQ4KYq z#t)!rkkZ%+l`3q$)&U@6b9ldk@c(Ff%ebh%@BLc@M5R(+vWF}4!XxM`A-xfo3uSX%I2xZPZmg zNTsy^VJ6v~7HLiNhpwb+!qk&8a#yC}=n!&E1D*6ealx0b^9mwCrUwJbt0&6_n97z_ z9l7HcIG)v+XI`p}87G+B+CDTX%WEN-F~sASilkdB*2wpzU#p*3ZtW%NCxBkCyz08? z+^WT`0RGp9-+vB7VFx{*nC!6(PHrR_sgTY5rJ zvF}>;qCH7LLixQC#J#j5dwMr2>GHiYSYCcZ<8!c~unv9-VnjY}XTcgxzH22UCIhO( zwDcTN{%8O33jPhy#AjF;cKH3iGo?>lgytaZH5Wyz_i}s4Z_{F)!kY7qD#bojx7D~w zP_En-Rz?(rItapu`Fko6(V1-My7DROQSVh$EP;S05#=v+LGPiGvN1a5wI?|b&%+R^ zU?0|r(LJ#Aw|DbLxMDdcus!mit8h52+nRD(wBOf)?1d=^%-)*+XYdn&8zs&m!`?pG zhYAj6VlvA15%yyS=nY#L3?@MYm&=>2OTHKJ=i3Ktjk)hmDhS8CxhLq1<+_=2S5 zhn$u9ybBFY2|+gJ+Qwf3IW7;J%5ZtaO_H)0i@2)AYb3$ z0Y)mRKNJZ#*5!1A7c4!~>_|(17-J5`<=9UQeMa0$UE7!pP#6nTd+E8>h&Vt8{IS3% zK-Yky>0_z!*l8B6xS+gvLTdJWe3YDHMCECX&RTNJ{PGQR;69{HLqGNrfRkXSCS{ zS)ID`=ilHVFylfOJo<%}{j9LG zcD`t#7jnHi^N}1Yu}Q58xuacPY>cSjE1C7sn76#wUG5X;6^=yJH^6#1X0-lh;c%vX z4nh;?nbMCdQY_wL7bAsc%R!$48-V?M2&I>$dsRUa^UTQUAWnRM`U}oDTuP(^!X!cF z?l~FB^NW#g1t96J= z5qom?xk^~JRl$yTXX7)@;IA*I|L`^E$t@mVyC`1Vgu&IBp~$?j!pB|FN*oPQl$CC_ zS%uNiI(*$U$Ca$E+Y+<|-`Iw1i+TpExZBNj%`}=A38p+uAhMnGo?U*DIpo@n6mU(D zEvu(vU(2f-4Z|2LpS-MT(OzTvafhXv(}>9A_et5?E-PB$2zt}YM8=#C&?~xujT&hI zo4+q`tn&^VPCD}I=3uxUMAyPnr(_lT?`}Wu4$rRhzbpGz!Y5>jM&uz%>3!?)Ju_Kd z1DWc3cpkSu0tnw6Q9sJU42B=Q4SR6*OLX9Yw(EVAh7=uX=tmi`yd8+PX5u$GRhRow z#H#E4vPpNm)$tC@Pz0_tmr~&DQU(o05B}9>NS9(M^rXj`Gmsd+7i<6#%Z-0ji-6#} z%^A7Pn}#gY${#zS6GVLYZVPAGMF!3oZ75_6V_fmyZdit+T6A?9gUgTid7ds?nbiV9GqgqV)4k z?X&wmjD6aaBanzW_mc8@Z-xbST!Hb+EjMruKlUv*ZNF@v@V6UbP$js=y1hO&rLD{E zC*7WegGDT%1ycmVLW4fjiTx_zZw282+>6$exAuQW1ROVBOYeHjF(z|#cu)PiK#2&x zj}Y}ar<>hC0aQrgDJN9bG4z?mEJw1h3$0(j(?dEb)IT>V71JJWT)JR5Rust4FL64@ zEDv}rfw)o%omO6Qd{wUf(pEM{vU@9g53Uz!G`{7~zlfM?F{JeZ+cF^1m}eW?n>Fn@$ebojXd35Z|}zIP?>nK_z?JSmDvs znynq@mC$PXAOls@3OGsWCHm{8CinZLqLH%@U@DQKjm{9kMNifr6K z`-Rs`fu{NqC^e)XM47$hkJ?RUmLL+R^6o z)ITQ$&by)a3K$6tF_h+ETa2=Q*}~AR4JMYJDO(GAFZh1!$ty$NAvytVBK%?}8S?V4 zCWol%i=kSCag3nZx9GXWgtp#3Ye$m8ImOXqWE8mAy7HWw8!jlluTf;tstO`CBhrc1 zeNK7)%z0mbU!DKC2M_?wvNK=XjQ<*3HkA+hS+hBcBvOxh&*-%qfw@^dsxbQD?>_(i z*n;%@4nang-2Ew5ajr3p?u`0s(_(rD6o@F@jQDG)rBbR^Ta1e}>|9&xC|H^H@#p5Y zU3$Vi&ij!B9hHb3A-dq^0{J9N#`Z(52ut(2tI}b6p$|Rfqt*#`TK;P}ZqKu8xK1 zY+xR@Af<91*B-OD0{d0c`nmEiZ(D?NU!J5LN@U(QJMp9(F(xL-w($twrxo!%N$vfI zGm`gow($Ang^E~zXe4Pn{&N%dXKy}};Ced&@DL?3`r z@xSebCWicz&03J&ofmuk@6O)mu&Qq~_vKJnb!*87Wz%`G>+=BC`?W#xEwIb!E~dy` zrG}zR`yB|4px(UiY49b=9+3J=Y zzc;DVP)h)Kctnz! z%VQMn`ml<GS^#nzj6@ah?-vY6%Hwzz`zzW) zH;#?Ug6x^I;}GirQwU&PXUpQIpMKXH3>#?^ZB5P_&35{S1!mZ%EU{Wi7l)%BaxMx_ z;L;!pv603IUd@6QGPgczLhVD}F~L*?WOhBSSR35@^-^SarDSiEW1ooEqM>{g@GAp| zs7qxCa^gIVZAXZq?3?i)aE$2p#%*#(KME_-quPzOA+y}KBOXsv4j}CLw^>Cl{djzelx}y_D*a@f zc9GY12gmgAxy9tPp0SS~fi{06K?i$Yh8YqU=uU&T1d-1m`|aStXA>@yHNGZI?jE7Q z4tBmkrb!A4E~EGtkWBXOZvEc=7heaCZ7}v6qX59P0rl=xoy;HpzN)qbLG1e~Delz{ zihO?ZC#iAs&)4631D@s)l$0tM3q%)cmOYC-f74*V(pc~N{DrgVyNj)>*$fv?W2jI% z8vubD?I&l!5#%lYuhO=faazgrD27>bEYOH=nQ0XwweLdo1{E;1}&QuhS>#}ZX)ZW6R>7>v!3C$X?% z1a&)svC_mvD)VoDyFmyNqL@m5(hRi90WMLqMdXHTo^Rquco~#NHD9q1ZdY9pwdTB< ztp4yn2ZsN1_SMPIH2pM0Z`$7)(Vf@hn-|L-T`#<7^~_t9=$msHZR@XKpm%$} zCVN^0b`_xrj#a@zIcEWy;(x&HKh@a$Lk%vemz0<;&JrijKB=)0-CDUoB0P zX0SIsRiQA*e)=roY<12h>0QEX^pY_(_SA6CB<(IT@g`m+TDX1d?L|Z=>+TLmzHX^~ zC~-TkROi1t%FUQ8I|I4<>-qtUX>soKwE#Q9i4}^AkCt4oT^8o|sU+%F3NqLPk2jEC za9_u^roY>{5f)gV6@W?8zQlbub}5Rop!(m8Ky=!T#6jjt2M)fyVpin0Y2x9S$d~1# zbD~u-*GFb$6WqKczcmufkW8{X#=v`0YX##@@Q5;6X$p23BD= zG0HRJBVs^#bd>@3Vb#aXaqi6NZ6PW0^|JCdMHFfD&2;NE<1jSF+C>jV@vr|}%#8#u zh7Nha)-8E??!#xu;RH(=7l|RyS2>3n9j=2GeY3jHVb6$sbDs!Pa!g)$I^GeR_tmD_ zAK$t_?EVo=>e;?NSY>@3yKf@zC854fAvgZKUQ6|t`u z3QwODt}Ku_JQrv&Ja%Las69K~wFtxv&t~EnZhVtKqqImHGvY-7dTfm;3g@f8OkwMUv;w!Vhu`*sfA%ZwLa@1lBYg36)2^PQ zNFFk-0SHhfdQgraf1!5tiQ+C33jbL}dbHwejDCdqp32htJNUxUL;+fEBhmujZVGU0 z`Y-~*sn+Ud)=~^M?7s@RKTTB-hp1yb4Qfr8g&x^BoIZ8%zg$28eyLm6I-NTM1DJ~a z4{mxVw{i2g6n1q3dG0A(wnUe-XzT-V!<%d^^B zM)Hk=)PSGyWY9s%$pV>#c0?T?HN)%8=eAVLGuo6!M;0dR$ZY@T3+b2i`sy7U3T4Y# zq7l7MVhPfgJEv|6Jt|=-(g<*AB|(m#vm;(xfPF2FTQ>fXB%< zMI+Kv%a-6l=W=VkAR_@I}$-~40G#kNmvtX=yUFvn~A#bn<^PGK7N!gF%& zSU+5+oQ)u(l|A?UNe3ld8DR6VY9GOq)Ju&Wo^h<1w$XVl8o`F_ttNfgCx;F)efVyX zV7OF`V?5%(V5BoH6Jbf}1a@~Ddsid;{L2hPx-J5)EF(O~N;xV#;=5>Q=^5LNHqf>D zCd6cW!4~7s=lDzj$0L7|@F|7+vs6JDRQ~ADHHKl=ISi6r8+ms9P)xm~AMt3^n)OP4 zjl|KtX$ZrPGx8Fr$JQ3Mbi0ooKkcE)LTr+scFZ7k=CX97u_#IIt#g?Lvg!EP`JaWH zc3*&AxggclTu4c0l2qb(oMzs*Dk^V!@rxXp9Y^_(v))lQ>qa@a!sw;S z3)n<~C^Do=u7~9DVI=gnb@xO2hLZsN@4;3>P?Z8-7{3Ycs<39lw0Ty=(irg61A6&Y zXegYU94siPhjcJhWi&71`11q4TU`P-phKy@*Ew<3Y+rEEN2|5t<$vK(Ab`Otc=W>& zgv_Lj%@X{gxr@xkXovETO~|=PyyBstAkT#CO-t3~_^o;-xlfc$3pBdQp*=POCo5`w zHQf-9zX4{{!`E~4RcfYq=@jMx7X$M+Btm8O-3a~Cy8+JK9#WIN-?2Mm=`PdL=Vv>| zxNj`)4FaY+%pX7G;0M5U6)hc-=Gll4am#4%1L=|rxBA!@N0Sn@Go(w36M|XD(!t(> zfgc!ox!7(}KEH|nW{m4gWESA^t$xv`@|stqX|wVvvt#h%hVgF`?a(})v&l_Qz}3Uq{pwk_C>tM@Ns2N>zjSqL zE9HY^<|8K|pLT6Keb91j$bOqOrds~XC~6Zwu4q8WCto80Aby(ux3u&$4QUctru-Nr5gX|9)~-7xS#`E9xT99}&7Z08$B_l3PTYX~@cD=fwumsn*CogJ%+ zc=#4KlzQJZ`T|>tOm?)Z0u017HHqSX8udjnGlXaI=ZAX0X9OhNDUDtyW9;XUI)y}y za3B+6MovNRfWnfDx9g9we*$zr-QI}+l`*v_k-`#LN3t0xc9# z$kL*11usAWN*L(qf!u&uN1Cd{EVg>IVV4=eD0V`op_+9n!|EjAanHn{qTj z5NxkD3gI0%qn5x2%%>*$L~atS{AKVf=wW~l6#RFlN-}r$*xjNRLcgZ;Xz(_APAcR(|RC;WUnuL5lj)2#h z?QgQp*5}J2#e7`Pv~m<$-LXjM)*19*m?xI$o*v@z0BZx{J4bhEC(RMtCbxV^LX^c_ z8I3!@zaY%5!dCOaI>;^ogTCvsD=G2)QdOmKB5Mg}mt9Zl-qYO28Q zhbDy|^~L*G57To74@qgI96tsdj~4npt=9{@$|BzK|5?-A83@{c3*CKw_h!pGeGvML zbT>bHy33lv;;RwZY@zXE(b3IyeAk#Cw2(o)gDe zmIv{=XP)LgM>oPOEHj!atOw90A)ZU!y~?Yo>=qidW9L1O%cAp*ulju;p%zD%h*^{B zB+{Gq43p(2V7R0lqPg`ypg)`Hd`&j<1lt>8I*|Ltm2~+8+^CFjkdFj_4Zaz$FONkA zfDBIN3WeWCW5!GPmjpB$3LL|}rv&Jq+Jvw9E;aer*7PiD&IhM5H|1LRg2Gi#eLkN9 zGpX=-MZ&!UP_-8451$zW&&kwoBJ6T@O(*ZQ>>ncMb*ODhtow=KN_RUUP{yc2 zv8Blw;d*s7b}VuR&capvhro#%-$2gBt$(ecflv?kg3*dMr!lG1RRI}|98ZJCkm6u~ z-cqP(u*zgAR@~&oZuxs?1*{kI zctU0o_u14I6WDoh{%FT62o9dk|5N&x6r00FhPRtF$}#Nv+%Qw;>HpneC%Zm1_X1Ut zoj(qS$&Z5wYNy^-k2weB@<(c<*XzF6F-z%zWT3e|^7t%m?fhAU<)3xLj!w^Ib)(Ss z8Ghh{li?k!obnqbUcpNXZnCg#R~v7+9nLIOaL106e8L|d7mPv?=^9e#LZ@Jtd6R}= zLz5#8vUHDr)5W5vJ6E}M=8h-L(Lev^9H87q_N+$@44K`9TbZl2zqJAXAt<3zP2yA9 z{8E&Gt%DTz7fN6jvNylRiHV=`W$N?%$3^qB4{J)E<)uaSv7ghW#49MdspW(mrEf-O zJ9zR4m1H>l3ohqsdknc-?~=6Dg2XSM_n?!g6Na7gs?Yc?%S%%2=>QTTvEXKQDK^e@ z`ufyACEqhOY$~;g_)mTvn0D=cc040M{aqx|?h{@n*!N{TV0zW#fM60Ov%*lC`=9ri zr#+iyT7VQfCpQy{XHdMlBi4s{X{>L`Q62{Q_$1JULBDq5v5t?5ULvZwfo60iQ3PNw zz1p6EZT|gW;DWzHp_1hCkax6{c%3XR_bsJGfjXQHlR zHthJX_RT`hS_-Q_T<$@bC1Y{f_Wo3!uImY_xIR*o;7Oq$QM7Dhk*E$`HTcb;da5RQiu9ao#z{* z_5(F;t49a}Q<8%uAR339z7B^3zuBUGHQh7* zpF)vYXNiv}hTqS>$@>j=_kr%S5%iqcqnz*Wdn~0QTrj*Ma0TB6HzZOeisi(nwUlS- z#_Jl_{SfnC{yubV!rA$$$reA@Vm#V;`2eGu8cSBBBr?02NivH^;>r6GQu;8{5=F~F z^Qk*;sg?3NU+HpZh% z?l6)C-J*n3i?g>sfqtDmreofOcy$?tB#(bnN~`qM{rS3^xwuREKGOS6w#?MKv2?U# zPCd-Wx8=3kOpPkW^^GgGW_P~>Bmm4vaPWFeb2OvPHrHH_bZXV~z2!F%Bi?xlDos10 zVt$QT-gOTE-mtc%b8M5fbj3atFP8-VD)VI7o4tTB-eqs=c{Mhw0$6-4Taupa1F%0h zy16tN&D~btbiUwaGvXi@fJ^e9j^=`l(=%V+n_-Q|J}js0ezX836l8qpDx`4R=3h9F zYn69O5s&WXL|pMMse*<~nkru>h)=iop~IX5JuZCmVs&2C31$z+yNcHX2C9%mOh7g! z=F`8X_X-nc@k8Oyf<_1&&Sfchagx1igKF}hKxfQLNuI+9r*`Ndv)@6-}9#J=_n z5t6YwSFyaGOsmFDVy)-9=<`d>TU?j!Yrca&(*?7zJyUJQD{>F3)f|7)O5d$VkHblWKfxA)z3(7JnyT!;fo z)%A5G?ynQ`1_FmZC zh_|A#>BZ_*@4~;g@xp*n>KN+v->5f^-rg1cf60jQFKO#DxVj9bpn{)BaYQtBgdC*S zP$5nBMfv#tDLz#=Q&{1t())n_QQ%Z zx}${XzjRF9U6y*+eM#wNlI|den~cc(Oi$Wm*L29_Gt%%hr344*=W3y5IGeMTQYjVw zK?Qm5uI*OlC2{m; za6YuLvZw7x``}Zm>wzX>Ef%KO^Ssv*NStXRs#`EK_t?9m?t$zt&kDyjWjB}HMdMg{ zbZY}tf*S7!0Ug9%BePe0$Np2}>mB`y^UMp1`a4$P-<_GOKfDD=2j9{RPA zOYTLekepBINP6e!v*l5Y_kyXJzl=(b&)gicB1fP*RiowFLP%W!TN>-qc^P`-ofxMu zcT%!(_^zYttB%+)2Snr|Z#Q$YGqTlE>o*LXM#-YZTnu>`u`VVZNSvBXumZ-!$bMGj*;y&K{LGQqv%`|psgfLHiAxur4(N+)Um{d1qNi5|aVejJG3 zJ*U#Jt&|MCGj2Err;UNredb1K2XRER-3^K=nFtfx(93GO7ti zd@+{9^T!V`CMDDN1@84{q*q99 zk=8loVJAGz{R=+Gts5vvb#VG;Wzpu5coaZ+8PdANRg{DzB7s`7nIk7x#ICc`jioY; zLGP5{Nn`n?$em7#sheWDC&K?cWaUnpbpIFNopg^I+xkMp7Sy<2M>NSn2Qq=JD+rQ* z^-{lJAn?riL$Dezu&&tpkB5q9`_0&$iZ>u%Hc(yx<}4A6{4)>X7zX)z-;m(3QFmy` z{HX@?eFDz5`=4)>G|Q~a}|{r)FIEW zxrS8SP65&yFE8CVPf+Xi>#p9(vTr4TMSYGTJ6a48Sn+ZrtN6Dz5_fl!LCxo(v<}V7 z4SiYH4H549JYVq#P)G{r9m*AR!??BAmr-$BoP(bkhV6JXS~b88979oip8kwanLG5i zn?ViTiwcF7PfkuZ_71*H_X++g>|Gby&*u3v(Rf%eKWmg#;dOTVMGOW*TW6QMYr)5u zT`O}$Va6+aw_~f?2evt`VeZMem1`akZ>j&jatmUS%>}&413z+K0Uqz~nj|EzC#FuH zLt&u->6fZy!>Lkh8zZDhz*k}!zi*q*(lwcCqx^YL-Y}PP3p($%+FR=4AqOk6lUhV+wu~YW^5Z6iz_<`Lv8%q41>-tb%eh8a->O;*I<^$Br*3wFVUiq z2ch>?qW-{#0qE{vllhkoO1MGOd_=_W+GuDl0%Fnk;ye#D`i2*?i2&7PPF*7B7(SN? zMNPNR^Aj9`Hqk}d6NCT^abN?f{UY@DBD5jES;$CF#}aX0+M-uPc&KSMA0N?Mk&mrw zi3srbhE9myH*cRGrDionC3R4_5|1SGft-M-}0;?+TIjQ6$E0d zml;2XUOW7j*L1@5S+u=KN zYTy1W48Y3`9&~71#;{^pr}6FafL8SS9j5g181biyC2&^=&$e0SC(rPFgc>BR)RtM^ zir=7R#^r-YTg3rT)pW+=n6p9E{ zKlrr=nK2{!YAv*E%2|+#{9CGyJwdbm38W=7l+8fdhi?b+ z8uPv?i^qI?+(~;UKai`QlbIkGJ^Y|IqLFto2)^B-Wj-nVa7npsF0BU;?qAU+L;Zt} zBE5IHndR1h0OIZga}&|Ll#JnV5Qv)l>dciR9^m zNO*|a%PbJK{#BRBz)oc$%*r$R&OM+R?i^#TYw(k-)JPq_i)EXSB4+ep=enBNoV->& zS~w1Sx1@^0nAG(BHq}FbTyZ9wMy5k>3BQFkaWUKLkBz4()_Et4O#KOv`{X-AIl%_P z#w379&nHsHWl%r1$Z3ctb?Ms*KWBii8(S}%Zyu9YKk8yx2jT_KRsb0KSl92C>58a*YH%( zzgv*g&Bgfk^=_{qla+GcEH0LXL*4MnuGME_ucbM~O)zpi-kyFJ7vYVQW!G{x$|Ibk zJ1fE=YPSENep~*)+*vz4^Vtfps?lJtb@kXVA#Z%$eh*R$ zow?=XMTxMpkcs=euP9~R045bh=%um4!Y#(fR7P90Pc=(V!hhlZmLnzqL$gv2b(o!M zlNNcm+9e;|1ivNfP}%WQKWm%%EfO#%wQON)C;6eE(nQl!93iT~fF$bd2W`=@iR z0LykMi9)uoe^!n{?#mRB@!9{onTZ`p=XcO`$o@odqyzozZ{$cXBu)BEi>P!n83ly5+GNcAlBX?e=_ z5{JMi-x^Qyq4LWJ9}M1kRsCFy*4v!9&DvlKSU7Q_hj)OYULN4QUIg!n>D~$=%D{2` zMxk2y!$IM}RQRl zN>)0)_OfYk+i@w{Ye(Nb?kJ-UE0Clb010hVnV4k zL^>7zTUb=*)~^=%-^J1g;w`s+#I8Y?k2-j(HCQ^N{xe|eZ7UQ0tRQ&uA6`&ea+$ts z*t)E3z4#v`iS7!Lzz^#oR(zi@d`u}!2DWaa$QM)+Bc#0I-dmjZF)T#G-VL^LIYnqR z_#k4Wg$=UC<2!Z?q>Uf+?Mpvwh5Gyu3}{1GH6MOkE8hxdYm@PHlLUZ^5y8rf4zLhR zHx}^dGdz!7u{NmguWXf|k8m11v+cUQ_sRdOKf6nmp?I`6hE9r)s!Ij`U=N9K*@=X% z_|DV!IE(gtcC(3#%KSWFD>11F{GA4j|ve=Wnude@(l`&LA7_&NX0d%$w)6%;g1vRNQK( zt0x@Yyyxiahh%DK>Dt>i%;Su1Q|kd+$py%CeC9d?qvO9K=*w#*|Gi1PSJF)Iea3=} zK$%^D?(*uYYLvsRa%&yqM8P@r`L1LRa%jcll5Gfs!c{D0FYqD`o=^o-fal76xKa(0 zuO3!XeDTrZ4>(NEigHp_9)`lI6ialgcW_MPq)mQB%}0?yFOq%n4!+d|h0Zfo9<1pP zj`A!p62RnRL&*PDrIe4jeFQ3sILe49n{N~<@5~{qGS~cTeR1XCp~wP`-{qo;PC`x* zu+^qj(Na-U_cX;F+Dg5Uf}-9Fq2Y7FLM#}{M+?M^Jjn=JO&e;18JQ-#9d)c0-Z1N` zWVdfG*`({0l?bCj7L!bJw|93sHs+z9!%39db2gj(CEt{I9 zFP*KDD%%kj%Nb^tRTu*07;jUbe;LD4pI|meh?{(Ytsa}*sOQR z3~bJ@HMDy3(1yt2+N@0tZ)!HnG-l;LnP`)V-t($ngsu`jrgwU_eU=8+M^YZ3Dc;_4 zt2m*FoMTY&DSxW*$*Dxz#b;8UiTV^&bD_;#6UY#<0xz;X-0UURZRIBdFJhu*w33;w zMZ8krjwNf5d+HB+2bSd1&abmNz}*-plgyfR+3Pc#x<`GOq+U7l;LCa zBVZHH=c2W8>_XHn2AG2#eO^9@d~yFXJy3qJ)Mx8ysm}+$Ie!@Bp3G^0bFlQ2(?6-h z8P+2uRTXe=Yh;Rm?iN(G&{5n^(W1rF#pb@B{0NICR1vFM&HecBhHPAbl-b#)8a?0d zS&Eit!+%IW*@79CR}d3gTnp*VZpsEz3g>lcdr?IU{(`}Uamg2bOxPvwG8atmiN$bD zazv_^EC9nUE9JCE(4Q%%7LUE^+ot&-eJTmv4~ zfzUn&211zLrTP9MH;yeO&`qePNeG6eAMmCflZyyNCPsP?Q9{E@Kc8kg=mys8%{cpqPou%=|FEDWyLX zwb?>jt%)#IGJOOTGJN+dl=?gq0yzK6qEILN=_nCjxCz9m$h|K}hnBA4R7A;rwRymt z-w|zrq38djUyv-B;u~!C{@?A7+vk*9bH+cM6y9b|o$CJ6{LH;UMj82^@EB5i?F=#UE7_p;dx+@m%2QF@qcjIcB+ zM-zY2!yDJuEvykvc29hkt>EvZFKroZDJNb>1~1$imYJ+GU3q1^Fk$GUR{MEUhmj$ftS_Tg@xDoCyVU+DNIZy<|Zl& z+)!fsg-C`&biEQws%6gTnx=05Gpk*&bK_sHmi@Y5{j55N3`sT4b{l|p*;*ka&6ex@ z0gz|7dZV1^{NGo<#4~PmVO;J*+VB4>4Tm_XaAwdw!FRhS@$W=UD}H`tT(}D9ZZsHg ztG5CYJK6(;-i%H5lzcnq%*Zd?@xK=FlcHhdKmU2ZnFsU5E~B%t@}8lzk3lEW$&R z&D~IlTALg`_S)l}bodKuJX_Qhy^sN=7kDq9bAA!crN*T&(be4-_p(8)=Fyu5ra=nc zUq!r030C-s zW^hE(77`X!DhaLrPkvJHtFs}aY-`l4S~QUVl85^JNPcRczQeNccgZ{{x+Ndi+wp?I zumU|smA$VK3mZ^j9_|T%dzmKo`yB7Z7)G^ z;JOJvl>&yHm=wM0dtv+fM(yzE6HswNaCn+ZLg_G+JYhbJqSllvj7*H{Q2Y;`noBBf zLvunjXPaG5a91XP>{xfV3jhcn_V9rUrOOtapW?qOHUpuexoW}BJ+K&GJKs)_zN(&% ze6Gzzio0tWhd%$bAc>|AuT)_Qg~e4A*CWdij{}ZMaV1U&l;FJE5jVeuxZfX{q-FwS zQeQoH9&Su;sKqy%PTIJd)|ikWv$KNs0eT#tXnNC$Tc8!ZL*Z0iQj2ixpqN9GxewpH zk9wF_hOM^U?9(j<@4>uh%y>zm5!5}iJrm=GQ=LhQQD=2?y|~($8{8aa8AdbzMM!Iz zy)(E_#Ps*P9hC#L{_V@#T#MEn`{P6b-{iZ6UP>paEEqfZYOVU4(F{AzVF-*Urfyd! zyKt5?u&d2VjIi*EPrV^|`VT2BWhTM3c5xN+h7%Y1?sR$95 zRatxmT>tiS@}KmNt^FZfyAUC88%}2ClE>dRM78@TA&V#s?j)gm>gTC?p81gVhDGHP z+PmyaM`~N^!w5ZDuG@@xaQ(~HVpf&0e+Aue!`liUg~&GkWuI0Ds3DtKx|>@>nev|M zy2dR0=eJdZKI2rtWc=c#Wk?V560F#^kPfX3&ZFrs;M@N}9yh09_%_+Xp9J>WPYz37 zL+mYSg>F2$?%lR<*KWy@!j=Lxb$t~(s`@vzE=v}*- zBfHAUJ`0U!8i{gY_bgnW)PXJy;zk4`_{kr45CG?$}$f0S%A)IMeG^?Mb>8( z9T6f-{sFvJ)d={scux=&8DYS)~h6!J}#{Jv6USR)>pb71PJq&sC|A=}Et|;FxT==I*C=Dvj2uKLh zB{_tEbV#S9G}02ofJk?DcXyX`ceivmLktt|_&@Ji=L5_$&&;~lz4yLqr@Ow@ahjHb zOq?I(vfVO5FmRp`TA{y>3(=OPUd*@z>id03W2;-9<55Nx)XpU zY$d`6QNuKop9M!A1gz>OtQvcn1s3#^Yh^1!gYRU+Fm}-CNQ-c|pBD!uNaa&b>-ge} z!}jV~gUUAEr-^V_iG1TMVGX2I=U03uS=!4^vH>-gHqSP%%};k^CGf>JaS_mF56n*#ctQDV}fZ5qp69 zu6K-?2*$^&S7jI$RNvcO9nPLn3>9c0B`Z7j#C7C0*_MVLfP3j>yRA754&B?qDw5Y=*+1Q$3}S0f}}gN1i6LEy5>n_x1uP+dkURbCXG=ig7RZ z{IalBeKs;4>ATPUf;Wu_)h!k>=PqX#{gzAaAnZ6}KzmbD$%~tznusTf=fdl8Swc6( z3eS?gcWm1q<2EZK3kB91I~0LMFS8#b7z9U=7q^bf>v$BjRj>n~<-*LLPS9?jBpx47 zum}9Gka-2jcHN=%~Z z*}tl9VgEWi_v2@MKl8SM6KHAtD%k4^=|Ya3`5vS?Fcv#E%6u=bD^(P*t3K0WDmskINFUQ!&kv{p<># z*Ok6eD$UZS)(AViXo%&NDg_p}3D>fIgeY!vZkSJ3k~G4JVNOb|Z@*s~mf|>&@?3cm zHGlpR{&BJSwUfFBIDq-610k&E5v_k?Sx+t*?z4&7B<9qu(tSyZ*gZc3-SLc&$1J@)%dJ4=i}?;YMtwp) z#~!tUIz31H4HS@}YAkAp&fMfgE=s{02;|=NzM%RdrcA+|&L({D`5$agKe6UnG}e-| zL8~oXmV$cFH6KL|A4!ya&(Laul;s6O*)#UhFYX1EC~wBTu1jG#@-a_MdnR>l0Wx(*fJZ;#b#yF%JfWCD0!Md_Yy*-E?b)BrF;^m&vHh> zF%Fl&=bbTh6<_TXOfl0$3rg^RU(x23g@e6dLDiaGd^Nu~BID8Nb7E0{;2sa<;xEbCKMpeK9dZO&KOh6`wa>regZdZ@!fNkk8W`^;QI(%A2ylySWZ(ma znnRf%5@I*0j=Vd*#Dx%X` zlCM)lbZM4}cfD9!oBuBPgo1SEwH7J`8gXq?SPhW=zxnh{VT$H+i#iC*)!X!YcjjUV z7e{1`Z}VnVRN~FVrwPi3<1+R=EiZeNANR`A>Y0E?(qmNfqKyJr&WR00m=@@i_vEG&gcusa8L>)O6_KXx5-{tGy;OQYb^To!s7x7abKhPZsdr*lxUIeZUPjKIZTzn@f){9hkCg_f-2`Cl`snWpbQM5*+ku@9^w(f`@5G$ACAwlOB0 zsgz;$tf})=_RwzJ53>Pb5bLl_L4NFu>5~$%S-y8f`%nxD2i7)WQT@u++mAaSLHlw_ zj-Sz@P>9Nvf1{=926WQK>ELW}b$@d*9N`XCwM>BU{7lLk6~1B1e(Ea!U{h7+SB|`? zIK|`|=>YEB5{lWnMFKVzP!iu{nD^|NnvEef*N-^iGRdxxfpVYHU7#oxvjOtgTV{sP zzgIMD3LTegmfaTfb(572bTe~QgIyH}WK%y_>js>K78T3rx)qnzQPl}+hiy9CMWe{W zN)s3z2J)(j;;h1DGRi}ZL4gz?!JkREiVcCrlrOY(i4d)T*CS3@b(tEQqpqUdd~V99 z8iYc%f#>+(`T*_88Gk_Fkr6PCg}a6%CU*LN~YLNxLOC9x}_rcC|IkhXq$b!ds zS4q7CquKzuUMYsLug8(j3Coo0jM9d$uMHVjhv|nI16e}k`pX)Q{`4#8cwHd%9;dhE zSMgO1nVdlL*8tE%CMtU*Wj-_wW`SQ{f=Q5BAWOm+7UjY8SwBvz~9kHCnOhU)-bn6f;sTrWU1Ku2dPZXT4FfE%*7_`&Wx;r zf(=-Lt^UPS8pzg#iuDoD1ShKjd_Ec#N3Dr)l;7p~Ty)(x!r*rlr=Ji*aI>j9uc6E* zoS2q_G$V*2%@PMZ8`M~1gROW9{3a>zaJiXAeAU*LvPg!g1B@+_??f%mAyonHCOsSa zsELrOjt^t(N4ejZaP)6xE+JPAVaV=FGtu=gr?2%!te~5BkHEdxsL?f9&%Qde{b?d? zp(W1gABoD9bAiCu?eVzzP*0NCIE?U@?{fZnkm zORUE(MR!T$W_%tHsmXl2m`C}W%`?w&kkf|Et-l+`F&*;rtSDl9h&zyTvM$yV3YLDu zT0|%xL1bqeb@!&e*VgzP+VBvQEl7*oG{&o#*h$+X@<=gO?hQ$E zhj$^bopkwU=6!hhSHK_14L}1{GwxG-*ISInMuVgFt+Jj~?YXDT%NG0h_bZ1L?(3{Y zhTb9JK3f5YI_7`q?AbdItyD>6nJov&lGQPff6p2d$ZCobpm2l?VAu_lN?Ughde-?g zRCt1?078|)gDXEjOFLN2WV}dE(YdKnIsYa|>un}*?J|6e`;;%2-6u`)Qk!>vLnr)#gb07rR{9gx&)J zcC-cIpN2!z8=tE|28^1tMN{~)?^k_cHt>_dhpU}EI{4q_(~T%Q=RUgygOUQ8X(pa; zB~qg!^z5PWzUE?zI(%fAC*Y%g&e|j8B>aS1?^U~|VTLr@*z1O1dX)0YaHi|p<)@%! zaX8TV>k%P^=T7712yI%1dckrx7ql+S>@mgkY6;W)?593bGRTK<;m0}*S zuGUG!`S-zVcOwtutyET2?D%&bjh?&7v|+T1O=g>uw?ed8qf%*;v|c1n5+tsIDV~sd zE%Rw9lCyYCz(NPkf^ON*otZ3$RwLwgm)mF~;UbP)v|N_sb-Pf7h8O|h zWOCrq7|ShNm?8>U#f#114;sW$Go}b3vq)C;alNd8YBu}t<(q<8!JVh+`NlC)NzoLD zyJC^VxMT>ygir9S);l8h@<+&r2xU#+oYO&hS2SD~_{ocXk$XL*cHXUR^ktDgIkWk- z(${x4ch0S+Kgi=oby(cc|bhGW9A! zFLQY{6{-DW#c{M@clF`Zt2?)Lxhj9>ao#FQ5wmI>@L{IDSK77n1aB0js))Ad?2k9VTfEScP@8<~$)GYic-1J`+2o=TSeEa!OPvUKid2eQzkfyH zx>(N+h{~rYO6k>Uyo1m$CN(9ojP2R%af2el!5{U#@8Ry(`$89^=RzNi@r^72L3~2sQOK0!CF%}du{ohs}2|&%?b4Q_aGx~^!sDt8A%p9!T9{% zoohT&+jiM(vg5^$|D5Cn9=Y2Y@7>`_RiuDsAn@w<)vA~7w+)ZoF}kRFp<=!@P*iNE zqKo+P3io7vO@=;#yHgiOZ2z*jC@TiBdJ)}x0B|q=NA33fp|^LRN6}1Rt%d?%RzxmmR`HJ$UqRKS-cGLX-5*>IR>f2?db7%pwWRqV)8#jR z0y`9nvV^`KA%68owj~Vwhj(I3;v1aVh^N{K=lzgd^tC0?Z-5sElZcRqodcEbUanVI zHxc`^k&>)%(QriQG_LePk5-BLs?9`Ij$ueK=swAq**Hx^XgDPa@J^=Lo8D=4JWR`9 zi#f{?Dz}C-pf;J`)@bv!>>xCK%^B9aO=MK;3J3t)`o{|=>KPq#J2gy&3vaQwZdz|27Egdr&#nUCa@9$y4Y!-;z^7a~n2}Ha{9e;wW`??Q z`lso0>lN5{7RBq_*YQ5LTTf0yRgFokK8;RRs;{Ep86MpbXe_Zo{7n2H@aGYf$m){0 zV1g9<(Rxy7^H<5^7CyYJS{e7R>(sVT`Yk2GBXgQV8~@INP_ zd;2Ts_X#zT3Vz#y)UIokrIQZY?VCd@@^7wNdN52;4`~@md4!Fm_KR-)`{5^m<(lz} zkmBDwQjN&J|&yLG+0WozuCHdvIMcv$V9rcE!S#%1JxM@_Ce zo3&x=Y)LT0y&(J8$nZY+nDZ=cZc^@Gr)dUd;IgfI?e%TTQIvhshhZNwjWlpXttWeR zmA;BEu*2C?i^$#VYxl811WDO+F#PsqSCF7cwhq;NZI${hCXX?@Xt;Vx!b4d7qd|~( zoxs%i|4}(`{soPBwN_kZ?I*d3**N#CsH7~lyOMakd77vsR3XQUqpvbSFR9_^m$T*7 zHPI32FBc@Fo<&<3X)_S}@hUJGS8Cy5HwEXlAK{yopx3u23BLi_pHt@=9FuLEXUVR` zxtuJPx6(1q>f-VHUSYxw@Xqa;ILLM)z?AHIMhY}+wuA4_*CdEIv@9K;Ln~6NBDpJq2g0$*8Hnz7KZq8@$>LG z!j$wI=G1J6aR$?jG5}!Wc~HTB%K5?YKA7p=^xSFzH$seqGR_P3k3$CW<){$sun+{V zjx2&vs3uYm)>JbfeDV(2v}*3KY!*iK)g2^w9yhb&R`QJ~m?R4D6g3LM=nFIXduAaT#S%T}>Hy`v`w!YI9*vs@t?B2Bg50 z3bkg>b5B7&1utJ|A@xAR=NdYnmHoeCx_Zi_x}c~$S$v-N2u>0X{M+gaY{H0zN6ERD zd0(aXXvr&TKhKIASow%kw!VR2)mFIMgS-^Pi$36toIX126&(Cg4W#AKXKmq{EpX~y zR!I_8Zwd>NoJ3n^ovEhbddSThsqm^G8ar!gOo!jonH>gt=Q~vEgJv*dG5-2{nK!BL zi=?lZRex&OE=&u%I&5&jq}ufTl)OCDGb-TPGUx# zz#r_|d`{PSgjCR`e?L7!h|}Lag~)u^>`?5VW_?tNI? z0pnERr63ndmWQDR?0nkEI7U7ie|||-?3oQ8zwcysuRr4A?WTB8q*kv11ZQFjE;{g) zBIfuBru`w#zh7O8Eza=^z_q&AM6X`xIPNI!?$nr$>t{k?KLrHrFR8gU)3*XBceQ#` zT!AH2K8>BWITYDh;GJ=&`3%L`t+zUYvaI5xmU&bH z-&JJ;LQ&@8LD_m=>U5?&Fp4K1v|i|G=PLtMilwT0EAaRX^gAj|Wmvn#^0M^~%ZRtA za3P?uBndC-#?fkY^t^UUE^*ISN&UHNYDogMnD_S34zUJ-zu5bDZjWUq6;?wEDkS8% z%nBbwuCeT@+?n&!}MJud|P9T94UwnCe6I*gmm zE~a=pPk{44LimkrVcJ#%Q6yN5XFioej>Vj$>xq6;;;`LI@faO=f^oQd(e5t)w9KdS zwX&*EBuCeA0+RFMPW6|IE&C%8x^-jHN!2zmjjnKCJ=0EG2Nx8NPMZTB!cIR;Kas4t z&hl;epEgd@Gh4QNNj*_&*C(vdswUwpY*leB|9Ji^~=l8a-VZ?!MxfK`< z=01+zW&&W)tB(+EWlU85neG4)g*LOAoBAr0c9e`SE#)7V>1+z23tmhPg`K-9zz+B< zi&nPFRwQ@`)7s^9riwo>Y4!SkSyp*bDAW|(puOp>DNvk)r=EAdq>ugV?q!V>sE8OE zd(Hf2h#~MKaz7W`{qFE35!D$otg4e?NAzh7wXJ9ncVV$D&kY5$*yl7dv=8ICYqCD& zN$n+vvBgxa)2%3r+^6HrjrYa0h9V*1NZ-y5`Sl+cms_EsIFGrcjE@pfzDSByq=OQpnC7DW3~iT*rbz%t!4e9VWmjJTj*v4}6aK2!UxwxiVg z19)&da@)!3ez@a)!>b^mgvxDc3v7LJv+B?#!m0o{U9MY=`oQ;tomHV<#kPiHuJXa% zIj27;JTWGSPvG7+^*Z3=gvh(mVQy)vBB-d9B{xA3zk(G@GLA z>IcuCRMLa{ubGRAqt3QQ1CCU0c}ADtt_D>}qbXv|m#DjX(B+;tLcymF_c8dupG~!J z-2IK4$sDH}NK(p$K|gpiV*yD@duEbHr^LDUx?u288V^uNfgUt&De&S=Tc{-ZT<_DV zRUJ(udg|F~F&zMz6CoW1Trn-(?;8Fj@UJ7CQZH^N-Rm8EYXr1Mdyl>%QWxk`mpW&1uA<5W;y(py}h@M&CUuO+H9mJiYtNH%i9b zri{J)PiCgaY$)I1%=`S!KG{p)(fx1NY%X70f9;YArhSc{tx`rdG#=5TX*-`ZDTJz4 zk7dxCCzcv81WVMd8qw?Nt_ko^((y!NgLinZi4q#OgQW*e0zyAH03>$(?gR%;I@P*_ zw}WRGru>_h6ao+RAS-oD=x_z##iG{4JgB%m({vJ*KEdZ%GKwlk3*p;;2=chYj@C)A z$FAxH!7r?aNLRGZ3} zZhy?t384%9W2^zA2Qf)u=@FAzC7Lh13q_fpXJk^IFXqUu5a z4<)lAQXSTBr$%;kHclXAs2`HnP=us~^!E&u;+0 zU+&e8^8BA4*_dLeh3w&uOwo|Y0o?T(|W7WEfcZCPZ-?-K4*K0jP zb?An;8O$rxu&APIJa%T@k7}D26cYqwm#Gi+>qKB>#6Pq13@)wXh67KoS#}$mHn_Nd z=(E&klBi$njif|6q`N^kJc8qc!V`UR_YFu8p{i>WXZuwM3m=nkK^<=fVa27OHnEGnO zRU3_kZ zgIt)P)E|qSnfTYah1F6t!fCqrn)|6cz>0?OUs4cXq@tI_TlG&vc%Vu@!t ztv#xZxoTPCza4hkLTXR2e`amjjjjFr3ws1*LNsd<>%3dBDcJa95{FLme2gkwBzs!d z`)|%5YO^+o9tEDsX`@TO!M&zE7xynh*NxJC;qjNQ3VHSdbycW&mHPALHwW1#L>&`* z8rY7qsTFeTw@NrOyyRk{Q7e&b%s5rm*~y7ycu*)GMBYpaPV5TZ@YtUYI-MnuSSG#b z3hBPI47m|-t-K~b0}cjQzCC(riD}i}xx6D8b%@ett$OC7M1?EIJqg-b;gx)d`Y^0E ztGY*mv+wN&uDu{GliQ%-EhI0JvpHORD&iJN(ZRAEdxP{PJe<2-OJ>|dS;XXbHdY?o z^O5`YuElD9USU2_Hy5tn;{x+#M7hex7l9}US`+3@kE~#?PYXWmYBWTmLV00YA`};n zv6SGS1raZ&XlO2 z?%t@VSWIfKk9*F!b+GSkkVvHO0~YNuh}Wk*-23`NMAVel=h+mN7jeSGC^AR`e(>AU z4Cc_dp}ZGE;Sk*b&W)1FjUw>+^aTx4eWKhaX2ZTk`v@Y#{p!t81WVvhv~hz$vPBP8 z3!>XS_b;q=x)b3j3JP~PN|9V%V%(b%5=}Toy)VV#0Fe)kbqIYT6YJ-W+ZlNck>2OC zPArEc`L8&`f7FqxbF9#WmgCh(z+-W^3`1rTBe2M8G6_E0T;b~wM?%DGxD4MV;ElO# z)m}4Uku8O*54+VkVMV%%t)xZGSZJ?O`4 zD97Bf6XN- zTh}Rx3VxsGoV^A5IEpV0ing3wG#%14M1bfRpzWD1<8KahaK1edU^|kw-9A*!c%993 zQt|-^lfHycQtDVpcya~NNq4n{AvauFXA@7l=C9-q!D^Ze*OgKqJQ#*?p4$JZnZ7Q# zgtoo*j?Pyp)gWlE{7*zN z{o!D6Z5A=^O)wz(6;+WRP-4G`TDg)!FJAN!-El2*diR8X2w{&?L;MX&8TkAkE|%z( zB{F_Zd1RP;+dWkMiTo~tOE)0+O;vX>hSV*mWZhSfVGF@TirN1onE216eP%lH0pIy~ zoCJQF5Fskcatb%)w`wFreWKGwdiP$5wzV!3=vI%4al7llt$vxxOBfMJ7Ux?3a(lPx zOJ?Nrqchnb7*yFImMO1X!#5+8SSV{^K1SQYc%Dl4#x=@XxW~(BNsQdGF8e!JVD-&Z zp1$V-^DLU)-*;JC{XCp2rY1?oFNfM=Y~2RI2K z1cQBSB*rxr%JiJe~u_HF=+YL8?9ErhP}ErmJ8la%1)SLI!}dBai~) zFM>lxXHUJt;~4aBempeYh+I9;T{mp(nG6r;qrcl!8~`rk{<@26?D@<61*$CgHpp|R z+GiH&7KPkRkD{ZdW&rp+_>gLbn=8P|FG6>K@LTSY(6YONl3zlkF}iJrx-BHxRpCZh z(U=A`EUmf7BWDS?pS(*LmexDTTbIAbDV?6k*&y}!&=JK1lt%ScgpB!^I^iWa2f|ZU zWN}7L)FQ=X?Kt#b#KK3;w|=T3=65=p${J&=?w0ZXKRT{0uzs6z=_TMTkaSiHFGtcs zbAH09+!7I z$;^$upw^}DBSeGGN#XCHx$h{=)H>XTidt2Tv7Zu@Geep=?{@%Yj5>2Qo*J+sZ|qy8 z#$JFXf5%XAWHuA)d$Lv8*caQrQJb@6(V3{Exm{VWEa<3e-ivFec5q zrKk=$LY`Lyz9njCrYQ8^P2AyEe3F~?OG(<)pho6(J%!K| zjV@{a<*FlFVker4SyHBxri1^m;C8Jx2X(^CF3im{3cOxC)^%jS`WY(T`8HYZM3vbou9ND_s?KQ(C^@ zHBD+CCPAiSrdxLla;rv!C!ZFq`x!kf%=pAR;$c`p1+5U^8^{ZftrktRHjY;R;%VjI zU1mynNKjgW<@M5i_}~Tz(-LOv$a9Ol3_{Y|Aj~>=2G-4CM}WXaB1OY<;KQztHb)~J z9~?i2AeT;w@-F7Y=8pZnSQl)YlnjU{(>A{T^Hn)AUa@g)n!E@^c|v+_!Qvs#u_Bl< z(3N%0?Rst{Q-s%R@Kw+D20Lup4B)Ru_O^>qC!~{BITcl8OOf45J`As&NV8>E*K;nZV1`bS6sHYT+8S1) z3oiTM!fz86p?~qovG#WnDy%hxh$!+KThCppiaP_J$EBZx-N&PBrcpOn6?IWZJxhV` zr?kyeVjHT+T=v{v6t0YgjAP>&YAn6&0wDcbecdM#v~m-@s~jf#4Rz9RC@mk)c@Sq1 zKx8gk+t{_L4vHY4u87|M?{$e9TdhegSq4J}Uyq-dL0#@20^(>t+V?ZlpLXJk3O1Hh zay;zs#1=1}{jQ_N4FQkwPoT}D0|jzX(*0DI$0ghUXyF_IK3e}NPhS(v(XNDi9978f z2^pn-6B>_>KJ~JIkmcpfz&Fk4OP&*kE}O&A)uEF*7z#anw`}fMS0^)6IXWnkJvX9C zV#`=Yy(F`N*t+NH(h=#R^j=kIv53)4%MMF-8Ska8?)vk8vjCjx*%XXFSf^y!)1^^$ zsm#$_PYiXPn0g~SjyAk5SOt<$lR?v(6Li0wV8WsEvGBC@r1E;htKX2>SoPTOXqh-n zjeIOwF#QwTfP6$B&NgyHej+xlvQ2%W`LCyr_wb&aNiZb4=9{}2?BJ%RA2x(D!cN(J z(fByY_X;1I*=FL^AFmxqQdpP)%|^-GNmXa@Sps`{#vYK$eDhh%^%eHow(yCq$lf4J zIfF9r4PSs_6uw4D*2!RyFaFRlFXEP|sVWiKwG5W=*qKk-9T1}CebaD!u(!SrzN6s0 zXTRubZ%;*5*<>l5-$GfEqbk5zLv~|un<`{7<`=sRln|?$Ls!sEFI7(cR;|0 zLfcj{q59YFt(TQW=BqW~?d>ZMO^e9(E@z?DZJ@2xbDXvDm!7JekbC0 zNy7Ugwc)|FaxV0li;}5MeF{dkn+b8>nXe~&DLo96$~I%}7}n=kzNQ)az;U@ViV8eg z6>58bV1RnBByJ|?s=96}8)9s;Gd{dL33))3EhHrSzIQhQ;cW)2v5xtCAhb(YOGLK9P4_+kj z@d2&Mt#(Q|f}G}}<4JQEp&K}*1wBBaf1_oOQJ=o85m(r$i`9nKI3yHymfZ60cRb-@ zad}c-U5ZH8wNGDVk%zum5jXM5k05*8O;OmDa&NVOl|^KMt~1+#DSX>9VAn4a9Pgfq z*o{?sK!-$ReF22d_ovGr0w>CzcsRH0^X`F&kC;`Gq>EHF*gF(9Gq;g^@SHGSLNQ3%$re~h=BPZT>fVT2}`NlB&orOfERUXB0M zFK_^yjA_jFL+7P{$SL-h&v^-opPL>9zNAq)oj;3Mc=b%R#mO3E%)AhCqu=$nIOPrA z(tkd+QW1RQ|I8>aB2c=~g~|8~2SkAR!8$%u2YnGFHLlJDoy-%`4V|KzrEofK+#+ZMmTw+v!dF0TK7uY;~B z=mcsJ_D07KMng>>mS0(sEDd&^8ZS8?E843ds-43bmuN0&5n4%g$|Aa*sMwBO_wNr; zHb!1mU>_rj2afF=FXl)QC{@Uz)#Z;rTC{M6z!5i18YU;6;i0@vO3wX4qvde!JS|1OsX5~-}Hal5O69lp-*!7 z410PEZX)k|fN_7IPP^KSe4OlNrrFL}@O#B(KlS*mGT(SgO2N{;^z^6K2#v(>7wX7XeG%yh|M{cIsa^{2)ai?mTA`z`~I3dqaS zY~ezT2cwxJ7p=NyXK&B&PxVf@A+Z}Cca7@>7bl5(VXcORtS*HPTu(-;Sz{Y>?k$t% z0`VB@52ii$(rCSeKObO!zheGT2^g&<;tlQfN1 zUQgRuH}A&UO$;UZN1iWmBR`Q?0nSx0adF(sh^-vu%WaIA-Cuiv|Ge4t>t^NcTX~hP z0@SBulKK6@>K)#Dlc6P@6j~Wd_e5dH$l~=F1+n=U#b&=h&yQT;8&AK?^M=E1Rn@`X zzZ4EJiU;hO@su7Deoiip>k|ulnRNY|)m|fpB%5EI!;0(JC>Z+XLH+;-;ozpL2Kw=j z{`MY9C=YeIu-4U2olGy-Kx$)sLBDJa#9dw-|2%o*E9%ZWamjP`@~$d2&XP?~qxZ!32K-!a#KLp92?q}Q}?GIU5X(ADpgigZ4u zY8Zpg#>syfjEyKx!#e`ynV{Em5*b6j`&k}o)x*EJ@XDfxb4F3_`bpX z5qFP9a|H?Y9TkW?(GyRD5pK87z6L+he&GQ<(Eil;XoM6IVYLBn1rY%ncWt!mt*+a?irsSjk!#9 zMEl#G0ZY`iq@<60{O+Q!-z!0mCk&P6(Y^VsMK#yYg?Q3+-QBtLJKPNrj^Ezk*xeFXl{P)=kz#5 zXVR%X$NVCm+%rK~v)tmtjx(8CN@qk>B#&ZjitfIVt(A_?QGxkU=%7jMZ7s?JBa(V{}~Dw|H<_nm`FdKAaJ%jdKT*2{9GZ?Qp?w)8Xhxx+QoyJ9kW zP>D-Ghd-pbWGOOQA1r?}`w=CZj`SIuLa} zk;bR5Ky&iH5mrSRN~i^;OJ3%mr~xKVg#>)AXIGPOi_StMDUM<|dt4MJ$+3wTA`BkY z$+v|5KEb(RrKmC+Fyq`J zbDVtxR~9WPZ|z=l7cGSyZmi>rd@Qr;EYSDtjkqJz3fB?As9#6Qk|7h43^XlV^}?5S zye$|`FB{eC)<=B@nhD_A8f95G&LAd6{QAY^gzYX{#)a4iIL0Kv7sFF9p{@S|jJnS? zk+ibqAyfVED}5opC1$GqKRC9I+R_qsQ=1L)ZTagBzo8rUM3cbQfaM`KJWE+e{=nf&?*gNy3pb4|f)Z0~odTmxS&lLF{R?(SFDHG+R)Q8JN_lhd}U7Q9cn zGwrD#=WJ@4->u~q&yp_P`a}wZp>yefu4>;^SoVg$*`1a-pJl7tP}Cl-zJn?oa`EOk`kGal^&;JZ)@r}antwx3f7pI4CXpj<{C(>R1uB$=gdw)(P9=J zkrrWGWy{~MJN*=%g<0Zr5{E6nt@w~79K5?CJUQF>Wj?BYITG2%yP6tH%iew?Hye3fLq!T5@ z+QZEFk!#!hM14I1-uPiLcU`6FA&(>^F5a_wi|big>*V76ZPL=_nce`O9Au>vDEi+(hnKky**G85?zh^0Pe)XXBur(@+Tk zbbo;vQ~IdTRcQ%PKFFx2LM1Ey<;468n;}Ie6I=O|JXMUp9HV+a!8kKeAD0y`w|X0K zc$6!FUAo^B@9?Vum1VI)tJ~&D`|BMpiXv*J#q79RL7Q^iwP0U}H|~PHA<)Oq*xF8Y=;z4*Q6O$t_Y9Zzo4AH>xoEY&&I?Rc*UwPVK33>jR@EBj57a{1j6m2 zt`-%2kYM}sD~X5Ox_ve(8rAT+MO0$y&0yE2?%rS!_4Htn8cqp+{uZev!_3EfPV&PE zGBCO2-C@X`_F7pD^6W}Es4Wy3#@ZR)VG!xaCGg|P;4Uu5U4#|1K&fr*gK|`iWM@^; zl4Qb#Qi{Q^#hEL$`d#_@qZ8u-%ORXtiupPl#SiyS3LwZH370Ov(|zkQH4X;_t-s4y zU~^E$$REGYFK{!ypd4@CH0+*^Az4t}LER3&l^SA2LaXeeit3&Zi}-}sVqR1zKW0H+ zs~BamWOh$r}^A9x6kZRNH%ASzop%oE@ctT^hi;7%sqz7V`ym)9Hs)qL|z3lJ&=7mV6`+Da;*G~u(w=zZ);D<^sEUrK33>p>b-HfJBGNd_F zG4z5wCg0cFObnGmKaftRomz#8nHIljvhL zm$6Zr+%W3#BWi%D*^lhY+Ck4}bBgYc|IsgZ>0U<_PJ^IHtwrVT;0-h7uVX0YKGB^Q z*%6VcbZ7A!b@}ZJ! zfJ3*fB0Z^Pl@ft#1@3U~3NJ-hvsQR_P^@SDru&|TS&0<*y}OY|OS<=xF<0-c{U*zL z*zwTpWVD~q1?^-ftaiZ^!1$(u05zZ1y>e{;!!U0@@QDR2Tz)cog1ou;Z)L?AaQv`~ znxf5DN;gj~Z}X3dA0GuzhNzMZX+cnU@CeSCd$ro1AAbiq4=?GKk)(LQ>;_Dam}$>^ zzl2C9a8jzh1qwMf_sWjB0_m(L@q}V8C`^^$9lk(S@2q&O$N7ZFUy{P+E_GD*HtLdhqk~$o zJ|IrNNhUPEJv>-GO@6~Y&Q&(n^he_VC|3h1GCk)ufOy@Vnly-7y` zf*>{0J3%;r(gFg~dq;ZjMM~&JdIv-AkkCR%=JEaBZ~n|&bIr`3T#)Qs&)(TEB2GhJYGWTo8$!3`x`0Sv54zQ@%tK>ED-C>DTbB7EP$DBkF z(nU+HW{n+$1Zbsc8U=8QxtE+5O*KJ#pKg4A{&l+?ey@KN#>lDj$7IL<%=}gUK=_azfgWD`pLA&;_4tDFWioS#QkrcOAB=r0OHwI~^R=@Y` zg@L73@$$sOH}xD4HUuP-=J4n|zFbUK7HYYFc9%%;st_+whE?ekLA~=xu(A!8BY!zP zGJ;!r=*OS~uVq$+38LJc^##`e8u%K%N3YDbvh>-w&pqSiuZlt@I}e_^0>KkR(u1)| zs*8-VXXH)-8i`r}6IZ?`M_>_dnUwn_kw-Ce$1ZjKPc-QfNgqW8rM-b zuRXj@zRr7`v>N9ylh7@fqnF;m3b+i@I5}jI_k9 zRlg?wcr_B)9=+LuVNV0TK1}R8cm)yRuR!06O{I4+SFw5bfkX&gvNX@dys} zzrXY>0dC{O5s-F%^&`QE`b$sa#*)oDfDqfhm7unp=q*9)P4y9y*O-C+pPQ9=_*`^@ z|5DkGPi949y6eOCzqOR{#_u0Fp&AtYGqRkYBYEdJXAjNhjdxhTHF}w^Xk&tho#XY! znp@nx)&ZReUTDiH#R5Pi8A8DNS~;~R4Mm(j7;xjt?~3uQ)@IMHkS=u{8%WN8p6tD- z25-92-on3U^^V^X{Ng-wr$BK{lt+U~y|bhlXX@`UE0~Keo;%g%0_P2{ZHH4a^|Svm z(o-t$)Zjn%VE!n8QL8o+Lhc?Y8zWFQMTh=#@{xMfz8i=mlGvrG>mGL}N4<|b4Nk*x zAY{m@NlKIo<15i_Dl0OQ$J#@zwFAMA29Bt~|Licj0Y~$cdI`ofpy&82wb?5Nr^bvZ zI;8q$E%Ucz)?J4KYq@Mtc9#@~+x{wFrLsgpV>qI=*jbJ6NcgwlN@62n6WidU>s4p9uPq?>3 zNy`=K-n4vdRskY3X((BI;7W`Sij;m=Zs_Q2&+-J_g@%yIl}f$v8yfOh zJCwJu8TjpkHCNmSmTv10lwn%YzNuZ#spt3F?cv_MdWjZCA^>j4*l+A!Eh)-0xfDNr zWpE^9=2IW|%j@VuwWG%6y6tRRa(5+adQ7-SGNDt`got0jiJ8N0$ikW(V5?- zT3GLWPM;0ETJbvHJ-sdc$UYyaF0Jtp=*}>}1F`XOxIAxBw$7NX+Cb;ZUmFZ|T1P^} z_}k&xwLSZ<$c*;xO^zob)YajE|A)8s z6Pr5UXF_?@9O;cVs?bBWcMLif0y$mk+Dgu2sHuyIfJ{%A)Bdm*s-mQ4Od2eDjpI3^ zlQTojnSSUMwE9+hF=wJ?U}}CMpTF{o-!EjK!qn*nY|S@PX8R)kEm3cBlKFQJ&t+rm zNmMPs_eRgN9PE6#f0`>RjMEE1RS(uy>QK9@8x#snOyb{C!H@ysfo)O|0WIoES`VWq z{9jW9{X=(z%6(O@;U`chF@$d*Q7lR2nSit5&|D>|RsNqw5PD-Boq_j0D(r4Rsq|YG zhB#1|bZ*R@Y{R0-IF<o=ih zolU)pY&@@X9=q$`Ogx|~O*MI!{Hqyu5cJ}?fQ{d4q@vbCE6b*PCNsZa3W0LZ<%0nz z<=ZEDEz>p=Mlw?(7TNsjZwSI)^gCw)djHq01s11Z9D1(QOEssB=4ck4RsFC|;oxPI zFBc(e`4hy$w_miLZ|bfTj}K`-(PLV(h~Ke0rMY@t%A%C`;z9MQ#~s+GzN4mZw)MN? z@$Se|cxC|vm#UL?)=qBH=gp!ax`J}u#bvylTSkB3YUr)`OgnysK^iQ)y~=F42;+=W z;2!wI738_g&hp@tHF(>|&0l(aic48P26rDV_FKL^@GpL~#LJXir!HAhMAlsw6MGd6 zeOlwML7`Psz%wZ6Zqg1WxUL4y;E9(}R0?E&s-j>%9eE zT|gqsz2Yei*#qBfq3JT-P~;*}3~pmDY|TDpSiq{Aw^J&u`|AxedDHGt z@bl9>_oJCB5M$LYl*Pt~sw_y_7EbgScjek;VZCwnh zk4vQvA~^HMv*pfDi*-~WDeFe}B%LwPU#(sx3x45bjZ({Ur2^^i>@guZlkfj}fge9_ zd9;p$$=h- zQ-9h0%d{?Hm%3dHib=ok^tG6FfaQfxdtIz80i|X+kNO?RZw~L3icX!{6EU&%^?|d2 zn+L9&>4KP7400;O555F_`D;)vI0BDCyJpjyK}9V1!MBk6!J7;(Y&Cy@!5&a);dV)g z*GXurm-^c1w5a|}VvC=r=jyQ|d4Sa( zd>N8C%sww+OggW@ZwK~gp&X}Iow#-iE-HHZjt;MW01^sdE6cUESRK*e)$=T@EfXk4H%GMJ(F|g1x2DTODB`WSp`fexXYHe?Fey zzDXwm>Ooqw`Z3M1p}efN+s5T0T)fBU9yDsybxw@h)Zec8u##+^O$C!S2HbQFNtbtS zYYPMHZWNciq}; zqs`veV+NnY=*UlZ+b=%4(c~pq%Y0 zy2jq&(#mSqW~X{}d%;vdS` zmV%<8#H*h@k(C@D*>2V@#-W4bF}DR$=cmI|awdOEC^)LTsu!)3xKW{`h-7)+I;->8 z-bnwAB~8P`$1HI|=k`j@2`!EZ>tieFd9#yG*d>!Hfzpb|Jw&5sGnckN_7bL`7e(C$tBYN8aAC`u&c=ABydcLhdtNveyLuj3L*H5|h@mKNVt(hR$H}-|2$Fzh?O#uDSMrdFMo5pJ;RI z?7v-LHr$5l`?JSAFxi?akvY%uk6_;-D+05J5R#0wUhVRC2)OS~vZams1KT047d^Mc zj6;2DMlIfc*vL`){=NkV4->x@z#Gl!MCk$@9S3FUzlGtVt@bC%B)DdAgB116;Dcr7nVN;h)#**`rwk2{vD%xAtOR9x8^{88VfS1Q)lMqVAHyd$k2gMmgdEb&))MjB0D3iXHKlR(4918&AUFGmWq#WHWvZ1 z{t+oiHPipB@A;9)(@Z^&^+D)2t8p3Sj?K-94oOILs6QCA*E?@qZE%rlvtP1VhKPxe z48%zWXkWS)yu$iVD%MOlcCw1vxAg^ie23b_*8gP?`o{R_uXP4Q%%%RG<bss2CmI8d@+^+}o}h-C;_>+4O39H>^5};|Uu>I0QS5M8h5tQ_+<$ zTD;5cyRf?)eDp0WAW5`vr1IDTzfy0qg7(bwhK^8G`d7AX&c{v*mNpr7Ei!zbj|%_p zQfAbbGe`dgQB)swoNIHbz3<&IO|B(1`Gf(YdKRxsLOk?k>cyzxnBtfg3a`RGL1*ny zF#wWwm$nxIzagSP6KE!n2Y_%XWWQ}jM!NNeiebf{BmE=j^1u#vu{bFt_baH(Rb@dq zgJ0)L-?i_rL0fa~_{Yi9>ip*YWB$*qN1DfPsAxE-&3%#%1G=iwkD~g(F(B+UtKZnD z2)Zq8C)4M7jP0(5Y2BJp<#9Cb*KlRU85Q|#AjxA*4mw4vZy@zw`W`Y?BG_~}@Z1fk zVyp8v2f#mowrRiol~&U}IlJ<)iQ(oz&`~B-oYuyg4bh2Fbr>3;$8xW&1YZ53^G0@7 z4lhot^vyaIe+CyqijM-W5|^j=6-N5I;;!r8Q)AjJaYLzv<#8 z*9W^hY?Cx^|G={c{B|#ISu~yK;=Qp)hsCGCVy|M4EByzv!J-X| zmz#e;rTVHRl0TF|zKlh4t2hvq^zjShWrV*xcrj$;cYT21UFJA+3J^5XixD$Br8H|$ z&I&LHWBwGoVvW860#ol)zyU~&o1Pa*Lxi9O<%$+=tT#Z$b)M4LnnP2{Y-`Z<*S2Fs zjhyuTE~gvPuweEuYH)x)7Sgz~8BWCb{G{zFusDN?Z>E+S4xRZBS&7`Zal$r6*B0m_`qs$86s}7^Nm7l6h#}l!XW1S5Q)w);%D<$oY_#@j8TLz_P{JgP?zg z*dH~i?23pM9hxx&za!=^`Gk(W zgse1q$~oydArfoYncaP?B1+AgvbTG5{{2^pIQ_scbECLRc9NPV!ShYpo*&bv)lxt! zFVw76hAEBG$QQ_1_S4!r&WDjQElT&FA!7q$A1`Udi;~G7Mem@ilruNHCuRT;AH7?F%=M-Y zTn~N!B3a^pn=Mn{2=es)H1lJ{3D4a&qzA5&2ppjVx9z+Ryo;ERcspgz{8L`Zz`hDQ zeuvHIQOc%>vZIJd8D0@Fxdf;CEjS76EF(h_=>p}*JM+I*n$%8G7sC^#JQG3talZ6j zk-LV_vx7`W&%9(JRkwgI1A+`>Vwqp?n>aF88KB@ty|ZQfp4yI7kpJndN~R_BUYKyb zHFY&dRKlD?z8tsjrXnww0tl<%-UTG0V?>v_vNxhdTLd5V(gx?r_J zfB1vR@h8#mFVj_*&+1nvLK2H1%^ZXuR`WkWeDfzF10ucsUgA!i+p5hxe}5J5o_OXK zxk8OrXIr*jM6WA+$dJCQNqbuaYg6UHEpZE+&n|7Icbc;)XZbX&beY(~vIer0qY}uL z4G&%>Tih2%!CQx!NhM!HKm8@z{(d!|AF?^Gi48%`p*1BA`R*<{^sEQC1dGJ*##+x+ zjn8o+Gt6(BjA8N`+Ig!CILgj?>pk*fXzM{mpd_}>|L3cP5z>nnf6a`NvI;x8{r!S> zTbd7-qr8PNd`R!OF<=m0L7;2;$R+SCnlIq!GkH-@6rMPKpEcK2UM(@IF!Fg0tfoA|5G7qsudXBEWUP(X%xnzZ&{#=rkI~K%g3`a zPVtHie=uEj{07e=j)F!eiYQRg;h_Jm0@y^DS4Y)6#ML3}MUH(oQ!Sf@VRJ2AsX#IO zQ0s@_s8qhi%*XWNt~7K~Js_gfZs`Zm{nXAD1Pf4eSA=^S>_TG?d(N_FuHC{y#i`7l zKTpEyZZMyz=XM#Ut%`m(J`-Js=2e>2WfHfjxe^l;mD9D^ zrCox*4_~%Q4cgnfpdOuP3b@Ae*;#oXa4mkof8m2Y5W?j`!;(3R;VCAq;WD=G*XCMJ zfdyd;V?Tl2+)7-hOWOqg`45ta`=(a;4{OD3()zZu*HWzbcHGo>r(9s}c;Zj`Nl!XA zf{WyU6y$HT@XE087^BxxCCV=5%ANbLaYv+P_>z9;M`fG!icz>lkLlJ@89xXIb^8e! z`ZZiWbb6@dVnXzFN#>O@H99Vwb(?J8W#X;5b+E6xPjl(Ycqhf{+8|Lu@s9f=E^K{b*ZlVz zs=p*4drec>Q$C z9Z6A2JfS|%9?s4Qr5fd2rz64G++J2N)Mkq9WPkQ$XnMf$O#9DhXq?f5z ziQvEZV^bsfh*2yA<%L%acraGd4`iDR$_A9uCUft{+-uB{5cd| z!nmA`O82y`Y@{}Ze@GG3Tl$Qw|KU00fk4ipI_hp2Ygl}Ac znSVa&l5RxscIVviUnRSOcBBiJug?D4V8`wBKYzWiMC`P|r_cDk)tko`tcC)Z|96xl zc+>ye_3;J7KQQ&)UqYE8vpAtCO^B%1nO(pB?7ZyJ1bMr=#@TS(^;UbFs1(<5MO7I4 zl@d07ch2Ez6-U#&JJfpgd0(c~6w)ypsDk{B#Fia;KYjnoA}T;Dx)Th8v|&r7?l$w- zDL`RNY@v_6fzJ2G-MbJQWUZA%keTY=?S&`k<*Baitn=TB7V%38R*2Cxsu}fR!aJ}v zB4!j;8G8dtw)xxe!yLUTJ3%@AVoOxWbM(FqQbxA|Pq(;Fm!2efa{Z0yJPSb>ShsJM zO}A%=r9NMGCSr6#sv)l>AYFAnc~6M+jevDZgGW=SfFc`6H&_^iKegr!r|#~wb4 zFgSiUUnnIW8RdzzmFZTiKADG3&S*11GUdF^0R?ATYtD9lzEBydRyO2S{YN%v;o}tw zW$W!5*bn5|&Y_FsjiB-@aR2^T8;?1gNza4&eLsZmLbm!3lm{{zS6;!S+_T!Qe+mX? zh1m2&mx;f}I2$Uo^%=kp)OO1OA>3W9~S-MN`Sl|2yViBqR56Ud&Aq#MmYUV9+7~VP5kagAHxRG zUfakil>;QqhugENk1lwL^@e&o-s!qZ$u^GPeEWp(m0IH~Pu|VW`ESE}Zi^M)0HG3N z=cY@6<^urKTCI#N$a?J?HzOLsEST)K8frZ+z&Tv%;P^_7t3Fsg=yMg7&Ff^M2aP*m z95n-)yLq?NZ>ugoCQ(mQN>%0-+{BNGllgRAh3^^XP}!y(bG>C3OmB08dHp4dLaY)} zku^MNyFfYjmkP;&OXdw#4FYhhc)AF2w0WRW=)W;abTI=uJOpvU{yo)4_o%ROsTsc6AZi_cPY1d*3#GHLXgf??z*hjEicl&IArv&s2XTcvRjqP>( zgC7iL7d};taF))%{n6F(GWV@!2smdyZ>=4pk~(;Jbab^EpAi@sYvy|7;rg3}XmCVu z!~&Dagquk+HHeX5$6OOLEW6H0*Ie||7rNA|qTXYZ8}IC}d3?nxue4=7cRl4o1YeZC z|E%b-=>YNe`Fs6$WP(5kEtVsbIZ6T~T*f&;4_swlEq`j+`flpj3aD#(k!zJwJls5a7~!!o#Ia%&Z=+e@jc-q6ZMihZzu7-@g|MP^_^BCP z$_HtgS~54UPcKRZodeZgW;qHh_(%6S-3iX~lwZWfR#Sv>nwzJFPGnc+b-fSc*kxiR zClT3Ij$@WO5dH3%(La|B-dQ`l?mPv^l;@G-gWBz9L82e5YK5bU2o=n0n_l57_6T1! z$>3aic6!Lxl5hd&67^DWp2Fx+uo?m~00(4dSLsl#DL8uc`W7qMvW9aDHsTwnzlYrt z^Wh5+b|DU3|MT?TzYf2`fj>D?7-;Tagv7}f%cpbE4~FFDK${21W)QOVo1=fKUW?U}3@eL6*b1Qg3l zgn`i7-;D$73y&qDE9E-V5NkrkGS(uSc^!ieRD>V0KU&pbT@X7VWbQnBE$Ql6l|n!q zuY2BTe&WX+24;GXa9)6?@KxNe-(%T@;8FRQYCxe3!0bO;^L z!f+rZUqXg3dr4>r)yjoogx2*6JrJNc=#>sA?Y>pe%c zzV&p_*Wq&slUV^gyt^R>Cjq;1DW4i_fo;|{Au(xT_3;j^806_T-4~0% z9$BXKq&qqC3u{bl2$@XNqy`AV0L7fu05_xjVqB4fNAA~`s&ItlJV5B%dVW`+S=UEp zs*b&=gkGo&c}KV2Y^QBOptq*VqPeFM{5TBqQD$q~!1Krf0O)Ff#^Y;c_voP&aIYq) zk!k_qnqOLoHGX$%^1Oz}#atj9erJ%GJ<9_zaw9ccL)42pt0eaFF!CgnwHE;2H}-!)OUkiL-67LUuLo2Lk7`5x zbQ2hMbQ?g9%di(^ovYpa2vq&6lWD%JT^daiiHr+BPKryy_GRKiM0-HSCrCHfH<~21Il@8#ftm-GSw){?g=zAqwnQte-R6o z6yBKwwE>MokA0{TkUX95#>5d9gT~RM z5;2L)`@E4?-pzsOYs2MLg=RiC;Ob2r`UlASU&W1|JR(8HTvBMfp=)Hd@49J@Du-g#X{wJfxVECvr0%)NtYJ%u{E$Xe)(uRn&!J8NQkZ2^#g&)k?T7v`mqT8-TTXmuODmu@PHjOj%iFdB@ILR_>jQ{;4DwP-g4CpId(w#lCp&qUll3 zyNEWDMHEFw34-54icAc0xTJ^9SR6(S!rTBNJ_UOEtVwlssG~9Nz}Gl2pkMa|#HND=S`}|PFG=pOqEx0x6V6ajqtWN zZnw>WJmUeEb%CMn9wPH*(2E_Ng-=r_-=BNSflBInsD2D@fY$U+XL7^7FZEQfO?;Lh zz1>+13K-Tf9G(n6D(O^h$Tsub*h>^G$W9YoetmnZF*neVR{6a3s46aij<4_D)=Mns zR`Zzwyza>DvZ)mdJhEM?!vI~r+1zdbZA>?m{J66=aIy5-CFE^n>9wmL<=M9VUQ=Qqr;ot&%WUbQ7&RkD&q>A-ctW09(=q|a zCzhQxmq(g81(%(V$K<3*XH$*yGi0m7rCq;~GIIL%oG)p_fO~Ez88>@F zT`#L4Urrn`)%@3Z)b`vDi<@y z)o}Q9cXDOJp@2I}>AbTBw5PR13p`gm z44VM~VZEzUm6GcV>9VLMD#tOm)%aZMZE3qDB&ZDE^idT|7*DVzrTz_mIb-#T*6nZGXJRXb6Kpy2q^uXuFE6Bk$dDES!hib@ z#gb*dc`rt4+-m3mWA&ouCTo9+plc4H3BoCfqmuPh!O=auOX=PimP;!Fm<&p`D(Lrt zjn$KqjjrRLYEN$H_0|uOdF7jE2Og(|%EX@q)aftRjJ;fLcJ90=b^@M9=n)ZFpSd%F{SBfJo!2!=e5jQnT@LOhqyKBD8NU5_Vv%$q-%clPr| zgX`Om)C=0bMtLS!NoX7gEQ-5W2#iUK^lW0z5OHeLMPu;}~4&G2ZT{ znm%*C1(K5W>Uph-ED=-JS!3(GQ0j#8j{0QB>fYR%1#l z12)cDaNmlC4V;#H@ysCV4Z+{udChB+UJo{$S76Je7n8SPTA^IEKlcZ*Y?R<7HDbnl z{=X4Ltxhbqndh7beKeAMac>u>r>!`=GvNtW@H4GyTXE6haio-{h^2DrqyF!%|Rqr%e`*olv`{l=4ho z;lJCed%piTz~AL1vk_xUrM-~zTUKv;IPP`U^wr*!3({ld`4R@t6wuJT!#t;%t6c|@ z^BLr!EYf_pI^8@$-I4VFekNM6h@9rB(1%SwR!)01Ye7vyA;L;X@^_nx(j(d@)UNv} zx;+)ZBFz`=XCCKS2uM2_l18(K>R*WSbgW!eVfT-zOoytWGWw@nOfi*vpGI6{m;0`H z`F7F;rch@N==^@z@>8H@wCaU*YeB~gzp}PA zj6LFF>+eLys_oL6fv7X_*vYP1-u8bSoI2Z=^RpVNo+MB#uJCC&q}vXsgbR=+nqP-W2>lrykckU_BTt7f8XLis*Zbp z@&%vRgYPjdGN%#ViBvr100yOkY?yB&%>d2z}`rISiDtb~f|O z(`3m#hl{#_GN%SgcSn^r zmw@=k^i?`|@%G)X(@g&5e@j()I=_fWnmkx^$WQo8xRigYymn{{@p+oGh)Q-fOUDl( zEYlY@ud4JzS|x})cd{9WYwy5HCXon`_4ky`Qc%`&HU|G#8OOHkr{Uu$mSyD(_itrw z7hYSRm9dQ5!O`Fy1Zb+RFPc4Vy`_gOM|07Ud;t=zZ#|h}>)QEVH9AOgt} zM6d?q4$5B$s_XRLRIh3?ckGy>Pf@o|GCY1`YJ6y|wi&qfMwH8hCtN;ahm+1v*fL|| z$0W5f1GYzXHO1JCGL4=z$i7vl`<|A;^!xAL;JU2`5u<1&d*$Bj7Gqna(aAosd*yuq zCoz$uaP)b#oXcP(q2)a6^rF2Z8P_M#wb-pnOff%39!J>TRZHD6KomA|us90dnldo5 zwbW=ZadTQp{`pH0sg~UKZ0{4yX!2LeKkuzXjwRkUidhzLtP=VS@iBRt(bxvVA$)&pcPI9QdWt;gHWpkKWYMsY){tt3g(&}+tS-di9qS-D#& z|Jy;PedV-kcggu7gk~est$hV!+P}kA4X;K}CS`KlV%hy&A8}Rwc1g@KIP;|s^H4T% zrDCkBF{D&}(aCae=ATLxwg_2i=Un@;|9-wWBFE{c@ACGX{=T-xd8&KvgK5Oe(*7aD z%9|`aubENYQz`PXMj&f1->~HWK^#3H-76DdwDGKwh<<$qI_`vy^n)T_`Y|A0F3R}2 z?c3dLbw`}i`5W}9DS{PxT3DS7)(tK{HagMad;k8Nw)Op*GfzzGeeI*AH)-aZ`+b$~ z#1JGes@mm9rX6L8O)<&mD$Il`Pq4{T_83t#CM!8TAXRMsGNV~|P@faP^ z#|-FIKkjzzreWFZ=0nZu5NA?9c1=r${YN0gRDAP_ZJ;Rc#;)iRj`O|iYbRm~sIzth zj?xJ&m2ysiX0QKsZLIrS?dWqh$CkL&kyQQ?gxs;|QGFw}rNA84@1dRQaOZ?|XoG=n zqtadWKAdCRAeLc88q852Xr@IT+>KhKM_2#z8^p!E-qc%W)8*J}u4$}sKpbU9aOz(Y z&gA>b1CBpxT3?!`Yq+OTqIfZP<+ym+joC?+Z8MV0XQCHWy&q<7WSd_VLcf;Z^)bI| zl{zM^zQI}DMI(f6meKAWg^=6iztrbSJPivu$G>wt_?TK#1*Pv@B%eq5Qo;o5f5(n- z+&>k04A@zQH?i{557`MADQd`RRmGU(0a?wU zN#vh^I8Df8%;VmLH|CjR>3sNlZ{Al%D>z;*Nxiu2?#^# z2I&1C>|@HmSCbm_h$+$N(;H8l;x<4J*^LAb43BwaHoRQXlbx}4g&fXw8!4LC1npa> z6iqKIIY^NCaWbgSt}Pc5gG)o_5lAw-SM5_fg%>1ER~~S4UiWqIJLx@6ejQkiU99<; zuJG-v7wB5na~>D;BxNjve-JEb4ApiNnPc5*V^btC*&@rRKy8dT{_FZO>eZ9`rcdZa zN8H?_++@m#mVFHj;zR?y#67DDZza@CK08c5ZdyVvPAMCPpmG`+OQ?JNU#G&fiKmzU z#Qodnf2F3*m85yp6P`+j_yWful9f5+Kl8lkv95p;ySP%4)+DAqRgxN;?!TUZXa8{% zay^f3pGa%owd=s*sns1 z8I8t<<^eYZR;<&78>N%Oyb=mn4_wCK>H%uSYxUb2Lo3t`Br}*S8gZ|ITKHsn;6j`{ zsjf{2rlxuvq$Dft$*~eFL)Zs6W7PErJ>k19`zO)Bxi7FD?VVl{eU<~0t7cT%S3Fw0 zV)0E!`{n&7g~!VDV{zXB^4xIcy4bPh=CVK&AkViQUa!<|WoO;VR*AGa()BPy{fx2# z%WuEJnZCP?i{-QvYy;l@aK6M7G{3fEc$}{e5!FSDu>ZFQcc0~)A|V_uIZKX$?-Hv) zzl0x`(tGBIxqOboQDWCA0`=}q5cQx_3W9Q`%H+t(b!o!SG z2jHEo{qTKLZaI(I>X-zCCY?PqD!j5YbNNSBCK?cdX7OWxM4v~2GI-x#8Q-Zv$3}I2@>5@``uTJYp$CCPWe|>zIg_{k`gf!u zT}cbOu>?T1< zlP+`5MR(VD9r&ef+bL%KRhigG#!+JID6Ug+zvWtUVh5QQu=ub)S36%S0N_xV^z~xS zD-biqy7~LZNqlHnV5DPS zis((c9i&3Re; z^x(C9z5JF=?I{`JLGM{z%~G;36f!Y$tpysj5I;zQ4~2H#MvsWSyP?$q2vyIG_m|QJ zN4-`*bfn%uRUz*)01l_+GY=4az2w6cWs(|P8t>(#Xd6*wd&^_(A=g z$G5T?^DRSJ$V{P8zp{JN(=#o`SR2!fIM>y95qhoOtsM~JFybd^>HOM~nt%mlf@R=EBPv%PyJ< z``x7{nOsi~qR&K!quaiV7XB14=sBGHSC??#MRzNOtu6O25g(P3N3tvPYos33WaW8h z9O~D}QWe^~TNAA4#}z9<$gUc!XK}0N=naF^*vjerfw#KPA84!!Z{*tb=8FQJdmdIl zfd=05ztUlY2o9Jz9Kk=^7*y}oinkN<|L?ns+u)Bn&8I`HTWq(?tmIk(Dgxo6Y08Xk zt&XiX9Osx?t(P+0#hvCrCD)1$=WB_cX50NDp}K0)(f8Spf$fwY3GwUFg+KW7bV+UA zq1Ll`_~swLodna4uhgXklb$(oO-7I&{DD97e|EdA74Ji<^Zx-$xJfhlF~ZoH4Uyo$ zg8Iz}9kSusrMp+5#HV30%BOHir&2PeHQiO;1Cn@tUIw#IV=BQ!@mC%_MKqNynDjGO z%#@;|KYp2EjjzBY2}E9n5-}>^eOYLH<*D~v*RM7z(hh?^LlU4eN=is_IsZUzR0!Rr zSeN72lf9cKsSE4w0r1D%N*9U6HzcAu!JeNFF!id%Wc!pQ(RvIDjDK+(uSMa)epJ|u zXD)8zM7qBeh3$wt)f_n29OQ&Q>`+RA;n;V;Mriw!ieX+>@;zR>OZPN$=J}4kzaw7) z1!%A>W)(uypkZZYQPvUrTf9#m(u_DZIRa& ze(L9qG^y8vM+z`03jk+cN3AbgX3VUZfC%p6`@cRMfcy-(r7nf1OIgso%=i@iaEqw* zn2oinRVa_$jTML_kA)H{@t&cuYkifTV)a=kxkpgb`8yCi@?# zm;~T%$D7^a?L?pdO6Hup&R=BT)3;zI=c8liL+4GB{%Q07VVruIQ_C`&n|M8ZCGd6H zwaH@A_X1Os?{b;^GI;o`@cP1(B!DdLIlvrCaN}v$*iLrncoubgP|*SC79saJ$Fzjy z7!3DEbE;?7C1>PA_?CkK(B^p%iO}M?)iJPyMvoLUnfM;Zf9JEVKFgg)m+EO>vJmv} zl(7=bs=Y&e{>@Bh*VQlZFh55t+eYn0{wz5gy^*#(eN0gF4@qU|La9cWRcE*E1Re*U zhksPpCw1a`07#aFL`U9HdP0)rQx(CA1aumYzEs{mF%kiK10^`g?;%a()DkzgnFCKC zkCx*MhrXo+nnJZVASE5%(QTV82j3Qh1=ZOUR(giwv%|ZT9`U}4*A&vJ@$8;4bl|dF zR$RUgJVMWQkdv3zuGeM*4j};N55bAF`6;Nu|I6 ztqWrR0A8!5T;3GUA=`!;9s~(Y*wAw*0f439`bYb7_J)bhK32pI^O|dM_w(ZJhksxy zhR5dkFn#hCe=uz|(8hz#{R4V^IpRIKx%x7|J3>aRFaR*DgdX0zdpnl^_XzEHTBN$^ zW4aru1kPqk<3k9?`=v#7>!G^&j{wF&Aw%FEkyzu257U%w2uT0=_>w$W2teu1|4Un$ zvb<#dI5U?l#%^s#y=_00et4V=BxjT9sA*0mA`kPQ>m(H*kVB9{FQ zFRw&sLyC4otQKv@0MIOHCkj8R<71tQpOOD;Ftxnc-*=rEQJc)!Gt;AF}<)q^vg!S`y_G~S%C~IgO)qUo;S_D5DLSECb*(peI@MQv=oy8H* zVfG}`e_LO>NkJY3OtfjiLh9;)kw#^T^ybMU*a6V)D5i&s&gpM@!Slfo%@7bA7$4E- zElTW->2SM7|1P^DPxUmh>kSxJAR2v@_;6Vvf**Iag!HRfBBdXMg8hs$C&6%p;Kptz z?_LPm@{5!9E6vh}{#BTb8g%o4ki(N-k;{99FOpswbCaU_i|3Ll%x>C8c2-pX$A zqgyz&}O=LYO;Jd*wugutk04s~s}T*&e|C6CA5ZkNg$%+5Czw8vQP4bmaoFfCHksCs=TL;3ra4js3Z=Lec%{7=1J3v!sXY zB=?I7nTGGWB&PE#G?v!4@rTCv238Z&@tHrLBqn{aE|OjbATK5wOTDb1yFE5;5U2S_ zyp42+op{Bvu?q63rJbDgtb#-I1y}zQ;tn!Clu7`2H`gZJa-c5$lR&+teri+ z){?Ra3LL#OYL@MEmc#}38Gf+XJQ00oHJ zBkia0blkR?HG#1v@Q0OWl3|6uJq zqnZrbZ3P6Bj`SK3Y0{;4>7q2HDIitpMXK~D9qEGf-h1f1iV%8-(2-t3hX4s7x$!&a z+;#8od;evvnRl`>d1vOC{p`ITCPfd@XG?tw4R#l{G!hOGxMP;@jxeXC+hu{TY5||i zs->>nmWuq}d~{0=`cooF{MOC7wrZAZ*r@cUSEDcDN7PnX`mSblIS(MW{tRNvm8crS zvv=cQ+k5+Sd4SLCKwfw3Ki6{(<9(%ln>0`R3d6a~!-iA~ufxnJ^kj}qeLuygk}!w+ zHtbpqj;V+8FT)^7{>4dI6rLp1dw;wG^_FE)e#WZzNbC#edqm}j!R5_7o!>I zI>N~mA1QP_~cJeyhKlNqex z7SnqNOe&{<=T1WS@jKOkSl_REKsR&Lh(QKO#H1s<6^~x56Jrjz}s zw{V2r9)()7?#a`(Aa|X4<4Qr;W^G(lm`s7pSG#iQq@ro$J$%W}{cP}8!=UsZ%1s{2 zZ_IZ=OUdx&g(Uo?%I-Z}By0@0*iVz#&fog#ddo6NUS>FPvE1@L za~7k;!~Mql&U`@cEq-Z|(Fs8D;+q{kC4&`K5p!9cl=IOWKS`JzAKofEL2lK!{4CZT zX2tUz!p*V?`-!*<{O=WBGk-C$v=~#N^EN4&JV!{J6xIqH?<95oZ!N5eC|aDUE7+t= z{&QST10aQ-880SzZ_KI?&rve?xgZe%nGcr!p0M-wyo@6CWEgrK;!V;F-_YPqEI=zr`?e^O2N?tA+t?a90pCU_| zJW_pMLiEx9Kx=po+V(zk;9SV}Ss|C_h0~mo!E!%>5|+It$s!0MY?W93cqUw zCaCN>pgcS2N>cMbl=)_gul9U@frZE4ggIr{#)zX;XfuMsBp>X!1|ZYk=E51RoC}i^ z#?Etdx)Z{fz4Z~FU0`oNejk6X1N+Y+Mix?bnw#e9k4>0p$-cW5Wov^Tz}L&rCOHnI zv)}>mj48|=(|e80$vUJ&QfCL_v)QU?|&oirDDPPQ@*`wSF8aP*|f}Y2@k3XmQ5&}cUh4wV` zT3@nbmjAoI{(+c?Yi)sMR&15MyvJXazQmcHV&S6Wz$R5)OTsgR5e+8%jIrX$sT{w* zo;$}>T~D>29_4gXsn1lPqQ#r5N<;?mjoK!Dl#@Ht2ch#(Ty*xocOC4l^040>Cgf|I znoZ!0Gho&gMx#uX3VId5+pzVITQDOdcYBB__G*I)-R>a&IygasXe6Q)v|}EAFNwO{ zA{`U>hg6s1(sS2exu2+ zrO?*lCo#TfAB!fs)K3t(fTx_eVa#?ty<#nSxK8`2*+Pvu@BXQxdL~&h#r&vbv3JJq z9*X%TU}vbjJ9d;x_;(V5FwST!O&mA@+YQd% z*jg})T%TE;xH^Y}*McxA;neRK2{BO`r0S4V4)zT*AgjY#M>1Gp8H35W8&};68|?1= zJBM#A0C5%DayW4nn@&`<7!2>ZA0{N8wc(Fm-y@RAlOqaf&$><~FJS&zh97sI(NCc* z5*Zr1fOra=4HZ3aEH4M#&f;8)a^j);IZqPtufnM&dq`>aaQj88_Rcr6X8B7+1>Ze< zOjl+R6Vebdtb!lVun^beJw+_`g)*b+TY*4g+7n96xY?Vc&de7yli#8;p-7DNIc#X{ zkKL+{E(zJyx3V}P@$9izvC?*oH$sd+|16j8tgkux>}I^LUwKG-^XJuJ+}jX;>V8_WE=hy5`5lWcyZp3O9kfe_w8>Sba#Ij&*-&*(_j z8$&G8C%E7lJ(ZaWmZzY3@t+^m_J`k#LwGE1-bnza089FbUM8928kUmYV4miaR$z9m05J2TwejQp-yTiePY zGH32?Haf93hTdKWaEBjv*CQCe&dU=_8p7F5eNhFQTl2RQVxOEp1Z35sMU5-LI%ztr z1_YERuEGp>}_E7F*GpEw3Wd$QTBzkiH_ zaQ^_jI|f|}74Yqo+4wW5Xs&Tc=IL*yI$-Ci>2mqs`a*qZYzVz#8XSB7k$U70w;ia4 z#rdPrLGBam4*bm~^pIwG)U%DXhk*UKeE(||R|>~NgOUK!748;$F4vmXW z{~hWs4{QD5GX#fX#70Sdc!CT%ZKfX6Xo)Wp9DELV0|mxPrptq$Y7BHOl3EaD z-c{n3HW}>AXzc-O(zPH5jk$8@`WG=tN;r>&?CudIq6$JwnV^rIs0pDxj3nUHYMd;F z)r`=t5`2E%+(KSS`kqus#A~&LE7SD1Ie9o+bV0VJo{-6reMuO#Z(*;P+IHECyMd@T zuL}Ev!gpFpa!sc$)Q?y~>Z}9xow*LsKQR_FbJ(?!=4JygPltElEen_{%Sh^X9n<%E|R1VBX$1`_2f)y@~A_OiIg8r&fEf$fn%#3*i zeS4e`1a+N!eqBY77^Aztpx;sama39qF@+MrT%ue+JfitkpJNK*VO}~ohk5NV9Wj`? zwLT$RObmc6Zl>oJ>!PK!k@cD#cV=rPJErQSTq7TJkO~ulU^Y$0h93d4sIAjmftJCa z{V(I#8_x3khO;Fh>Z~vc%`1rwCub->b8<|7SFvvLt>mfehc{#PW`j@|;CY`>VY;j1 ztGr!TdRqCH%r;auX43}Oj*gy&pPzVXaNb(16k4y{5!l% zN04Iz;2=tkxS5w=tmfJ|rqcD%_e~=l0%JeLOg~kqHx9KvsaabI0ysP@9OeyK0{ItI z{JOWDC0}9>E$ykX$$aGw27H+_B$3V+Y0gaXl%y<1l4J%` zL`88vLUy9H!s6riCZ50D1L0a@&aa6-^2+FzFLCX0aElPGU_e(8E`OEB_?+!v{h+0- zvwCZzVP)TQNyi5Js_tR%{;P}+pNKIuKmNNQvE^JfCUXqIYcO4eh!K&pg2GwZG|=W1%_{ZFq3%z8>PWkPd!YV*pgGV6Y~f?`0={kQlF%V)H8997K! zX$H#6!D(-WE>eAu{-=2{)eAv4aOhrq#u#HUQu!{HR0y%V39IF#+QsDjXn)MAXHoW4 zHy?i{fn3T{TKQ+9x0Y@NP*pnvhf0$+nLc84FKpntN%vl;JJHI8^@iTi7Xr!ZS#7~G zt|1fzj(O5}3j+4sX3<=x%V#$NS7KuhPX4#Kf z9P_ArAn<1K>rx+6mx+2;H>;C9MIlp*N<;dl!oQ6_kCRJsSdT<~p)(U4%YY$vn-$-jCPq$hE{3Mqp9~vpJ@Q5fGih#zI;Jyi*vAGe-9r6= zn9$hywQHcO+&_^YLr>u9ZL5}XHrPiyy*i^^5*@WrquAGmMf~1M#8d*r!vAMX^%aAH zmYokV9@7H%_LvToL3b%i?4@k6f|UAqJL&-|Yn_MIY_V?jWfF^q<{g79vEkPJ-ul;n z8LyL;ihU#VwXZ%4_0ztVIQ*77W%vs?Lgia-zTnRj++^`EHAZ51tZVZVz$=%5?$7yT zB`s)}Ip~{g5x2`<+v!jH4EN$X!!H}B3=dDeIpPSuMDg8{%sMJP?plZ_079#2|D*ek zP#~7J=WzFXgnb<^EViGRuY~!+JeMvNCNdGkRm-0!{QrKgfX%;#T&;WlBSwt5nt)A) z!S`o1cCX1E9%#>fBfc+26s)xI;E57R zPsR{!dfOUA+r0YPD*F)2@3rn)&s5>iq&UJQH|e{50cdvpT!WeFFSYR*Q|V&Z6RRr; zMDt06;b#m-h@fUo)i@vW-j1ibu3AIL_W5N@DlfkpYfUEGaRSnUl21w`r!6X6{nG>K z`DBk*?259WflDm*8}VG&siMT+^;`M7(oQQ_&SceJkyim0h*2%xx>ie z>&bA>M-o?RC!Q!3K1C-b;`ZN5Y|-kp{fO;h_gQ1;5l(LkMN{p6T8P@#VG zQYs13Q2kPxYA{EjU#^3@aWMaOSay7G2Tnh(ofxWwQxFvDBc&s*PgT6DfqrdL&-CbYqUn$h3iuLtD2@4=9C(!*4%=R-i zUGGa@K_T61m|~8W86UUP6K>}QyPPQYK{(D`Fs%)QGQ=F%f$F8ybZx)D?K<(ST@IC? zmd)r3Dctxiex27+5mBecmFcM)VaHb z{PT4|$JO`)$v>*EhTm6N>s?U(@I;A$6hHX7Aiu8o3%8y$Esu`vnFVs&tlxb*>OA}r zDB}ANnAbGXv4{HPV0kavb?VfSFt+^X**^YOoQ@FiQqsgU(c+lWj>;iO=AG8pYt9@I z0riuc{3!mP;ef!@!~0}8%-yhPp?S%kr`KT+Qncy``N%!Dm+hw&MheJ|^~3JKh_5b+ zs#Vs1WV}PAz1wLu4#a(%M$bapbOC2D6&@Drsj&A!n}8OPd(bBykq5!muY$EKL~m#i zhG3{WUuDwU5e8kk$NX2B)SpA&P)c>(8V$K~WApz2{I*EIA&zW|`fe^p8x&B=9Tz%G zm>B1==({1kLmpH!gp~?TS=PR%HB2|~L>Y9Se?iiixk)bCF9U`|W;9$lNw$Z4$T+Ms zlTzikEKnvCa;H*2Jse_a*UbP%LS^&w)Umd}#Cu$Qw+iG%$X^#y**1VjERci&OtQy; z$1w2}_OL3Q!l6sYLKxiaO+jO+8TvP6uguk6Ndx5;AyB&;B_QOe^1;4Ue`ihFg?rZ) zkP8!Edu+Ly6G$kjrh&dJb*%FEy*6P!JUa% zje6zH&Sk1A(LCbb5K-E^D-ThqFpfWf)u{z_&KrnU+17pJBW zmjyZg7mqn%btSCQ$zHPuya`ooj$- z#mI;N`zMF>Cz)SXW135j=itzYn$mAS+S4bp=A()gG8+8@cm8Q|#xR{bX8fh*CN2-0 zJ7k?e|304{cw3xyVi-om1{r2+emBP*nVnw#C!d=Mod;tH;qD~jZ<1f=G<2Z;`_cRG z(jNiL+STi?P2Urg{tDzj{b!AJ>EFiSqP>i<$+AXh%b{u&(Qvm6(pKfIFn>}d4`*o|i}z9@N2 z$>IG-OygMH7|9HwG$_5E<_@XJTOB&&a8<-eUH;pYf5IUrJmqd*uY1EWMcS5Jz@2X` zX%=430bf{fjq+WC{Z*=)Cy_3!PV{$Igu@1)BP%mniQcPN14piznNh3D!yy9@UN8 zMKos6Kblp>hAjF$>q*0=mKP}@vn9U;TIB!sT&=H zW!d$FSqED3u{iGMEKcy)N#+9hDB@K!epw!>{1Y=^*(b-T$uYui3tqML<0e2x>?1y^ zRP1$At3nMuyN31*%RNm%30F|qzYFi+1~}`v-~E|1Rehn&;__)n9B;9BQKi=hEa;_~ z>$uX~CCnrT{ko+6`~n7CgNF}0%S&o#Az{6;?Pt#`nY?c!UuYBWB zUDU-M3A2V~hjHQ+FKeet!)8e57@Qf?x&pJXDSh{uxnqx`j0d7Pg>-cpXEjNPWE&;{ z;x-I;5>0C6v8p>a!X1ak_i%vM1-QqQbZHNCrftLWN@}As0 z@qD*2(|f2boWTrpc?;CCo!nb`CYlk7HG7m)ADe6yC!xLuJnd7QRC@HG$8)f_hd=XY z$=bYtBs3Uf%~hg%C!2LaB&^A)#@OD>+foyl{(-?v`jzqc1aB@gWKWsRQ%Ecda!F)W z!slsYSs4GTrmRFtZ-=5G{nU6ZHAp@a@M?_68r!IlT6b2+t2izsH~mTQPlpC4ucY;1 zK-+-3LCupbbD4o>+N5rG+^MB~3k&QJ7pl1as;r7l&?N|*dy3QBGdJG{x4tJjMYz`}B|B)U&mJs7`>b%pF*Sy-3KK%t{_RB9s&YYgI zFo2qjQYl?{* z7cn~lAwqy^E(BW<#llMAjycc@&$*X^y?~>-(f!vZR01CU4n^x25hYgh^u(nCYI~E}M zXqhSb-(rj#OToqBcy6~M#<>q_Fut8N0eWr{ii9K!K|~KFnA7>pFLBy^&94ivg?46R zHl=`kA#>Pz;yf0G%#ba4Ygnz%-k!~$*NR(47}~WP5efZ2ksY_-Q6i#M3zOVz+qcR% zd7}S~OoCLV8r)s_h7+bR_Ev4O^IF;{40sdnJ1gxJX)!YL7)Z}bp;-v5uTdZl{LV)b zgXK7mql+Pw5Rf@=!rwOA2i_vTPrd5{?@vJdLEK%!|_(RM$cbA`AsUQs(*8|SMnoY?#aS}pARr4zYPv_;~(C{snL{4^{ zQFgz?Pw?{?dw&Wh8k$GcA7mPWXa%&iBHlh5*RMk14SOeScJgIyKV@`X zbm1%;ETC6u1mayoGf#eu$#j#tng@n>2a#Oo`Q%^ai;xzmEjH z;ja9ald5r`(!`?C#y#ztcZx6qh+%fmo3pvrCzOqEJ#iYjjFr=VeUJL=KcAz$U?55AEIwe->DyzJV9Mt4oFKlM z`5SmR*1UZNb@^$h*L4;D;d3utamt6Q3U3_F>;-3oPV_;^#b*2IP#>s+&aPYyJFUqP?=sg1Mw0-}F;sQRJvmjh7abgkugRw7~ zMO+_iu2(^6%Xd5TrQ>$mi}r+|13}LU1Y2PA8!66DB<2LIlg}KKmh8~KjXXo+v5hNk zqk0yUxP$c<%7G>4!Y zHQZ(EQ?&iB+4>(AlgN;9wi=NrQOTwTU{*6;z)m7$h1s0|o|^TSMeq=*dM%|4(CF%5Y(i^Y48NHGQ_Q6%4B=_<%P*qe zFS=!{jy}-JKzTqQd)-Rct8cUYk1SgE(A7NMi%$NBt~+@KK@JDrf6Uya?(J`y?Q zljE`pPvOV`t>+O~m4r+_qPCPWoBM^m+1ZOn9K~51U$TW`UQNl)zqoGpq4Hs0U+eln zgtg;RcN<1US!E|jt+c;R2D&Efm|Snj=c_lw#8{F9`HdTyHA<`JZn)r<$OXeK`Imrg zzcmn${U4QB^6HQLU^oIy-PBBphRJI$sG9Of-DwC90Z1g@8yI_~6%L!+ogzu0G>J&e zS;yxGaIZXS0^8TEt<3NcR*>>V)0Q&z7wPcgtk4EM+rR8&;-dewtrB~|5*@~+!KAnS z$Kv?6+ANsy1ta6+*Cz$;Dd!=QulX`^h|33D)Rh@=vKX5wR)o`S{!RFl5* zTP~h8@SdS5@a4L{;gJ#Wm$vi^Q=R3fq$3X)sQ=MA->L?1;Vh58{PWkYUX+eo(Elgm zeXau^)Ld$~{(V8fwQ#`c%$6-bCnmMOZ%M1yEQQV|fiK)GyCLJS|E(9I9>cwVsEQwVk0=`rmamnR?^gqsg)bkURHXiCt4xfmItAg|h?m4aiF0tusW z3F>^eX&k^lkfB-iaDeTdxtY5`T9W8Z!wf>Hd+zqMV~6^pCt~C?cuYJo66O$?)nr$k z{+a|n!pyv?v0UzzdJ)_h+D|eI? zpNI>eyb#rR+Y+4>lRFDhAINcsAx_e|tLBj=rxy!}3PIE_84vd274LCtXHwTi#TyjU zILvrayS4b?Tq`?m5Q~Q8tzkkOB-EjHiu3l*?5RkX3C5CzbYPv>Ru&}{-pC9?k|zG1 zcQ=7Jr$0*tL9$KIw~=#Mz$^DfXK?2Z##iO@Y-7rB=hxMzr(Kr@7pGWV#m#?esE%Sz z+Zdd^ zTDKmpH?yH(GhfN%YYEpzQ zs%U%jyBR2~cW)?IDq#*uq4HKe@}qDWd4ajRsj8nB7Ks?6U0(+)Dfo}RTl;&$9}9cC z3`YuVqho&sLMWW6^W@j`(zWr9q7?3iqc#lv^9hm_D=r~YSAFL>c5QY$&G<(G_$O^q zCSI#Bx3Q|`;g4uv9l0sciKs&hUE_2O`wn{EL|5#+Csl6=X!2hgUeOhk?4g83I6Mz* zWA`5}ihmL+$~-US{XSt}Ygk0MgO1YIg07A{G3eCQD6^K-fJdf9-@&0H;#;!%l#ud6 ztu09`8LGlm$`dESFcDmKQOA?a z!uf+wYK~0>5a(**EH;;k5~_eChpXNjCHkKek7Xc&?%;3F^gDHC4cA=v+GT-Ys*zRKre zD(mA{RbISayoK_G%-ni1yYGYEsWQh#^6ZrNk=LX4DSW&bgU{t;!?aqb=<9PE$yn}w z=AZsaB#QSGU-~V^*o|5mxtJ!L7z_t=aGLp_hZ1G_6Qubhv%a{jbRP_%(0cT1)@ZlH zd86X=!p8P^FURz`rQ!7-R;co!PtTSqDk9s;nfx%eV;4RPFVIq9Sl*Jioz$y?&r2^~ z@1&Rw^9aZbfI4n7F)qURQtMrLP}kRKPS>rU!qTjNJ?5BvV@I3{t)O+V?rqiSMBRsLK!lx{C$XB}qhRs%+)eiZA@{0FQk-N6Cjh{JFV4J$xYZVi&0h3XH6?xY=!0I= z3mibNpr9dqK)^E!Ex|mZbW681tK<)#lIg5_JZw_|_p)ez)E4X6Kwn#3bQIITnvqUT zqa)ayK_KE%)Ol5J{N!x~%MqbHpH*%_hA#27RUx-!l-k!^+bgEy3}$aVmRPrIp>pXF z1}TjPzL=|~Anyl@dxpX{MV7Gor>LD#lkv!0Q=bAwaL8Ny#Dms45X|2G58N_ z6mU0fCk6&KKx4PYn`2)w?^hlJ1ix&LPW7!2KQDhR0~Z4YXR`|BPQ#lYZ;2$l1n=6c zfD$lbFAcD8d(Ujl^=QiGZ=kygciiqNriPZ!d~>PvpwPWW^4kJ-g$l_zKuWUw=hJyhUBF2Aftc{v8p6OVA$XT0d4@K2BAEt94 zY%%-A(_9Wa@29b_+9hu5eo2}mYu2{>ur}=rbLUJ(^y4d1Wp~oFRWV{=k?pEpAk#&2 zh|v54SN|CBM?11#X}ijQ-o{R$UdvnjdCQ4c<=8-sYh0gDr$qnYMVrD_>I>9>yJE!Z zdW)b-v=G!KF+)bb@;|Tu^JSPf2B#4N^LSzO;^q4p3o%e5!lv;eKbYBZewuKwI`xk3tu`Eo(xn=Y8@WA~>yP;o$SX-^@Q9o@)zZ z@O~HqGy%kfV}bV-W@y)gZz<{>Hk%{tpwHg-*jeuiEDKqVaAf<2KcR~+O7ZWvB^IH5 zPyo!%xxghVTqr^9byvX#Jzq!md$T#u6#5^H1ilIb3{;OVV>q_W>`QJ{7wq?{ujs3H zqRAc8<&STEL313xhbJ$RIyaKKzw4n*lnbC$`a0V$O?=AXx-=6X9t7vH*MD~ zbu%sXq`fHehE0zQ_W)rd*OigXvs)kv{7df6!Isug%RGZr<4QW$et#&Ybp2$xk+FF3 zZQH>G|6-qmkf+8s;UX$F1MKkaGR19W8&|O`*_ETQmNZ(A zA@Fb^C8kN85+` z`HQKiEnd;iva>W0Ureg%&?9~vI42o6UR~oBDa>*%UN*f9}M zEWN0GNBO*w=lt58)=oI{**7<^G;_s~+ z&zWHCO#9@-%Eq9s6M45!uAi=+yrnx#9q9K4Adu+U87+SFp^a8?Z)IF*EjOtz>Z>X0 zm;1JH!*zHQo-c}&7u|hsjIJM8*b3N(bi1uxu0zclCn3Q0OZa8oJ*m*v=X`l&h)oK` z%}0ab&wxhG=sH3d#^-p_-Dy2o50x(!9@5c-wV&5l#WE>Y z$UR6>&BGTJp8nEDne{PwzQ)#&@n5M!v|8Zb-krep0MG@zZ=FF3TdzW~)PPB;@S|?# z%)@U-%^3rKKX`Zqi`q}iO6RD*!5=(xy=GCWbI2kcuszX;f*e9oY>!RxBU1LWfy!3z zdE8-J@A{bB%7wnD;7p~XaH@e2skVQAfT-D}BgUzCwmAJ!pAgljCp1zF}Vzh zruO%wMq}R)M;W(*_g?Y(d(+=U#Gr4R{f|*>QrU+6Aw)ZEd))&ikZ#ChoVvq#NVkh= zZ6F!{u`IKcEH3tyHcZ{%T?NU-9V+~`s3&HK{*E=fFWV-c@H7}|bb1H)y#>0XjzS?O zzEpOVQ5OW#_Dyl=Te6sER0C%pf)aoS3NF0X>yBxazZ1g75>lf!VubQ_OTkL|4T@LR zzPwJuniQOmWAa6d6;i(gb`l92@MR7@7LmHqR|>s}-!*@Alk?WHgO6Ht&FP4eixnZf zIt&_48Nho=EQ^kUne~->n$bN<{YYdYsDiVo@y+9tzQ?h6o-!1C@y%A^{Z6Q94ZkBBz*Ul@v9jBL%25DI6JwD&(J9H1l ztJ!s1AeOGk7Ggne3NMLt9p2xpo6`EC{O%#R^gM+i;0f(9Dnt-=FCaG6kTT%4b_3)g z0TNFWzU!b?{dQ%9r|bZ&2)N(0R~~C^iBtDDp7e+pHA2*L*&Y)9aR=Tn%YuNFk1OK1 zWD9{SEwyw!@r{-qNz~x7zyX}CGH%m*InJlbm1Z)rI(hbMI>Lm4G7LTEe}{gDIa3a~ zu(k<*goVgyeQs!Io)JGhFf;YPXuJcEAHmm|VL3E-G~b1uhC<~A?WiLIb0~y%$>Z|J z{JS`ucIHI+r3{?gxt7m$=$mwbY@k+%z4lEvd`f6pfsW$1P}868(!)40tUXO@H)UV+vY2TRj)_>FJ z7ueKe?N7YeCnGH{4EDE*PuVS1f3s!jKY^Bl0)(8GhJ(2W9Y|?>e^x5_DB@G>rJX|} zcg@x7CTWz|;$WV*Ta%^Zwqqp_CRqFI7c=(t>o=e;y?_P5Qjpi^usb%&>+=&sZ{_I0I+F>qnSl%@P28{LI^OBir(1qYlVFd|x)lukbv{$}Zumcj|x Fb zWO9dT!+ior5sy7h80n8YRE{Niug(g!sQ{gMBI)<67Bn0ED!TOXiKnVVXC)h&9=pSC z0=9$50Ff3sJY-jhut+m4vhMX_U|E}8xE|vSXv7lY49kCP+}4#a%0Qr2xLnuuO!ZwC zLGP#s&$`@*AzgH9zr*MAmHV69rzEx!jh2>i1O;SI(@x-O%9Uet5&^o`Hxq5_fl{AUdEcW(kLYVSfQw|aU0L1NILiEd8K zuM8dWCy@tG#wuhvD&d8n?e-22dELAKbNIANM#J627hdXIFs~F1*10MLV&;p>rR0^w z+%`0R4(MR$E5|#P_>6$y?(7@KfTH_o!-!8u%hxx5w#f-+H%d9kUz*q|0_j)vyXnn5 z(QynQb zkA`$yG>@Rqd!&=2UqFVET` zS>`;IKp-D2$a8%bECc6T0iY|k6g<$fWaT+;HK+0>HD)UR}It=1`@Hmt~}>2k|eD zrk;xa;>aVhpeFe~`qZhH22uPu==G~;I)x_g8MeHu_|Gr0lvO_^4|zP|TX69|=gP-7 zFWwvYiuOajWhnKT>lC!Ne%Q((Ec@g&srqi#seFaZAsW$okw+RhjynlPh3EoOf0C?! zQ(|8C_${9D9vbWHdsaH~+41Qx!myQM;?_rFUgg8+caBkyQTe*arX>I^88tEicb|+t zD`33gsGqS=Jw)cc5cOqHEWCbfkp@EUHp%Iy0`|u|C_1>jLR-OMg!mM76b#Bes7F%M z=LCFDBk}W37Xf^ZCj1lX6RK*HQ7^y|qF~|2;-oDDov^B_KIz{4WbtW5M)(&VX^LAO zau#X4u3-$1;SMM=l_&yJ*ogfhWI7h4kH8Zf+!tH=>4ZLjhU=Q$7gy7QYR@12s-7%+ zeBYNzmT=sYM|>63$+0JsLe2-oW71Nu`l65%IfNg*k8%Mc{i;h&hTCiQ01o4aGSs{V zZ$aoTv>o{42ny^;4J%lRR5J@Yu&C`qpg^fC@9$wQNZ%9OLi?)#`v$Yc;L{C$Hlw?= zIKau$O(_5RjeP%`#?lCAz&e?N8K_lDv?;({jgF!8#%+Zm9;7q>^hxWLT6Z0}>ZQ(4 zIH_j-uNBpyYjpO(9R)L}27?G00w4Ii6aaV$$5rgBNDQ0(<&%QtBdDAM_zx7~f9-e1NNNS%eFlkf6PLgI@CClh6 z9?84k?7BENUR7 z-)?Vj`uUm_TUhJt(&_Q-P=HdR1u8f6_||LJz>H`8-V8O+9h38p)p2Wa066C^-IYbU z^$VJOO~Z8>o?SM7Xq@TdhXAEEpc;hsQE@jQG1Gv10T3Edta;~5xsYdy?nKFwJIUU_ zJ&1(&R8x7w7$9g^EI(bCXWU9q%-Pk->7z8?QYEn4#Uz95Em3vLi_vYQl_@ z)ZRA9hJX*QLls zyYP?}CsoKqt5`}WP{(%NQ=0i!&wQ}8XP7#V_l(=8)dznwugUMT9Tr~5)qF5i@mjh4 z+ts7I&&psWCgzz@g`0KkrjZLX2b}5xU#~Q@51$8S?Tgo=^c|=_yRaZo@*TD?NH+=r z618|^)!v1UoWpn;b!Krg-o2!hU*{D7NG(EL9tX%LeDn4-4I-?azZJa`Yo78}S#_B!5vLV@iT#@GeQf`b&Q(tuFU%V3t!ixjjjxbi!~^yAN=>F0rX7^ zS&OO#g9LN;vU&bc(^j6LO!@SRxsg@Q>Sis6(|EQ4036QzoN;}-x#rF#Z|wkX$d^~S zsrhLF_^^L|B6b3}gq`%&!VJxjg?@BxbmQQ7l@#Oq@NXj<^Bv0N++5>nbv}eXhn7QU zmC*0r#p7Nr$ZPWs&{Eu{tMIsS4W=Qo(b1$5&WlQT=s5`F74n&C6jN)DnHNLeTli};6D$Qk9gMVjcJpYaL^l5;Mc*NDz%oCn{!V%Twg)-=Bv{5jP#j9I)wrt+Y zlV4Va*C+*SZqV_KNZX+JKWuaq-q`XlE)OfkeVXKK8;*MPseAR0GgwKjDdr;2@c-y( z!AxqpTcGVeWP0Wn-0`W0=Hrga!~Y(ZZTTg9vO@0sEIN57OBk3^`NeeFPbB5YX(vzZ z#XiOV)l>EVUr({P2Br>2?c%u-*JJ-w;gKz$4~dmu(s&U|aAWv{gRqSHHXVVcKVg?QP^xvNg&+>L~QX zw8b;)OJXjM2Vc%8;|MdN2VMg_3yp>40eRDXQ}@j+Fr>={Lc5694c1FZLsh;~y%;)k z^`>3N$|_GM&O4r57D7rlCd)3s{H2_!G;oxUX;Q#9t#dI}9}@=VRouxdH^k8RCPgXO z=NiGqa^g*>&7ke8uFV4A^<(CpLAE%I68DWv_&z8WVsynfLb|)Ojf9ATTJ@B?sk59O znmPRfV#eWwI`2$%2P4tr(oAMup3N0txznd;PH5R(-1D2JJ}6La&#bjxgV}FcI?1h1 zfA}ykpE``p{zYewy;W+HKI;M|2xdxtmcrA?%rvqW4^n((;|t%{@2t~(1lIu3uiu}- z)jCU`s%Xzm)`A&zW4mXvhK0b4Lv_;@(BE-gjVchvl?M4Q!4&k&G6#it^YJ7V$`dg0 z%Y*n}dG?p}^7R)3LB_kgPI0Ib)kH%Q~p@42o* zp?t>Z#&0x{bNIT0p7md-wL`2D>@4<#QfHu_dq#0q8?GyU9jm?!sEf2(e%#008Pas= z!26|Kv$VopG0?&)65fVp9uo_6LeeSGWVN~UBXf{Z0$iN(d$=1pk|8D#n&fPB8%L6H zWtG^eZ*sJn`$iYlSpVRjNy+#m6%=ngPF^*_GVpI8MR30%7DEZtU#1BM&P5HD5_U2-7OX3~Lk zmj=V=t-vcZFX=t%R3JxwGeE?6{flDIE4H_`gsQ&w4R)3Bp$L ze!};zrF`4>9*ls!bgeShxJaMyAa2}Th5-KvU}NiOkWCkN$qYkV999YM5OI*_R4kTh zwh6!W^PDecpzGK+LPHL6GorIUK=w=#cx0p#n*O~;hPtbSLRfZm51b;YxlKerSgps7 z%UjmXw>+DYc{Wp_*j{dygP}@Hcu1SN6LPN;#ALIUJuFZXA+&ua3Cr~fi66R}O@*Cp zSHvSyG;;Y_cT%(-sl$mBrdH8K7ExOLA0BxXL-MWCqA&OhQS20@Z(hMaQ%oA|Fb3sA z;(?m^O3zs6Z#U0f=ftBWSn%k3@T3?Kg>MktKaU;2N-W}oukrk+UOLRWc=ZlPTO8h~B*qAQrI z?BCw9jD30sTt|yLputiT4xQXPFuCfl4$djR%f}plq2{iKKd39KtDI(iq#U)~{qEt+G~o)jn%mnb*nPjT38cGK2lr|YY}I?IRq zUc>stWW%3wKJl~|thQT!6PN`=Oj)Zx52gFMs}+n2DF@HHs)rBLZhB+6wtil=1753_ zmtYoB{-yc;X4{WtwWm|p0DY-uhzVziZ{gzU9##F>8tEcV*eKzpAR1b@h5Jt`GOLvN zG?Y&Gwc4pK3BhaPBWarqini3AbMLes5`N)Ha8h7X!OzrPE57PGfP=mNGn%`C2w2H0 zlGXOyrR@#=b@fD2b{hSj{J)4=rd$KOivo2WJ6tC$JM1cUT5TdA~5cP9s5dKLUJd?YCwkC8(-j1 z-}R2>b~q)SNO@%}X&(AGa0hvO30lHqk;u*||Lsc_t6ZtVB0twHeMgn>u|Uk$tH6T8CwIPzJhp$I^2Cg_ zcbHq9Av4}oZsis?Jls)Ss&De-t`DC=`zNVwp*t}|#)Td1P_`}wWvT$46G0;4vmnra z->^PuJ|SM1mjvFu{09R#vhO4QNY2i<@=fzdG*H66{5}kG9TvZBPg!2L1Jx$sp3mwdMNrbEUn;QCLLnFv*vX6X zek=F>oBjNLST+$6%lc>SkPCzQ^)h=_;}UyIMN*`QPQZp0*Q)CAn>= z#jJPGa3ud>xH%tpnkDOQh$Gwi3(55?t3>Bdg?_So(rPHRfS7vC9GIc) zpz?mz>(Mi!6Uh-4pw-I8WG$-hdbq7maTbN`7_3DBoxhlkn6LAJjAl$&pk;2(#G}2X zG`0GELo)pGRY`!j9qWLnu@cHWewtY=khJziG#I=Kxvt9p|0X3IyXS#_!-C{;E;SM*6<1&=@PoQvb8)BQ^^v)Y)XFiLD@*lt6Sa$Jb;Dc+O3 z(2~R_{7!?y7P@&1G9hE1WU%XSrzH-!YJW0Art zsfOydHTuTVHpA>9_&q{ex$ypR&R_d9kaeMQzH@0MS_<-dF}1-ZuN9kXg5dukvMR7^ z#F{04zu2gttgy8TCau>l71*!9x+8lMlatf0kA9T438BGpjJyIR{c^q>D&eh-K& zd73uu=^>tpta$$7C9|W^AzK>D99C$tikYka&x)17JZ8pod*&q|A7*z8rT1Wtx(B8S zA*0$}rKa$Xd53U~2#fa!#lw+_&9ma|=1BCIX2iiCZ+saCp-Lqh%a}O#D1~2T=+Ts? z4ZLxkMIiX6637M&w?Wh1I`X1^WziQrbt(iKo>Qi8{I&_NRArwcj&w zReLd(WO`ec=~16zjrnRaKdF!iKE;f!Ma&Dn&Xn+7b-4kN@EpElJ#Jti|8dQXxr&7b zYv1Qln87HqMkS#8T=n7-dZ%C+$SbN4=z6O()VJj{CLm|EBTN3)km-+mkPzn>E1jjQ z5#Wa{ttfx=LHAjc0UAd%ct>SNP3b5yBcJy*`iXD!*w63HG;B`{3yVB+oilH#*e61$ z4{2hvmuOCBgum{;BS6j_!Sg2MjlAwZ?9sTOcCbTQ{>b6{{GnD))PE1@kUuT&a#4oW zsQhsnCOysDMHlBDZRm;3?7I!g%p>_g_QyhsCjNsX$z77^U9AAFdtJ2mVy!C&XCc(9 zh(GA?9%dx>g_vX947MRh&sGx(HolEz!MKu&jS{QXt4l8}^O|ymD)PkJvU24aKWzA7 z^F5l&B~t$I&3Lh936-Sd#Y&JA1dIpY3bCFvp1tW>XnqXeNmZl7{zqlv-J?F4FDEhd zOdO?+UN_~VwyX1fs>LV@eR6qv*oJnXte&$i!f=Hz?LV*UyNO@Pg2kzI<<0_CR>hA0 z)$ao+F1VIGHq+w)$=mK9H6{|_K^n_<7qKcnxrW*cxM_6i@Q)@L`tpQ7F+CZN7=s__ zlXn&8*5PK`auSotheQ6Os>j*+0ILnY@^i1kJJ0myuniQP^2FL>i}g4UXB@Xkncj-} z()dZVP3ZB$e<5_4yJfOqw)|0$qC9h?snah}LW;5Z-t1Fd;Ztmo8)dFmZ4`v+i#2&r zhNq0|2b5kjWu@F>_)iv;v)`OEd$s~^Q5B%k=kE@YO&$RY7iDayRl*!QVS`w+_iD)V2OV7y)40jF|D=%ZN$D166F>| z$8K+#$9^>Hd&sa^@6ohAcKp7x?dvhp$P^2q)X&iZVmXy)Chr%Y(Y(+9kEE-+i->+0 zLnvR7+3;yZkUg%ySroD*OJzYXp+Em5|1^Rj-Zg^Gb5OZ2kU^0?j%W8;-e0h2INNTd z-R%D0j>MDwJ)GWDf(P4ePWlK^B;~_6MMKtI3jUL%)Oad{dTqQxjBh5_)~MKEld^U4 zHxN|r_TnQ-VF8nv( z)}zhAZ!+ca&-H8K@ez?F1L2ON=x2zEK@2u($ZJ!jeZb za3N$!RD+28ZGQpn%|(wc2ki$`S1IH=6ze?tne;07v-ZmOH^;p=UM9CJK^C;_vz(2P z_#(@k$#%fVZ?|ZGpa^3z@9~{URp=7Vb(Z};j2yx*=Nl58SXA})MFktXm~kxk+83^K z#a_mgI9kEr*qI8UXuPZ z4^k&10tb%Ph+-`X^vjpPR_4;9pFNbsGm?>hChwk?-PObY>$FTCQ|NXGX>F+}&^yny z7F~%4oKzkVgh*jnh`kG_AgP}P4@svOAE*|l=%<#?S%^;{>cJDYH81MwmLIz1{({^?gxMj31A1B;US4gC+ zF;>h9rjzJ@B8jmWCi)iEYQ`G=J4c&L>7bLiUR>G-@y zQ?gI#_y}B6l!7GqS&6(=rFwqC2YNggtOer*H@-_Lk5)gDHqHBOw+3e*OF#VPAdO@M z$4?(Srh3=s0*-DX+b{vFg5|ZopEkzs<(!7ndKri3h;k?rCIODr9HJCvSq zr}`GaNau3)hRFO8X>C+J;m@N`d&M2$Pc|5*`BNP4I46>Fl=+jeZi-q^>!;t!u3=5~ z0-h$)_bxJgqcvfPF{kOPP`5wwwNDJ$nVb8Ix4yGFjIjJ-DD(i>2_q)i`eB^fen;Oo zeP3oYBMT)v_c?})gQBGuOOPJNn0w)eHR~CXIg;G5WMBG;4OYTjmOPmhg=PaKyWaD? z8wzAA?ESY+v<%ln`UuS9zrUKrjXAuvO133f3(AQ1rbiyZUx)I`JaRX}l}ikK=4s6F z{W~OZ02%Ts=epOF^ueepOi>|52BNzaxH55h%$4rG%>bSILt7@&vLna3b2adyK&6s` z@;+$Hyp>RN#~*?dsh+faGacR@wnU!=}h*A4Dl#VvAHhqo(e43jlY;Wx!Wq|QjZx&1~E}T2u z{`LIqkp!XOZ7xCYqABRTq7lDoQW{v9L-I|fZ#If!JHcfHE*qC+Bf)lWpv94B6Xw{M z>;4hPaO~lWa}o`8P#wnUAnpwVRJi5A0S5pznC99rlBd@yPaTH}xK$hVEs*|kfoJh= zC1ReX&>VxAck3bXTQ(}63dehA=H!-pCVj^U=%2R9#X=nO3IS=r_74&ga&7KNGaq#Lyx8FeO11rkM z71}*L%8s~~u(gquxdBg0Hd?oDFU^<|a4Q}Jp z`_8F>AW*lI%O!7qlf-Qmk@sUIFM%6)cd<7&Kja6X?>irI|y z#DFXtU~7t?vU_#vvZKZ%W~PwV7Np3-?UI1bIMEtLuZ^#(=F;m*h7JX*KJ*EeS8kVewoXF* zVBeQ4@*Y5g6HXG{y6#AV$e`bx^v;ug`s@1{>}EAQIX_OknbExw_Po1!-JDq)^qK4w zUNJmfT^n?r5BwOj^TZE`b9{LzOqj=4TISUBgtXHbgvaisEA=tqDKVY1SKne_FW~Vh zPT5V#N3-zDv6ivb%l3$E=JZfjM**yZH!jqM1Yz2QTjDoCE(px5zv%1!ELc~9Q>3?9 zG*zZcPP29_^&}0r4(22J1isd8s?dqO61EXYp9?4`S0fgSB>YxiLy`B&#Pu&~yRu%I zlc$)Pez@uO8-<1rvwfV84h7MsfoP=LtIBNz{`fvj=h_der}F7n2feJm*?sX{W9i#L{_p?X;dicBS)$x9D zP&FR+);q>br*gOBHjU)lj0w(aMk3i#TtY=UEWHJW4o5tlnfmi3f>Q7wH0$adW2Wrh z3bK^pkr$W?7x67V^rCV>OY4AlScb`a^ll(ZKQ*p9?i~oMR2#!xBGc;bg;_nTTR|=E zpjaviMQl2r{9=(X5dCngJecr^z_Z5h7A^X=O#WTJuu~@Qu>!T`K*rC{LflTDQ3%KS z1s14e@_p#t3iw!N^_*|=K!1wJ02!3&nbfunM0Ck-LeJvy!Zm0!-DlFyLNh?1-`Mmc za)O?ZNN617bc=iK|NLYmGtg1a33SH`0bjGu<~K|PkyP4Pw!BBFi14RE1=(Ghp8*9; z5r-Y|v}Q62dhsHJUX3aSbR<#~&&dCuA=d{k|7So8AaqaY7?fc)q1o%E+gKHL%Wdo+ zTIkk=JhJoq^DKEoLE>aQ1SiJl&c3>Nz6y*;~Ff>Afa=`V4_Zv>_cL>v1|P{%k1aL80v32dCwF!2Bms z6WXL8^?f1smUGcId;H1fnG~%o6#r}eQWRE>S^34{!#UY#14}l17DwcMR(`!Gn;riH zwvE^LMLa_a4^+ScBgKPw3|ILBDCo~?COK=stmqdaF{!hfwGkIRleUpdHj_4ei&xL> zeoR<~lf~wITpRvAZ@&o{PL3)lEShi9Om)dbbFujnu&f;ahY*d{AHl=7D7IkFM7Gl6U$o~OtK_;*%bTbZBsyC?HrDb%!Ew9WJ5%qXZ(=A&J<=(crHU__iFSc-t&41 z+&x($|7lF*Ws`V}M7+Orxc_2aK3t>Z7FegCUoTx->kZZ%!R@Z$UlNTVMOSlimsZQ9 z`9~hFu%8jbP=C=*8!A5`g~|O72K&NcO|jK^(};&%k2={rvx7k!Y@zEK$-V35Y*+a|%8=pSRW+8%x{($^Z0(fB}tQ%oeqO885a@D&n9K zgO1F-)}^}n1o1xl6dwL&ry=7k;CIl&R{@P#xBcu09Ngc@pHMD=^g%Ug0O1+5%m5?O z5?1nR!maG0PSGeAdf^5@oN)P!g|`qH%_p_j*O+4ktXj85E6Co@aoGBuXwvQok`PGj zcUj!B3sluua^R5_?_wLyeRLpi5U3jNZ(S%K9q;+$x0C%GAG-tpIbm&->FpnyiZYV7 z4xNpdHXGFaNyVn@WhqYdz+960ukA9>2Qe9F)U67kd;`e|A2elPgsg9E1GNq96_}+- z|5|-_P`b4>3-;)DRJs@+{mungOEH75c`aKo<&rs*Ouk@^3}>l&<3;;N?Ai;!Dv(Vb zIi{Nq@?W8cros?nmueVQWv4-W)H8M-t>h=hE4dJogn1XO3*M-e?`z&AzBEa4b zN9O_R{;LR#r)Ptu@s?=YFO0&NMl6_dJkF5F-b6OcxxAt_=PAf}d}!y_S#o zlB1ns^`gi=7vDu?IGl~yhh-@w&9pD-G~wHbpcZ&K6PC}RVGB{(ckBhdMTE!@-_E?T z@w*x>N%dYDxdb{~M!A8bZeAYREC3=#R&Rk#H<6yG*9mFAu?=;iH2JYQM%>9%cOY-s z=1V9$aY0ft+e4CD)K<)lGL_hVq3e9Lr$Vo|aK)I*=`OY!;&16}AOkL@hvCnGETW}% zBE5*Cdi>6(plv_={6J`9*~X;LXB6H|C{)?Z5&K8fTgQ)&z{%)KZ0bGDD{YnefX0)& zheH?o& zB+;g_t?(=(qUq3BmE|UlebNm~dYkVh0+GwrGvw1NeV?{kckmo`>t&8x+!CTB|2e>HWc}|;kE#f9w zndVT0td!7vb&De1@}(p&eJnJ?u%S7b<1+kN*ZZ+g@qYHPDSMG;r%&a}Wv4aAXEaLa1gbhc%;e<5t$3LN|g&A&g9G1J#G00g4mus`3 z{~dlTj>YzGwHOKiJmI{=4D)==XsQ+n(dO4{oL5TV$BD0k#R=W`0wr)hCb0p&j^h1VCsV4@J0l zQ?%s6M;)~Eovt!T-yL}-X(F&4-Us89f|n$AjOg0}gzDX2j9;=W+XG<#T+;u}3ldizYOEJaqVrx7^j9Smq!F?v`bHn zRepy)t&#%6gJ8+EY6NKQ@&YwzFVuVPy942(Pq!a@?aa)8UHdm8u2M{CAVm|$8M5WG zZWL~{nftI4iFdpZ&9blS*1m-rsrdN8=>c1Hn@KifRVu z&0jIl-Kr?#$zWQHxo!m8I{p*>3)j3pEk<$eXaaKWzaJNxFl!o%v zG>l;q(|Y!aq%Pow#AWDLp0db(glT@^%&UfTWZ^mkp`~E%q?zXO9F1kAaUl^yRY|#= z5qLifBKWUg3pTZ48mRC#Ho9SjLj}6%^yKs0(5fj3Xo^r$X+y@E21G$sAEgEOEK4t{uy+1KzO?%2(arhpnfp<4`mXNA?y=n4N!;F+L z?Z9x->d;55+4Xm)mIw{d>vBvzXD^VEsj>cD^jNvceLB9$;8%r$NiGFVrIKUc4$A8- zaUnM+Z~Ovt#Du~BQ+9X+uXKu%i%mT15SNkQhOmp9MC-rUVx0rGU^$N8x@>Du_^H~T zex!N$>IYmnJ>%$057Em&fzetcF*gCfJQd+GlR_bf6HvNjTN(0S>Vh(mS|#`~UZ0zwJ@|KWVO7Gv5 zsDfk@ude-I+nf4`s}!&V71YpuX_6di!LSx3T^E2 zJm6T+2+5c3t}QhQubyH-$-B&b8YWN3h!;*#eytp~{aab1@)X(5Mp4W87@{a{oTazE zuW!t|9l$)yNpDGR%Vs&d`xF|&zWgLw>=*erD%Ji;W@YMR6E?I*3y>AFi~iZk*)tZV z72Wj^5-FlM@rdv}=czAq8m2)n$ojS$gud_JSqD!@B1mgwcj=7#s-$u|H(a9m?t6tv z9p0SZX-=MQ>>Zm0O&gmf>c14Li(GT&-6MGKhWaMLZ3j$n(riVp-rw*&&MW^zel9e`PZI8oK8?dU=8F zxp!qu%k%M^s@rcHXqRb7+??Ps(6rUesFLgFJ2}x?Q#fq<9m1`*9AQJ9@MyQ~-mjK|BFVpVQq3_>lL@P)=H>{8mm{)U&Qz%17KWPuvF|l}rUSQaG@tg*q=V)MWL%V}oWlzl zc{UuvL6)IU6k~8Uq#`IA_K9TtF@2v!X!cj%=RDhIQm?g{iB)b-SZ=(vPPczTk@?p0 zhiSkC$@d43Qawl3-CJ-ShOxoq{OW~G;0{<9>MDBjtd|A(CNMjXtMVPO5J=*x!DcEG zV92g)ZVumTzqYUg!$S!CKGB&+3h?4j+Y{U!GJ2Y`hoOI{6TZC~jfp>>Q-gI2DAh}RWR zC6xOxbZO2S4K)$djsu$#qF?R_aJPzye^LG>o`L#vA5jID*wJ+qbqmCfln?=b9FjZZ z#KZOtZEn96Os>*w)tmv|K(RKPSt4L3r<;*8)i87}Y=df5X-s#&4-fORNgCGopz4h< zbJvAJixLrCboV!;P6UPE1HoY-5x8q9ED<`vrT-YQ_u1ESCI0XTpdjtmEMEB9r8Nks z&p&VIPY3_vMNFv;@cDo?ud_%&YWI!18bTzZdHEg=x)ttU+E#u&twuE>Nyb(fxIB~;Pc3^7LKRP+gUFo@59;a>-r64Ix zMrt?5pj~x;>)xLrt+8g{*IGlE?&M#{c19rINo zQ4fJ?bPj*XXw-f_;=$SQ^2#wT_-48sYWnV?g0&>S_hvP6dc47ncu2J`AzO~#;*Ho*Qw_8dKKhnVJGwon}F?C1O1o}r*A zxJd_>X7sU=NzoQ;GYAq`(pggKBzoDLoInJu9Vf6KIFi_~C5;xg;_N_L(AEfttSXTLv>`Ej`?QixS>28RUt=ab5AKcry?YE41;WA7aC?b}YNg z7bdsA^Z|>n^hB5jR#Fuc3mgu6yCied$+k&seyweH7aWftsT)l40~2mknqs%!5sSwVQtM#h|uRL#;DAmsR$#h zLw>yPP%8aq7K9;o->88>L}z_Cu*G?nC-gUVwDQzF(I%s6nDb`soyggj zYuMM>dPK|Q_H5|WdzY{LCJ0{Xmw5+q4ZuoUqA#^{I%~>n7q|P(y zU#p2BqMg~g7N)X{-;aeSh+Cv7p9J6fR-aCBXX`ezPaMS(Wy1CC|`#AxJz+*6}-_MU=3cQ!5AiL2)!`d4i4-|hcymTV8yI7{7j zy*6@^mK3-O%oz{)m-y5BS@zwhkFQ-0VJ&_^A?T_tc)O;SttUTD;zXub=O2-5*_VS? zv>`CMn+5BZL7Lr~^_*Ot;#ar_R0Oc>cZw7h`fpQjy=MwILdNOVeJVZ*{ppjtZB=YD z-A0jX?u-9FLk)q*PWu%={pG%uc!v1%`z-BZp$IXz^KS^TP#uTvOZeXwZrP*oz%ud_ zd)yx)!x?S<@%j&{cy7Vli5^Dh1S>?3p{g5wD!^`2Lj&2s_nG7akP#3i$+ z(0D5HE&@B*F*-f|jbBpZ33^Jfodl5#k*zzRoCbdWD<1C*&2eTsRpA_iD!CN9b%`+t z>=cW0-ZaE6C$Ke#N|3LIgfrQ5jg$i|H>-6DizsJm zqkAv+W-etJo7=t+u6--(X-F|wOs8ivVWn5}HXa5WGkdaxqu!gD99A88sLF_>oh7;4 z`kx%C=6|mZZ!gyZPV6ekvu(E?MIRe8jy=?cF-CS$DBpP*)copx3(bYGsP{wQmM;Sb zbLn0m$4kZVTU2%~R3#icRpBVY%lN08tMGS*R}ReY2UM;!ot1gmE|y(Yn)U`NlPfWu z$jy9*m-+g)4Xt6PK5E?H$0(1(%}ME?+>r^}k_C>d9@R7X^M-rFCVd-@X4!1-xtu&d zjxutZJHhfEa@#z16Rk@^i2W$%o-nwcX~(fwo-PPoqjFnGMI?YEy1m#t{6-*mN)E&B zqY*Q6z)bVZuZ(*|Gv_ahbkqt9W;CW|z|OmQ-Z6cA@9!LUpyTDdbR2MLUcs3Avfzmv zJ|aUuM6M_{egCd@(4+DDEqAWka{LeCL;+S=1sp9>pw zaenus^>r?DI!iDwiyTEm-T(97o+>{;6M;z;iwRJ@Rb3dt*SN41uJQ@_knB$1 zVH$h(Yf9`bZbtT|;|FSJmIOt1%UCty@6=DKK$V3BN&N3k?G$56r#l3sKj@5JXY**mvAJeYL(LAPJKmJeTe3!#ukjJ^> zLjf-R3?0)I4^!MSwjT|GM>O)*uSwYAEivCzTO0S2-69E9>q0~Aod0TO3P(Tw79O|b z*@vRagvEY5&%OxB=8p4O!d{BT@wVmqUJ9`A9Q!U;CeF65Q@uI5+}uz|a9-%HZmu&Rm$o-m1g07_x zNWkU_Hv@7y@1M`;0m0uc-+`_Km8pTLmeMGx^Jl$;)+o1=jHm8gq7X*_%yj^4-@r~L z50b*RA$|+w3qV5@5V2{|3Ie`1V$(u0vDVte7KMIuj@sE(UyU5FkjPni6(-e9aJ?8s z@Yw0OVU0XTtTr_et!1qz=%Y6pNS=HA!^hZ-SD^TUI*lKlDY^L5P7b~Uqf~HPvyHmZ znDV^}fA3wnFrt8s59ZS~Jq~KUj8Z^a;51R9PnHWvbuI%Vz#0PKQ9%7;7D8xc`(BYc z1QWWm>!CWGW??@EQl$p}JDV7!+QPpC|+R@&&j z&gYSt-1vLcy4k*EgKPK1e7WS${K*dWFI>J5tv#5G2>&eMxK~9q(C@v4`-pFmHuMGQN7`ZWlUy@`vgY-&k%w`qg|}G01GpyMHCJliSN%(6i?Bb9fU@4viCHQdPm~AnKA~ ziFH4*4lQNCFNBYU*!hLXOhC@jSQu&%j)k?{9IyP*byx=2?v>&BRF}~4O4xW|5zR>L z!r55y3~N{!1Xk`)iG8gk7(z&kq51hU>Qe@1bz@>F}J11pN?+7>}!s{Qv``k81-DtONQM zhI?M=B+z$aOy(Ne9$Bd1PW&q-XR#CwZ#zHM42aZ)O|E`m{sKChdZw#`MUP*Jm6|lx z+|v*Dh$N;d_sDAZV$PbfmeQB{UpF#S5W>fnb77^SnzP8`gK*6wjp*za*5Q8W7Iw$dK_0&Rkp?}$NYeQ>` z{>9ZV*)JBiEEZZk{Bg9VO@+ZF!ta!uci*!~bMG`2wu>PXvAJ{_CpCEYO{-SO-hHUI zyp^){?ze*r14H-s*1i_(-(LgixU=Cma-7K-XN_mMyvNI0IG68&yfgdoKm8I5o+fuV zCio&_n5RTI;5ATc@^cm)-1RIhMTX%(i}KD*AWpEo)S(5fmfbmf1oTV;jk+^{pogd% zt~A@IqGheLf^T{oHv@DG!D54yS+<~p`>+&vC9LO-pc zB`kM4GIFLxXP6&B+C9JZ7fbb5PO${9ntSSgH8{Z|r*kJUcaa||6|+*hMx2A#rB0a` zloHnO^DKeW!ot;bD3zxYx+yCb#AhjFbvsfEf%^+?!q2?TRT-vWsxi0vq2lfs*Wt?u z+$gH__pA7w;Bo2ej$OrSmXn9eg%q^<6yNHJ*m}&E3dDL$sMg69@yA)&hXhkss0tVu zCx&<}dnCqeI?g|(#q3Yu3qRlA$_T6#LH?#wZ&)N_X$FY0V5IAKbzp|#a+t0`o8_Lg zyPSzMq8a-^q@kbE_)*EnYFVec*w+|o;?ggZ6xG$E1mO>nEeA>D{h@!vxY>g-orOPN z%ncPx-l^F=KhwN24e;z;IW|gez#R$vyhm{?y1hnEB%!3W4r2G9TkCYMoxu&cEc^G%6Yup`REK?;(x zyI3pW7<4MM8E5I{Mi^OFOT-yYFQC!=P%<^XiNBpfhMN2lIpUtA;(n?jndz-fE0~LOv>&c9Md#Soj&8s`-coz_51^1 zr;cqoL~S48)nbq_5RqF85cMfzlwsGA9gAp*uk9zc zS>acW$-KH%%jNOL3Jp~G`)Fx z-VrN8Z{0PBG7=>2xnn=jn9{P^7E{SeOR)cd#s#yusK^Pmp>4fwZ&e)o4OAw5pZNQ0 zM66Q3Ql^KtK{ugT>F1e|eD)(8P$|%Pk?QbD@^_=}JY03I~lpl~9 zf?^+iYP+-!Z_TyvP&sIa?-NqWSuv?EXj-3t-;GbM{Hg5@ix%~KF6<6u4|eJz zzlUoi``7&KMV-u|K#*Lr2%1c>Gdu#T4`$hkeg~Z+%ruvGHGO_!<6_i8+I(;VB8Ic#tE zOnEvGiaFviJ#r+0&V)zGF)2@L>s6L9G33$*iRDZ?4}S)Gtw`7%%YVXR22-w|PYkh8 zr^(Mh?2;ZIA601SX4m|#Fkl;bH!Q`|ud(|~voVL6swW}-YX;wwkGFM!bDtncSSpy5 z(;=N0H+A{7oY`0ZaWL;ATjP!Na6d4=|96uA#9bcFVANi_?sg;)T7DVketCylC2;e0 z?=MX?Rq@auh%WZ&e`C)WM@$>$kvfT1zB}I}nq~0cEXt8})Y|T7|LfSnek1rMg<{^6 zfoNB*3i%8e2A6|NjC3%WSGkLH#Z&JDF}Efo`|ag2?B*Ffw51<*^BfNLq2O1-x*YQt zHdd!ZDxZsHG_UxVsqGs^c-74WrY5NjTy_k=vP4F?{p%?O%|cXtGEGQ335-I|^oTRd z+lTNo?fL{!WA1}EB18+I>=o0Aw~5fS>+?Hv+;egNSJb$?5w7vCqbvT()JdmPyw4_3 zxn(osvo02`2hght2HylG3$JZaeP*_-IG0{{mcA;Nx2Y60Ht1wC1s9c)s(lz>98OU) z-=nOg2L1vp=_8f8MmK;mnDKjE*Y~CvarGeqdn6`~raMsbIau#X0K*xRpAJ`>+G-4! z9)<3ViVf6WJTHaqJgC#b*CoBrmhj3>S2)LLt~U&8n^S}~hi-2-hxy;>-OlyS4{SW? ziC}SB@lbmw|S%y0NpY`~GWu5}z?-j$^f zVpu*fY?5Q&dX_@|GbRZYic|NmrV@Es@Q|0#P7}DFomS>(2E$zQGY1}i1Mp7)CpBMh zu~Rg&{*hPZ@s|@jI0{MueBIRtTDXDqJveVZdE36@UX-?f{*ohA`{>7ktWwagyNh1N zZ-Wx1DiYWP&>n+Avt+PTh5F&c*P6_{ubC*txE*R~1roG(lR};Pv&7pG3r%!OKcswrKfBSQ< z(pSeq9E&?}%*%_O-cDT1x^YX6mi=+`Sa?mY^^n;;PMa)xGAd$A`RPa2g+HD#iOo-* zl#hPM=dXz8PZYBQ{$McAGYw%3=oWxxNiGFO)Bj`%%)+nK5Y2_wyNi|n%F5LuTQ2D*1V`()1_8EKbrmA z3{NTdGKbbC>-BzdV_queA!QBdoj+Lwz_ov_?}g->^y%qg3Wyi-wwZv`?i93s7A+(t z|2V!fH{QF8!PQ1;tA}|nis;N7n2uox8q39dBqw5FZp7NrD@#PbZUu@>9Q0Vut6lzm z^TU$G&@|USHvG-_=o@(v*C0xg*Q0ysBk+SCcywI&;Av_;pNv>nlbjJ$Eb2!tYuLj# z?MFO7>bXG%Fnp2yy6$cR$S$87VS~=_N3njO#j?qCGeBpTcMdNBupLV1RwB)QI#=vP zt{#w{3LI7eV3Cv%HECFNRRP|33S3&E5_t*vlRma}uZX9XW#5PIUQuk%YJy5@>@1vo z5I$ULb?kpklt^{W^clv1%QrKDSR-1bQ;tnd;w>K<(Dkq;xN=d9_QiX88jeA>YQntg zq*d&HNJVzO(f-+3qLri)+D?FO23np0rF*6@GzX;Upz_jq7`sE2ZRvSayBDkuRN>7^ z#zNaAuEJRirC?L@_DW^B?kXF8K|xCzRVTi4e;$(8OWV97_Alh5b=huz{@tqFZS|Rt zkXD}<#b=?ffu$E64qXLk7Q9zW>ZFnRYq~9MrqqCzr-g0)UI5scs5(#fQUo|()$`^H zT|(*hroX7oC=aB0Y>Oy-NJgx)avmCkE`;8QSFp$A!o3m#L6s52pfz|H)U#vj(M6E-ZXC()kp*oe+l|4j&Aq zZG~rGzp9btt4WkdY`%kj#C20;zh`!bw|s|NA~ZL@AEO7)e%O?wi+z$5aDyzIy|OhZ zfkW7(Lda?sFRV7>@@v!}rH4;r+tZ9b~(WV%K!nnvMjlkRsLe^H!qQRHxEAdS}0dLF81$zzPpZJ4h7`e*u# z=OcELNO8R%(yo)jyZZ7_4P*+>AADq4s}DYpqo0cw1})M95W%*_d{y)QmbhW;zWOT| z=;X1bAm0!;?xfe_u5^km^VxZel!IK@#HQq_28mS_e;H5RMQJI&z8$J7I%Wr)3K$=d zmD(oRVuzj@piAHj_gV@UNU+N=(y`nj;KPV;e8Yv~UJ=Wt8|*38=v{UxsMfozk5%h( z-tC}1=ofzD5Y-;6_@Aj`{8!xTuZ)T^^5Xk7kXY{mm2kGdap^Wrk~@Ktod>2LCEiF~ zi;hYK92=CiLRuy*i_e^qHMK(6l2{_HKvfo0_Bk7D&$%+<8m;$1HF7Vi@@wK1C$& z2YY)?k;J$RROhCMYUp2qfo%W1xIw>IW)rbKh=k&cdv^y{q6PuB!Aazf@|NguW;g=>%+$9SJb&L~O{%U2Eug4D9 z&sr@1+mp0;*`pv^Ufuq3KJM}oRKd({S*n^PM|b}|zg$aP0Cg4SPPw{J*Rb7r9WY35 z9`pbIX!`1aru#44S3wb^yF);_1xB|>BQ2fMjWkSDy1Tn1M~B4dlWk9yP-l{-dcaZj3}vz zti+^Q?iW0gGgS)xxp}yV(JbEOfP%)KF=&cx<7hk-g}0$Jf$m~StWY1QZ7iVsPqXn8 z8Zd~#`E(AT>o&( zer)VCe%9|T#~W{9HRP|D1`gS*|?F~Jivc7Lb2v+jy#mfY)2+_ZgGHi_?^=UwmALrHps-{T#iY>m*E z15D2Dy16YSCx)%5l4@I7$yFYM5}z1WB)_IB>((L*+08?D!Mi>L=t%gM#RXZkmA}^VY$a@FOUKgDd!=#T#6@FOq6-hQF5 zVko^ND!r$8GfA^uHJ57ylW?zv3sS050YwnOb7e?lX>@gSOJR++D*<0hP{Jvpb%2iW z^yo;j(IZ=N9^R=rM|@8=PC}Juf^;dXDOuAb^nLC{Yv^NSSqV~wsB^+sG)F<}r@6_g zIq}80pPz1(p1ujO_A|#)F@RWr&(S*5q=Nz4m*RUm8}u^lW>c`MhLv^01lo*eFh|3Xc*BP0y)k z8bzvq1jb8@Kevth+xr|+m~=1`XSKtueVrtcKPOZ%&~?4`d7W!Lso}Q!7Fih)+5bt1 z35nhyUvXL=+xq8=GBpJuEqBQLJc?J$+9YN8Dy#i2w{7p}q8ltWNOLaboJ%FNZk{@6 z^Wn^{{$GKpjAX^@u(vb?zu+jz-|`Y4d)O<^)!*tz3m?5eqImnJIIO+Z0F8zLy$x>*ySd!hXa9a*2hH7Qw!r#d{E zVrO!QbJY_%3D%%a{58RV(K@@dZ*#r)iTov;R$k4nCt z%71d@(nLR^yXTp=ETMBT3|;y8igv#=FY1byt(y`_+*P_C_LVdfpo0E>}qmIpcw`*+^mQ??v z0BANB@QaY(+qgiaBN|lYxNU~4?c9#G9wcmTdQulZI9mYV$Ja%Lft{k~fsk(bzWtguiMody3nd?PAUfg; z)baGb?V_d>lKVIfyWvz_`_wVsVwC(HYnu2lQC1RJA~=3atf z(_m)c8J=x`7JPpYk)JV8&Q$2|4X#?6AVOTLaTsR5pMf1 zPELvIzTC|-tBb=q2}W=P43T>}8#ji4l-FuLW*$%YUaKu38I@-`SlH-C=yS;-IJ>vK ztO2|9Bd}`Xe~IqBNNLZ@DvQm!1t2C8o4J&RzB%i@Epc8xU3xepgO-DUwa5mHT#f3s zlRp>7&cmj^acHwEBsi=pmzX2!VD~ zuwnY#Fo+OTs>j{2s@qL~*x`woXn=h{@JHef9(NR?O-Wq0)4LlEZL4mRhxY^vri?;v zSXEaJ)LYj6*ezv*UmvVEz_mj_UQbAYmUZ|78+hu*siE_*n*7Obopp_3O5)lg00!)Ufq^=PvZBHRWn&!QSFnDn|lisOW^ z@1fyPDgBd_j!MQi+vS)NTPYUgmR?A0j15&(ZK1m;6>k;2pFJ0~5A?lY8%^gzB)j1} z6Jgm?cpfE=0>v(`p{@`w7^}qT1RfM`0{-*jPLo<6IDFse87PnfHWJ#0bgl-jp1zv( zx`|ZfF!4QLyK`h5MyEsawUCE?Fgw@;C#g#cAq$VJf36nYVhQYmwp+m=2I3b@9xd0G z6KpseQrRC>3mx#JS!a%!uh9kdJF`In>wate&;k4E_lQ5EV2#cIg1|$6XR((nd@t>V zvPN=jt%(b`X1;iGG|bmdUbhZRi99z|T$}9`t9wrBeq{2l30AiiK)^ z$)5j%8<0L;XP7JOyeVAv{R7F-Xm@F~E)Exz9%Ss>TMN1qPu=MS91f2UolS@rB#-C3o zp!tNm`YZm;9BRJ(xv|*AI?JA!h$NaiXW6oV3j!0|FXx^TcMWcC0lrr7(YKwQAN>^J zvnJ2+v7)uC$O<9B_QJ&#xr&b8Tdsn+xi;X@D@Ap(jIF&n{)dczp@SLMsaXY*2;>R8hWT!|4|}L+ZT=* zIaVN$e&d1Jh5x#$*g>r*C{ZO**;CkE1(Ierc*vF`ePvBfV^>49-y6O6z_s%Q<_o=C z6!mJNo2hEgp?Ti14x>Cn3ZN^kyMJu8+`ya_JJCKtJl<9_Mf)I?u@-WL;AUG$JK}mU z7@*3mb{Bw%?|~se7820db?qGZ&k5GS%-x2J z-(M+>K#cF(FN*wPf#?SFbZfbR6BMB|uKssyqYAg;A?EH+4vi0^OiPTY5;F+|>o;kI zsW(d17O`Ev^xtt}&?@#_?2tKu%2L5K*DvJ~9FIq}u_Yg|mtrp|2wqUBHcHHUP1&1h zARE({4s(&`TgS?`;}B@kexUD*>Wg|K_)<7)L}I=aS>f^vJa2vKy>!L@){Ku1r>rtI z?6Z_T&<1QK*YWUubmh~^X6Rb^>iZ|Rz2AirS8pf_-uz8^EcwC*A44W4;+4b?dNi3- zm{XjTxL+dO-@VJ33Y960KUT+elv+ntQOVXxe|uSFds1mqKxngL>mOzH+~)|!F4^6U z=ke8~?`u!bsj)Ds)OPmYJsM5Sx@sdyP91+zEy)}e1m6T-+|oQ=UJRH7NWoiPcfx$2 z=W0)1g|?$eQvLLI2o3J4gy>jJg45CNB@J$#jbnX0k^WXBsNr0UDHR6@Plpbk1okoC zRt4x9xh(I2_QW<>m{Qdqqp6TO`<}qdTwuHCxrA8x{nWCO$+64f6$w0u^j_^LIG3~e ztrVOCRb+;)2@?01>U4fqY4{gfnyqIMWP>b8)ghknYI}5jMRVYHh_{G5j=8;M+J{Q^ zMq#8@zEwOTJ@bo3F_{vyCKWlH3-mQ}^feNP+miknB3a137~#L-xZ_I=Q-{wtH`^l3 zLifK>TuWn*@fKb201rB7w_>|>y=6#Gwyf5A-!}?U(Zk3H>?{<>0oOML(z^ioxyhHQ z=VF8(!Y(FGy#2&HkMI_g@zRi$RTJ2yH6>RT0s0|t;=oRb)Ssl>hdbRaMuh|Yrc{gS zHp41)NKLDMv;>yVx)DzIfro%vQ>;1L?mFlv%RZrbfpTbo{p*ew_-w zPXm;Q*V7MDeR$B8Hh(przQm{8i!d5LbDzhV&gLw02)lW`Goct9e#;DN-}-V*`}#pN z^=g#~qn+$7ko%9xxr zxHY?33hnNLbw|j+s!xoi54P}CcN(J;zDtpYyu*uFSZ|eeDwF7zlQzsZJYIrFubsu{ zfI#}IGU9hP%r($s??di$Z6^zbFi76CRMxa3R8wD9x{q$=FgkL+T7BDW!L_-wky_;D zpFJ`~)j{T2fk{A=KiL|{O?y&qzNpAClQ)4?6wD(0Pz5AjA~AkFNNkj|>bZeBUx}pC z1Kpei;UR?F3T#Vff=}gt}@=2khj}kibj39rb*2O@<=G`i)chwIrJ{JCb z7?rv3bF^Am3{IeiPlV;j;pi6_uvomYj{`~p?Eu~-jEL_G<{F)R>r?-lvC?4@rA7si z*~$9XjD7!S9-d@B(aA-2zPsCUntiGj_|OT_jH##DKCD76|391V4t8fnwAlrK4krGT z?1e%usi8c#t+!gb|JNm$B3%OiwMeN`=XuC*x=z)p_M^t~LhHM!T6xn$S>*Fe?;aXE z>(50>Z;mf-SMEeLB+SU8)hayC*8)xdwdFw|GV2}zryb8eLYviIV>6zAkp7wW9rvwB zDQ6u-sDJx~e^R5;hd2*ulf3tX{X$gwF+pJijM<`QJ$ip%B`MPmR>uwM1i60yr!B;~ zn}Q|#8x^9h)S>R5ew3nI1-Yy^ose2+eC*xHMFZ2L z3p_ZH#w6z!89(jcrx5q0-T|*Va@BL5T^7C|fq&KKjbc*Tk2@pVe#1+p*kFm#K(5(d z)i5BD*jC>K0TO|rav!m^CTZs=e#Sa!z507a@<^?obN?&!;rS@wt*K6b?X%h&I$Q?y zv|rZVWI59R;Sqk8{Y*&1Ong|lT<+L7x8hc%gShqWFV_aejy7H!cT_bt>N+X~s1w=u zzl+eqm6DZQ$Nlhz^i8ZJzVySy-;=C*$Za+k{30+r*!}DH(b3W0tfQPY_{|X=Hdvg- zI%Ap$pM_Np(I$QhHqcEXUoRdlyn#_K5Ls)Qmb)vSfrgAXanZ#lmBrM*xRl>jP#K$a zIW9NEdmkID__jZMLTf#ab|2Wi>Pa8GjYWn|(m21Uf0N*75ofWMZX}7L5Y1~Na8`Ue zR>#$C_JZ{NXqf?w6b0asZk7OZP5!BRz3ef4?p;_GA+J}B57&fic zVI*1izXWBQ_tPwt5thsVtX`rK55rI>amEI>pw+!XI5zbf+m-yiY9`Dr%fV-pF3XG- zoXv4Jh01v}v_1M;%gU5B!i2N%DuFd^KwIIP1FieAs)Fkn%#y(kpAn1^X{p${5={n*gSEhk zlH0R31ZQ{JREG=JtGqzOkNRIM1tCLk)l6zf8YKv<*IYseAGWx1Z#yeVY@Y?#Ybc8O zF12-Yan`%#j@|<-N3N9xY)_oe6!MK*YA8od(vAtLTi#l<=}ZK=>l!{{RZB0WW|5Yg z5Ruc6K9dV=*&gDt>~3Z_k7OV~ZiEr-vL|3XsvH@d1{<8dx2jd%tNvB7(Xgi8@cbtF z;A!%Tz2|bf=Xau_!PIZWCQbjXrM`m6VWhE;Hpwx#8}lt2h8V7I-bd;%nyie&iCFKL z-r@0YqK3HC3z-bt4$|MMKkplkrl3ph1x=#V{_k-i>!1c1+XT_qr5|BiigCy$4nCZ* z$o-&d^?C55LjG7$P$%~n_{W-s5J#VE+%SJ7K0t7V9$remThR}_l_U)-(+Jtm(|OqF zv1vD#$qE=yMK*1@wBH{-_P<0iG8~y!uu5OX)VAFnvFXQuS4it)etaxerWJ6RmfTCc5S)Z3bsITl{ym%@`w~Y10C_{Esr?)-m7#AW`g1x3Aar3oesmF5_Fl zyZlC50PF+ePt;s)g(rM&T)VIoxc_!a9Y>2gi?zlvp;yUu z*;>mFUyvo5^I?8i&oa_6pA%~gLu_*Af7eXf_>hldT@=e6@|4*wBmD~btQ!t0TB(jX z9Vlyh_&p*l_w+tWmvlp9OfgPOQu@=sy`*(=Ip8`?`)ZR}*dXYbAv#N(rM7Bld*w(` zBz?}_Y_E!09Bp7Y>Yjs#$L}kM>HqchsH(T+!w_NJw>Qp5WfxsZbCD8-w0|yOggYCU z1Ia;$o~#;4W?ght9q3>oe zskjHg_jd1RRAtW4UXed)MIHDUnFplOd}w=ygSUSZ!|Ip^2i+fP2&+4=Wkl+Z{cHR~ z(gBwSkOhsj+ZEkf`HKex`2700pBGB{Ks!!pm+V0`ZUs!2JCl6am^}tGWZvXffS$lacLE=H zt|T?DCEGU$fmNqAERIGSjR^gi=Zta`>5hMPF6>Ow#pS>PLv??Ig<9EOBP(CI!&03R5S=b@Ao$>J^v*{mVTDm}7;sL4t;545BWx+|kl)=} zStgyqV}e^DXqIxJ%m)?wViL%B?V(AKJ&iAGlH@DQ!98sUia@S&R1@=`nW!WKNtzKfKtt|&8;Bd@F1GBhCWAw=1 zRr(T%$;4x93wHL<8$Y=)=UOA7$>vy}n>a_#>3zXo5ZLTD`<3-M)KLJ1Bw?H*%kQ zM<-G8(YAbO16}NbZoTIR8>GKbF{<#6N0fz0wS$+3#gFBP20DsJQlyA8aZg@#vQuk< znP`a>xVee6>kX&F%_h6A-^A(+Q4ukPcnlQ>b5_HtLA0IhRguHV!>P)|l+EAq{<2G< z!6J-9#z23E4gq(sMX!J2WuW_?p*&ouEW3!RezVVQOKxJgozVvC%YeF6UUKP0oCTL+ zUVdaY#Tbk~J^z{=bLt&gBHnFcY#;?~!L8Z;DIpS6o4XlhMplw`kXLg_YUtTY7(Zk0 ze0I-FZ*dyR1@JTswk^Frchp(KtRSxaF-vR^Za6(`j8X7m!ig^JqFR}xd0Gr17WB6d zO7@yieG2m@_B%g2ZdG3K)u&g1h)p+tZU0Zqn=Dt7_GgmC-DN#gfdn%p#FMKJ1C|t- z4HCk(D;Bc0JF~hA{vi_nSFZLh&Ndc%LGSm2?z+p`T(90d>&aoQIbL#(6WdDA&MR!c z|103_N7m%-s_Kn2Z|tJ#wk{DIliw42epOx|LV(qV;ry9*y((t^vtXZb`fyY`=wl5D z0PmABGB&O#b%roYkaB5&OIN7>*3XrCX_996)J1Uc1?fe%6%l#5>_%O35h{x2XM8Q@ z|6S{kX-@fud(6Rf_M@_IjSP8znO>wvCq^U)?`3=;wIf0 z8Kgz~)P+KsHFEMdm-li;exf5dxgAEBxYsw?zzek2E%`J#B>j_Mj$7Z%Q`z}7Tp-cr zuWV?IJ}Co)7Z>zdvIwtQ!;uk>wzey&Y>a>%KZ)4Y%swVhZq}Mpi$3JXj6!~CHSS9c zce2bbyt8!aCKYv8f+M`MInBfp?*PS@>^Vdb7cUbnYZ!PH%}$ zyM$xyQ3JiNzr+iN8+&NzmG$VvEt8u9VeybIoc8;#+f_tk}_a!*88!e1361lXlTB@H$8^=bVBY8%$Wk<3ZPYL|4Xx zwGcOW9=@i33GQ35NjEWELBO@rlRhMM-o=S|Sb9YaX~ez*Peiw8R771;~=crLlR zk<>mvh%`F32B5D9!Qfu^e;YM6gwowz20$)#Y~#1TRdL5UIbU~vs?SZLsEnZ)ZKIZo zmuw{c!uVvycXgwyz|;SW0Fy9_dK&{iRSl4FXXcW5hY=gZdD1a5cpZVe&9i2cE&ujh zBRPnrJfr9Wh(h*B;}c1Ob$%qpN8yUsm`0+XUpGf>cxaHaPbU5iJ0O&*j1hA^ngBor zR!`e+Lf{LVV95IWIpn7(jmAN>{ds*$W&$;`WPv~LNt9(w(0rK-%B+%hC7rL!EBnv4+F0Uf5oNipv`6~c+1G#l-I7Z7E+0)A=Y!zC zK1sn5AFMR?7U}Unx2#YmCM*mYs}7 zGTt2-Z;qrqe6e04-%A&t>NXtjXDLt}S;kMqYcMrFRpP^*v@$VX<;ztcQW-rJRl~C1 z4%L(v>0Qa>fx`nlI_M!;-e+!@zA^TMR9w)1%``MdEzT2jjZdi`GFCpCmVP{ zQ__ZX3A>5@z7ECQpyY6@JrHBOq*5@uQaa{oHI?|#5h4Yh~(fmT@@+$lKRK_Y^fVq z&Lc-x=Z}`n^g|b>)Y^S{-P};i^Xh9LU+eR9+Tv?oy)(t51+bY#omIn4v*o0K*Xscz zFW%*pnbi08@ykUFW(%wGHJ+cXF>|j2JZueOBJ_upDsAwWld8KD5IG@FDG<*^BGTI` z9vh{Xin<+P#S>hr%34E2doQq59obfYCeZIX7kQY_=}eB4eU`Y#@er-{t3o_?MaY;# zsXq-%+UaBrx_dl}ClOBBEpZ1^|3<;Faj`qp^;0(+G?U8|f8`vOSzMZzvBP;DQb>7nWXquxcTvLTwEe%#viXr5>+U(wr-LIw=sne{pTg4 zkM|w`!s~?-_QCADFRi<;YG61w8AYwGI|PY?<#Gk4&>)r+rOpq=WC(4gIkE|qP2|gMGeXx?`)>sy)C8! z?o#XT^>Wk5EsnU>qhP><$iWU>n6hmZcq2D<`+5zw(MEzbAN2{LsWw2=AcPQkz6`Hg zKrNQ!+EXE0>1~mUW{ROjB>HBBi>QTM-seR8LDQ5UX}%NnI&jovzUVV4!PG7O#vOdj z-2k|8?|j1t=Xr?1Oo5>b9=V=^516dVYfNN)L|OF|oAxx@o(Acwlv0Ij&lf$&C#`X$ zbZ$8bhaB>>au}tUtE!AZojGDGV$;c#J^J3rWF_Jzi~O2P)n#PnEh>NwiVX$X4@sEcm%05&WB&i;;Ix`4>~pL)`l6Kl?1_ryAX| z3A4%kiyW~AC<&=lQ)zyrFm%tzkxy_Rq=)_0oO--_vDf#a3{Y0kK*~rMCvgUUe`MV8 zQY5uX`U`1g#CBEOs)m@lwo3%?KiGWr#JB!TCmeNnwqmYpK5B7V#bMF@rvP}64r!8t z{DPF&WPIBoC%vnoK!2&H&Pg+Ov)Mm`vn5svuuKxPNJuwX=@;T%Na<&6O4iu^@L^7< zeC|RvV{{q)n^tx93c_(6NAn9D^?^2J9uJ+sJ9$E@>)o~KU#(3|{2QU}pXQS^!uUyCb0` zfpf`pnuNUThpTNaY_D9~uLhkr7fy-Ogf;HX9xS=gCRVwzfM1$a-moT;MYn4>Am_oj z${2G}r=OjOd|ppQ*JQ}tcfp~I;$w@}(46i4^c zt)0P~?#2DYE!5%`EqskSw@Lf04_OzN&jUqY_@+2gI|s@&HOdFagi-1&^(NY|mw)d9 z`{b-6Jlx!frFBbYKuap}aPh#tZtzaH#lNAm_QrZ6^O=VvHcMRPV0(sdVJ&8uEw_JT z*2D1Pd3S^4(t5F7?JRN>VK(02ch+k^Uiw%Xv++rTtn+S^SbzlK9g6R^&x>#W=<3qu4%#NGZx17@G)I0HKZxfKV0ZX1EttI{tAT z_flV@Kc&_CvRk)YTinu#6zPFjcN@jq+{$}h-?wP3Jpo5(1J41<=c1E7Bg$F4VmLft z*5uRzO9_thQU!N6J4D(;(uogg;6BUUMfb)H^G}s84EVsB^fzjv9WuV7XUG#%U~;&< zk@Q8VA1K?sEfH99&{Cn$;C>%AR$~B?S9N~o*SqCRCLt+|(vjCrVxnUy7E4&Z#QWf& zDm{226YnQP1z{dIxrLc7UA{H?Oh@gT*Y7L&6FFLM-umO5==%-Jm)fr-_J0w$xI9`j zKWtsT-!DyKnVHhgiS6`VO5~Ee$OOMs8AL`tdBZ-mHqhA%7=YOY^W%tpKp@2jB_3~a zhxa2G`~n20+7h-S#&N+$UB^qZukI9gCS@c(Qd}=r-MqcU}AHap7Ujd3m zK@@Wjld2YX`ICScXP5h}ad5?n?|99IkRd<9{NWPvdwZmn5F~MlI}X0_%#;Vsk9ANy zE6Th4!FY!67Vg2%)m)a2StnqBJ05`ChZ$3{+^=O3kj31fna%(lZpb{(os=@EX{u7X zx&G>q>(TjMJ8ma?|Y3 zxxPC}POWhaZ-dC86=73Eo)8Qa-i8I`aX=Wfy}=%T@bL$IKmsZ}K|Et`%pz|kt&o|S zdpXCk5Yg2G?CZMLvhT>RL)U{j0&`tEpvKCdT4-m$T=e0s_t0{Kp2*Wx|?RaTxHkc-;y3Xf$C%{?c3->_@5Y z>w4K1mW@K<3Wp&R_+mnVl#U=yowAN>{^yZN%SkCR(KFmEB-W>8Zlidw4rRtn81!LR zOZ<33`ln?VF|86k5MmK`;LtmDt{2EZ{$LU4Zt)J@YWVGl#=k$H#kB_XDyNl%#{V;? zfS1Uz0K4z4k)hzyaICbu+LQR#M^DD}l{++L%DCW@pK1|1qz?hB&x0IyGmnCKB624c zbrP3FWNA)}%8yTSd*;Vx220iD$7(&lm8vg2?#83ee+VuhK(~CZh)2Y!P(D8$Ug7b) zrMHd!H_^QAvb2SJ+~E=O5J2u#6g z1=t1;+-Nu3pL7@f&Lc$gmI!ztZVy;MS-kt2l}E^CEbVCD%Szb(z@8%EEwcV+GxQEd zvug&63F!P1{TLYoQV5%CMY}g2qPn$cpu$;bz5B|p{!_CW{a)Xse}U)&F}FKM$cbnV z)H%ZYqD5({H(lb7@Xeo%p0EIwwRT(_u}MNv+~>wlufGhbDD1*gB}3oQ{kh|DR564i zw;5jZjtgNVE9GCXB-IJH^f!2yU6e5?q;}z>@w{oi$E7C2a~Gkg{`CH-359pTAuRub zpGnu%oFCPKYqN7`mWD_TS~N_HqHbl%Gl^MR{Bo>Ay6{xgefP8q2lycHFohj!u2Jk|pYV#K zfUNc4-Y&MYtLK~EC+q-MH=6qk>=l_{D9#jXUQ;<*=I*25Z8CBM+hdB6Boa_U!mEo& zHyhzo6|1W{6|(oBPjefQ(O{iwKl8%19#fN%dZ_(L*yCHyA{qqvqm#F_G66s18)+~| zOu;YkuN*8ZkCApnmuNL;u0d_?pnS3CL1!D8;JiX!qgaP9r{xYZBH0GS#|w?qOzM7g z`ggZ=OC4ukjo#NM+!4fM-EP9myHSH1_ME-0XbT|~#pL@>^InYsP8~!6h~k2F(T6m} zI1oQTd(J+N!-4UWxZw-$Y7VqPyOhY+dqI6U16o? zw-+=biremHCyvz?+a2btd=&x_t%B)=X0KgvG7b~GWQ>K@Z|TsY=mu)}aH;*! zY)Tsi6;yJ~?qdpteb0Ai}n(q|7RmpDfYdBXrK8h8{D}w;nJK9<*F+)dUXbWXRZLWI{ z|DmlzD9Nu&ok}*ezMg%|cS69bYxd5}l61B5to6@2xvFNC)}|a?Qk;Fy75(oV-bt~5 z-_)QWF4qo`=U?6LA1=5SjRz@3R>;lZI0=7rEuZmRj}~X2Rxi6`^pbm3d}G#|cx$>H z@yCY5!6j|2$XcG&w~bOx$gw#80~UBq4pASq%8-)Md(&T&^mTXmC(vDdA*Rya9~G?x?R8rL zSUg-yramXPPX=zQxlfFB*f`xjXR8>TO^TeF4F;h+kl>d5Ti%!d&k_b&86RG*bljk< z0^1%X&W_}1wrjn5hm9KiR)i=-j}hK@cJ((b`kvlYzIg)FlN(xp?E{{bK-bCBNQpkR z0SB$O^hxCH+14geiQ~sy-!f;Y{qIlt4%wNZZ?LC`4}t`aduZMhbTlGkTR60_Q};O{ zBBF2>>x^q{m5cQ5_LjtADTANWFR=?@BE0?_H}n)+W7}^wZI34KWYK}FZv*p9imbF@ z#v_5Sd6aI@H4qC}Gszk}9O_gR2zltT0fe)e}c_lRR0+db6F*DQ-HEp zwA=ZQV;9K&A*uhFm^_Vt3~PXLcUgk;vU_>N6G&sTik~lla5_tjQ?Us+r7woJye9iE z4P0*C*Q_kq(31O77Or8HW5dc|4{6`6K0y!7oCpMMEm#W&YOBgnK zetb$uS9h`cx}Q=O!lzdk{7b2rn0MStY^a5_a`IQeudhWzEx~Y~haNN-3!$MUezXtj zZT?i^XbV(3CSU1$_;t;*?CZ^QXz&dMaGHh^^5!NvII1}Wq4qKKe^4l@7%mr}@yAV; za6?;{D8X;d^ZS4Sf}}OpB^w7rXzw3w;rAc@u=pVPe{ESLhx5lRfe>-= zuNt)dpB2UKFOzAz4L@9+1z|B<3BYx^1Ehx!!{<)WzgZoMwSOMX_h9O|?^|h5)5}ti zeF)(?tJeIeELXDhez7q4)Bop()Sl|pbEb^jLvt2*Auw2zXj@r7YtgVo?55BgS__{Z zex2UQU+9=BBitbB1cQ0fu z^b-5FTCjtn5@_is-bBp{{t2wsbmRf#rhnDYzK}*lKVLl!AAdWMS892_QKLishg35B zgUaNb!=BDTPJ_(W6D<1@_EXPHzf~{4uC$_~DF#*F46Z~G*ly3g0)24Kls+~nLi+%K zI)x#zOVzs*c3|GT>HLgc**=qS<8kjYJWYMpdz6vQoTm{rwLV@`2Ncgif~S&naL0nD2&X{mgmOtfz-g*=SA?G{&{-P6*|Eb zpYxk(U#ifX|8&ky~cM}{}VO&t0#HXS2&E8KfSz)MFfqAxt2ojf@G{7cT? zp;_{7zn==*7Piz6a4A=6|BT3<)7-yY*;A@Gmq^J7`pZI6Frm=v;*hEZlkak)TyG>Y z;q;+cA+9Vh^4AI&+u`+|D1+li4!0d)PDF%FMnyM!zre@g!6QJ|8um6;vOOBw@jHJ3 z%F>Mv)gnuC`^jm17P3RL?eNvM+#N;e>rs9XMmYKKz^Z_{EZ|w3lUJ6gKm=}Luqv5i z^+N~0k^U(yJW5Mee-VS9nJ5>s?EE-z&sXlr>{K7_bf}AMj(Kd&L@H4ZA>~8t?KT4A5N=i2J8<<~4bONwwPJDqZ!o3B)o(oQ=jp=KF86(ym1tG7ctn zbHoL{oaqr+?LnW89hi*em0Z1Hp8n_SY>T_+uV&-LC?B~1Bx^CJ;n>GtWQ1*a{Y(!P z9qvbwng=nO_X(;@@u~`dx@#(j(r_=gaZ(Q&!Pn_1f9R9N-QpSChCS!Y&XkVe9xN9y zF4+P0Dm(DtIu23p%Ns`N--+OmXXd`fNRT%Qa$U&zse0o>vl)9+HbfZ^L+O_~ z-W{qoQ6Xw74IF(0{UJvgxJhl%`NiJ)Nlp>Q?m^}D?zP%Vls4EO^;P(dV0BG`hKHCV zYhBg2SnOKxZTVvmao$QZ@Oyrtqoj+xu%~s5{MY1o4BdEIe;P$%&W}=(dF0Wguf7RN=Ki(Vg&XY2Y=EMIKe-VeHfMlh zj1f)so|GZOMvS38k3eF9hAO8j-i-rmdn$0Mto zq!y-m;OAR;y?smP_;s&fm6NAP?SD-0UQmX2-t@*Scyi&M0non|1Yw#j%1JT$%GJa% zCJ!1Gn*XbX`R9?2(r3@p!MSEb%L{{5x&CLLIVIfNbGj zSXA}CLwVj^!MYlg8$RsbTO7l0g&S|AEaTKNSUH#hjn0>)cSq0EcIwHR1JG_@ZwcYb zGxhUn#U|n5cWT}7zM629ZDnGGM6C?&nB$V@-H9t#BA&Ol&mrI7$x4idQ{omh$gFA8`sAwd3N=I(R%n|0$CEFQP{Y4@YEAQw}>l9w{AJJX<6>#qq)#8U_rH!NwGiKfW+C?*M;jSN0 zL53$SMI-8abLAPntZ5?*HIg4HO0`b5rk49iKN7J-`^Zs;-&nrcB3N|TZ1#Nv<`{&2 zVO#jTrT*?NhF`$IxFS-0Wz zR(Cn*YMPLJ46%b5g;*j^Sf5|E3EZVd|0s?fOcMcz{v8ir$eBjSMg{CC>^z@KRT|gu z2X==kIPpw#-I1xbPNJ4?krceo<|ni`1*44yx;1Bi`f9pSiDqNMdF-&2dLn!-l9rSl z^s0WlUEffDz?mi8i9Y;?D*8A|tL*FJ6~6uOcs=-Jc0&YxC$`HGCl^=6FCG9}feAt4 zXnV4KH=~_27J~ccpSig_Q2V?n{wiDe-ag50Yx56lqVI73(An2Pznabd{&P8!vEE%Q zO`Z=6s0M903R2|@)YzDW{UqjY3225tX0jI0*lu7JqYr}T{RxfCoXe3-`)gBHt6G@i zz#q*SVkqN>&MP+dQM1t zCa8P7EvL$#QRh34l~Rt(Q$N!?dEz)Lv%O{}^G(gSD1e^|>AFTcRT>qGowyW&N0KW` zv1Yg5Ie!E24c{mDqRMllT7Ysk5^veNB`iGlzCU@1MoB%c{^!Gjk;>Iq0&M?TaJ)Jt)-wW z)GvsGX#oG21GFz^z1N6GO}4yw&v<9Lh5MelD7l{cO3ip~I)Xk@De05L83$iHJ(NUP z*Sta9I7oSVP6ty6FU-3!W)xk;P0XnM_**e0&(Z~ZKdL6Xgpp0nsQRy;>LDKQahIJR zx}G+`cyJ4gQgu7q+uX5?Nl+}Mxv9MnV=;Z^x&oJD>e06m^vu8=CFInwi}hjE#iux) z0VMvh)nTm&*XC3#MquRosTMCLYo^9hQfmq@ATn1wTH3u&L zyU}(qYQeOcg9q0U$IR)*{E#hDO%IBVYck!Wx0D-aXsSYAMzNz*V5F==W<`AS@@$7 zQRBWTve<5ej=gglH=LLLJtycz;)Lu+Ye&N4cz5_(L(k^I-3DCo{m6W05?-p=>KhKE z=#J%!`SyF(2v1kE_#YO7S&^HmLZG*dP}7|%zO(rN_esROUv{1ri)ZJvu7m$&F7vv) zm1bSM*foarJEbDJ8Sa~eMm_SYKbqgcqqjSSrf__J_&GV{Vn^&ch@Gx&MxoSoPXdl< z>2-Tbmf$nk6!F`R|o4290FpYmTG%mfLFT-h1zY)M=qX7i1*EKrp!VgZHT2C!Ds$gjw z_6O>i8F4|$kvG1)_;Sld)5aP-hA}ViyKSvzaa83aJxy;j)n?8`?;-d9y@qCZCWuk{q5~H9J7wjR`(*1_)_d^&+e)nG=xX6GEkMlzrLiDu6G~H*zPSow6#j5D9R%`E zRL9iD7p8@!Yipv2kT+=b1Ac#&s#_ycj-h&gCWRW^Wji^PcFI}-pl~pxCrkhdvxFY> zESRU93XViH$dfKMO$NH=ww`+2ws)BpfoB(d!4H1jw|x&%c4rZw%!fx%(TmzA4uQtg zpnP3m>DS$4&|n*38f*2J{Ti*e*Z-lF-cv8UnxZJ+f{B&xHHz;y8T>x}W=^b8FR4K4rXtn?YG3hA zma5p~dt_uvzV|h+LoQ0g7+H*klHkX)GdVz5l&eBLJrXbA|3!B~4#QZUshq-BX&;b( zRg7KrD3bCw@c+{dO@IBCxhjK3-g z*#{U#^R`$0AhJu57f7wK+wTeE9d+Qc8&aq$VU{@c#u(aQ!Af^|fRntdKi@V-Pv#b; zU8P1{4L~;4PcV{O9!<@^8-G$Ic&3>i9$Rfx_|XQ3c6JY+G9S-#Y8ch-f-~2}M~5hE zx)torzNr;B>JVpJQG#DApzK6V!*v`tHW88Vi7!-QrVcss@sFKd#Z90hnrW6(eXHEh zcc&0qda^nu6R3;%mVN1~@m$_1Fgc&j{oiIaJe8R%XX}KqcyMU0p>wf+^}x#8$oz=M z9!6&a4f;EMPMEbEZIs=8{A6VPAu|0;$)OhMu^^`CG#JKHuK6~hgM>Eoi<12p9YJ8S zi__rUO4I(|vq(BDr^QvuvV#XN+En*NmdBMm(!agn<=G;C>3 zrrawSf3VCphe~WkKCI8WYVf;=#7OuNi2S-JQnl+D37GcBk25l^EU`v5l7u_;=RWag zcus)<3F7?{AE;!xd}ycN{PV{@U}#y|5GCuIE$W4+EGCIGEV6{gkp5UO2y?aR21iNp z!fT*JYPlHE%%AVB<+;%O!XO3w^hqgXYU7GzW^pqEfbvJTe0!$pD zgdMSiM2|B9IZZ6`POVtUC}|(SKaeobEmlsm*FY0pbD3_8?oZnU1O0w^NlGF!1fUWNT41`@&+)d)gU02z^feS|X#GiaY?s3^iF>84{lw-^Bz_b!UN!@xjDkc#w%=FT6TMpqTdLS-=^wBBdRbE~qBigZfc7T!T6 z<8QNaigEs;qN5p^A*<55!Loi9svTF`{jxefN)-E-`C6XslhCK#dGU&`lI|YA2W|m1 zH#Kn?;;(VW&&(gBa2Q?R}sw3433;KPQVGo%>EWJ! zugmmL7vi(TN6V0*fBGY>J`oeUO{2zXKks`D(0U$!ZN1JmyMdAK>O9rXCF9u2d1q2p z6|(RC6=Ja&%Lh1Ejd2>j>J}pXSkg+m;02X5 zcGV)-eou6Nfy#cDcAIND2acczB~y#gFk$fZ^k$xaZpwJcU`)o?o!G493wuy$0XRQf z3Hl^eIaqn$_E59a@pT-?jks<_pL^_;QF1DzO2qU35dn(n*bG*567<;|$%W`4o-QTr zVJYCi>4bQz$EhJi4@1EnTiVEwHRH2xUdH~znH?*(fd11)>vh>2{Oe$bPi0XC0$AT2 z;0%BI9&AgS_`Y65P?}q5)9pM30BKmQ)I6~wLe=JInxpH`JE}=_rwZhv)7l^mz1t-K zfL~wiN9P4=xlQwqt_gGvK7`I5Evn#Y+#O{_e4RGZP;#nL0`Ta#No-lnev4e-xJ_X- z?RM*Ny7WhSV$BhIVOU@XYp1MoXt?wxkz;=@h1LZ|(iYdwP#yHf*L?YDqq=5h^*wSa zWC(3(tl^prw-308wM8ERMbKc2#w>EU3{kCEbi~(Dcj9%my6=8wsN?C=(-mIQ8ZLfP2Z-%& zG@lO0_d@8j=P`xG9$dhwE+(!8s`AXGGR;34F*r^-UL+<8Ic56aw*ZIAf zTDYn(71hHp`(0``@;%rzP!uvMWul4Qthhsa(61RfrS?bp%-%7Zncv0HYgcYue|uV6 zqOm?ZfN{aS`Hp~nYaTDN7qG!agvggM&-&~hj0__+>NdgNos0@8olx}Y-+i3yF^w>t zdEnK78)H`TtHtVio0kE;-#Ps5Jo55Qgs(^xx2_m+lACj{GTf^u>V7(U#iXq46!|K{ z;!jt@sg*AI%jG6Oxy&upS^lG7Xf^G0i}CMwl~17U8l77xI|{lSNzbO)2K}{~>X7oX z5MV&0wa=q9LfgDoG&5&w18?1se%wR)IW24G)>*CUsULBp&9!bllv(l7yByrK)||MX zOTzV>zqK9-StwgrtKPRqradBKG&24Pz~?o!C?yZX{7E1t>@vEzm=d|Ad_=2igseo< z=d$qmG`G;{0!LqN1(V+Ow-u@Iv$h-j$o2DMi-R@nZrH=uzK}30RZWXl<$h5qEp!#D z0KrGi720U2vPpQIjAX3ONz+1znkN?cNI$+!K0%`&ftvF6V)$gU8E+v14U{o6!4!0H zA2Z+a4zN?Eo?a)?cQ)(3Sc`T{T>-X?pR z?;n6b0eX6WAD;jYcP|^5mhAf9;o@lzYac2D>jkN-j{x0oa~@5JCO>3xuTwD>jFK4( zUJJy18zmS`FMRf=?E@tkMGL>K`!Zv`6L!o&$C)+W+T3@{d_XR;W07V`HzG=-9%!>c zm4z(ZhE^Z*{UTBYZlo}|*80?(rbOMcEVZ&~QiM7;o>rUAB%rnAv|%LN;1;yB2KwY* zRi=dukPwo$9vGlF4p{L|Dr&)!#LZ)uEa*mgb#~>B#EW`=p|{AF^t{jHL(iZ5Ik=sKfnXGxu(hdwxN z7=uxTC{1{ZVD*%Ltdvo=L>?SCvgTGhnv|Tk@UfDt$TL$7Zc6SEQj0j>Ldz>qCiG9F zwjOa=7iILWcF{=|-D8>={_=QU+N*4O_($kS_g+^kt5^9S!ta`14(uTvdnVFR@m8yX z_G^wt0HRFQ=HwwMr=^E8mY$cQUk}^!Vu@6XG;F4*KK1#4p^b;U{3(`)s9A3qk5~QS zJl01D$$gKNVReRdT+XQQR6*_Bq||w(li-&MiAo2(vU|Z}$65CgY(A`PF15mVH6>)@ z?Z*i(Jn)@)Q$?6ZrO(mc^M+a-qoI^Uq;8IpbGB2xLUr@?{4VQ2nZDUj#pw%TNZiaIQ(DpPwfry_x#;0JL@Rh1iwIoC}1 zQs~Pd@{(gg>5LcRhJd=tgBm-3lb&tLO{+F5WWf6?CXV8N()vIDpf=QCdwPZi%N0;j zQBZCdR-l^dZ>oF;cex2%dX$v{_3SMbeNQk~l*ZYi`UU^W;B7aJLBkY#aJ9N%PcQIdQDjPM7;?`V=${PwbPZxL{ z4|)_H7!_!?(qiG^rAOf}Te5qW z8Qro|J26$75lO+9$PPUoZ|O8!m66%PD&t59sIqr=X?DOo(B&e4VRd(IrK_GMh&5Q& zjmUzebXGZU4oUM{zQ87N6$cB8XsbsNQBrQbzO`mkBHOP|Y8Q+l4j(uy&R+8wd=j4luqcB zJJA}bEtv19YF;Fjupxq6;im`2w=w*+?Sm_TZ>7-h|LXJ5yuBA(l$3+DSM7pSt?z0# zU6OT&ag$Gpl}lAJG|sW z=Z@yzT$Xl0C?!#v<$Oa!WbWP50Nq5`{2icXm1e~u;vYVPIB;{-Bb9SU0p*FhFO!Z; za|ZhBEbYr}N0gUFBA@X#6)t7fP|2W7-^$Wf=#8U~_(F~%UJ}<`L1g@9Eunya`H*D# zHqWwcVUzgm7@TP#Tm4z>8m}acyY!E;{gu1f@Tqh>dZEycZ{FF`z&b41Y zw*X#=AFWm_au=~jJkJ^L%l7GtcU)_YesudxShvbtF`rll)6X0po=5CrNxwHpG-*cg zH6=Z#<#UH@f8ZaTd6?*x{~K2#s{9KFueQA00=ra2jWxD=HNY*bOp_|mP{|qR({>L9 zlewW${ale!WQ9n@FLRa^Q2 z-EznmH4KH9MD4)Fb(FVoAXQSR2L4Q02;}18t^jcVkh^t5Ah@Xg1-TbJXu=e?rEJoF zAd(r$FoXc|(z%U(z#p+_GicLVbDx6B@)r24yid%lyqS;3eSbbK7C6I2ZuPGLj|`KZ zIljHq0btLrJTOj}d;AfqjhErnb7zJzr2TE!HzIVmXqokX&`RrnGaf|oEqp)70=@Xl z%2=G%HRLJd;v^EX2HyW+HXEsl^ymuab9t-zZr@fd4qae)%lS{y-BRwny#$^qDbVSG zXqP-epJ-pZ98PBgvc9MqV+=)TI{4#1Ge-U*(abB8n3lM&Vzf%yeU{9!Of2y653@49 z({B>u$%Vh5Agm)i*5{{2%bgnyq_c6)Tz|*h|MLVfv_MY8Vz@on;)63!)N)(9SLjPh zN=91gheLm;^-Z!>iFiQHl0vod+0x7Snz`|xNz4)4U_-cg=OQ!85?$UOSVkW%rpV0jM|ZXNzmyZZ@q2?vesqy9i-H~6f@oQmC_74A<6uZ$$tm)7<$;HSoIWzRWU^To3HR}f1fnJ4u z`!{n=U7ruBX^JjQYU4R++ykb;Ct5G+KKigZa$qi(5t;|t%9#?6j3%6-iahnoca~B9 z5DHKdT1I79W6S|IPS&>i{y$@EcKJW{vMO*T>acP+Xk*X;bZC;WQl_w(74Qdcg80;^ z{z~rU!~Rw%ca#vVQ@uq?RG5Pu6)6xQcjVB^Kp9KkPl6=P9SbP=Te%Zm5*Fwlr+}ib zFQSt|8z9y-aXI_d)fpoT3ympv58*^VK?oC*Z`b*+*n%jgAWWJ#^v@rw+TRktR&2ua zb{7A@Y+|DQ`;r9|#60Tq^HMYI3l}H8qnV0BB8TX#c2RW`ZSTL9>3z5SoI=5J$m;g< zy(4j>g_a`$?|&kz0Epv+vKTkp(KEvAPQ=bUY4Rc}mSHg)Zf(S+&zXW=Uh&oCgNTC) z*Zjm^`y1+j;NSil3#KT4K`KqEMbDq!?s99K<9aAyIWVawRq8;7b-3`OS02 z&?;gkticYpk6)41#rktYOhC@Mp5`%J(fa?a1h@-LE&EfCs^MQe>-M&} zn{2d;Oa!Tx(wF`ya3@nW2h&Zt&*g-d&17pTvCWD`^p17OJJai1JO+OLolbEC4^v8w z=D=UbARbl>HsPo&IlF~ z3DoACayVpTY>SiJ4?Qn(qs&%Zdf*Ac&e8iFzoUBYhBRwW9ifNBBR|4*JXyHWhxK?> z=qnHthh>CYTHz-6&UNQkaC6nxgK*G}V+0f7Jb;odHMeReniiwOxLs<$5*y)^?|R6u zk>b=mS>shVLT}uNKo7h~JlWo(#mDSsb@<>jo=)bxCqM~&rwIUNKp%dBnax0S({)7V_6DL zlEZJUTj&ysLo2`TwkuauIiUhc8U`9iqzv80uCU?M`U|=;O=@zWEJ2ekc|w22WO|e* z%t3(c7?G+Xn=t0GSAww{_8c(}2JG#tiajTYo+5;qH!rCwsbKl3V91nd$O1Z1a_bG9?fg1brg;=6G`!K~o`UP4 zWM0>+e#|$Sfzy>5^c=}jpur=Z4LD*(V`)U}^#-C2Cs25y@^-}(;pufXvgFyN z%bjIuRC}(uEI2vc1$g(Qb@XYP5@Mhw?eP{*74Q~=5b*ve(Y(fTLG@v=xed?Y56j&^ z(r-~%!`3|DU0hW%EJBwu$ib({FrQKkL3Q2?$)JGiOJyO4XkL2&-m8ZX3X<6aY9e2y97=u;`cvC>GUYQb18& zPTKrk68nq;t=}F6QbX=+jjl7rN#9Dkk47Go(I;-qf z6CF#+jGrHT)xZM2R2r$`EkuX(wy>uZ2nCZp442`OpQ=RbTjGl>jc7zGQ*ux`R|86w z^ySpec1)zbcG%i%8!)=>xsW^zl^E~! zOjJAZO?8EFKRiC;?A$Gbm8Vf9Jp2MQj&c1%(r4H8bro(t@yq(nD<>jvHwig zoWvUD{1A-L)kI@_9PuTq=llH-ux{5M$0E0~^&&Tu`u+vbHYpQa;31J`zR~XwN22yq^U*+D95GCwTt-3r%WRZK#oj1%D}kLZ04FLNQLK5x@+%WthUmV4{1_8sVS_KBT@qrygzl z@4*Cjbv*tPruHG)v_~40ZIk5s3CB{D!hQIZ{b|u=)x5L4l_%#*@^I{=eg1#(7Zxl; zlyQ=9IA?92Hpusjd&O{K+aKm)=TajRI^=p!G`^_sO>FR+x2YU;g3PioUBx3Oz8BS6}pLox{j-{tPKl>U@1XC0krGEZ1?t(z05e2NZdHj}r zX9pTU>yos~tt4RIh-Cz+$fO9pHdJXeEvGL3tj}t+zujjio{TI{HQa2p*WDxr`_vD{ za+u^i@j6w`wFWYPb~EF4N9B6Mr*u`n0hm@IKy_|{RNmgPyyviaZp~;E8myv6A-K-2 zy?)=b{El*v1E6vWbgX4P>vykcct2Y=3>E6txN6k?8NGzcc{?D+H2AZzsJS^C3QCoj z+LFoJtdBV_+*{peJlq|&17))-TcQ`J*`p=+>_Y1yRi~`g{qpDgZ3-*fGebC#WR(_) zd*f!z}m(TI@>dIljBNGB`da zKoPsh2D8AQnmI0i?VB1LTJ;s14y_Rpmy#-+Hj(4>`K9{)$WFQP!1nCbea$gYgm`7!hDgn_T@WJd`7 z_7t)LwOzZ%YrNHxff0J0m}jzXKLbIlT-?ZmRA5%zm)g1Q8_Pc5;n8+eqkqn+#Pb?A ze-x;y-0n{@cvCAE8ho%598LB2@Zalq7cmv;xN>}%XZFW%8NK!dOc!D6Tz75j+=ac~ z+4uoJpe}ia?-L|=8BPHNgCP`zeM9K=4@FTOScSFOAuOIWBGdq&#$Z%g!AS0Tp%h(% zqu+&sEd|+;>_7YLrNZ9wR@Qrin*hbchX-D6upu8k)mR)-_xBZAPcJ(bzkLrhOttILX>2`zxb3LkUxQQ$)HZ{%?w<1P z?!?YlTTs#OVHS{dJcz4p35%S=b?}|@^wpe$>DI{0NJJp0} zG=n1!OSLVF7~LeBK)Q#2kR(_6a^LgX9+4Mv!pvUn=BF)k`_6rzkBuN$ zps%i35)SFMq;;T729CeP|5$0P9`JT2;7uIk|y5@O*tuK7d zo&F7qn+Nw64A0wxVoN*#F)b%dXzs0iA;g}y^_CZ=D}W=0HJ9TkB#l*$7AP0RW5lK_#d7M%kPId!^Ug$UM zR_g5h&&*Bj$v*bei$_0XLgKkYC|@@{A~uuJUs|%lJ83riuOnAH%|t(doRtNL7MB zsvqCgo4um$s9xwsx^FO#8!ef8iujI*RAw1FDVSi9#WYR%NmP0Q`PKv~4?A3H4nj12 zld83)a=Qsl121*#`m^OoJWqtqH#B0Vyko>MGkRSNSAl2sM>^yD)D3rj=m6Q#EYYq+ zojblp(n`FO7s@@-d@xUG1goJM?c282L(!N{i>7jyx=-&mp&aS|#uq%4gzaXZ)lDiq0}!e801oZQj*Sr6oYjzlzK7a6Kq7c$w{f)Yv};9>!k;ecH`FR^G~b zQ?p0Rynl^GurX-nES(J))p>4`RK0g$NnA*qJ7e5k_x=f#@~UpbG=umR@wti$=j-p` zJhU;mO7UrC4Hru*Kt>nLJ@USThs)jXR;gF5f3zw55y&WpwySFYbH@$WHLp(l@ThBs zYz_`m)|UOq4JxhRm3PTAiEW4&==!xS-HU9suIns{IHa(5l{>HR%+?*zw*3`e60szD ziuvS$7vFX0jK1;|{7As}+q(KhW2+4ZozItjOP3R!w8t5xI`5i7wMxYy^|up{RDRIm z{B2!H{lnq9BC409`4@o>FWh&ZE*N)LA57i`FP%hl`PDOJO%wTxg$B6zMHg{W49KG% z3ZthyW%lxt_?Ny}fHS5~*@Im^fOp6{;U zU%#93h7!l}Vq~Lx$>er}@4YiNJZn{2#1B^-zW~nBaT(0gz183H7RZh>AwrNp*#YSm zY2m>q(Ek8TaJw&XIv1rk54(zB=ZPI$dANn7o8zT*U-FBOVH(*_^+*SBVU#{E=;zs9 zk%$vTbGQzHW^$Fz{q6^M+4p-J@+aS^k`tP=P$Aif? zBo8N7pM_g2KjC#X!+~)?DSDLW>;s60o?-BJXxX#|=c_rIXzj@37~H}Muzkc1!C6ge z_QOr3HYH*Ei#Bm{?|$?qve7!Kt07#qH1BT3UA)fV*m45?S-tg#)JoULMyDpFa;+$& z^H@GQdc4UBWxX-#-Q1M-^;@fo1pr>g9tp$lJDVUBB0l9`0?g%G>DbhbVG#smwFlPe z0gVbh`d#uS(!w8fz-f`^v1`>Y8ffFFlx-(z6F=$6&wTJ;Dn_U_`Xs$ET0w3h<^BrtbSFiBTNY=s z*41cUN#FNX4pc@!5qkwbQQ!hwj>^He3f)8i*x3r;R_>f7iNO=aj=@q>&C}*p7^7?k z|F{rOB_o8uulyxYQEfA%N&z)GX0G)I4jg>CYzF=cnDv`wX3kIJ$|Hl-=|f`Ul~pXe zE$ri^TDXLLzQJ*Oy8$@$yUbDrEY||OzeXg1lhAIGDF#5D@jBx z;Qr|k?6$do*R2x}uK+#7E1)Z1@l+(`o!o2yCus8|`6R_c*W!q+%Kt|kM0C&R`O_mN z51m!R{39YlvWaFJ0O#3SBiOemy3%f_(_agP-ot}yON$lbzuk`K9rCZ{hePG>_2wKt zBQUp>qb{Q3-Tv-irv)fO;)^x6Zm|j)?e?v!S62}}Y!)+{3#)q2gdrMTcC1`Zjn7Qp z&PnIZA{?gCLF-hpem9|C*bhD3s7AC0cjLs@umFMGMpwht-E{_shHvB| z#)Sn=$UR=R23emB$4>{`cz!e~o?3Z$$Fg%hVSL6j!sTW6NtfeTmQEyHOMGyWslBD^ z$0q15D{P|Q*J9_vO;cmFr&ry6;H|2Ost?N-he7gltxhCkdnl)ee3i++ zOLFoN&iPBy7Yjk22xt|;cPuTFnYD$OA7)s6`-lR2Nann{2_)}f!P4Li7G@T{^&&y+ zj=w++PyejyHeh(XIBezZ#FW||w7>QSv`v@Zj!02HBfxDeIH>foL@65v0z>Qv+%jfb zeukA9khAH5QkupIHChZ@Vw?nM5Fl7wc}pHn7i z3Uollwj%FG)f)eRnKop^@1l@PB^EwbNkuBl6C{j|u`e>CN4fz0gkvM)JGPVv98tGr z)frQRi>$WwM5fOiWel~@| zG082#At*%{e@#Px2C-$yra-X?{>*e<&_8~vP>iV17GrJJtsx*o3^$-y&)c7uMm4XE z(#*l^KAdm)Ei^5o*}5EHMu4#qw*rSw?WVFC;O;ad!V`Rc_~ECEZLwUBzjoUhpX960 z`=C>aihL{j>Q_6i3J56X_Y4MQ?Q+C?iN@!~0^A0@^`@}?@~uP_k=A!0R2jYGMBcCF z+?;iuqbyPYRC=%W$mHt#`YhHyH3jiSt-H(1OFkwQBrBr4`?2%yU{_91Mu{g+)s4C! zD6j+4byI-!h3_*JnylTJdOhU$`uLihl!mtD0qxUdO_~)*=zp}8XI0O2Ma1{WGSE!l zVTz;D=d@+qLWNlpidLCqc>BWG9K{O1sQnfO!2*%}?thhW*=L1YqEnu`3??o9=9<}s z5w$8cLc4=f!CNsj_+tIAG$de5!qxWa;}~QVMexMD;kiiZ;qt4&tqyI_6VSi4nR&fB zj9X_n8laxNG&xuZ7J{4V1@J#1(D9VVI`eI#NG{m~M;ib@|K>w&bGDEx3y&PTF+Bn= z-fy1M>(e*E9Zh<;dCyD`uCiG$!4BVVN=nv$u*FTWsLKoX;AQwt9oj@GwyOSACzc20 z6abRmoPue2Ix@-G1TrGg2y`)$$il+am$l<8Rmqz&irvR)qv)?xG7ozU5#=&B3Se{k z$6ED2;_u4~MHZ9>otVpK!y;q1YWaRKAPH{yHous)ueK%LSglVxc}d>h)3U+;Sm_A% zX)X}ah8Ti^OfLPWQnf>BaLn@-6)n%4`D;xst2E2>wgstFWrU$Gg?Xs8iFhRUnM}O& ziS!&Rnxyi)-W^d6k5Mr(@|_eRlOrP%{pX6~$Zt9@zWroq&8}m}ol}p172r9Z)80tI z)M>*yQizB@0;au%AL+$V-Md?BZQzR7A5n)hV<9I%l0h99%euEo@PH2TI=WH`TK0C| zbnR*AvTar%xDBX?cQ#9k(%ejZ!JdMRjP0AQqEMBNL_*V?>$Urm`XfFcntu345#3(v zRuoH4O-AQ?PCH{taL^1C?Kaeg95ifk%?H&5r2B0CBnrRi=SOK#lz$1~3W^5=$XXw&$PsxEw1J^yhh;CN+I?PFtd7Q%qPM^O0#d!a{cs zpQ=+b&abYPiqxc7^(YF70~AjQ(x;VhA9H~b!{MLow1d}qgeBJ~%DRYF$ivE5YQ%y# z#EH9mSSJijx>r zl}PD=hOd|!teEtd@~y+#^Di)JDCvqX0$I&FX4sc6kxZ;8HYdx!vE~EWOV?twCcY|D z*(xuySg6RSL!Xj^k*ccwUAY|n+4;oO4OG&2ea_#ivh7nD`GH@i3m zDjR$b6>pqBrgF^41#>{qfZ)jxJChcQOji|WH`+L}!xok@@@~or(HPH2LPP_6HuBV0 zEb~(<>^9sA5c-RQdH#;V`P@=?N!(!Y|EzyQPXc!Grx3H;8{k8?3M^KiRN& ze=OKFD|c@f!6Hh=vu>!iQ)RMQu2r|_nNKB{4&5=8w7$*t>U8~_R_-vA9Fr&&OcNrf zmi;d=d6m6s-UC(SUq zi4aa`AjQax|}q!q9m#ZTjVlj^EFWkfKueW(Yj zu!m$7@pa5V27&riUwqo?a-_`3K+fX0q|x?vKuR}$t3;8T9X=(4ci7yP0BZi9ILY8@~Drt`KbERtbqirafp~A`(?#WQdVhBN*|Np(r zkT~mYUYzvn5~~8wa{Wa#2}y|}D`J6HZ+GW1!)A?;jG@cwFET=ZoWztxxjQ}EW_K~p zsuF8;4{@Mk0V(hKMcsCwg3K1u*RXXau_Md3e5Onq9Y!F!6RW-bxO^Y0lHmxPt3EvT zkv4Ki8xR;1byqCBFB^S!O_W?LRT7RTB-#=j<(MP1+sHykP9@2A)iiyJeM=0|r1VhF z@{_1SrkMS1mdlF7|0YaK@P5N0{4~{QKR8iZmwRdp|JX>o4tBx41XU{3B|a~#=-|GJ zW`Kg|{ykwpdb1jE?Y8jY^Z{Z{OQLT!56nI#@Ku2Jt_?LhsnC%Y`BQSf;C>1wYu?Yd zmJVPgGovYpyRXSQQ9Y6P;XnA*s)ygY(MBFxcMpa2ye5TptX+tgoklQ_>PgzvX)RnkHmJAaGGU$bf>RpV1aCZ>HD=+_qVsxJx)5c^w2hM{fG^7C3Olu(JcX zVv8W71fhxp%a^K7|0rJ>cY-D_>HVa+9^M3CqP+ulPDVSNB7GY|I!Iw+FIipuwLZA~ zl4oWI5^S?!uNTnW;_P!;Xdo?ZDxLh%(qfURXW|jkZ<7t}N_)Q&5&B=|b~G8`Fw_M# zSs{2Y4V~=$y|J@7tk%Uq{1EFdN3NQg%Bb7{asn!d6ksm&JJGk$$)Tkd>Z^$uL ztb6V05&8O`JLx+d$1g00Gbv@mA*m} zcPvE`7MqlURF5M1h0ftBt@gm0iTeMXo|!w}dW2B*cf&9vO)Fe}!A}+i)363&;q+44 zvM-OMR0R}#78ET&(I$%cMn1OPW)eS*wI9Iv%YN`*+dnTlltKL{1(9%^r~pco&;FlL z*^IYC(+xfzDl7C{v-ys;Ubixzwz_~U8%m3di_NaNj-Hp@v3$5i+d&qk<5-vcEcw<} zpVWvo5DX`lCw$Q>aLSOm5ZJeuR6h!NF}~bwp3~T4WrE4`0gotY>nOb$7d%M`gk0Mo z3Gt>!z^E?Rk-MYY)f?Kx1saw>a@1DRnHMbW^gew~dB604a!l&?b;mvLP98otX$?PA z@s07lXWPUP8iNqL9xW%Ov*@d7F*nz~gjDKu*(k%Z-qv2?YRxIX70F+qJfVjX7~&->-g;4`*61I(&Pkl-LNDQE`COoU%nwhQGb zFlvorA3j+8Bq(p(@8(O!4+baTuD#onwd{Bvh!eXy@$Sg zix(D4bIc;-k9p(`iYF~JrLFsCTedyjxJ)bt^bcV^#~=3NCu@&{aB`M8V7vi>6H*3o zQoru%J9$Yys%F*?(3amj@@`ty1HZWNdgr3J{*bu-PJWV?BK$O6V*7`w7lQ}X>(`09 zuln7t1d=Bd#RrJ7JnDg=^7gA#UyTKZnf?X36`j=VHAM}x|Gwk-+_>cZVqI?fLUHmh z^kZHzF?FQbosxCrJH$ix&bP#GCopJCl3(M34TZNPMj*Y~14XuCB$8)C7lQl~L6=3q zv#>36GknI5o4 z6G`)1^BI8F-=u3o_YjkJ4bMKt=rZ=>>F^$zzl&${CeI74#?~A9P#wBXtP{ETJW)~o zyE>eQEEaT;R0bXD8P_9ZgscH-K+yiyTtUOp-TjN(YTZTzZnZLBiBwup{INuR}qLXGq! zh#L1I7WDT(4~Y6s9VHJecUat_BFIETpt02rQp3&f9^L(Wi{enj1xK!e6guuTrSgX> zf6ClW89xZF;S7PHv3`}qETX_}K*CyuV``Lt=<@n)IY zjGb89e7{I022AeTE!rf$QxLqR5DU#7OFjV36ZI}c7Avg{nm_Vws@&Os0zbYI{^9R@ zS3ALaIrH*9BV|Y>@oQdlrhE{20d1*17bS62uH$D9g(zJ3;(e)llFNT)$iJ~edgATJ zz3igdBtNT$Yyo6ly`ejhi)E#_$CjUF$Q|DuqlIk%Qz*giGbyz*{7let%a%NGQ&8Iw zT(=v+F;CJQpyx<$Tgy+UPpiH9yFY$twJFS$Bn429eyPJm@uye=Zt<)Sc_|EgBls_a|)FvQWJUn{1n}OGTnk;9% zIm`FkZJvjr&9@WNAp74(o7k7$&+$!DervHz9$m4$Zlra6{SR}n^UtZ~bz+*3fS5{= z>sEm;+2^ywZ}9Y?#$$c8Y%C27C)#J^dNQe5CU@OFBb^b~uOON6rgZk;OA64f@$6^S z0INTFXPTn49pRoNw7v^{oMt?CcfAk09cr+cGEz?u?!NfONuE z-J5hLGdZTLDggqGn-)fPjk+(wCiJiL<=dCZ3iEG+-kW zt;5v6uk_XH(z;S@pN1^$>P;j3W?uGDT7k2y?xumX!%cJj*0ky^(uI>Gn^>=#?+Rzx zYgY7R30V85bC*6B(>X<0q2qR{a-JQ$-F{*XF$=LZ2WvapgMR7~CIdcPC$#3>eQ!%o z{4Utcp20KIeA&A1Yp1nPEzreIn`%?Cwl9*yesM@X`>}Hduj_rv6Gs)D`ulpZlS8p# zh~fZID=#fB{ZWu6Ttzkf^*wO=G>}z$>zNJF;xVqmgg2F|d1W`+Ya8-+5ruBqaJy;3 zmp0&a!k4^o9>bkig=)?6Re?XJEOp)n(tEN;O#u z_a6T@y=wY!G4Qd_0So}yHcWKI_cIl9JGL>_) z=Zpe~iTun0YJ4V$I$Q+E*4*UDV%WWhp;P_8tx03bpArXs85j*Duq#A*^nJZgLXD9; z%64S@C;2K3dYL4YmRoUF>NB=^^cC#@y|I_30v@!17P^me;&|P9Yc`>SCrg#niE`w@ zv$j1n**BBl=i4#Cl=uzVMN^ZOAe`r6|E)~b1CBH>&EODw^!8BXR`yieai!tD@kt!n zSq6+HDOmGY3Ozt6LTiFzg1XI7jlmpnMB=zwjkMcCb*uoP z|4)BJjGBb0dpv0a85c;(Ls9Iq`*J@C@noeL$9%_LM~H=f7r zc$V+W9rG9FOL=(n9X^4wa`PJVaSZdh0TMW0pOHRwcd+%_3-14(!HHU91oYs3YywUlFh;0>>nj>Lz>q4miH`?m3z~bw+m#i7U_;O z)cL@t{lfq;Y=$fU`#XAeV>$zD^>dNax z&>T3nr~!9ed5~|nIhYBBc=7R>jew5L2K&&LgV%06c5rC^pO_B}3Y$-vfzdqJe9X5h z&<}ZB@2h}-)a93YaY!OG_BDzt$%1xIpip)ffCshAC&s?EH@Fi(Cg^`NyyT4yetU!(7jvr3x z3rjk>Y|$^w#q>t;Bnb&!pHB&wcyTOlppJd?%H#!Ptn5a5 zN5LtngbR>c_x-T%onrI)g530#m}IYzAgHeXv@Zot(rDQ{6oo3Yj#rOOF?E&7B#isa zHbPE3*=1EOr5F?6FZP)E50(}^?Z%251o=!~wC#H$0j$Ns;(P^+CIrrvAH6YLv}`uc zYY=v?i22&7f*zY||4eTVI92O5*{@lg$t;*KFktv)TAxP*_1uta6;HBnj{Oay@rDs; zd^tmEQg{jAw%GUdU1+%BUd~C0*@gn|G5^w^DXpiHx3^nal$&G(!Ao$2 z$tifCjiefIX45pycFv>M-e=Astk@edz-Gwn_MxpJ`8v47>74ligiQdrxs4t3wU*n= zJ^KJ(XP1^PR+b#0LXB+Z$w^wf_ZmcN3h21D*r(oIf3xScX&Y=Y-xR&wJc2@sj zL%NZgbaywB!sy;$d*1!u_jteVSbVtmTy_4g^ENUG@3B$GjNDFn7EWqJo9YpgP9IRl?ZA54scxM|uZKAzn20OZATrh)2yH;Rxf z)Yz_6L%xsFT9|osi-s2DK-gj@&hrJ4kaoX0sws|0>aAg6FhfkuAlr>6LE^>x8I=6R zsbC2F>Sf}E(oB#ZG7Roh5HPJ`mYH)COFHxEUP{!Z=KL%{Z{|h@RZ8-g&lJVut;!lm zvje1u#fCbxqhiy*)WxhY)mt;NK_8P0V$hbvKh3A$K<=g%a(OYgcoGH5v)bMIJ8Jls zpqr8DCGnF?+Ms_SSA~ElR&S+y<^enb)w6Vh^w_faha3n^TDuhNoI`@Qz8~j^&IhdD zHEI;0y|nGx!c{T6CyBBzle9zUJzbu{ad~Oa|Dw2+U{z49<&!tytekfH>V~xo8;65K z@tJWPKwfY}D#Gqjqw^+Q43QRFjm(3aFabvAj9v+E87 zdWY^|5>9&u^PgAfTzK1j;SO@z{=RV&|Ky-1G$$E#5eH76Upr21a7zkzI8Pk|@03V# z+RL78PEqFL_>j|Qh_#6kO_+!7dB#pSa0Vud6T%C2=C*CGH=RauGQ6{H8jJm%_sRg+ zF4Wcx$M)i$AEEj5#F6bFe+<@lgR^og$k<%~ZXj*9zgv4#>F|L1(GW#dff@Dja^d1INCkBJh_r-?%2gy?MKXD&D52>sl*4)P=_~tQs+;?s z3O?Pmh=6GOhc8fx`7>~m*Q)5g6ZKoIY?Ev7`N_eMV#$>&`Kyr+=wVd(?Yf4zRU>)R zCP+YbGuC7Vh?_az55Vi8)HqvkpOg9R<{YwkM>W+Cbh2?SegS1}oNZXYqhc+`{{?6P z6c-wq^a(z*VydwoxNwHR%RevE?xxx%dM&u+K`}Wza^cCi#cU$ZHuj$ z@<{PZ!@(Z1QnG%BJQz#lZr793aq-A+95{>^6yq+>04?^HSDBT zuVN>&+cRtvYnlx88X(D*U5{NN~?K!qOp!y-{aQJqM zVj{4rwnO(1exDSP1B(+459OCSJ|R$(NNxYlojANm?L~14(hS}lB$lUT09$CHRhS^i zo8-`EAGMEXw~l8cYjwgg0&-zFLehQL*vocH-KT@+3=93!bi8NM7={J4&Vz0qwc|wj z%Uor9b&J@x&>SUyabbO&6y-k76x$F*M2kbd<1-Vev@?KY-Lhta{*lmg>wii7Oee0` zq~K-442p-t{uH`9zU45LqXKue^?&PE3ayf3yDvm*NKy-DFBZTrfpLwNwlP%|7H8kFZMP1~N0{Qe^PS^uf`;k&t+Cqs%nopz8CHV+Mg4&t`OhO7ExQXWv}ILaaJAFyI!1W%qP)$&)oa;{x$|R_w<&25LPd5#ipT zhUz(1CUO4OSZe-zsu7WW;w=R%I6MMt{R%d zsn)A0GHI6%AOk||@!H1RDhscR;#{ngpA*2OG3vpPFeemCUcQjNN0_EPotI?rp}hNZ z?K(~2FPlwAmmJG!2791iXeF~8XUegol$8w2RcKEI&0LtREZl>j7iKA!h2Cw1MRyCg z`kp4kE%SI?Gk;iok(tjXgWs(`$R}1><%J^)LSR|LQlu1BHkY{@!OPZ4=iQGjFsa7q zv8;NX;AhbdTzRqV7XO_rST9AD&;>oeL}z^`1E*`EzD}-sUV70uVk?q__HQ#7kuOv{ z#)u5`d%1L$uqKYa3Iy@9B#0;qf=0IbCn+P57x|C3PKq-Y{&Z~LEcsY#Dz|9X$UUk_ zq852sd88whXY*NRuMzNWXa0|&ELwHoeEKn>=T}g_F(9`ka$)W`xGK?0RlU9`v&*=- z{5PMjM)N#3M2%m{&H8VAGL`TR(e@Mh^8KU}9h(rr-ArE}^Rtgnno<>;WOxwZ{kn20 zXO*TjI$@k}F-ECBK`(1AzIy>C%_S?~K%>_ED*x#o_D^a65-RqC`>MXFIi84xo{$os zf%uTZh02yFTt3~U{eyG6uw=tls+Ua-iMtlnf-LG4X}StEu?r&{m{37+n%soO%nWfO zLmS4L#7n77Em0}9PgO~9J^z3a%TN3PchY8IdP<1<9+bVTOIOrdg>m8bGec7 z?|r^EdI;h1aDWF%&KN|xrW7f}>%TyEk#2Yr`V}^#cKQ=X>?cSpPaPWu}*JbRUKV@22lJR##w3jrDAz zmo1^q<7TgHWTGB;tcrJ`H(x3OXVuVs7`h{#j|Xb#n(~mh-U~$&z`gLal%h-e&5|K1 zLxkMn%X92_7-dq-`ohj%-_6aMTU4uKhe zm_rP(SKn?-4}GObfrrX|R|A6il1nz_xSJht$XpgUV!Z6%2Kg2|MA?~zPW)v4{0A5T-(_DNK*d=kw-?EaE8The zGQ*dLmW#|j1<0BH;~}db1b6-x{?eDW4%F;d7@#@~nV;pFq94+m$J@5t@sp2v-X)-S8D9WoQ#^` z#1xAZR*Z5>QN+bMy48?*x!tuq{=HK*c8a_Ww%>VL_1!8~PM0aelRR-!gbT<2gOOf5+_%L4dO}<5X}A!({g~w zeSUS+apuS*ukf;&V{wKeo;+LQBn_YEWMYN0%e*JO-p(Mo+R8%8X4m~^HJ>q``>+$6 zfVe?6r@w`|lRCX7H#z{#?t#LB>L!=Ius)e6db@3Cj?B|HxHM;yszY_HuA%jgGH2e1 z$zCNQY|0O#eIi2EM(Vx4mzMZKv8WcpHE9&L@z~j#1 zMG>WL%nLy%Cl38AoCoDrd9}ajs6IBm477a;jsuHgP$}Mn;P7e~$@(NWeGb%p2c($b zt-*@v-M$cn)aV2ZT}gOEvGEN*w`ta0G)VUmCrRD8LvSd@aiodkB((pog1JN z^&5xb2U)ftx;GaFRx2_72HOt=@w-8Dir~Vh%-3S*N^8K4RZuL42hV9Rvo~n^Qez3j zkW6hsAM(av-}ru4{j_oB7s}Q}&|jbFhFZn9;O6GWr}mFAe{~C@6|Pd@Ld`ddTFzk% zblxsbGwo*F5`r$lkk#93;|rw-R+1chl*{^#F%FILfvfTmdi1{n2dgen=_&4B1|vH= zz(W#T{899TQ3|4A3NEHY_-{`I5wYxnqfBx2`5aqeJ2YaDuehNY$#NpfrsC0KjD^&) zzhZexCprlmL8`gN-C8%YuIZm_u^#Ud$NvzXLk)=9}tT{$8#DQE4F3Hv=2~V;NWz_-B0Y4 z{hVh6i#nj&;2@nTUfKVsPC)Y0rP|EaHLbc?V9)viCJ6T_pYjs+fs-D!m-0)TQObWn zU|LB6bX%bIe7GWR(4LCnw*%Flx(cdq;0Sa+eCoanQFn5Q)~LYCGs>>no9j{CZg?JP za0s#-@P4w6KN$Wn}Ii-*9jZEq+Ccc(BOVYkrxb-NX$EcugLGea6N>#!q=2MGi zXw)rsg1hEXET|vF+IHcUY>o?l!Q6Jno_E!SX7xB6K);wVdo6&zJAFgN{ra_71G=`px1+>D>{%|wox zKLB@@BEQ_w`b^6BH|Gni`pT^%V*!=p`*j$$%v&WGbsen5w zl?Z#0&E7IHDS^9w1t6}9HvH4@E@1BmfC!5-e9tw|UB}@~%(M{^*tWHhcf0or0SyZD z>$iw3RF!?9Do?77HwJ4FsC)};&6I&We4EN4}5r4rPX|?ISkfOeln4a7%0-mB^(v@0KZeY#fK=z)Rmluwb6FYsn%{dy;< zWWQ2hR_`l3mczUkdKLzfpIu$%APcW6YgsU7n?E@F2c}T&W#P6R(&g&|-_uUhW))V6 zYG37Ig!Pyd_2Hjpiw)r9KOX&1#H3{giKcbOZ%&M(0I7erHUw0yT*iJ79TP- z`TgU0n;f6{O}hD}uKC493*pd{nsON_2!n{a)<1Ngic5=Q{-fLiPDf#n{l(eshE6#e zVyK7Fef=wEz!*pu&yD+Q{xn9`c7|TY1tTqdG=rr=nB7E1j6zTQ4+B+PjiJ8Ji?a(Y zA+J|4s+z9{8$$L{uruWNQt{NJu0QnGN%B<)SivrU{>zE?kXY>;k56K@w`#^u2>Q|H z_!o$)=DEDt%XQS))iODV$pxz^1Z6d8IOBm(!_A4hfBh&s3co*7o2TMgXH^Rq(#4y# zNT?H6XgL7+;;U-zi%&;feZhp^B)TOij?hIu4&v04eq0r8!SD{(i6jr=kg@BAd%ut%_ zl(O$U|FAFmlIIu(7DGa!@6ArZ>zmAhfK`VE@qgjK4&+tEXf)FC&fYO+`g$VpXH6+~ zkZw{rItk}qJQqE-+lvch-95J|nQ5BGn{l*lCx3D^q%A1oYMP+8 zQ0qUBcS(LK!u6mFn|%K?-v((r?#-fGN6xfbqP!2cJ?%aLBy&#KuCXMcK0G@K#LU&E z^_@PiNdIkTI?Ys*I%kJ&npDAWiftS(=GG`iPva+R=TYq2g);Vi>-T2bU(ZziPz+wr zR^p^Tt4$5&EvfgXHR)d2uq}hUdKk_}}2%!@FlCb3*hT!-l~=F#oA{%zP>=*U*CPvt(OX;Pg6YxU{#D*K7nuL;XQnEyw(*~-d2h~;vE z&kS=hkk!U)BlG7ybhfU_$gDl(RDZ!f>8e(x9WCzOc$2MtmMQ9mVwFCc5LSojV#9gF9nOZE`TqEh}#;hkVdnb2%| z&iiCTE+!;f1zTmiSm52+ej$wUI4q%Ln{cs0Gaq^3P*&Yh|1b#uRE^4@QD$pqY3dh+ zD!m{ueSwOy8l_}l>E;I~dJKNY)vIo=ve|~kukqIjoUzvP{vkZZk)QFx{d7S5`1m!J z&YyC))dX~?^f*%qMFiUGZ8-jsJ=3&uO0Z4oQur{8TKWeHbQU?GiO0xq+*-LebEL&^ zoC8H|yUGz2p^sEyuS8q&;9+|dP%(oSdu?=ld0GAo?qr4JXYMQg>P|phSq*Nlh2Y%+ zLn{CkDtVT@zt(*p`h$P{(`}wEn(o+;Q(V?#%ke_B1cf*mB@&7}6gI#^N9sPu(Di$V zdQ~lAn&z1O;Yn?@g#>B3i}mmX+ZuIQt zUx%&GE?o-Wl+c;xp*&ozYz>~NU11_?&wjUD3f2$y_T0>AD-nin?K-qcriH#9E6}2Q z-L+a*h?Pc2KzFCmvX|C*fq&$v`k292P=qg}^-{z>A7b%3=Qh^y@ji(qTdW6u}j6Z z>uUA_TAF@PV#<53rGh1>;-kYH))&#X+nS4Y8(t*r+J!M?D6ChMiZ5imdmg9QIHKF% z#WTp*w?l>}CTElQ(fLWZrweW1hti&WsMFt#+ikBC70BwHwU+XIiRXr!vSCG23(XXO z15bT_K%r-Ff{E}hKdyws+=oB1cH6Vl8;)(6^ z$d$u))D;Y_qxr|A^V>mF)nEnObH1#$ZRcfxb7i~ZGkPvW9i;Q^_Uq3Upg!K<;MsPAYDR_8y}|FzS&9sfB@Kx}Wsp1&=$j3clqLjY?Hnaba###5|be zANjEs2|%j-p6Hc6-syfX&AZ2U>9$E%1o6OA9E2g87vn{N3d|kbSQm9#<*r+OqgDm! zByvY0;%7J?ls&)B7j0HdVArdaRl{b(s-$)?6xw6R1%wsz{;pD}eI2SXiv0We%$y${ z>_B|EZ97aw$Z$o1^P13ajaa8WH57(SdL4u14eK%E*Chjp>8P zV~K7+ai*FHDhIU0e%-5SNz=&k8Q==rNHI| z|A)$D)e(Y>tlB%*AR)WVd8HT;mwIEPlb>I@-S;=bKi|8C?v=yUp?H}~b?F(;(($W? z%E0|HMXfStKZ8g!KKbzXBO;|8&@xqIRe9t}y~7kAWq$uC1vS|K*OE^nl^1K{Zb;;Z zX|=Dr%N`T$qB_sri_hs6icMGb$?2|?-O0WcMXxIPX8j? zi^uRr*Z7CbU_0rr*97?(3dYeHYJD90=y)h2LMVBqLt7RlkR#bZ*M=DyYP<&r~R zjQBW@L%I*MFguLmYaHRr$Y;4_nlF!9p6^-~Um?T&()l&WLQeWzQPn4yw&2grMefAfJe6x3&vXF>r zK0knk&3;|=Pk5j8s(J2pxVANA(?e(Gv<-Pql4@vBqgmt2|lW_~G- zILv#enSMXKDtLfbzpQk|0gVEJ6LFU~3gorsO{X*ENc;35Cc`XCas&pF7 zp9M>a;7iNdmxAJVnul)J=x<+{DF3oWu$#GWkEbcPUoq-8Z)iky91*6@vFv88uB?Em-_`I}Kr&^f=~ z)}<|XGNnvcqqYS(Q;*slSN4Xf2ZPO*vT+4<#ArMjdVFxZA}wTXUJf8(ng8zvK!s~l zXhG5oGmcD_v-caL8$He-qNTymZNCK#;ld{c*Ek8|?|XRbL+qyvYt2m;R! z1Lj~8d{qk~yc$K)HBk|?k(>B@1mr5|GhY1D#OTl6+Iftd`73St+)(T^bDnd>Li zpy=$0uL9yvGp#ZU$h)K#4n<9S4Nj%oA1B%6q<01^arbpr-6*x9KvEf|o@XZq)m*>& z$^ROq3zx)zt{$!)^s-X8p!Z!BbhQ@Lv`}5tGTn-xT?rU9FIY7q1p3#Pw?Z@lRzyHg z5&I3)1+Y`?C7-X0K1s1&$XmBoi`=-F*Qd=w&J7&hxj$rJf0x%{XKdZb7%tV7pm=c; z!a50weXVyIMvso4h{ul9%pL6Op;!ZVA%N6c;BB>$RxX!xVBSecbswTcEom3D{mAzk zJX<6+<0@-*JJW%3m~lR*JExgH3r0h**9u4k?P1D!u)u zi*0cW`2<@&t^cQYYq(32G5jZs6;u9u?&+NFkF^q+5&x32Qo>Q4pfa(@8fjNM!Vs${ z&81vY$HheqU1P4WudIi}J}$1I%-iVrEw$$~z6L*5q=zh2Aq+VLQ#$SP@aW(`$WuWF0@nI>f7>d zU)g@IwP#xcxiC%1Pa}>YJ_n_8r>u!G8p^>n?@&6K)ThYibRIJ@y`9l5Dl60d9A6gQ z_k&C*s$>ln?M@bb6aC=`lYK;9^2axh^Hz552Q4hhrs+v}*Pz$wOkgWz^1{NX8Wl3t zKF*okTzP6sp@+X#j|P={1RQ#C;zD~eWa^7FoqWf$x@=CxU(;N4u4W7D?1J6j`z_%U zs!{!x|MDdaNYI!FydesEQjN?S%lWg*fSNAiG&;G=@^bV0l6pDBf2$5zz3jc(V!Bm_fjsOCKOguW zJP%03J7%L(2ea0$)4P~1`Pej_6EwRQE&9|i-lgOT(I(~?E>U=@Q87KU4OXW_yqU3@ z0NHpp4e44i(g{=c#u$%%tD_YegwhAzZ$Q(fu4``#6Ha}lmz^&!*ey1B*cebRV4B`| z>e2L>K}i|`lXt3T5MJJxOg+!~+l`sAuY;PG;;(@*`zHUEkk53=eH+P7&&f%a$k4aw z#nqmp3-vvKl+M?#6}8Mh7)y0yKOo{C@d(CzNEo^~c z;}(2%hb2N5x0A5l-%|e+U`WXNgVAiHnU+c@QPEkZ_D8uy*{ogRx*Q&sQV8NfE^`jB z(KE|NJ)8nk*}tc>9MUkG=vtVaKm1WL64=Y!WlPVQ`tBr4!J+<+7FnEl-Y38F4=c$I zu}x`KFBuXMTXvfk$fx@M=F5p`+I7tn78x>XXmPey32BjiE;$U9k>OiWKZ# zGrjtI#lkJAU1T4{L~vt%V`pk@^@*F07vKOgX_Sxg$@gfWSD`W8vS z?;u23OoKIqzYVoyMP72zAp@TtPTkE9~$@A>+^?RY7Zy)ab%O zC6S$U2mCaoch7Wvo_pI-Y;W@8I-5zyr0B$Y?^8%Ee#b4?bl0rh`pjlAFBw&w%$a1Y zk=VPeUB+Qj%}rxHh1cV?09d&P{yugrS$4tNGuwC&%N96ma!wUgrtH_d_Y=fbeo4v{ z&VI$GNl)6G%bOF%sD@%mI3L&-zaWv#`O7@f%}r*c*Z)h98?#VG%w2lM)}J^)p}s9| zs$t{aRclIwo-rz~>%=J=_{62XKa=+&!1f4N3vKWY#9DRARF>T%J7&22 zrPz*t?{_+s6lo}Baw`nBa`3G&XZzHa+<@D9)|0sq^E1;W>%Kk+Yows0uzPb!)_2o~ zVg4C@!WEDU>$>E_nd_jENcxXz64cVldPa`#+4v`|*#VcgMKn3*`v=_<$4RK0cO}Ku zZAX-7QORf~#x%%sSs#Z(cqM+Rf1$8|a76>0c9=&ZB>UjC0}-@9R19Qnsa3B2dtLCn zwQf>BlT2g0pkhfANIC^ANaRbeZef0_#OrxV3#-+%vqK%!!yN^AMN#~Yg1w;wH5$s_ zA-}QZdr}0hESv%~Gxd=g*+8s~v0Y8(*aO?OoESr3whC%M{|a@P(-kZH*Le&)6Z5$m z4GuY=fygZ0%!hp)JcdB}9m#a!@^!bB6=zS<)ixB5#S=zZAG-a_c0cMmFUh~mg_S=} z1?U=tW74+~XM6Kr6f!o^@^n>m-pd7bbwQ4N=oD|^atgBCCYcW&v6246Dg4IFTh=xJ zmAO0l@2_tzEJ=GH%gu7xAV}w_OVj;^+hWovLb%j4UZ2#IE{M46`WK9FruL>Ge)4-2 zZvWFg86xaw&5TZC%?|M8Wm0QO2WIHb6J!;2nTxMR@T1+{ch;}g@2meNj+9DjZAH$i zX`{*-@yAQ#*un&=^N0!N#pn$?Of{P;;CxjTnB3_9jR5qAjqFPfaw&p5Gw3-lW!3y2 z6)tV1`p!5b!Ivx)Se~(cQLpsNvO+s!Z73Cl)hGN;tuCftaux`U4@CNX=5&1lvciZG z(gK7HFRrfI=!{d;fA3~^^;`=V-+$y#!YAia&HPmR+NGmsx?yfxhzFjr_{57kW{LA$L^sQ@>)1Lj*as^fg)#O5i7G+?}(c)oI@)tW?2MfbhYT^Z$I7Gjo;Lac5{M>WVlU0x(r6B-}{o?3=d$AU|&+0$# z*{YFLJS`(F1uId`qVt)Ipz99ylyE^y_U7I3ggDfhpXseIKljz$fYDr0 z70ysO@6@IEDiGJ#iA*~>AbnN&^_-Tl=ZN}cze!IKihX@oFTm1cqRug`9e)S+(lVXBfAg|SD2e$KPx?@?CyhI#yc z04JG*{WPt|zjOKZxE|KgKKaX2ODgkD^s1e&e3JVc-BvIC%YUSHK@}%mVOp*vES&#s zw{12GkVgsI=bpz=jtSx-$J{Tp)g@_jiLT8#Lk(%M(_v9bLI&^&Y^=ij|G0Z1D4Wgo#}%U$ zen*K?orXP}@0|)+4Tc-?C*WTqroDoJOsp!=nvs9`()6T9)+Z^N_;`kN0Ub_d-v<^5sr+l(k$)o#9~JF$nvV%#Hios?FTct zf?7M(SjGin?!D;5cB?qE)C1gIVn;sA8T3I}p@)(WYGtA+K*Z_%6!N_Xj~npnJC=Va zM_X#px2JsHCn*Gdxy$h$RXV8+UUvTDq1`3MOa2S{FgW@b&BmeqKCj6$zh!Y~_=Mb| zjDierAtV@PGY?GLwPOQvYuhmz?dsH@if?H-b`u$dtd}BA!8Sbx(+WNL0=m*P>`QnW zQa3K8jpx)S1C_DDc=dvS8n^bLRPvANQySmt$wr>?2gU5Izsj4WHy{odHwYMow#~qI z%1Oj{JVwt@LI=Ap*tfD+asr32)26nJeB>RZefaz0S`uEUJ1V#pOXl!z;k!N2J4K*T>5wb=;4QCPy1C12i)+kn4(uo_Lf>^HRy*3I5^%pFb`jH}^?i<-( zSFoYR*Fst5m%5}`BG|lR%FqN-&rW7; z(D%U&U>$ge=*8=L?mky8Z(P&wyxMG-;Qh1bcbMqW$6wvJ3>S~FN^`X`_3YSsrAv?5 zR3l`hd_@$p|BE6iQ9~Lp=qvP{^v-!Uh=_(mt(GE+yxeH54<7mb@nT_bc<290?Do-F534tQJQZ; zHpaS5?WV^}FG$2zG^I_<5C7{57ovYwqqh?*+CZx6$dPYla+&w&=dJh0o#tS;b&T~{ zm)tE?fn|g!i?Q)xo7LER^vn{0Xhzf^x-uHR zW*}i35BA2FEJcLZIJB#=`@evk!62j)-qkG>UJLmD7EYBEN678p6~4$(eo&iP)dvDj zwllsY6j9w?UsO13AMs1Q8kPW4dn#ZMA5Zvc+2)WEkG>%qoRu5om5xt}=#+v**F97) z5&cxAx~eJD9uBX=9x9;;vnjShBQuqO$-$v`x;=4NAND>`bz*BwAoDj?Q* zG|{Y~Y(dj!NZlYcV2JKz$f<*Pic;^2Lu7ALK0ryZURC)1g~w&!NNUEOuRh4cK++%6 zxt?um88arj-pu)Q@EAQvrQuLrx6q1|fo$CgnwCrZhtsJlgV>?IllR;t?|kkQ47$H+ zUc+w@rH3zdx!>4H7lv(}!(iZ`YB2jZw`(+EgueX6-aGIL4HtcLFTMRDx%hppbOUf% zDBGUd0FKtzA0k-{>{bh3QWCv&IXnSA4qT^1Y7bH^uNnd^r$6-$jaz~%CSDTq&iV;q z@(QKA!IF)AlEs=A4?ouHqaUMa&|F#N8PA}{GTT{@RB$i|EoE(yWbu90CGd*z8Da>{ z7@V@51jF!h@MiGF^lKj@l0;g{vhQUNbv1fBtfp&H76HJ?J+<{Z6Zi3u=R zl=biR?sXDoT)Q*C%E&JTwan&evbphgnVX*WS1H7DBoj0&k9FTzh#A{Ts&|Bm>7_rK znw5%FF6A-?Fp=(U_I9VBufK0D%!)MHE{`?e_6K`wh)f|8_?!anx{}>sYE5EFj;|Pv z5oaSQuO-_=SG12!qju#fRhH)&(}YGj?gnuj1eLh59{ay9Ao|s#7qCmzz&rS@T<{7b z^zCR~OusBfeFLJ4gsB6FYJ68Z0iDykRG3A_8PCf=BuvPX+!uH4@{Oyy5w)8>^UXcv zh-uL^5F_tXONo7=mw#%Ei*S=h5;MVS%(Jh#p@f{HMP!PXu%jS=`)?}l?JaoyQcI(^ zlHQ>F8s(a@(7vVVhyzE^p5!=tIv{(`9k`J0sw>f039E=s`s`(UY zdTHV_H0E@)0ZTcn?8$ofd-wOt{T6^B`P-)!x4-QVYHWp~^RiVNcze)zr=x08aZckG%b6IyXJPLQ>p1je zFDPEqS+tGjg?nN!tk?%1ws_&C$T247DaauRKcs$3isAdop<{uy{J4}m#TB3}`(=Sy z{5v>f!x>BQ@TpBVzxIK_p6(i)A2Nb7R7dWybJH zukK$aUFtrZj1BH+%;k~2&2P3BET+IO^=p|32YTH&Pa%K|WO}Ws3?NLSdyH>@@s-q~ zOqAWv-`u{(<1o*( zwstul!T&ST$-sd_FFXBbG^#p;*1nwfP(GQ}s9=d;o`2(ZM*R8QbDf@IUq0)hQEKwN z4lUQC39V4(Y*tc{X-FO))z}mn=wC8MRx8GNHw6P6w+r;^J2}r*mL3dUdhGt(M)4Dd zt++HDv_Ir|a=K1jTkk;5kG}qw32+6af29zol)I8rKOWGJaBR6;d49zxlp#7x-AqTC z1b&|+gQ}s(dlgp{TddOhxq=?rXUni;oRhb?g1z+IRG=n zk>d1ej)zaH!;+vO>~aijLi)oKp_abTQC-&Zxh7R?K}X+;0&)s7`U(8yK!u9I<$LT zvLak40_#YGX&Y&7;404@gSr%!-{bn51nm(Ce9%77Goz>SFd<~Pi`MXx5V0lvh!}H4 z5Rh%5gF#=$zM17)PCDh|rnW1NrnhvWkODX`Zj)A`*Kx@`y_<&Aw#qGfbFx)S+5!qX zq;M08&9ey_fH&lN4)CfQ1l!LFDnX@ES((y9l$c&LG!gCnsth)NX)4=SBv;yGwd#;t zK=QbHH}oC6i`;EeiRFkgjm?1B?eKA|NPhzwoip!E(YMwD1$>|;*0Cnc+nQ$373V!^ z;6hT!>HOUEq~_VOqAB6{QzhgIAabw@*ims=Oq=#$U+3X6txA|qdQ|8H#V;Sryl&B^ zQpE^Sz}d+-th zvcwuFIA_tu{xF`OZu~Y-?3VzpwgtxLnsOJe$V*)Dy%}_^ST+jX>B+R6gym+dSxg&? zTCd}ZUo8CJx63yWe;z%(bTvqCyoQdqIuM~or48O6^~-Aw+h{*5A$7! zpGd55(-h&M^}u1td)=MAA4;Dg~2haE*O z|NJc%jqF~w=F(V%`>xrZP2it$h4bKuZBaVa7S`vvxwc=sqDF`fvq;>jQXHgkh|vy9 z0v!rWKHh~{aq;Y83er8 zi!I}Z=|xPc%CP(VE2TRr^-GbHi&iA>l3)1pIv8uNo0sm=wJH<9#X>1kxA*zMU{Eul zG_v*t+y>lGO_SYAR-R2 zV7KiC0*8yOH#3cY0>fI!{pT9{9(|;XBnZsCI?ya8`tf!Zo;*YmKv}YnGxboxMi3V{ zL;v;dW3*vlujgrVyR3rwH&(TwZ9xp|AI$c1V?i?*HgSls$@^1Y`OWf2`7sKE#??g+ z)%Bl{a{Dwl5gx1&ZJ71EKosbxpS@xA1H?KQtCRe4-!&#t!WC|OQV=F)f-P!7l3B>_ z)s`r8vEZir{>4yn*WE7R&QmA!j<JUAIt*tWD$KR1XkFuC z>~)p@mZu}~vVW+JCEE+vtk&h%1T}uU7_--gA=v$&4Wy9r;f&~XG@>Fn8-^I< zrCZjJEt^QG;we6wrCPeN`nY@zy~zw85-Nt zD}MXJV8>T#Hu85PDz)?5#`^8I;0IJ%3to|ZHBp(_{dUIVBCfl#NLbSKbKYL zjbWZCwY28?9{|WeH^0R}p1ujr@e(Io%fkuOgJ0E=|Y{|=pn2z3miIBvO;WXT1j4&gB7zacXWR3H0l2jP;l;t}$ zvv^)L=ODsR&}cAm=PUAxMAA$q=?44!OH?Fw45#59V}uz&!#E(CAZwg|2*xBz6p^;T~fd(kNk^K@_B~e+EWECr?#; zm`8|EcWdMRk3%WykFdr|CM~10y8c|4h2LK*s9^xC~W+%1#bMvWiI}T zzaAfZLM9o^K=dl4Lkbh=o^J;~I~6 zy(eJ3tF3N_Y6*DFU*CVgKTLiXzv}P{{U(fKzlIj~kIThpKPrC!W9!oi@X|y?{$*%>otKsYn}DPTIlm{ z3$CyKIZ*$8AJF;__k^jaf5=43>!;dZWx$$4U2pyVKO$cJ{f~9i-~X)^c)!A8inu=C zV9n_>nv6~6^N-6E;l79i08O6-9byPn#okF8Mtzol;v49xv zODeMBS2;>^rNjZdTJ#y?HGl1&jyoUt>v7QP=B60A643jP0`K#`w(4Jhz+Wm8`Wtl$ zk`YGlKQJOp4L;g*iBz6WRE+VXOhasoBw$!m6Xg&4Ap8S2CVEcT0fzFo1=x370Y@Y7Q~SUQ##q-Xv*$qvQ)EN zyo*0gtIBx#4K7-txK0q7Pr#t!##G;IAy=O>0=q`UW!gdt8v%miHmJIKqwt{RjbMbm z3>^8@R&~J*BJN%42Fxv>+1K8G503-T>=NcZyaZW0-SsqpGEE4r?$e)g({2b0c%^Sfpwf(?DsMgbL zWtEODL1#cr^zk6)4CBRT;)IZ)0u-@J;TBWs`-RFc_uCTBGAqg^mC$U1T-PNU;C22RgJ77a4KcE?F721z`BWIJV)`30fwnHvjdSstxN z0Y(G&s5xL)MUaLL@W8Ge+JxcW|33e`pN3LlwQ23lsruKVw{;plU9^%3re zzI2tWY-B-4TD`=$+)4}uqfGEuYmPP$>U{91^bYx+FYH@rwYG}|on|F)~Xp1$_^kEXx-`~^*4 zw6}jc_GKrfLk@jbdeV~)PFpTsSEy^p|3`oMFX`M5{%hJ`!<4o-;&thiqxK56^!e9x z&7k_%e0|`nny>ASd22fP(48z_T=OuP_-t7Jh%M}FEqxvEE!#c;0oTB%+j!>Zx?!z4 zL-t+z;N@5Q6gN2#XAMMe)m~G-sl0dDCFwn1x;!n|p!&)o=SR5yCTW|^H%m)*T$=XZ zb?3C>7MrAcrB~`?w%DSsm`&=d&?N1&)$2c1D~)<+HxW6Df%?;s?_bP4bT|+ zScgG%{wW&JrpjD($Ys8^*@PshLFchWK2BSv%zDD0WV{;U0n zEPM)eudaEFYh3!80)g7Ze?3I-BoSBWEJYGRW`|51hpfk(X7c2U2^f&3=hTAp3o*(s zbRW)hAp;F>&>vfUcRK%1KbF4#@CIo^^-;KeT$v{GvS4|7c(X&&+yC^iw6VyY8U7=e z7NOc6=7JDhA>3v^{qQ(3SLl+L^HKlQq2nI!dX9@yUe(Eb$-uDuXFU`!Pa)#KGjT$M zg0XAX|LFV=3*+d6Da#;MTUc1cF(^$l2by(_S8y*(tE5rml;bk;vS)t?RNG|S^`Rhn ziicVHaaGVL6K2C!Hy=Dz$4I<-p-V+O1-=9UN*?9ksBsZZ@v5a^94+aFz5kw1i&>xl zHLw3ie*OLQb$^q6&~C3v&pTKSrl!6AA5Oox;9cng{bctq(s3_)LE5MJ^8>aXCS3oI zq+fjH-9`S(U)U{wQrExuRO{SG+(7B}>p$#ovt$2Tb=T$TQ)fRp{oO^u^cMQl^z~2S zb+(Qid&bAp%U|~5wDT77Nh&nteVX#W@{wE91^;rMsy%48Gtw!3&+PAiL+Wi(*FSPC zx-{pmb4dPIK5|RQ*Q@+sO4C;^mAqMT|0~uvusYQ0n#3oUDFUZqL(rt@Kx803WW2H1 zzz;#jZyCX}qP3fT{LS>?3%~EZae`f+mfrrdm!#i0VlR2aNQu!G$b(*tnf{sPKP)Vk zqvN)ELRz&Z!$!kQ%r*Y}>9)WyV_&%H`7Y~RYWwZk~bOtBT9qv|Hxe>HhZ0XYC0w@GIp<*PXB158t<}a zy#E$t)W0>$1n@dzj8^85Ns9Yq;(1(TG;qWMa~KoiKA)k@kVXJ@G={r~6WI^koGy6x z1?g)Y{8K{OZ)y6|KM6GqpMs(WJ{|sRh>JbCPTU@iJ7Z2-uvOn@xQ!ssi2O(Udb|;V z&iWri{u@H7&N@}U-POgaT4W66EiV8=jKD$#owTzQ^`|jl;+L?^!{rI8hQ8dX#9YCJ}k+eY)d!> z^?w61KmU(B^kBN@?mN@Um8;C}IiG+0+l#fu@O3TK|DtThrfIz?SpN3 z>-zoABJjX{_ojR9xg+lny60p2n1cK__WE(=|L&iEBAs?<$D!07N2X(c>x^{l(MP5y zP-cK%KkxMPzAybO`xozd5!ZVk{U)4@`=8sgST|n?md2Woc;3I)-ul-Vz~{;IpT1`Q zX1M>zi8S4ZCnNte#Q#Shc`!XF_f>7kue{VfhvuprX-N79K5VEjUY~+%*P-Gfln@Gt z0~%hS+7aU`_`*f_SA$V)*R;>BySABb_X+fcOgk=l|io>1ogZ z&2-imFHaAMToY|janVBvox}kRFHqIfc)*C$wm3;}WpUWW-cX`*sW|vQEdT4tK~!#7 z2-Qnv&hq@b|H~g;k?wuq{#ZmI^%}PQx}F}oWgdFm&S?Y20W841sjq@(|ANm` z=D#S*`4Zj58W`dhF<=$cQMVB|<`SwLwj@&y6Cd|El4wP|9|wC=cnUOIz5*dkSF}sx#>-(zc}r?^VX6=zlWuj z>HgcVOV|GJ-_ut=c2@fOwKiq(`bfOt%sb(f=D#XGwV=QMSt1w*-Roo{n3DA`s*r>6 zJpY(=ZX@SA1U2ku=(ZYjPFVBPZ^9|{UyM*Y7NoCz;_P(q?aR{nr=D!F)_61SFj5+9 z0>9f zzWc58-Vgj!lA~v9H+(TY=j7|t1?Rsf9kJQEP4VV{$y9F9tH!zhweG(LUo2xpH8|2| zeCly?nO7VEU|;j2fYvwd)K`jQ=2v_uqd-vkQu+38U9(obI!6V{JpA~kv_cgyXnK=uR>PW zgxVyRpMUHf>4-D`UAud~>!ayAZ`j9&ngW_V&K4*(*%;n`t!4j1e8=iPJ<@{fB&-!$@CW47sxsQ)qKe^^XMe_{%=xhDZCE*CShC#Hjr-di3xeF0-lb03IneZf8HyC46P z^yN#h6FbGgF|7Zl{OlkdwE(EEPNLt06ZYRk>s9>`jP}y@yg!M*x=#haJN*}<;#|tp zX>#%D#shzxA4ygDa0)Lp!hq&sL3G#%lb?Szz-*%5g!G#bK4Oe8;#21T9q9|NlQ+|$ zoq66b(x=}3T0NA4E#>~VZrWt)J=4L*{Xu%$mzSq6e)jX}M45NX@^B;yJJD!w^nrd8 z(r-e{sre&KWl}aE6tN+XmctBl-a+?9)LCVc{%@D5UH@j6`>4hX4xiSu417N*X>C)5*D#!_gsK!BD0c=OlMA~6 z(r-eY=5}Ws2xugXTsu%LN?eZm3K24oeiN!q(f=sh1=zj8_Cv+t3go*Vst9;)=s>wM z=$Kpp*OR?;^KH_TpMGpQ_piR0uKdv3(s6m7ApiWQ>AnATd$wK0-~w@aQAZ4it!ldn zCZykl%7P16rrLi|qBg=&;DGW12o0tw8AHH@l~AWq3?YcXIS5~s2D-e|SBHwrFrl*G z0@msBAGTIa9TXa%yx=w=@cIRVmB)P+^QGzPRSGUj!$ol;fCK+Q1EFhEnu@^`jiJGa z4;~k3j63A3KV?m*{q)g=?7Ttv%m~K-<)ywk)VVSzLO7*70)gX7sJOWclD?EfX~5Q= z|MUUYn*G3}%>e(+kp7AY#F~#dqZ4VYdqNZ(>!18xFjRlnu+Vftp-fQ4UM^Kvc~CsQ zV3jY=^J!tai2}cjX(JB7YyppjiDAmh^Fa*!87HlpQ2z@&Wdfqn_UKPmD&r7Kk>v0F z_qZV-5+KIVyUlok1w0r;BaXnb1$-vtpSCjZgfJJr&bjImm56`q`&XPj7zT1?iwaIw?KxsTg1lFowh0R1Mc+%eAJOO6F;)r2+l z|K@MJD}CsiV&98CxGbG=*v_^OC|UF}hUmi#HDO`e_J9|rH(hmay8C|l`U+^;zz>A> zRyh48R9V84=U4n$4&`-Dz@#E0%A)OSHUG3-Y^2|Wo$dtwQU0%C{!2bJf)Yb92)4y| zF@|7)X|x%A#lepb8gVePMLf_{kMuE|=VLtb6DOo{pRhLl&-T+c*eTMYg}?J>X@`y9mY(xb|Bb5e{Z;zxwTGwQ-NXKb2kg!KS^W(2-(#OYZ6d)Mlm7$# zQLimk?tktmGIsxW*z550nXmk9I^%>tNS}8h?|<`G(`gr5;Hdmh zga0rOb%2K0T1_}18?2)~hEK%UIOmo+rLS~!=KBxN)E&AEf;5SQLM$y{zQlm}1oIJm zIIhY3Iya=+gJf@rG3alO8ABmK10z;&H>e77=O?5f%ac%#tL>^m73rt7%paGp>H}_U z#UB|mejk|Q{#!obv&Cda$b6}zx8f=FT`jVjxoo`y#KXh7~J@r4$`yaxC+R~R? zGIXLoR64Gq1kOg>$633RlK=bCRo^_9ic&iIBieYB<WW+d@d6)VoSPY-2@KY><^^3l{)?^fReb}5O=26N;^=@T zu*)BHMJ|ANfrqe*Us#Mm5n%;0DE1$8D-CM#Qdtg-RXS+WVciodvyUlH#4%Z3(I()k zhtT;!Q&}DSQ3h~`B?LY}{~?79ZyNQDqpBBUAUEo90{^vjFhA#+WDOHYBFk{RsmGWG z!I%^`lyCR}6<1zoQvg9ga0$3~qYuCjIr(?fnL5udpZ<<~nip*VKj@U9JW+?>I_8jk zgvRf(Rf(v99%+E|o6ur_QB6J&^VuQ&Cp7sGq@3U?IB4>sEr5y(nGmyVuEW5uV}7BG ztOv5S)8YTdPfYLq&}+ipE8^LmqMYV3KUp}}Mt1m5O%n_RTbHJai+M+8Bsn^WwWZvV}nndg7W%>2bBxBdsm zarD6y=Eh>xB9?M?h$UjiI>aKzm_x%&21rJ>hl9NEgs2NjU|a;*?3R98Kr#!lq)*2E00{}n6!_lVSFvve}+{c>z+{eA)ynRJ9YfI56#{EFN8?fT8@L( z-`e&6!H0@rdF3}Q@H;PPgb@c#URt>hf!lAd7z7SG z2=%tEG9=6bb#B8@FgpWa#I=KQA5a^`=5@A9ulVERF-hCcuDUthuRKm5?)*w6O*-nU zL;6ja{e~ZG#MQUS@xR0Y{y~w$h!67L{esQW7n*DF$~{BI>HlVX9hUxR*COM4*WnLm zOkVz{#ea{JdePy3_D=kollhPS(^;qC*$8z4sLx35$8DK z34BnEVhoM^=o#*Ngbc`!sKiF$BcjT43L+;(7`=w{>R^fn41Qp)Y+1O zXt-0}h#3>dBk{TQ%mMB7ljgsz;aeBd>}ir!*;S~#zjHJwnDqS7 z<3IlTm9hSzTasXtGw5UU7=+^S#q>w7ZK3iIH(5XoMb(%0`sZ^L%crJC$G>Nh8pP)M zpK|_VZI)4hU83puh!JJz@Xorz+{765K-d}e?7H6a=oSV|r+cI9_-Wd=9DD>bXTpx4@|fA=@MGb7xa`Y0iaVop7;S9I$h%}(?%C0(H6p=rW zGT(rw#;5Dz?;1b5W-ra~A!E}J(!}p2G*+L%Jw1MSikz#*7NCzuFk2iAf*9fn_T(4= ztIktp%rPg-Y_}t+srZ_pL(s1wGG@dj^h_P06(yvbP90i}j9hj>y( z9gGI{pCx`<2a@@XY8}QL;=gzN(Q1-3wD%pOM#=j8((#{N_Od-O1qWmF=dT6O&aC3PL#+t$A_Wh>F9jT@rZlRYYa{w3?p zS|;7Q0-BFnzdd*Tv)5J&LC4)6IsP#-^IrdR`~D|hFVqRItG0$kRYXF$>AGxdi5r@G z*80aziC?@cvF`y?VB0ziBJw{V#MVcpbLK>W1ngqdH6Hz%ALCI8d$q!mt^5A903VDY zr}|8{LXl7bHCL8zi60$NLmqv_EN+A892QdO&!4^i_Sp+HFIlWORVrJSiWTGMFG_Qj z1MpLfPa5$t)7+GmPR&y=CqQ+Jr-(D|w6P#_>!fF{G-HDM#DvLX@Il@Nq2bmqTEwLY z!wO&&sS#U`{9!#>H}xp=#VLsN2qFPQ>i)9v+le~lEzF-3&ZqN#6Mwugd+m|Tf!}ES zXP3lmPfTGf=-jhW0hrq0Y&g+uLK?I_o0z&p=*yL9b8}n{-8$f21XC9dFva=f_LKJD zhuV&oCe{$*@%U1|(T(?ulff=8{&@atiMGE z$a7Vu{+>Jk3l=Qw9WGQ*)(&5%J!$^yn;vu4)-i*i@9pc~kb9nSuP2(NJYVtk zCzby#eY69 zxp;J%pFJsl_#*28!}iUNq=tvqrf>H9&^fAgLcheJhuRn5dKv?-s#;~q1!VM4)=Zl> z5XK1*i2D;Ee4tLCn_E0->6H}N-m}_=c+rLqZYgA{$i3>>`THw@)s}RRB z&{(m>=FgAn)YXCL`%MNAEp;Iwk*Ge6F`VneD`siYIzNvjsTOCQX5nLb)Dk2d3( zJa`0D`5JV@F&(I7>o!!eq@Ul}&HuHF-luQY@VyVkD$(JG)uaMD*U*C5tdqxD^hTgy zA=b2rnD&H4~jT`vk!g*r{BLWzRBHu(9k`ggGY3atMiD!^K7t^=a^ z^=~V!S~QzJpZz|4$v&)Hy;828DOS8X9n`ofHE7z7>eejF5)jN(f2S0RUCSi+JcAP{F_O;g#^kr9n^)2)_tUu<|9G+A8 z_?5YIjV4t2fTCh7clF;v-_3iER&CxzMav#ehg2;kvO1oTBi)3kT4gT4U(AX1%RZrx zXH26xvp%C`n>GjsU${mG9@3Z^wLFE6YTlUk!30UP+`azopbbBKL(7&gr}aE93l%EN zHdRqPTYlygDO;Cg;9Dhm&VTXgJleu@swmSAt#*L;YxUo_at^)o>QnU29IQ1Zsd~ef z)a4&UOc`kpDn9DWjpd*@4gC@t*;k7WA!JmHplTPvNbM|dJ z6|LBW4ym*cZCkg57S8{W=6yb&zW#Or{kU3lS-g08&SfL2)8tsHfB2zP%st^VtiRer zMRa|%9EmNr)3LLiRxf;?*Vvcn!!P4&Y~y22p;oQi@>-_?>En-&SF0f;lwV`UaklYG zzpizv0}}Cy+E@p)nu!sx##)#O8yrp-oRVhizUz<4Yu+$|pef>y{6TE0FY*Jtjs|{} zg1k1b$7DSe4$l}#L|^GM1v$x@6I{+yqBs>{S`-dj(vuR~Y59^Y`e5FCnmum`EnV{? zZDjcp#VS(WdbR2321ig6na^>5ozV!WzwqB21O79uUA=<7`F=J1v}Gq1;wy?QQsheU zcs`4lu0aj(dTzP#4!eA9qs0qn(<|?NNV68KARXb*RGbc}+mMbwvMDvKS(*w+tbVLP zjm|%F*?S6#dJWoc)PL<4G|8Qmf{s@8DzJPQKXBD&wN|)|bN&_Q{7cUH*PQb;F6R|E z=ZA348*|Qat2WD~#1Bi{SoMze%jg3<*NWHdQsoY&I_0ru!{$y}zGyB@|KLM4 zH)b4gC08}!oU?K|K8w|$h3fnr{Q^ZxPqeALcC9X#W4amOh<2 zh92(IFQeU8pI0lj{x+Nj4(Ls__T$UZ-GsM!uh5NAb1EHl`8D}yEp_!u8P9q1{|{XM zic-^42NOL%$W`t+>UrMP^zdW1(;@r0*N{a0;`L{-mTFmLAi zZ=!bw9OK9HCp=$@#Q8-wvIl?j(YEj3rzdW|m>zomv#2LTX1)I$JvR>V9!I@KkD@b= zKE(S3MvO~~8OMf8;A`mfS01Ow@9#}BWFCVdcTLt5#J^=BjXGE7e66eBrN>-OcKy7C zW>4%*JqM4DDV#BF0*$}@QmWPYej3vMVycXbDJAC&`9#v}=?~Gvy|2r8E_WD41F!ER zbJ^AeF%-tP`AhcgSk+a#2X$|@k{%lOT+E-?fmQVPwMp5TeTar6}>~vE@SR zwRI`4vDat6#@@x+y5IQ`o!h=K6~MExEq+)r9CeGD$1T=q{S$t6#P~!1AP>6HaBMiB zJ2V*S#QVyifi?WVW*?kG8rVTM8jcMIlxA1A$cJO*2M_=G=#fwXUUpM?*h zu&Y$Xm8KN4r40Et%%@xLeJ4_J%8`fDKH`jew2MFVz4y+GG~}OSXhy`9K&I+(-F6qz zm@CepIwkW<9ze}L{tfi;Td&a_cE2IREW5LTKq2kbI3p% zn?dIbGpEuWn34@oqse`{(lM2aNzPOee@IW9xX>y&Gh8W)2^7Nt(NgQbiJrdinjm*= zsa4gIQgh+wT}u6TZlHH(?oQ56r^DPCdAO0R|K{cI(aBfe2Xkb3+p_W#x@q_|bYE~z zQW)Is#i^61ALIQ8-A7j+TU$=xp(`c6DF56kDabxQa#Cp5W_s!2fpqp0-gxJ0?~@Z# zuo1R&@fc0{H19SJtPc<^$MG-2Je(iN`pe=Zb=Df|ynt@`Q3A1nnE65eFeY^9(>&l9 zr?a%iuAh&LCjIOy+;vL=NEVLbR?uAPcQ3%K1HPMXc%aQ}14`2u?1*3Jh{yVw$w#ds z*zbP)KhwNdo}~dD?y`#qy|M_KK~IkvFGpt#8c5goZAVr3yG=8hTr{mq%kv6i>`ayI=#@AQx4JgM%S11xL}7 z8$1HdHG9zk3~}pEfhGzV;wg+quu%wzMGH+~dhw9TIAYCsqY?O3~&ePf(k7nLjWY zFW^Icn$m^Cp5bp{vaf%MDqf1OYvNWY^@E&e$pK4^zvO{lMD^p?22(!T`TgtEzg}tT znK&ucxXn3q?iItSdyn44r@Eq+_55{oe)9`y5x(xm7*DrF+=_%_KExAj{?NM@y6XE= zu)OiFDM_t=m`b0vkji*s{FiEW3H3VCbJK!5sax|#^xVha{QC8;fS*&93fr?b0$V^6 z?n$qs#pX*ipu_rmUBydmrzLNXpf;_W$4*MsI`SOq%wzkqs|Qo3Hub##y-s)aIE>C8 z{-pl0m>b(-nsLAoU%PM;^{-ooF7C7YC#7I?KZ~EQh}3JJh$u{k(+HZs#VBmixhFA;>*~+yMKh>Nn^URXKX?D66to{8a}UbH?PvPz zmHVlB_sl1yyl0GkI*B@V=}0$CTO8;ChjSX3N6`_hdCv7;PtzVBMi)&LQQ4i+=l`fK zFQKS8{tQ!_{JgJKpU^qSXU_S6vrkWx^V?|NW~WR1)f`9l=k>{2-BKlK!?M?@!daP5 zN*OQU(7~6{vG=`28(A;(g)s{rGK+2HGqU9mP_~`l(}M#$#!gBNyX)UH`o!8SqT}9R z?#Dk2>!$V+@rz&DA0Pkl($5Lip)~B#lKvxRYi1RG0tXCn>qjSW*$J)$9Ra`<(*%K2 zX(k|fV4CTNDy0ES(-KZYquNl7au|qNM1X>`fI)Q}@_~RuLBbZs9OBjwrhHqzqtRVk z)3Mo}l#1JUdhkuuXrJCRc@CcQmreWy3sOZBoijO?C#4{eMZ>%Iq#BJ!)4Vkrv&C&N ziW|YWW~jv%#vJ0-FY!ax5m0UY_jOkq=*HwAGeT#!@jJirfXl#dHU6_nXtq~Oi5%)D z6%Qdu(gC6tmXEc-LJF1_9D`Sj?|Y4gZpJYM)ot zJO3;?tX5^(r${l{xpf__$a<5W8PO@anksADnRIiF4{7K>j_`AuHNc8yi(gIxld@Qq zXiWe5Y^$I1u*h!=*MD>~bvSPjj;K}tcj#Z|Hsg2x?%5vE`k)_b!|@s{0enXyTNT4> z6gbOmUH!TLHn#RTG@x%EYSFYll`7(w&aS;~p{$u>=q5f%l_efbp!4oIhu*yI1b0Q4 zk4U;JD2j-zbI{C&Lm;JUXs=v`osDQKkp*l>GbK~=N?oBtt_26>LZ%p zv`GdG1SFQKmbq(fDLOGXcF!mv4pz&1L-q`E=ts z)c?x$&FK}`ajpI3kJI`8=s?v<^G{q^4(Uefx$J$KIFwIbO;g6Kab4&nzPi>PQEc%11wZ`8GXshhJleCeRN* zjUn2%2+!rY@wt3;?_AC0^Y86KZ}!7n_Fjl%E^o21!?ebo2h(XMo=k@xT!RiM!4DMT z^{D`#f?xB^e407=UK%mR$4m=`^2x6y^x%wfRJXM1#~wNX5x+I8vHfCeY(Ki_oO7wc zK{aTf;zelZwhg?-zCq92>f_fn_9GhFxlyDnUA zGiVXwIpG!0EpemM#g(P^>aLojeBZ;C+? zC2X5v#;Z5j0Qy%fevgI^xiNNCQL7%8(}@S*yI}Qk^XkR^q*cv!H0t8h>4XL~s7ztJ zwy6Ej5wRp5G{cu?m+&QLfc1ZvqBK|e=o~45>*DH9yLN!mJ zF_(3s)(2IfefTc3^*=17SEoNiJs))wnps-)7)j6mUp>B}5NERyzxPvfwv*od_f2&2 zXn*w=UlK)w2b@EvH93?jl;nBIZR_UM^x3R8=$gS}oWDeO+|q-VZn}>~|LtH`8{|^h zC67@pnUt>(J17-!3=yvN0q@RjKhY;2O{bAJ@JTmSqx*J!Y1H4UI~%Tk4Ag&R*}5jo zd0Wo;89~m$vx~0n-YqrfkF?>OH>E4*zq_Dm?$bBX#RuL`6F5IQ?rqFNr;SV1kNCCMG)NBq95{+oy7Q0W|M40d-$<52qf_b07LG=+ex4IjI?v%~hgj=$K$7nztuGk(Izr@JC zMfHf{cMdbi;@fBR_PY8sVEkfw?A9CUsJaL78)5!n#vv9cx-V6(`!~9B>^2(3Z)UB{ z82m4qyOEEqCXB}GZ=4<$D8L`|3KZlG5Fh%83hz_SGvq5Qv;z1*hz+~~9gEqLI(ac* zI@3M0-g8I1ORqdTj!rtd5x>DKpgu(oot9T@Jy=N2M zd9|On^!$Z1@>xDHWb2+m*xdFdJrJCfYJ9~@^y+^fpyQj>qf-0rLxp)QC{?X7oqXAN zdg;IaiLN-Db^9?iW!`$qpTB_2+5Gw4I?FzoKgIaIx9?uRj{1$(lTy9LEvBKpyHc%k z2kzq36W#sK8>w}(Bd9`Y_KDZO!u*YMfr9%|nOa9t zr#|Cp`qX#n+}OC8Ko@trjrb`w(yZ1x-%NS9#uD{MlYW*5ov6kW5 zHL|T(y;|UErY&8vn5IsfM18J4p6YaupOm7bd(or+;1fS+upP9k2HZ_cAG(gtX?hTq zE`(RJF#baPOtMqy>0#%4iCJIBSV_h%p>F$@1_vjlT3s-Nb@!$-8`q$+`xc|3g^E$x z$_LXqUHZ}24-Jg24w^B#4~<)}i3%3rxtBjbLvIe)s!yJbos_EEp)bvOc?9)p%g;$F zRg?;`9JZ4DI^5!z&NTDo$7onPuVdoK%V@$ko0SlYKXyPbjGRCZJpVjB@XWJro5aUc zXu``(dulxO8FCGkIq?iSK5;5NDDYlSP{o#b8<)IG9b+e@PNAujM#xDi z=>ILoKlJ!m& z!tCk)AdElF9%HfqD72GB2Gu1v%gq)c$Vy>U2llE{HKJQ++8yEUP-2(&`a*9ZP9&;-u7h_uWa0x8F!zPpC^3_b-93cjT{N(Gpan(Gk>l+*q3Pc7JLa z*c#s9EPCwoO-j*07?0n0>vf|ymrbQ-@9Rn(kE=}wmfeSn7As8nCCq|_it}?^52LRC zx`Do3^dR+aZ9;&r(uwV#pcNLkLCqHTONLay(A>n&iPYLE)JWrdt(8*-?q5=W^=vQS zYYw$Mnsvc>XAp?tfP!Yq8C{%ijXJQ$-Wq(QLB!YDM=4x_-tc$$7D?N(EJ zJkVqSrP|{ToiwiBIj1zXl%sso8;d2fCrWfJiu9>oh~~wwwLu=x_J;az;U{>n`{_se z`RAQF)@{uERpXb+x}C(DX8Z>{wfIW$tOfPyUsK+rV{7Q`NH#tKHbx6j{d0y{Ix{|y*%Dz$YGx9f}}E_hkU}C&HABI;8J~d482Y@))<&5 zT)sU$$RFAdtFgZh&Y&AIFcQLrTW$|J8qQ;aTlk=f`QpCGDv&8^#z(wvq>*MkAhooZ9}ai*$`R zV<0u0Gm@(ENh#zVEnG)leXJL+Gj9mJ?^7$OQ~k2q4_+5szI;WgK0jaT-v@OjI=qea zLs#MQ$J2w)ucE_h?k~KNKT_*>{AF5oqi#HxWsQ=#OjL${umLU1rx4t()meY1TKus3 zO#Q}Nh)?wOcytxj7=9exG;WTP7hFe=OzA+^@H(jD&Bd=Kg-6#|qJjVYiH<**pQB@a zh&vX4L0MzF^BOB_5!To@zM#WSKEzyz!!+^dYW!QiSsQZc-ARnsz#4L6+cy3Nf3y5D zhUQfA###(Q45J2bfuqr5&(d_kOF!ej-x-UF8yGh<=ri5=Hrv(K`2co{3HXe$1L&Z! zz;>^7?m2{RKdlKrX9cnNToo&O1ikj?Bx;IhA|Wr_C;l)V`6DvALEY zv2^sSm(U>vv<4umKi|>FzK;oy?ArCGm-`$i@qi;%FX#KNLKW%a{*Tl5i@MMq@R?iB zhu)%2H?@~r@hz{+&KkltjrxuBS8|0t>G1|MY*;V4xM>yc=4@DP4JL$&xSTiPoP%HD za}cZ+3t)vNC(;1U`4G!FpI56;%*i-ID({Wdtw}U@QBA&5>2B&+m4D!Y|L||k*FQhC zIITPNe`%W5l-{AyGndflw)%yP)!*b4nl3=i8uYLD?7vi}_kEuA=CmgMpiAy@S7*(+pGG{@ zf^Pf2YT}DUZ1J0a8mn`ZfyA95et7jq)E{>88b*bmd(n7IQ-7O-k~8j)io}F)EN04S zj#o@+(b*Z98|)UGaFZTTaoFX-%of!L#jQUl2<6}blnvej&(N#~P&UW|2jngEF~qGu z1)6N6&-AJrsspj*#h)&pVa-E4CC{!tNjho`8lY@khX?{sG}IbRY7Pi_gN%nhhIrC$ zbZoMw6K)7i-4GpojFwJc7;}h6`r%(T8)8L2plmP%JVUb{K-nM<9FVur#}K#v6lk)M zKGn;I+Bhh<16=^}Kn=ffea3*3QbgSzd67>_6?Hy%e^Tt_t4t?d=5Oj9LFd5tC_;;-f*?yu%2?oZr^>NCCk4QJ^?`J_~-4DnmO;V$g$LuK*)*tLT2 zEA3_+47nd0_zag`9A{|O11KBhfjd26%pq?5DbQpieWq7kQXT2)pNtW9G;Th7Oj zo;H5=B~#n8;x9zy>z&S5>HJL3KmQi>I@o`j%Kb*k^VklU;4%Mx7fW{j7#QIX>--pT{=uH=Nc}O>(BCc)QW4Q=pOvMkN z_Wh>Do>eya+PmnRZ7%LK@nelW_rb-~zP4V~Cb5UUyO`9+%5>tTy&~6M{m+}S#-{oa z!p(Wf=#TVB;8wJuqvf~BhRNh#FQdmF|a;`#V=PL=3EFZ{YBnc}g{f5w-Oe;e|H z52m@$YSPo^wI+V-caWHhO8t9I2P{on`$_77lTxtba-zOr@AXCTm#a&+FS(5li!?mWU&5}G zBXFl}TX&?8Rd>4hOVVk#jig&zdy#SH{)fKWs=i3hQ45RT;gTod&gVV7{yw_B6U$tG zuACe924$2)wauKbTwF256M+@U0wfAmnGSBl^;WcQk{F3n>~0 ze}(#ML~hS9TTDg|;|HzQdC$>#%_`~4<$79;jX#9wD}U6fd<)N=dix)~yhL*o+Ry)p zr2Q<1^rxH8PBh{R{Y}Tu6@)=Esd8>_vVdq7OBiz;S3&rOEov+u^GiCSMJr-+0Y$<3 z_*I$O_hT-*c%U@t@9y)*>zCALHY@;(*;_G?L} zUj7};{qQw<^O=z}ak{%A1|jjov2QqIG<6;^o31$daH*3-8pa(}*54@tVrh3`B>log!RH2+r3q_SUpuRef$-gpU!LSUAo5d`48yQ45jC{n6j2RYNFJQU`KI4)8c<6lL2@~+|9zc@h54G0qa2XA5 zSw*hwFk2yip<*Rzzf${AmHo?78NR*E-du;ih5XQAz8O!lA3UHX8)%@$wIlzs?S#qSzctW z@sW)OvBd+zI7%FREDdXT7xi<`Q+jRAN;;!bl_)0TGXr(Ij-aXM^4hrVXCn#Rv1vUm z`E~(KnKG7!Om&xFaZk?+ZlVL9;ad_*rT>mSC7)^HM=V0|@C&;?=g~;iFPeFX{3QLK<;!$+>$d=Tuk7{~w;B@NExAm1~{#g8~ zS*XyI`}p^#3m?9TuCF!FY2d5N-|0%%AICpu5`}EuSv^>Pa204uKABB?S9a$U)H}lO zQGe(C-VL}CZd@5eK2Szv-pZL$FN~Lv!x$l)p`~BV17`l(-z7|0B{QdiE}oZ z3^G;v6e{ipzJzK~7zCd>2pHml+c7lc32|exH%S&3d;ks@;?|!N3}t|`m!bzyH&}K- z0CP?ODxMd=<-rg4|8f0MGHyB5-o}kVjl=5tC#t`Ot9v&WQe*c_m1fitnLPlC4_Fe@LYk6_81KL47a`<`7`)$v82YITnlj+ z^C?fJ_)Vd~c+6pJUSk{!c|zQ{5Eq)o2Y)Q)5V!uEU?`)p`8=@q&a6R=8$|Fbzhn86 zuKxu~R;K!`&!;QzolR5z`y6%T+g!so@tOuS_U$DR$zs;{BQhhXi`CuWZ&QDh9&g*2cnK>gVomRike&94LY8p_PcvT zYu0RZ;~7-6H>V~t4pV-1Yx}gVnre=Ma_~AR6#h@|c%FjR}A6S19MqDmGVa|ZF zcP<{qm0y(?sMLvuo#%h~@SXQQC)_Nqa=zw}sZ^u7+zM)b0fxBh4fZ9<_%(Lgm$a5Y z>!;&SAO9cs{s$g&hUAY&QUrmg{2|Bqvs^PbaFaJ(K^9`D<`sFcnkXJ)3J$bNN~nHt zLu$)EcyMxy7IadJ<|$jdM)jy^jjB|!R55-KuzCbZVAkL}nu|jEfSdn@Tfc|6UdOkz z@o)Zt-ghT}$WBms5y8^NUv$5S>xZ9$pZsHLzHQ&r>r%{<692h=VfS_IBzkZQc1?|^;hFxo-D?vR*LYi0)|_OQU_3-dd=vXeh<+6 zdxNX6_!hX1r;nzucc#y`*Q{9D_-#@onPrvnM607g-{Z0BRV&H~NvkJ%LVnZ}&FY%3 zcquwuD7>5bJ$=73IsUB^kfkv0cdnvGhIWjdlp1jJBYb6Ey~uVb72>-WzxO>;JfUvj z7HcN{l=xW!G?NSUc^%e|+-VLB+B^s$yudMF+#{ucEHhHO8CQbYQ%U#*E4ni%1dJD9 zf}Hc*L8wgXPZC5IGXd&mS%I)Mzb1f7tvGIpZ5_Qm@l|7{Tkb!JN@HxexDCR#_~c+Z z{~GWQESzVW=DbR?m+cg8^G}?#e!$&~zhKp4Xne0)k%m{NWzi0{VEPfg&x6Jaiq#{1 z{2FUmcf%3$?!+${LMh!Jd;xC$2@-j~mm`uKPL{!6G5)j5qa#~2jg)xiM@XbN?Y>ER`k{3aXOOQSEQ&DqMtpVVi32H~5`wG+@dM}0 z*bP^%y!ey#MDG3y^+y~s26#MJ&4q^e(1!zVe~tJx7TISQ|53nT+RiUEjxSg77pO!B z;n|;Z=fON$%kNez;W;A*MTs8<2M?P z4F|O25W=^gx2>TcmM*5HJeR*;_8o1+T>e(rqn*jy$dzh@36JvW>uJK7<1a{cTc1ue zP9Nr;_378Njr9~@0oV)G)3$A~T$t`;e3rXi`781B&6sOCI2X$rTiM;V2}%?X2Ld~( zK7z~hMf{-gCh-TjW>@QFK5@4+uiK;y8izuCWrhAo8+QcOBq7dW?%|OjaR6EiaibX& ze%5C^uwyjv7;gPuEBEn<+?@E6{T7G8q<-_upz-b8wvm>vSWb&qE~jr-uBP?;?423Q zjZ14HHDiw@*9@sZ2`XnSDbF_4j--9@ck3pTN{OF#ZlLd@UZQ@NpGEzGtC_XDfIONMAa|3%juXzjUH^kzna%vPRv;C(iun_Kih!49sH}7 z%T_F-uUD?*oUf9czZK*>lr{g97FgPbUldh+pcjRWlpbQ;;G-ESy^bP&UxANAM z-;2@lZ?gcsMflf^^KJQ>23*;L?pp*3S`8X?@2zxpo&CfvD3NX)BwErw{4yF?4bP+_ zC;nu=#cePr{c(3)j%4~X?vf!FH!|)na*T_}(a$dLd?ub7wD=jXa^i zHEY0+o3O*AzfAYhG0v)D@fWR3EzahbpeLXpi?UX3r8brFvy5rfxK;hnET>G6P(aaOEXov9h#WW(x;^B|t7}1W(R^rylKXN$Rjz? zEG6etf=%YF@B|$>E=1tvfhS`RC~sC1bh)KTP+=09t{a)>KfgSON|;^8c6>ea)|g@I z4>a%H7iii`<7wiQ*VSI8z4JHT&FfH{AV=wF#%Al1LGY%=ud#{@f>RULjWn%K86<`DOsikTjR=1EM(Vupy}drbTX&~bfUrBVBy zNj-0W&V}~|J@wKos?WEo1m*Z{`x9_z!;q;)p!gR_M2P6*KFjm(Isah;A9Ku>Ie`yK_)9kV(8F}C zIAT`Zc$57}c}S>is5`{t5@yFk-|uNYsfHO;|I|+}=`rZA7(g{#;@P&AKKkebn)K#t zbpO=ZUMLL)enb(|SGX)Su#+4!KcvssccA8c5?B2WV-9hmC5z@q7OI!g`|2&$4J)$f z#i;kULHOog=aa?n^=e$+zgT?ChJMiYJNOiub@NJUe51R~@R>n0viVbVZ6n--6gIQv zbwW$jBIepv78u0r`6pU)j##qXdz4&PuQ=|4htzBU-;R})O+Mz zG}cHpPoc^Gx{O+t*X#3)9EUl?A7_s062_QdiFiP__zmuE{6D*sD3D7wquFg;a{HDs z1?*~5=M}n>IhN&+`E6N*DOrD<4vW>G=`d3w2wtZ^PKa>8;faxSvPMhsFT9n`j5uHH z5HQ>L`Ila!bI+|C6MlF$;^z#QP3Xxv{vuSTWqYD0^a`vmzx;vnHL3&+_KoQ;v|1EzMTBai{HkCRq20U|BHLyLfx;piGRktfOE=x zQeQ#*OV!^}FOc8dxOG6u@wsTMyB9x8`ROck*30|)`!xXk{Bq-DLQ$Rn=7sXfkz+RB zO~2>lMOLS(b6m2fJ*OWq(U(DAy0~r4 zaAGdsX28apkJ@9*^A}tHMm_#Ek{8t)I0q>--bwrO#b1CLbQ?wI@`SwyeAUZZ?(HlIMZ%8|r>+xK!$-ltIb9s6&mqoW(wXp~oH0B9&XZZ?g!kzH1 zV3`{J#1#Jm;A*}FvI2jd3^r_0;{($flA~Ovi$7!({~~P>V!1KhNq>|5elBYKT6Z01 zP@3dV>a(E!)?+vwvU4$?8Tp83xBj8Bz!DXXd_^Cc!6d)QM*SQ=(qef``jV873^^oB zpZR1mU_n1TGQOmJ@Py4|O_%U*{*ES3{tul$`h}3y;WepM!wOW9KZVYn`3}u>RLH5_ zFkeM^?DARRnYax39qK#rLwftdCfUcoeh%F<#M?dUkaDryTU;>Ps_S67@8pAM`&N5^ zIpPlOYh8@yT#bPOMfQ;gr^n)tad;gGvK(sE%2TsNS9<#F_vwr$7E9ldzGLWwDgERb zSjLCCr>*~%<@DU-i5YTUgL7UnCFkn1j?GAa!b#APGwV-ur_EQAM|FgmSC0Gmq*U1I zHaj{xDHYGB)&#s6<9BO?)ym{U{poP?&EnHof93dhC#Oz5_E>6G?{Hdrz$`Is>q^Rc zB02Uk3ZK+j*M=1S#EA`w+0kGqxhA**MnQ=_*tA6iLP2>_bCQ@7oNMz5qL4M8_0VDo zlrmFx@o)Y*i=Gx`ynACK-x}w1GBjJ}Z}P-my7 zJeO=ffg1FZ+ml{RFBH;8-Kw0AssHKR)Z(V1AVi`oJ@BgJe#iUP)|wvh=+P^W(TD${v(5cW+4aM9Fe2s&J7$lfHV$`XZ?aF=!y(; zxoOkGWiDft$HRqRxS}`>#bbA>I5fvgyzHy2Sf{FbSyEDqA!SFO`~>1`STaxbu;{; zhvSl(8BAjfE{KC_QrCunpF52P4)vXB8GHv><57L6e9VB6a}D7fn!-e?WAy{q+Qrl8 zl#L8ZTiRAnAOE&*X~aLT5?;}IbT4XKo=-44&1%W-_u*%j9avgVTxp#U zWg_3q9+XGwtT|Ra$*}=$9aXMhj(j{RV=Y#m{M4@Njnwst3*A*j)9BbokD*nUwct^u z`A=8>pT2&bb3Qa<&JQeA0;k#8p=~r_;3?E|ruvqKj_^pIFvEr!WIK+gB60pm zjUkdU-l==)htZhFe2LV}oE|AA{fk9v-_x*fz4^&L52fUtU?`r^xY{hp2vtmH$jZYQ8KNYQT6Sd9haRfr=!mpR|+I@R$oEHF6L* z$`?WK@`@rQ#mW-k!Y8_H2q{H)S;F_LPGL>6vjGDrIBdWHXYXf+0}9TX4SLOLRn$Wc zEt9c@H>5mixbwToiIY<2(9r*UOEVsQfI7BsNtJL?isP18^6~dpiFG?3D~{5~on0>S=D*8g zw)N**gsg6y<2`l$?_c>KKj8TJO!%e~?YF8wryyuJFoMCLJDd4$^T1yB3_|~|tt;ru z1*V5%h{nHr{V#D8T{V8VG0?n~&&K$e^`pNmYs{nz9!kSOU7? zfU}o<{3%*g4&+SotXp(SkQ#Ta{wyk8fKHD=i`$^knady~JPu)PpdqH@Cj2laQVd1d zK_MFsn7T=x0=+ruljp*S8uu&}#;sv#m*>X>{jjMr&Ni6EUz3VMd+rS2_7(3WPD-_* zC+>fe*4%a_b!~MxRp*mZa%-39vz5Z>gI}`2E^t8JiXK9@jqDy7y>Cbl>UiHP^yQkZ z+%1d@x6*<+uTaO6yU~#7?zI*)nXhat!G1v>{KHl#zrU&U#hD9eJ=1V2hxIceIG}8> zEOsInVhH5WOy25kOc)(T^Zu&eEG7RD1UY{d8g!43m-v>xQS;WofNY{ed{n(VR=%4! zDb<#9{sgVPtruM&IWJBHMHKOGclOjfi`$^Y;&hw;0(Uchsh?92#%}SO?)6&>Q|&gF z(EO2C8i{DiZB*xy2Wa*>jo0Efn2a9*dA*Q&iai{hdGMuUr3W(_LNryqre z9X7}t1ScA3w%IYDYsHHC=R8jSmR5z!gXm^HQx82t%dBrx@SM4p4WlR3Ib)EsX1KxJ z#lPXJFZ>l=9r@OESys$J=mHLyZi8)a4@cQHiIq7_eJebH=ISGG4A@rp9G)z2*J$rADvD$187$ZkVeNUdiMIi`zxi6 z98OO^bPbhvU-Cybm^Jbt&5NA)w9ydD%;ZGDl^Zpga|L?W_@8ECYV=YmCd=`pYAzV6xBhI}+92zdr z=RT?jc%*5~!tw7u7iu@;c@r*!PA<8LEM(b_@Q|0`CLJsDGKp*U+SK`_gIW zT`HWj#$8BVc&l|}C#uEod$Xnlx5)`dCmQkE6V&pjdy4=7KmbWZK~#8ueoMl>*qo5A zx8x9d_S=0a26epcEc8LLNf@8;-S~zbQ%Q1F18*yA3+yh)~PZ| z$*%rp$Drm;eSmcK@f)AFrWRcsAlo;tqKpYfLlyLwVR>_rir^Ii*SzQo+sXBkB$2vSj z=eu7I9CXE4I_bsURGS}1rvb4t(YH1w=bs_x1#I+y%|<>1%n5j=WdTiXYhSoG9;-Qy z4VRcihhi89)-Qi+` zk`kNaX`* zJ@}edZ^=j1-K}xz7|RCsA{w+xm)Egt*Cv|#`rE3C_hm|9;`nB9E5&@;&G@$m?|VuX zcezkq7N>{aHn3EG)%QQH{&9cy=A&>fM8hi@>sC*xT9l^D<3D0aL2|xgibZTf0jAra z)p&s0KnB!6c%fw6WRv_j&aRnmlTSj(8c&EDpV>BlzysWxB^WToeZMX!rY{M>7uq-7 zW)FS9O)klAb}~WJZSqN&)R*Kp+vX4Wfm;h>4sqX~7e8{5%)1?1pXJMPYxxEr@J#$B zmkB}+&E%8N`h>~x84bK<+x$TvaBE@B4R`Opge9|m)A0L_f3uQ|+m6v6`7L-nUpd9^ z^?NKXnaM`)bp40b0Jp8dxqvzMOEAv3QdZ^2QV5#+Ot4 zmOH=@H@{4`=}W@UCU6^D*~M=<%|>?6c$0SR7_!C_;?}<>#lK^7a1x?+V=B!LEe%`o zY7MBBBg~k;Ku%*>Y!*)vf?av|L{`v@FCKru;Lxzw8Of{WH^~nx5X4f zuqAnn20HxN!E^cUU@kxYfSr_rtQrCgai#iwXfwn;hhi2#AU5;CWWWVn+j@F)TyKXH zwQtra<%%)uH2EY%zt`vSkOm&}$8bP2b0t&!rYC9Fj#Y=nb66Zn$<{^53TSFfjU>XZjbM6V^7k1hC_=}ZqLZhF5 zmt(Av572B*h~GkUHpM7(AkR+PK|PA^ch^4bIjszU zYlakBE)0S*cdSx9RS&U)sTL$G;`(L+U%(#E+*&)%lyI zpqYIo!7uNJ_tX0ur-8nirrddVr16Y42lL78O!1o@gBjwNoPfvp4E`SVk6hpDk!bv7 zlmB>4P?wy>e`;EDbz&+x(QU#T57?3yVbI|w< zrsI#%_`oI_px~^_U^HJ~%Qe#tm~P99|F2*FXP>&CF6+{gmMs1gB zRWF+J{0}d9F=6kc${Sr_-ukEc@%6uTP!H*&vJ{O_b$|9!+6U{I>GUMYlyKc!)Z;JyPNZPrCS5E?r0~ z*O(KkuvVKMbm^e~(8DjSpf{eKL}zzsOQl>MLR-$fj~|3=;!oj+S!X(&%Psp|MAi6q zxt!up&Sm(PZZo^-;zwT?lj6SS+ZH`dL*11gMD?lt(Fcn&gfgekLE|$R@d0tM z3oY15t@${o_)Wxg1E$*$H|VB=?@sVqOA2r*mS1|g6y zUvX;P?qre0Zcv{|^tA;UQ9WW^fY>s{ZxY6D$AC&>VXjYdgFetCX6*S;@REnp(3?-= zK5cWKypF~$+{g#M{$b8fLCy>3T>ouw!8NP9L8FJz*nfHaJSs&=7`s`}xTE-sRiSHc zn?#SD)xa2NX#WeS$5WruRvF#U;o}eS-5vjk+2S@B84VEiojqjJ*iajTz|jl>5BcBv z62V-^RK@n+G^H+)X=Sv6wN33%t!s6$1oI_`*a;NeeZ@$8X zBOf4<T3kZ}@E=hb)CiTvRP?yG6>1JV4gL5ts5sX{Xwb%+D0NV9<~JDe zK?jJaB?H$Y&5WoeEqYzhcSmZDso`+bLbAc zx{LMnxpX`&-pMwcf8PZ;Z`Y`bmIQqn<1a{sVb(R&Q|fNe=~ey8r0nVAx3~?$f5aNL zbn*Ls%auZnzXWyaHiF)`=G2I7;+Six!muf{d?$4K_$5$@m3M%GGmk+R527>%b zIet}`C*B&}7*39)+Yk7Okxz-}L_4l>QT+h_Z=6GTXZ(`U5IdKCR;`YW4YVFNVh+Drb zL#)UCgolr;*`V7U$4yqCsR9gf$F#fpEf(`Xj5)-uA7)G@#H~Lse$K$3y#6+>_NMx~ z=@Xmt@%WA7*NuPE@~h^5@bE{u1W8sU{KYvX+_NvxG$L{Wb-bUYqKWG#6 z1W3Lg=(+1^(8CJ?T7bs%+j~0n7YnDeXWjHVtZbs~Ryhv?>O&Y}`Qtfm3- zYu~?<`j2@-NbUQ-MQ`wr1ss1=ecHE(+C)yQ#r;zUjb=RALQ@48;!X&<%GC+x@}~=< zH6fi|_@8brBM-aywgk*&TL&#yp&p|F%7%Fl-2H(!-@4Cf>;*Mwd{ndJ=-%$_DZhII z{#(XxzCowyHE4YyO!{xJ0*1K72U(K|aqE}3lksydQk%ZlL=nxF;efJ<81LL+jK)O# zf#(10NJS7gI5On|eQ2iMj*S+=5I27Lpo0#MKyXo@AAUFm(f}(TbeJCSe(GF3Sp%zx z+hB-qSo~)6i+m8=9vR$DYqwy5Py*s!^~tkz_LH;0MxA@!L91@-O=mW$LHifh4}{>{ zo%r(=u0j3z*XgXyeR2S`zW;T80;5_5J#vTr2Of`qJxi<=oz4 zEpHb zzC+o6$@n)@V$RGuOCe41boI||{HD!^p_sjB>70twI4&m!7~)ASnU0YYv`lj%liV+* zhdzdQW_=mrFIbtbe)|$H_QWu%-E}gp;DKs-eP=2*|EV!_y$3yh>*>d7L ze*v~;5#4@#XS!>S$u^?N{f^{c8P#L>XnqBX?ei$o)rI1egv?#33TI` zRjAb!1L%#|pUoQ1nRGMsAi*YQ%fM12~Zc?0PwD`s3U zjJnjngl6%$wFVh9z>Uk^qcIm$pxdI`!MH;&2#Wkb?#xawjA+^7IrPIuzV-n=nDa9?`oTmF?Y`$ODO{2AslhnjEWB6{Y!5kjKLVk5Gfs@whbyrrkr+1&A%X_`Q_Q zA;~qzYA}jBir?4LWT77oFvP7t>5utk5PZgy#7ze*cq^bPC8Pm|9)qe=bHF|5mAyfu zcnFN=f^a$gG9frBH|B(cag{MmEXGZxRo-#|UBCfDJn9eXBl!Rq(@N~OpDN}33wMsD zmzQmmxHQY?a{lp1`)K;3bl^p|Mf^m?s8}J^v*|l}a99Vbe#2B=k~L;1+pT;5V%C^@}jz++= zX~fuvExY*fT(RX7ZuM8U@k7zuozah;TdePSU=Ohxw73oGn!rbi@$a(i{Pi)zQ;k7k zi0@APDi=n%d$^RJMF16Njc{7x6~y77p@p;%*Io`^8eoVk75h*o#Em1Qg}C--_SfVA zO)jLR;-!u{izW|k-IV^+jNejcjz5z= z(~utSzCQCcy*-OLtbcmTdn+~j{Kdya$gBCXdS#bAjKdt8EQVYqeIa0WOkCKDCa_ze ztfFZ&VzIMkxIxfEJY3+UFIWJT7kfW}QgO8v>WtyeaeE0!+7so$E-!w5@!iYoZ^Hp& z0e^ej_(%N7`EN^(^l?JMIcE7X81{vDy8fKzqb(%{p`%FUy3~>3WKlSpx;0-&y+&S7 z^(z#nWs7FfoOhq2iBn$Z5AAo-E6-jj4;Z%jFCuAng=?Np{o7pVu7Y}<`khpqy8P=k z>cqET?ax=6Y+d&qE&lXH`gfn+v6Bv}Y64BVy)4Zgcc*@!uf3kqUHD?TFbLUX{NWow z^jq8pLz?1$?)48bpvgRAxTnRxZzY^IdJZmXn?8(g)*b|(N(whpf1;rc08l0_wl2t72oRB{^;gZZr@$BWhdlY1l5)A*wM>8R!R)BU)0jvL05jge@4?eI^k$r{&%PUEfCk>^m0RxPPU^=foL z@j|qPe|_txHCdE3>lK|dkp)FAeQ*Jtb#yIXA2uRm){HS*sq!@ zj23ZgMzy%zU6C`J&N{CHU3KlPe8RFel`C41Hn01E=Fk0r?!D=Lew25zO`|cR z_oEM=nWVq!S93G%aPh&)0i~mxf$5I_Os11dpGb$7FHJ?=gTBLD5eJyBps)gRrX~Xyo6?M zmKHtIy>x58<1%iki_H%(qd(9l_+ckpL<63gPD2){x}Fz3NykjRkPfVROf2VTBj?u) zrwdvfN@a@Vr!DJO(HC<+qx@tYP8vSN;i#ByOa5( zl(tTE)3EU%G9;bpol(W+;uc#Tyz*iYIS~`ZtZm{O}ojjQ@`IJ{q`UcH?6sB-qKi+`I2yG% zhw2_yk3T%`_y6pD2cR8Qwf2hXl|TXsgai^G1cXTFQY3;jK~O0wpn!@BPXQa{7atEn zirBD#iYP^}0X~{f#79s;1yMknz>`pRFGlfec}mfIUeY$FACt5H=PR#uxKXCa>uh^F?yhoC@XaKu~KtE+!e1K0@t!M)Yu zpzh#5B|c|*d6ziGq<}^PSEE$o%C5b z{li}iOh1$3uiREHeb1z7&d)x+3pxK)(6^mjbiz)uiuQule_r(0UBfyJtVJ)9fB?LQ zz=8PVj|xCQddTbN;Im;$!JscsU%>Sw7_|8Hy7Bq5uaFIY^d&ho^vmgA{D|Cn#F=uz zfm5{3(C-S`_J67VFteok-|e&hrZ$aRJq#h4uZ}ZlWFz8SZ-5w<9>t@))8mox17bjs zG$UjO&eikN4qBQ z$C|@?$gu`Eua1uG*Yj7YJ&%ICrMdHdWSt0^`_}7`I$CG2?Sm;>0C}Zz%g|3RQoKY< zoZ$K*Or{Dnl`V)Z5b_ds#@}2OWsOe90_=>wRGv5?bW&xCr!=#h98%eYT#J&IurvPV ziYRMzI@ZK*=|Da?(%hywtHVn)Do~iEH<^iODjTWw7sK9IV}tFK zGkhvUe{H~=W*&mR{;RJm_z>|O`WiIU-cl;2gMJVi&1U>y@%NKqYfsj5e7Zyb3z$`ns&pfBp69m3A|o7s+ ztasdaslF(b!^yJGCy&!lj~pap2B*pCYoBlg_n;yE9T+7^+9srLfEb_y*Vs?`yRPN) zy0VsUG8H)N^O^tlwTy3#rVZNE`6-|MogBBr>+-vweorp=Mg3mk%J!8bzj1=>yW2KJ zTux7pL2v(&$sSu5J4EOMA$?oeM~02jA935eiJCn&F<0m?8gn0Dp)boP?LnpvFVSdB zbsC*ENGG(0)stY$HV)D}?W(}iZxX5t?iKcl2XTMMglV_up+tF;g|)yBohd$WmE@=V zXg%nm4dc}l^MMMW+ZdqA?BbO6jmXV#=p_06b(hFaKRHkSp6_ttB-8C`{DA`wo~P&SNuXg^V3%PT*ra>E5L{CI#sqCr=J=u2A%a> z_v#hb^pL&?1_Oa$fY*YuGp|jopXVNIL8r0yk^y6PlV2b4XL;xOp4VTWd$s&y?~P=# z{xp3DP*YtOEdo*m2@297A_^i(?=6&1P^pU2As|hp_f9}MNKupy0qGqn(h0ry-g^%< z5FjDt<$v?uOy>Z&L;Fu8h8L4XY zmMpuE#xnV?_|DT4&t*r=EuT6d8wfNty5eYST-m#0og>~rtO#)Kc`BYWx zyl+bCyD{Xy9GR{2Z*G}Xn?p!9CFw;uWmDC;;rR<4+VTVbxJJ-9B~Jksj0>mZC3Xjg z#xMllzn0NdY|}6lq(voRtWs8v6?}(h;dD@VaNY^gGdH&gzi`XSDalauser>{h54?6*IM+etv{ zHRq7cGowJcSt$|d4{{EOu4&h41oA+DOV>bSb`lI@jUI}q_U0Q)BtL~{Uyd5jn)FS0fJwrjH%9#)NYy8n273NHq7VG}VdfO)%kk+QO!X^TL~CG5 zVEE^pAH0%^x7Z1KNBDF18v`9km>3RiR;%4B9AMCj@cm_F;kaHHABW6GLPahNMo?~P zZodyiwB6cDzB9`yk-=+(9NDQDM-wA(sBBr7wwt`Jn|!F&9b3Uu0rZ5y{9l-wNZ@tV z5yY|50bAqiA^sJyL8tGu&y&A9n$9x@!&EOzU>~G5ujUH7V!l?7%-$xHlP=xrf1XQ> zb6a+7gI~Gcs$I3X;l9WN{R}w*j;wr+kztd1Fc*wRSZSHOQg1n|fEw5?AMV7yS2(v! zzSGaC+P^c{xF<2193w9^A@H}dG?oXNxM%A9+B97ZjD+|3-O~J+pqovoLpD=;BTcI( z`mi)U?L}8bQ(J$xZWep+{Ug@{z%~ZRcrA{FD<8TN`kIQ5>nhx|5a^9wcv9;C`Ws>2 z(DzGoyP5WY7PZ6<`{CoZJ4D6>NS@IHn~>446|D^|fqkz}R-#t_9@?2xn!l)YbCd^5 zIrCn*C?g3zggd9c;|qE(?L7P zg--muk43-YvOPn@hS$=RgtWbqF+qJoyS1qFzo1Tqet6;Kj4rUe@@z_{RLe5L(smo+ z9!fi0<`JDZI}H(ngFdP_G&FvvVXu0681IW@71!j}Tq2! zo4Y9TT2`O`r)AYCiLu==WsCtt16EQP_Use>&-kM-j@j%_bH6+;KeYyiRvl=k-^8pc z0+=gtcUp7o1_*LK7+KHjU_Yt*_iFMudb*-2i4d2-nM2MCH zH>mNIbYoXA&tRO>(ra;JCpiO$e!+veH7A3$6d7gH0Gx}@ejBLYNkCfOC(`3Ln_s7o z>q@-*Gv%CK;(LN6vHFqecTd$Z*zj^LI*FM&V-}W_p57bsL20KB8_w#WDV#i)=VLE=#0Ww86jt6`FHp7b%AMp>E$4E?kN)cq-@L0|oLb>A*?t@%JfT_3XuGYM14qc;ky8_T zj{XzB{+Pqz+v3-8fYN?x@0y-U9IH*GvDOT1vqnxA(^C~FqlHE@*53mDk1J?N zh~z`3WT6_EtjYphj$&>6Jlz-QpUP;Q@K=;}+7OGp%qyzR;R~?n^X2_)aE$4mBl(gB zzE9di2wLVbh1b9@EgDo*l=^O2@tpJn9Hh~CQ=CJsfnR0xGveo`qyMdJZ|B{MN zaV@gwKAoKqsN{li(o)=STL3%%t$KHEam6vuP%VBc;9*0i*T~xEN;u^$(BkRkSEG9k zJ9M8*a2m?AFNo4&NbKarS=XwhNaNe0)OIiYcxuz8+No|UD(BTcCDkTfRCI4+JE%T3 z#eW}}P7ACeICXj}#Q{@CE$0*~H9H8mP32+<*tG~^s>}1oUlNy?u?&SL>m+3T&e)H$ zH|L3#&=o)exf4wNpVawh!F?~4OdeQTTQi}hc&bmo8GEkmyV#;UY3ru+)Z1ZFyrq>=A;3r&@r^SrTHwAVyuUJin~ zN^c6i4Ojy>?Ke-qVuw&iJ~nCvc$q;~`LVh4s*_Ku6kXx**8F5LY~#)*^r1}(m_J7e zj9-+dI>bm*Hp7B{6*ZV_lTb7I7v2s2MaI@fn+Cq56Iu^bq!3*Xis=U08wzS*G_ZBR zPHD3>x<421mIy8&AsjEP>CjqT;To>g+t1#|rQk z9P`(q9>*qserpIApRETD-~W0byMCb<>Fh9R z7EP{lSN@cCP>AmFM+<2My|CN*_FVt&J31}nHupjv_VAr61h~(|Dz*;HoRz-V&52yX z1V4OX`1}IeViI=))_GbzWV3_L^-OlTy6rlfBJF(Yq7aZ8Z7S?C6tYv0>q#SvX7xqS zaCn^O*V2Pm&!=2^@1<;Jc!6AW<3M?97PA?T|25rwFpOqIt3Pn}3rPBeWd^gjMZ2wH zyA8a)gD!nH%Jvc(jX&;tO!iH@)?klEn~gIlRjDr@6QYist^NCmyy{U#~aUJwcELRk8dpDYK zXR9aOn;}>tjSTOThX4vq%mr&>V_AC=rp=Kvz&>ZU>pE&Ej15XbeNUE_ax8}+s~ zB$;9YQveS|r>{CLk?rwHza+EU*2!~F^j~&me2DEg=lq{Z{bdWo5YJa(c*!D@XLuB5 zGZO+-^!qCRRFFv9nHGS~^8S4bh)fxy|I}1`hg3@k{{*0cw}75!3HEbPRvF;a(x38V z@qxjAb&V6aZ;XHq3x$0+*x#ZEFZ^Jwy0wN~725XZQVZROotlPDO^F+`o!%s(M09X!I7 zK9Z=Hv6wMwh)ACtr+da$dlUXEpExE}#I*Lu5V14|=XSuNdGwSbngiwkSYm?SpBjWl`PF&ig^xFRJ@2b0u3fRu` zXDyq`1paU3OegkcY-ZRD%aWJn=3m5w)W`75t;B(1rFx2a2p1`ss6~G4qr1Y4p=v{? zSAYV0@SN*|#mOGZcN9ee7s3i%9a>rciC4kLgw?5O@#8N;S0#7I#Zp4#a8q<$cN`Q1 zRK~&U;iS@Mf==S}oi{5gc)h$=NNaOkQ@!?zr~81^y_^@)mvm^$_~>SJZ3Q zN2({pqQ-AS-~C&>OVL6QUd7e`#;F(~cz4txud45ue!W?Fa2MW3BA9{9@IHxPPi)GB zd*J(7$X}i!fMe)5z4U##u{Iu37>Yi7Gm==ew+3+uqN{+M_;TQsh+L=ZZfxXk2_W^0 z0Xp?n#3xHCX=R%GQz1JN?etDUux>bb{{2>&j5*_$#M;O31QVtG#(nE^qcTp7#|LnMxI$khkhy~|Ki@kef@7?5kI7S!Di^`8 ziewo<-qz{{Th6>TY{l-^^bm4&Wv-m z9azIo2SalC{=;OA*Y5Jhr7%ly!Zp8i_zB5b_i7YL`!~tVLeDwxfywy{F_7Cm64%3c z$V+?+o|o?~=9=kAZPZZ$BG0>b7D3NP{J%eGK1w;?>SK7VW~83}B-4kMP9_Rot?2sA zu>;#--mHr{X`8Yj-qmOryId@1kI_KBEa3~Nuu!Pbdvq>i&gOBsPxPFdSKQ)N$1hXe zu5ZsSHHh#2M>ikC%9-6JNWp&X+RT5*De76Xp^x0p7$1OCg+Lquj0|0O?ni*L6?@5Z zppEhK?E!RsFU8#kd9=kEIF$B1TgmQ6y!>(kuopcS>Fd8MyiY4R`lPnFeZxDjY>;d0 zW)Ee`fW+^$LK5+)Q>vYv(yMWui$_K=J|qGS z1|8$hr*?MHV*&BBsl~Hnw(;1DN<_U3p5js|x%c05)FxYI$o&E7=N%4- zgWhVadbC154Ycm<^3H@cenEv4cx28o?tEcj+rc4C|UEe+fBOb zXNvZPzFtX3+j+KFw9YL!tTt+11lBHxX^I;gP%K=B9{F_^;HQoZ)GFNJp#L5fq8q_n*$KF!w`#L+Tg{HNG9xb@N{zM2&y{&3Q*HtVAcHvWfd~@gZ zO0yUCK(ysDAU?r`&X+M{O(yl0gOfO%L8(`xK^O_@e0l$A4&uYN38LU29))$3iBN$8 zNtqw#U+%c?hg3nd4FlvUo53qg>fFyIOnfN{cjGpLct?FK@~9)b@7RHZ$%RB5l^&U6 z0(8OPD`vpAI2nP20lY`?GH!XeWb_^8!V)bvwY?r7&&!xr#qm$Z(35p;%xWq_(6N&R zKb4}>68Y-)+w;Zw8OE4&TJ<|#Ws$HqlKN&1zuVqnwDT~LlmdwW!5%39*}5nZ$9ho8 zJ!vAHjElPhl&P`<4?aj_3#n2}SwB@+H{&GDf~~g{2j@C}xWnoEzd3w+=^8v4k{UpEoQR`v|6$PpQcduMR$^w_|ba!#!UZmEf(E_9nm4*cwgs$wOB~)mz z8>icG^{vV$3Rp*3gShEr`_@W-%e?@rNiz6A|0lg4D0KdA0Q%*x)G>M9`2F-vJZ@BC z2mH46wLJG8^jzrH4f)vX6kRC8IST_x6VG^sSmRZ(n^<6FrtPY>6}#9|=$p8Oo*ASt zyhO?4Y;HsS#g>++RN7dLbC^JMUb;rx>j`*3 zlF1J~EU5jc<<)V(V=e<>wxa{hbw``vPbMRVCgcAB3bYMj<<9!2Y@Uzyx@B)%+$rt> ziG6a-_;=zZ_e{w8n#`#Hi;d^c6Q>RvN*6C;2Uk8pzxiCZN!CebB%zZRU-V6NpDq|K z9G*T>wf6yvBI8TT4Dcgp@IyFg1w7<{zrn7)Gn zK};F%=BW%vWEZF_s#A?oL#O?Xj9CIaV6WujM1pvc&3>uKR{J$XMf0ZJ@elChcZ=82 zKvSE8SFU~k?VJZ`vgC4C9)xtZcln&vv!n>)+N)l|0f~IZ7)}E*UaK@dUg~0pyOUa9 z5+X#i1ZZkK-`C5)*CuzzEX>P7Q?$-AV8@L(mt_OWsmZv)QZ(HhwN+W@;{xx|wG6LU zBqrrre7Lx>e*uC!^?DmBlE!lHEt{oksme;mUNOpuh-WsvUv0$>LnbLsJkc9;o@NoW zHxh*GsVD(?l=?aOzrTohFmWU<@ zgp4c0VVymd;7Z-EJa8e9zERi(y_S9w#iI4wPxj-CN`iUb{OwYm&qkn2@I#-ao7>9= zxQzV?sf!r>WL?>5wRwK9Nvy=46Dx#&ylc#WQ7MnB=i<^iu8JMByCk0?{4CIFhKv85 zbnVaZ+&_{tb4oo}b(fy+4A0|afH&Al1|a<>$J&qW$PgLo7jrtzw)bgksQkiM4Z?cf z+}KWV?i^Uj8-00BOYkMT*F@m);b(0}spk^dwIj$sUjAb&RK}?(g6DsBRhOu@eBz5` zt?gB*#x_g*WM2))$hhfcOO>+=Z^;>RT+NecqPmJLqh(53PQ@EUXkG0^<#~wZb<_B+^(LV7D=ebXd zIQ*VRPLUvg<5J*#x$R{NDQFuXAqO_Ls)HPVxF~Zt{%*a>ejPu^#6!U-gyZDu8_a6n z@(A+3j`b0CyRyeVGHl&AT`}5)<%}#gL#df1waG5|)2~Iy%5DCmOkHy1fNXeia3G@f zrPIOxn>-M~`0#V%wDJ$AYhd#--ltXC#q@7Z-LrDK&7SnbiWZlX@2k49m-lHyZk4~T z*+_dC8wb>?y@B3?lW_Ofn>MLv_5l}pHRHoIy zHa`8I>47}q&+702n0XP)-C@0txs^g=jx~5L0Ml=z?sk^nPn+S&kc8)_MUQ-}PQQS{ z!w<8|Ss=mz<14osSgZajTt&gQsbojy`aNGPWNLk*Q}sjAQreA6%(yXLx$;Z-xl`#1 z&2KSlP?_9=KxSbFo!Yx9@4BS+^EO)E*wdPE6%O!I7FD&vgC06H<{`hXKD8#h@Ke5* zK3bxbwpnZQm@@khK>D(dbnEn@AW+!93in%%_2a7BX64gguP6LJPFDKxj9k=h&cXcp zDhZA=XxeibJ;FM~tQx4KyWk#P*rrdqQA@PLVVaHXZY6%K`t9fR!=yT1R#qZy)g8Jr zmDmVlc{tO>{af}^n?+PQJSHjfx76Z6?ngfT*#cv+^J`}5kGhm^>+FpGyi5ikY&tDO zynX0Pbf|aS(94AQGYds{{11E}5*=1K!U8g^ESV1|5Ocd<$P~)#7!ZFT=v0yo^p|cq zPn3`@i4@&?4ite8!i|W`*gU+m7kJ9)0ni5GN7rKJ|C)rDLxg{HBs}HJqw9N!$-VJ*?dJQ^EU%5g4B#d~G$UyxdHPG&ZhVDacRa|!Q2m5DC_sQA2yKGy| z=tJ8rm#-e*6YgO(*tFb>H9OCaQ88}I40L_{rYE$qjT@)^pBAB{GZJz|dJAyqVo(f6 z;A(y3=#O8A(keZuZ=0LinZpaE2$fdy%Q*dQl$G8*b^Z`>8)XOdtn~&FF;hpZyK-=` z%-9BMF4qMo>1;kCskwUmLLq}yVwGg;KXSPO#!)kW&i~H^fC^z#D_Gh7kxOl&MUEE6 zr`;q&u&oHW+sL-Mx&(zG3cr;K_`42kr8J@GYnVozWNm<;ReCTBr@LH7)vF2V5odx> z$KH@50}GwXAaQssv3%%CB47YRM$Q_x)3@0>r)e6S>~FgnFQ)96 zz_v>p_9RZ$JQVdL$^eJ(*-ab&?e08$RCiH0bio82BYwPj>mpBj(RZ7|3t}30qFujIzlklLCGaVi4YCT!{E|Db;SL<@LR3O@%Q$?a zzpmGbqCr}=OXNDvxmZ>U=MD1&z^3thu*jXau9+CTlz<5)hY?`$Rk(EkqLg=)1%fNeL_9yOr z#=SQ+zkkEtm)KKZ$~?bJmwzeB(;&US_ncJ_E3F1RkTF{+I`lHy9z!aUd$4l+5#c?OTkKWA}}ai~s(ByjF2_TzdxBoo0syRqR@0FFZ-HiKluaxV<^zlfAb7&A&}; zgK&v49yj&-%h#0Y*vbr;HGWESBa4RqTIY(WS0>zBYaSoNbm1ptzk#m4jt(^sW`4E4 z!7zS7Y3KtV)6xL0?J@iH%V7KG0^b)lq^Q;9vW7l_9LQiwSdT&&(0u5Rs&tT_Z+2s= ziXp|I8uv0L>=?rJIMj_l$A92)k?Fvn(YC19*Fi^UW$YcLSp+FN8>Qw8^ETDmnM!F5 z$hbxZ+Mgj&a{wB^3ORdVtby6_=Z{C{(#QpvvFF)!=)D)?^MU<4!I!+`CB*a|y+1xb z3jeLwc>Y0a#DGomXX&dOAqs~LzVh7m*tNXRQ)CyxEO$P)KOS}>J=cq+4e4m_8zFb? zHbPBxJH5S7B6@x7Zt?1{_(1jUjU6KT+dxomj(o5+>ckoegI7Wl{P5>C3ju4&R^9y* zVW;8Q=Pwm_D!-q*0iTm|jHcbBqXjx}_>mdLAwB^)0dfxX!J_BZsA5VT_;vWLJm)oc zo0P*9#28Zr;!6Bt1-oKEO}?FcF>sfYPoV~1?mq{6ri6zms!Ll@S(ke@v+MLB=i6=QM(~eo?tPQ4oCZhj6hW`Y$(-OIf~UX*p)-tT zrgpg8o1%yiF9J$;eIP52TB}|spUot^1Bre7UDI@OAjG($5yila|0iF)!4gvE6!75F zD#S(Tfs$;QKz4#FbH>ao-o5ofXFl0ndo(U_ki7{UsNs%XKnfZC@{hEx>V;N&-c0PG zhZ-oEpJvB#2hNUP1Y@cGOEP`^mWu$dL@K z{+dG~q>o??gh^55xvy1?GsTT1i~Tc6`Dm!X-D5ONXvHNfXn8=OsE^+`e6T#eL4=S` zyOCq;Cm}<0erE~pcCHql#m}ZgxROnrlkEQ@Wh}cR#r%vhrWaX8cy0olt??inPf!Ww z0f5}=TTYdsf7$b0Em*EnDH`T#K&^&>>_m)Se9(HVu|YI;`0W24cB}9AIou#i^IgW$F&`rGn3Ra^}As0n3n(X*GSgz>%w< zpwcAC)6^W5a!a5&Nm<(oh znVSWH&@DTx33-}hTbS&UnDYFs3A+}uk;NLJdJ7%Z?ZocAYb3jS^vPGfD z`1ZGEn(Iuwl0i=R;dfAKg`jgBE%E$VjH0XpnmF=~>tXx=;F2v0E$!sjPrhVXb z#s%s=Q=r-UDev3eNh#hz#LkM}^$&U3|fj#%;ej24u6Qg4Xc|COh?% zLK?xCvSQd?&H?^64ByD}XOoaBd}abUE%_34E1xqm-hkSw{~XBfEM-dPJbQ!>eC1F9 z=_!|9{KGVY1eQKqb#f^u`MiDYX6zekVu4!dXYTzjebPH!4onPvV$QNEMl+^KCp z$}L3>9RA4MJ#4A~HCVQ7Bruzm8_m|xx`+CQ^1a3)H|Pg-yXimg`n2IKdEXldNLM=j z#u~dhif*rlKcIIbzHZY+^w5d$yY9bnuID(FFG?1ibK)n|E@3Q^7M?mKGUE^b5_`Y` z01|<*G2$*KF!{1tZd^i5#SF0S3t5u^{uMRMD}CG2I2Af-x_%kq@S5WOd%mh+V5ar3Bak(0%2{)5MYZ1H z(kwd)h%#(Ej7NXY{5-b|KQ`$>dvthD>+toz$CV2OEhBt(LWF3ytuO4muvTbRr8T}3&RAh` zTs{7qdHo5`%lfaqWrgcp0P84n*^0|dk1C! zF5Bq2W1dkCCXLjKo&d5xJGUN7P?GzI8K230>K82RBdCEHmLzO0_XjtK+2H5<)ZO>f zcy_#(CG@?5jlm~Qwm#SA=o0U1u72mJCDAWIbj||1Oxi9sswIu&Iq zQB{j`=3-NZdju_pH_L8_k)acj7kYLSHx3LR&%Na3;U6^H=<&%3bnUZMTrJZnw0nc? z5U0J+&b$^??nFwdM$3e$9&(e)5uixZv)xdq?y}!|^LDMZ@Y5NKOp44jJ?iO4EjXv) z9}8|aLl_i?8dSbD-wUreSy}JI%ol($)t3>6uiY+PtVbE7A#DyDzolU-WsV@GcW13_ zXF+Qk>FCM~q>TXcf6BkpUnGcmwt|%bW@m{c7|~?9;|5C_!7j2;l}&SzYGA{`2-m5= zv5JWk{~bH>fiT(6XSWk9$VZQ6Z~s^O*PFzt>icH{m0r#g&G{?7NpPo8iy2CKY3wO0 zpuDCw$Vy{N;GU}$D-vB)U&{!Qf%4oRCu5Y&jyp`wvA{da^^Ql81l+M(FkC>&lV3$cw%ZFH%Z;B3{L7MjD*tE$yW4sZZ<3MkF*0RS;3yA(HzZ zm00Y!8M^I()xek*|1QNHkq`>e&IAljiyrhBz8SCOItO-~fXPd0IJO-FZQCC}Z#)-} zua*Y$29^d@oKjzI^_6px_*LGV_Yno>RD6#TDHGe%d1I>ZHvq7Fgr5V#blU<>JSHL8 z{b|h^&u@>b;!5=5vL7xzJT)O?GmYuSd{=Q1{T4L7rWhel%Rm2Hz4c^rAV~(QL9J*D zH|(`v<%tWvD?p(9nOfM35KSgNL+7OcR{a((MC;Vm`xWmlxhlln05Z{tI!m2dA;!MwxaWBOpgLQQp z5_SNA=)@Qz!d{@)S;1NL-)1CwS6Pi)#+5k^pQ8sT*2#Nzjx>-NecLk7M&qmrzkds* zZJ#vd56Xq7Wh%$5{KG8w)uvQU9P^v@ia@h{mHlGi){lYsU-w2hIDrISYRiy2XL*N} z#SSiJ_LmVZw6h0xtgyvDZN~ETdlUTcFqeNa@XMw#RaCAgfGzH_Db=BhJbS1MU6Kz~ zjU0_4$E0xq>YPRJ*}A3XqZ-reEb@qhDr~cdwamHMJ?_q)Tp~W@Uk~cjuf29g| zg2aJ=#G%m8M_q-{Q}hQV*bLA$phjsFp)-^62aZ@mw zxA6d?p;V3A8}s>4(Rf9kRK*Kt=e%`*#UZt8@I7*pt+hvP>pa_pvgIq5veH|pQD~*s zbH^)u%Iz*XBJj+J_w;^WYEBEZJu$8YmWI`;Nj!oaAc2jSb#2<>i@sXS;%>fa?_g0g z-4~`4JQF$c2}a8*h}A)mRQ`2gaKhrZ{{*m~AMM3Ue`6hdh+Kd0al&94KdwkVnMlj3z5H-+wcs- zSr3iW_5l^Vhxeuw!NyJnPB>!}95pwHH}gCsbDeyPtBUM(0B*gtt~&93zYF*#0{sch z^ln`KT(~rJ3q!OCDuR+%pkrzw zg8Aad_vy_Ygyy?*a;l*p3_{@atn3?7Cz$Rn|3P^A!{I0Xtf(30rdXBmIP*4V+43`9 z>X=XG3STx=l`bLZ$3Gf{^($;}I`2y1_e}A;Mq0mljWk}pX4lefxnXSaj>l{sDR1A- zt+IE$siHJG<%5@tnCebI*M|cRq>MW^oJGRBB8w#3OqRTsUGV7f7pOpcK^5c{lVfQ2C3;o`9>O{;< zWQV&etp#MK-^*c@JXU(ap5cZ&3?csgapEJ$BAj89MyK^be0BB&AVPkw8@gLWQ<=+| zo=4}q@?(jRQURbH-d=%Vx_414+Moc)jcM2R#1F5A;-`CmY1;h(ciiKr3l+K=Aq(}& zGUQPHrX-AF(gj0qn+ON>h9H_AUiKCE@_o|72a^&A`rFDtx!$xN8fXT|%@@~fFD1*r z!p0f>jh~$JTrIyk5&3kRVq-sQB>(R_XrLHO_qY}U zf;M~Yi(RPZQf!w5qEDzDlSTENp=`>=^5p$elg>244&A6CE-1N{^mBNiSwR%6801@i zdkJcKCdN#EoBz7RG2YxgP_@?%H{Y<@PUItm@r z;W+PU^u|-aupkp~QTXn8K>)Pbjd@Wof`6X-O~2L(H}n#3vw8q~))WEM&Yy?IfbnsY z3mYNDx^dgFL{6yUCrn(cXgYCwMXy)jFWJ?+nOvYw5o#qzR^xZ_Ll*Kzjrum!)9?0i zuc2u-L#$Rh`1y<3-#2-*xOz;e!IANqKr;?1aE(XkxK1Zy<^~(&E%$V`jiC5EbyL3? z1%$`PWBjVH+58&=n=KQ|#Se3p z898=Nv>GHK!Sd%Kqx1z*ge`ZiJD&O_9H#}>Y$*nbOw`Q6)?{+fw&dRYlWo*vwk566 zt#n$PA+qPAft`_nfSbn@Njl4N2A3AN>`0i>HdI*1&R5}RgN1c6XkMrU@ne@w15+l2 zP;4oNT_;|@LG`Z^*BM>GnrMJqCC_iiuD^vcsOp1y-*!}mH^?!SSv)Uwa=~W`-|bm; z>!dH9xlJDCy!b8ph+n*ETE#BIh3=b3PfYjSh$MB4Eoj0^f_6C6zCoa^Qm0@T5ck0& z<|c$YT{mOX-~|)kMs+;Mtw|=}_(LI~-PTDAZ=f)M_ zWqcxSqMEL-IKun`~0gI-#+)~-55zvVV|TR4KGeIM7!g1yE^eg1#D zT|9D|>Yi##D&>$J?;8FF$5{T`OSLVEyW2i`uJx~`$EVecFWPzU89U>o8|nH+W`R$9 z#QTTf%pULCR%HF%RzE^_+qJ0HiUcAqnQ3zgo==o)_XC&hc(HSTF{F(qAyIlHlK=b> z%19myfKGmxq#D;@+ba);;SqHkvMlj+{~g6BP`qHO9J|Gy5HDSS&-vU?vG+&9Y)_88 zsQZr4iwl;ji4WK6G>|9Mn1=ednBbo@y&t01*o8`wewyw@523zVEDez)pQmr(#wn+s z=Uv6t`cBn_Q|=!Rmj=+ubCxB4Mb?D!F>T|M62uihW3458cBy%-xFpqVBI(fnvKPId z{LK5RRqA??-Bw+Bk^a|z2AMrGe3;FyfGyj zuDy`?s~%&s@)sA6g9rLxevn&=V6%#Iy22}+&q01u5Gz3b)$d9S`tySCkv7~3Oc%3? zZ;fzxek!@Ehlq1sh^f(EIXt|`iew|);~%-fj77oz-)lMk-aLBqxyS3SZNJh*DayW| zCtDzEf7zZNjlyuIgf0u^)ut)a-pFM^p2rD^w4v7c_^9IBZr4Td47IBS_!1e-zauEut()XEzI@IU$`G7*xj&b-Ih}BcRyOhn{YO>YR_%`Zu1h&oGgDL2 zDm*qB1w-DRmVhF=Z?lgKu*?LIqUEy3pz9~U9aKILvS6wur`*~y+1ANWPryWPrAwy~ zD9}=K?^aRXD3Mu;JWvX!A(7_D8EbmF=awRF7@Y{14facG2x3E6CsmCC4>{Exh$7-c zd1+X1jaCug)2l{J2@u^Wf_z8#ZNlcZ-KOeEvI7?4H}VY8V{x6x!*M$ot%qmCKeq`y zq^`AQA|Gmm!9M(Bk?BUHTrr*1?&+|(jV_fzc7x$rw3Jt?+wZh&B&k^R+#WTZWHsX; zXR&nKYETGdUmOe{rks<@IS#wx*F@2~uJKs*QkE`U$%fFd-Xvf7N4s&%keqx9wRY~= zXzRI0=f-~>R6Ba3yw<;TgdYmTtG1@0{m7C&;@3Q}VTeSqF%x6lXMZB%NIi6RFL2f@ zQ<)Slx|uB{r{tV3ipUe(dFz766YUGguV1$FUp)GtdT%OaQa^{4D?h(w`Ue4d>LTAu z9gt1;lP4_0t>Oj5qb+L9?N77I8-DY&Qqex%tXa`=u?vDJn;yQg%Od=Gme4AkN5B{w zu1%KW%db~?`k;PU%NOxHFWFmGQFY=cuoTjAB!vdBVthQ;;NTLpsD@13ZK{UHA9%6s zZZMli&-o-QDty$Pik6?d4?F{ejBCD#ct!_0_IWAPHr+>W<4?dfj-uEKq(5Z%pOHvn z4LHvAJ}i?^mZ=)%((m~jUeNE!nyd02o#~b5cBGq~*GvTkichu2rD~T%Hj{T0+-PznUoJNMfWzBG#zr_DMfaEbWbBkxC*m0pv z=_A|bkn31L)a3i^H$MmlN@;FNkq(DeJbIejh_hIDizaeRX<~x@0elAxKaX~FT!2eMI)$~8<7H!UO zRMriA)ok>*OY;yaLliy4b;2DuO}{|h_y#dxD1p2tHP|cb$Hpsh(ersfZ~DXm@1CQc zec@bTu$h?&I5Z^IkL63ScTBh8ed@IRc(*DO_MUMk zsk;OJSkBK&_xGpxqhR8H3gV@HpH-jv8IkYnd=*2V^Nqb?#%lt-a(5=ILRmq}NjIhOcO0UqZftd23# zIbo7!EB#0TenZ;}0oF9p-%%Km9637g3nxwSfkH#IpePN_aZQUFMR;;%A?|sS)7MgM*%i)>-o9_8L zRzlT^1mm6pq2xY-`;+4oSh93t9#uwx-8!3m!&?Bd1_Q0SopYJ+%D|;@{dt?5bq*7+ zGQ58ywo`e(p)+KM?sV<_Exp8~240{U3H?R%0s=9595=VqJZBihpTRtD~?K z^Fwt^uLf^-SCnxCDPZ5ffJT(M>3iQxgX0lP_`jdex##uxzjg+j(0Wnev?<0_$y~m^ z^Pl}kyli(jYor$1lsZ_7lGWQL~A<*xBi|YFoLeKqF~v z{0kIle*J8@RAc%LzK7*UF2~RM8pJkNUaRl%3E!?Ot`nq{T9^lXFcm>z!W}LDPyMY(mo`0W;i2yMEdnk$9VTx2A4Tgjy^BT8$NaH zE1K^7%BQl~Ora|UIJc9Z>467r8x7cVXi^l(Mv-qxINGQbciggS)i#tp1vL|*wWLZlq8N{<^<#2 zwowtMKmG77>9X!~0f=F$v3~mhu0;K0`^(gDP99FYbm^GyLuwxmojm%h7Vd zk-7x;H~{&or8ewpr(O@{63X$oPbbaSfj*b#BvPM} zbV*z`>eg>w9nQkW=70E@NPW_6-)UK>rm4q19j$~UriuT)(UX&Xx;FWnRvFR0c z!-wVk&A*ctsV^nn0^UoHKEiQA$2;V>)-%S;OG;@32ptB`A(&(6HlSYisg^AM=kT;& z4`=S|58fs9>A7^PnzvlhunLDOv9EFHo4o-pujOa3))Vo@i@3t$SvpLsWwyFMe7Aat z!)PEW{d>Y{3D2BrYcB{=u#8MT@WrpFk(D=A*f*`x?g7rnd|jK2Av>`M7$o4M^+@Q6 zhE;N_*DnX213w~mA6%^D#obus*z2+y(b*1zvey5@(|d=r`M&?d_14mA(N=5Js;Z)t zqNtS=MQx>OOIuV?V$TqYs=Y_8*lMQsmfCyNtUY7T5G0b^zkI&O@jQPg*PR^Kaoy*2 zj@O73IctwA2cMWDnm#mBQ>)f#W}^&mPsy zCZWg$$7J9Rr?f$U#IxC}WOt-B1o3F!S#jmUa>k`j)1Fo^nS{8EfgL^v%1dq_NL8js zEB1nkk6hL>AshlK2z+3>-orYLrguV@-Law{fs^$K29S{5im)Wt68_t8AKGobT4 zrxU%xT7iGxc;~N%OKz=ScHF4gNoMAgO*mO62nUgR`@PQeToV@+M+eOAJ0CfusY4Hb zs*xFt5PsHPBl_?q#f6%Wi5I#6MIZBF}Pr@aL5bDdJ@Na%ta(DU0i)|MVB%)%#A<5kXrrYkkn# zxns#=pK|gE{9ZXsBWL&ctnwW4*YoSEqsR42RXS8I^2zen=5fSmqT!;_ZJ+-K>hYVc#X_W$C8Sh>H)!ExGES>FW5&Wo#9@E+i2c1sK4uPbfK27c+v%ir z({oA7HEz%|FbTg{dEj523GK}=FrO_l9Kp?UVGin|1wNULVYPRSXtH`Tv%oDXGKPki65jcz{3e0VDC~5)gaE+15aEpUC6-cOe^XknR3L}a(UmcFA%)7uE@{aVB$yU zs_TNWZ-LsUom4kO z=hG{6DEnsIU;lkn18*m1Y7x2l1*k0m7Q7BzdQYI)B0I~AY-fLUyb)F5Wx1`;OF3#s zjXQCXx9`H1DSt36mfm?l=QaArj^M8H>9Tacj$hNsF+kW+<_ZtrbKo9zb+R+#^EX{G znutI?TsHMf?Do6gA5%v7)%^PZTMd@~;`!2Mb-MYBP%7SIi3G7e{W9Y+XK!`7^>M1_ z*L~8*+q0_f%jqtcJI)VM5hVe)@AXiy_u#2%D}@f9i@$a_dOPg2>6~FTzqze`I#MCc zI%>87c_@i+H$}@2b?ZsS?@MOHNdKzKL&aNo$e0S%uT3hbi2M2Wy8ZRA>fkTqR7*@) zbZyv^v=NpfTGNO8vRgeISDCt{utv4K$RR7X-k%#)ESsR_w&X#Qv`ovk?p-qf4~zjp zEQqi|3%SpA9WSO>xZZ#HU~WNIavb=d51o;R!g(O+3dZV@9DdpC5O)O2Z=e{KpWhrH z?%566tRvf_z#{keFbX!erCmcjj2$0hJHGyMGQCzhlfLAb>qYr9>UlKlm6tnEX;|iq zUmb1_h#zkU&HJSkHuNj{^`^4XF{}NCD1<~y?qefht4>Ov23IOUrYNIUayq^lXZj5y);)Jt&!iC%JGSXrn>2p@#2m=hVjM@pXp7 zeUQP3*CFLlREX!`+h~@l`6V)HN$mZj^>8VLN>{_y|2n<&;Agih?>KF^c%1_ZMA>VX z@Nin*{`$AMB}!ib6y-?33n$GA)U-Tc`5<)V^=9cX+jpd=k23 z?eCDw%}v>dFY+W_8f?Ig;VvIgdIwkkc89b&6|=bCh zxTpL(q1m-o^)NPFGys!)uVuf<|}F_ zRIm(W;W{;aJ{^=bwUS4XBwnBCxp%>!*IlT#7ogxDmWIDVYrq}!dGzZJMamR2p<)wXNE06@{P49Ca$((+zBlUDM06i8#r#8-cN39fIn`|L0dsQ-`z+XaKEtgYJN z^ABYrSwFm_qekGYaac6d0LL7ZtK;%~X!TH-p%zF;cU$DYhMIRzYmN}>9IU@_2Q0;# z`3T%>lLRAzwd3mS<&>1mBuw$xLt*0Lgo*b#dD1$yV7%yd+J#Bm1s+pFmAL+>qklzb zJQE(h;>L9L=&-{(w{5Omn{>q0upA*ga2C(oH!4r5(Wnh%S59^RfR~fK$pUY-nP|!B zA0YBGD-VK>z{~zofn`6Qc9s^?!0%;8*I*_n*{fL|e*IH&g*Jq4^oxJvO`nhbp2gG_ z(PIVQpK9=YqM*1foPPW1V&|ch&CW=xlFvilkJUE6Z2=v6*R289YrpDk$PD;XvO*h| z@6VPiv>ufe6KuLk?a(c}93>9v%dD2@Wq6{w`?O|*VZ6q*$-6D^LFxT4owV|x45fBV<8 z#{83AuBhcv@IqUfv4u_*-Z5FX)=y$**Vvph2S#G-v`{g(oabwj`(_aHTHUwkqcbO7 zLt7y8AJX>M#hCo`4_YoYW*`#r;vp;9`qJ~so7hD$U%r~-J3W0%pJAG$HDcPpYzch= z2;axs{Jm6Kt=8-C(rPwBeff zpidK$w)dF%QkE_H;g61yfiH247RaSfK9hLHiXiCE*vZM>PW##AUDVYdqWj?>7h_X; zX;YdT;$6B&sb?%TYVhYCuZ3|mISN^MF2-}Y$TR!2%%aLLooscf)5dj~7?!UcXL{7w zK(Jfotb)hU5bHQolImY;1wq&@n@2CU!TiV~Ok!t!pZ9b38s=YbhB9R}`-Z0Kig`7A zV{)Z`uw0hBS^eo&(a5-DrR1?)xf)Jk)i5wcWn(;roEGO*Oeebo=TDy<7>RAp(Z)!> zEzPi608GmyO`S^#yKOd(N+6eQvC8uTyt?14_D8SpZZBU+%m6-^_#4z}NKN#Nad>C0>^pP1?FADtc=}(o{`bf*7PQ*iBs^F z(vs65?aheF0dL~mBLh_Pbn+Q#x^szr7wYr$4F9{%8}P1Tgjnd^rqsQf<2>6jD@Em- z6i-)S(UZZ-)@^^=dhz}++FEGy{P+3>FW)bLhPnW%)OT!4_O@qx3_q4Vj9%TU6V({B6&(bWPOUqitsRPgcqX-)GqB#!L?7qUK2jfzYv;5&6mU0? z@-~jT40CA}IA7XF%u8Qx2&^m4-IBbSDcn7;T`RR}*x;61^vFcZNi)$)B@w_?yC%X#S*5W6ztKrm0S4P{u_bt z(>oe$#xQxcPe`rmU;gDiT2@XTb-MVEhVei1mt38ik(p0vW880u#77qs$y0L*%Vx!$QrN8VP66*+u@ZXC0iUzOe;Porlm_96kk)obt3L?X!8>u$F|eR@?#G;57qhgO|J*k??J1^+cB|*c5?F}S<&vDX0bOzEof6}2 zDQk^dA%7=deUU(Z97A^$D<>c z+hQZ#una02RhPFdRPCiodT8uO^vxrt04vTdo1|2av%nss+Z8Z~qp-vC#)KnXhaARsYdj zvXjUSaHL&pUcgh+EBg}W1Jgrb<>NMG=O}T@rEx$0r4^6pKKgaM42q9);emE1vqAMu zdj^naB3mo5{wp9~8p{cbfKNyqPSOj)_4cfFR)m6UC_BF}f&cX%yB-n`w|O~Yy=GO` zx}jf=(IJU)e$gI%SRm{@@WjvZ1G*G-Xw(V}dTz6LonftCgDq?+p&X7S>dd`7{$6A~ zqP$EU#@pP}(+-)UPfA15IUksLcrX0$97fDR3(3Z@t$CH21d&Y}zgaYD0hVg9EJcnu zJWIgfNyPN$fz!gwxQ4)u8zd@Iw)vYk)-*b$#gZ`-D{e1pj%#u^eB@}vJ8zt&iL@m8c#5R4!KO977PpioI^*5sS_Y9VNtJsB5)c+9eJ7l%SofX#d8cW>P8bDV5ifgxh{TIC zkxD*qBQady!E7t_@zK9{3nch_8ro0aDnt8-J!&WlUIYIoM)&?fe&W$)<`JSwqthxL(v@ zm8fSlPLjda)AybNO(uT6yAgjB%*qihX&a-pSDy-gty|UJh184~79ncNOHgTomv!~W zn`ah6CS7{N_CBO1e#o*-u|t$Bph;|jB)tY)G$|G@PO8v})&&D|bW(qk?m!K1$Uhz~ zp?0?$+u7(-dBgE%_U&RZCf}m=Ve-ximc2&)8a3{^y!Np>N%(+VHEo=2TT`T6SPaD` zd=;?fxFH9%=^9udW1Wu*FIK1OGXC!?3&47P`YtN#Vsbhd(=-Q79`W5YmDGem}=@qB_HyYD2|9wDOb=R5ftBPdt>0?wZLJ7_+avrG|;cukq=1wgG zF(Q?lphu{N2V&Fx6J?hJlYnpi*RrKL8r7*3&xxci@TLuphN_?s^Ssy*p(Q)Z7y8{L%S{ZoQZy)fv; zn=EvopY8`A7y%Tb-h_gkN^nZ!$%ctl_ln8Q*2!Nx&^#{Xh|@@r#XguB9&3?0Q{=+^Brc#&}up>Muiu?!h zCQGGEEU_+nTPPjC;q}2pVZqPP^VMESsi2fM$KNWhtu`D>mb(d#DsiM4NNT_v>$Z{? z{0kk{F%wE8_mkfn&3>9rtal_K32%Pj?s<@=dgc-7V(0W9Q^{voJrRv80-SdNXOuIo z^T~-@J9cf9M|WO0B})&~WA=nYjQHfc+XI!Q)5meNx%LXT_1FXi%{zUkZ2s(0MbgH# zpkV22xhRnJ+JS&20(jSO~V!)Pm1dDF)l2c<=!~0H6#GR6XE1zDqhN7A#Aj#>Iv%Blq?My7hmjOmG zmhbp>9XY-pI-@e695|}YemPeR^z2{CHGDjT+7j@Qnp5%RculA=AHA7-7K|`hlJ1|_ zS6~p9>r_K`K#j8$`KR>t4jz6`-h_=Eiy3b_nE4gLKfPydDihsyK7XV^cW`(yE;0Qp?1<*ohj7OtQmT> z)1PDKkJND#A=kdY@U|yNmo&U}_$IUQ`rMA)>U53-Ru~VFnLm@EP0T$4#BWq z7pkZjCH>YlJ*gHHb-lvD?y%3kf=D0p1PX>-SYwJkyP($LI-`Se--Z>mieyM6k z^)agiU)7wF=KXw&j(RC@NU*itqcLHKtfMb_?~X0( zf?emQDqHZr;W)po{(GgIsGgA#emqP459iSnWBkc|D^h_hZsi6V8PapyCY< z<9E2#D+vy2ojC3uRbeb16F0yMx2g)-ZL(p}&yz=Rb3@5xlj(n*6n^JlCFdFm!nBNJa21+=29sV~ zLrDP&ZR(aXQkS#Q{qjxGRn*E6;m=!6W=QLLXcNxNhbPw4zyk!ogUgnf<1ara?*}(e zZpx3{F!`Hv2UVuWzH_uQw^v@Fm2hZR3fXx$MZX7nk7rEOvVR_OM@W~L>l2n=k?_QO z`G%e5k9)VZPu_(TR!8YgliJ?l3-UcJvZ(x-4mF&)9j^>LdZ_@v?s7&*Z#0?Bwf{Hz z(J!W8(;lr^40~h2rS;E>%%&GGBhEbAQ$4%7&GS!l2ka>}<0jlvYQ8yac9s#2mXpbN z%&pe?o!pw%3S|&|-%TfNv0nv0H94=;sX?%}wc(EPgbv)<1pR zJQ<1i6&YPYf}bs3U92pA$%H(RIo!m-axg#TlhgqhtD(@`Hh8`Q^ zJ}v392`)>LP~_42b2YfS&t%OK>K&cb)BGl}z~ei7cS;lEW1aCO_r>Oy97T)r?6s&} z=&%{5*Kla%qr*v--m`KkRZ>qrz$fg99l|ugagtsgO~_|Kt^7#8D8J0JPq^+vKkZX$ zgiKPtKvdk?6ms|9VtK%sd)Qxh-|phw0{YfgQ(q!h@zhO_HOGGS`iI5-27wc&N3*{y zNCcQ1Y!= zXmkRV9_HuOV^i|_Ax`yl50LJ0pHUxuc!BVStRbHL4OAor>s=6An)Fu5N>cZyh0z0c z=S%7(zK*6kYe}DtX?pdh148CJF4Uv{7cPd7$wOj&f5djt#(QM^Mm=-iiQoeJtVnQK zRFyi+tMVqTm>NxNQ&OLX_Ptim?0Y+vC_9&SZ-9H}>(3V;q8pRlfHm@ z%`tM^`1YSMMzf)B;V@YYzOrqEzMnk*(8TrrO4mypMm}Z}6<08Vo+tCs;<+D=0EWlw zV-k2*4fbY}$2Q%pWtD|4(x&?gS`cw|AguGH^6LZaq&m8}(C>`|n#S7O2TNoxf*p&2 zOMR7Mu@-OfG{%?=e8dR6bL}5K3ba;w<8rpI;1}5SYU2=-3iBaBua&(FLQx%iBtI6M zL=_jpa8>SgnX-1Vm?je+;h-4{W!rNaoGR^tU%pD44EQ_d8qxS_zv<$sMpzV@dD&#$ zO-jrwzgTYDgT_me_wNh8@^SZz9f@Ed-D6Dl@D=}%wiNYu6A={tT z(NFP-))-U7-uj?B6Kzi{^efb-OLEa8+pSMFTRb&_m2MEg4-vj=C&j$^nMdvo1exT% zcx{<(`&?nQJFFA(8H~aQX0|05glHBu8hCS@HM>}jmaImj0=C^nWdO}>K4_1S)nzM%_U$!1; zOC4GV0@C^^Ac4GpANRi1W@wku1L6`_jN5-yKQkYHL5jz@xRVUZW#ps4XID=kymw zkhUEPrt*A38#zwTK?VFpG=K%kW!_HX;RgB7b4dQk*jcNHVm^shntl-pB&sL(Zu4gF z4J2I}Y6b7Jfiz`unun0NJs)D6JBF={1&gmtPpWy1_<^=;tf<{91N==11aP5@19#dJ5y z>h3C{I6W!@NWi9BbHq2AaX^J`oG2eOyeAz-?%_mn$>+_+Q3qCj-jG`N9DR^S-1)Wc zZVnI9VGJC#H*ylA4c-(aDz&}`s{BQR{F+J0z5&T;m^D;}R%)ee6t zw}|Vh&dS@q0a(iuEI^xh0vL;fKY?RH;8c9o566G*=l$@aC&Bze)z!<^z^4>TMev42 zsS)wt?f$gVvWI_!&`>{Fa3A|fg*I#&gb=Z|B+RsZ}^?t<> z7ftB96rg1y6sK2x-&N3;6Ds!zmR%Dz7pQ}wzX|%y_@KCD#v4QXy(oKnrA`&0re?TE zdlm*e?Qp|!pwuyr1%j-&iwFO*UPw_aLXN;DcctTYpT*?{LZtypA`0?kz z6Zmn0j3k+Rf4GL&`5jfb+}iL-_8=9|^m0JW8vBVh+t(YC8saU9cX@StW34X(ax?j% zgxzHQEI5e=Gk;UKE={kzuK!r=*Q(mjxRJXl=f)L}2hOr@)Ve8pA2YH9Y@kbc7U z9-L9XoA^Wt@xGZZu411QDZZ9(5)`-=p(2_6ppcE%P38 zV(mER#ZEotz>Em60uTOUiU4SCjA-r|R6r&h*rLU=E*|o}@hv-+PSz?)oVqfBlU^3J zS~|nOo4w3J95=Q;4s-H7NKZQ1$~CCk{fA(^cm>>X2@@8SyX1580f@5`0>7G zac5q5nzFS8bbj9ipq4_q;Et9mDCtB1Mne|r*Cs3-Jkho8!J_-K(vubRU-~@api@ng zlVSeuNRy{1d}^}6D#SOMFqzM)M=*IZd7F1o_f>tI{r!bMcLg}OeA=$*g5lX@qpyYj5vmbaW> zx^Jd(YxYW7!fPgc3zqWSyp~GrPAiK46=+Uu?mO^aY=qPXrponkp8WKSFGl4LEgv?W zbyT7Y-~-=t$mX5ca}0W;K{e?aTTE{9_Q>Qtbh!XY+1s?Cc2_d8xn5Twh(y8N8)7nxr(v4poPPY;F=jb#ZTT02J- z?eJCqnUayDVY*LByP=#LH)Vgp69)g_LMHVCLO7s#QL1th?Ccl&WVBv(uG@W~`|mmI z5@nTxV3(`)#ZS6x!z$?Ho{OqkuNTejyKolEDl9pd3z$fg6V?by750~11cPG@LD@%g z@8#&z5uwM?f&n3<=w|)enO9+=JWI;e_V;5HovsRS%!j#T@+2SGa4(?t@|k+m0Bp~b zJia;bp8xJUwX{~>?WO*4^VFp`Po3M+)~^*#YGn@DZ8;wbEOk$zG_-Ou>h|NZKOXjg zctMv9lhe5MTnxEy@%Do8z478O{ET-2b~3%LXs}?>3eDOH~iX8Vit7EnLKmB|8 zw}vu)DgZBEoa1Kq!OSjuKWiT}D&M4@Svk73UoJyHIxaWI-&Gk4{K;v$YzWU^wjc?x z(SgX<;rujAk%U2SUao)iSxZCbYfGtvaFZrhy+W?f!}Wze(7`x*qJX~{;8+s}e$v%< zTPdYDh|6qz<$~lqCbyZ+bqsZW1~2oA)%IC^xJTMcHpA;RoJNuwFgGqxYZ486JiL8q zT-uk-0UlxBq|+yCX2{)v?;an%P}?VgU3D*i8NaQ zkB6k$?*Dat8m&O@#T4xhB zE``clED4-`huZ7Yq*dveYNXJN&B)%I2MV?)OnQ?^93yzPgn`Y;DPUuQrWHa$4HVPc4FduiR6t_!mhMc*WQUYMpWm< zX3cBi@Rz`QE%2oR?lj}FhRSVCpqjmUeYb@gz1TzykIUES)8lAu&51;oFAzV2&-nR$ z*_Q~(a+?y-Qk%{3QPg*ayF53krJEL06fQoB@YV#k41`wo@scm5hY4;xBkqE#9I}4T zLn5B($vSCE@Us|vqzRa%4j1wqwd5_D$+gdm;ieCZcJIH}Jd?_YjWJ&ObZaWU){dT> zmEPg}K?xJwxa+oPq08E%pLGN6FPO+n{Og8kMfqhI{5yN^H6p;lJ>orag0?^qL^wED%$o&%RLdw-{F zWtmz?``R%m;<5^wm;PW7RJQ^6&0Iu!3|!Opihw;wov5v!10)1GB9VQHLCc?Z&5pX; zVO4KNFB)jSvm5A}7E<}~R%V9_$et~6(-Y0cf;iP{Ev<5rLa6D7ju_>fhF4Ft>{Cnj zFChFphQ_c0Nqu)Uw2~{|Ln6Wo3k{Xe_LF2l7f)E`TYu`aZ@V6&xEF6|*K%*P1&WeX zY2g#AFJBDbCCTXgX#@SxKjQy9z$|V)_l8S_91S{U`90H0Ubm&ea(I@UnscJJQ@Kx& zqf=q+S()5t8Q*#$V!wI7(rY{Vt>3%l!ACo#*Rtok9ofn882;f?qMe4!Xk5s>up;dV zuLz50zb2zEJ1d%pe5p7|Jp?g2=JPxk9#ZVHkg9*PWt9F&utJf|GO~0A(56(ArZr1hS4LOzNc>3ObA6UsY~=+R=#QIq$}_V;D$Igi&o zzi`Y6gg}9-gDyO4l8&g_JxD`i3(Zn|z`G4#-u;Qp3p?Rhb@PO-^~;?O79upc#Gw_u zrc?98VM(iDzj$OI{4!Hm_+osgAhpLb{8ZjRqZVnwqC4!>7d^>4OZeC6#Vco8{{lKy z8eQ| z1|H?mUT^hu&m z)LU7ALeOOy=BEm&xfz&5JvKn|`hmEC3uU6=?@a#9)@7#k2|ca&8iAz#c95*CET0ed z$sfvd`-C|tvCrbnSGAoB|UKD;{IpJ9nA~B^Cb}^|Zz;n54SbeV!Vc|F? zGuJZNO#*LkKgoOsbpmQQ*%%q%HOFo`%Tf29a$~kV*vW(KBOX$TAN>+P_Y@WDO&Lxh z=gB2fxnDK@28vr)`cPI1+eusLT+_Pwtd-)SPFHL3%$?6K6Fs}H%(K3Q6FrT&N}S42CP%~DT+b_W)_3EL-_Uyo_o@dXVl?V#HEn4qopZ82WYxDpyz7IJY?G5- z0camRJnaYurE z$iiJ*nL3!O@}dS4c7F0?rUnyNg}m=I3y+s1cF$n8R7zu8v+sBvk$lKF+6BJkX4J$! zce>9`Wh%*;M0{0Vm9o+6Gs>-X`>REU$?V_hs)^~4vbg&>t>##zhTki@0FL{-29z27 zV6IR3&st4U;=jP0>J9GAMkzpO$zhJzJ>gl|@^Oamo309_QkyFJSHJ)J{8iQkaourD z5Y@eX>$VZnGxm4MeNQqTZl$`50w?0n$ZWmZe6Y=)JSrECVyxy|S2B>SnzBgmS z_A#Y)Bqb1@X(&yqt`BRlWgQU%wefBrSWiUZWSkfEM&i+DR|W$Zf%Bw{lmX-EHwZc$ z>UE~qUwE&_uut69$BBM46xL{(dGQ2?y@`4V_BfYWF9S~R+cl~|)x5mQn{i0ENs{Ae zwX&f}t7@T-q~B|y>5+eED@^SOL{R&s767^i|4NJT65a$aTvIQu?&_|Nocx%Zr-e@X zT#-b{Y(oWW%M0q+-d7P|A}rRJ{>?5Htc~Mu$3TeyO$*kVv+r^V`C-a%SC*<5_z?0& zib<31>+Ev_I6{T1ZY}p-22oC{qX~Q?E z(I`}_xA*mZgXT-!hHu|2M#1>Ya5UWfh`Hsjt`x1E&YM1ZHjnSkt2TJ&Q)Eu}0Minn zihJZ?yd^wfedzu3>l*_sTlT>|4Xz9YN2YrZQ)BId{i+5gU|8t}9k4~BO6yh^hO9Y2 z>ot!jr7|DQcBQVrmg1beI{xUofovmNDUvr|@>Rl`yqItIoAr2`XX0xa@U5`1uqze3 z!LR!aT+D{N^rwsI#m!LtAr}=vn)hHQlmZ%5Q(HX_uXy@{c*UjIQ>x31Mg!x>rmnwy ztNHcf)g5xu{7gZ8AAZUdoREAQNY`3PA?y0sc5N)`bqA0FkCuuoW4RBg zqwk(zofp65$4Akn4hb!x>^xV8p9i&$8=P zA>lANSarXh$Cu1#i5TYx$bwGDrjNPJc%G5{(YNbT!hp3GP}79gMx1L%5v%p=oJYLS z*6V9axCB}3gt+mDTh?8#K>=j!KzY5fA8;)L+jQ!J{^sGQ2l5#3K_71yOcd|QRU1HJ0Qq5YrfE?bb=GT!u-oeu4+Kp>#ZPs|rRJ?8Y zb})yP2Bi3R>o2=fpLtF@BI`_*q8~vRny|7!D!=**Qu!f(t%?oqmgd=qTAbEnZ#;VB z+I)LKNH~6JybI28{>GPsZ6IoKdIM!_5?ZHub3@_&vPJ6?HI%AvGq)sgn)ZTF$D)q= zp=zhMu}Tn2sc>hZpWO(wx{1xg%bUhNK-%Xmg8R~jgTybit~I`WDG^c8n0I9@M<^0u z--ns~fKmN3kv-hRV&PWwjokIp76D=76rNq=fd~ck;Stcxc+>Uon*f8{{H<|3fI1|gOSD;Jy}Sk z0R)$HR3)(=9i5fIec~epDKDin1<(=~*2NKI1OT0WESZwjJI@uV) zj6fD2-DDJuI~kc$e`<2w?d;WCN6|mhd-2vy-@m_CxLu|dw!nyE^i|^2`!memE&21) zcy(bmC+U+swUUkD+yd%5*)=uwK^moNg|K4seA(|?fN%RfoljrxN7BxX5hT{cH{g30 zoD;~h{rP59_q|1~%QL#Z-zzi1l$1BED*Q5R|Ax6$e#!V1$+t~7s40Xk~)BUYa)PD zkP0?5=dJJTF=>dtciN#*NsD!Xuh5>5luIZ zB)7AL@(_BvZ2Z};_5{kd%lYKf8mhI7tiB4Nf7ydH_Ue_)|8V@R)m%#6ZK1}>4%95` zNi58V;vN`z9GuuYMCAbz)co+ufbEL{J(02#xabC$FpcHf;4Cs`KYu~GWAcWw{MnQi z%W|-I(!@@G4-^7lg&D}`NT-3G|`zcSRiw$1wKCO@OdOWzgjw{nQ zpG4jS&IlnmV;?a zzuRtx*tm1a&g2zbxgQZP(d03$J&-PB*!((5P6-V2EY6M@%a_{U`4=To{E8jtg%4TD z)&2{P$b`YnpOi(@Ff-nk{UHkd5c+i#%9QSz>fwNg){gNpWMe~((;N&lXb-;LQkqju zk;b~Q!zQ)$%*I<81Kibme(e^+BU)XGZk(*1bkC^Y3zBrY@`gcYLYzkYMe&KlK+EX7 z?sM6;M_qrqJFX;s+q_1>uVh;t6H-1KT1vA~VVeO9@m8I~BHhH6vtCSo`Z_gvb?$JK zag4kXOP{iTd{kHdZXjLs0ahDXoC9sT7^;{m5W~jR?m~9E4WlnS3?BjXmw1nJ>qTxE zKTk!v4tGe~KqPgapz)}1Pq``A`3}U$=%5Kt97AaUYUKOgP#^c;_%P))!9e7#lB!CS zornz^Ku@}ewn!qb1$!{KHMNha=~9~rj8Tk#GLW<}pnJ@5?^;lg3^^>-bFzodiNiZ-U{kVA$OeS**BCiQhglV}{crA=tj= zw&1#yM30EiWSG{Of~1u?Xu8N@Ena>FMY^-~d>Ae@?ebHlK0l_U&GGj!L~rchwzowX z6DrbA{AQCmbDe9s!G~Krzv?bBpS_6r*!V~wK&OA=Gkj+?)UxhuBSjPUxoMwCY=RA0 zMqo}{=)MVzN2+Ui4#GPDOEla`_q(i+3yarG#XU_cL=?v2#b7OA=`=CAKZ&)(z_lGL zyFiBZU>(J%)b`W6JMpw)ERP%~To}824=m)+Eh(J8X1|rikWV z*W?;(;#|P!{jV%>M)s_5_6Ia80(;`A?{p{JSur!!pXz7zoyL-T0`K-Gk=JoPcXPZn z`7Y_w0oX=edHnlPR=Qn}aL<1-FRn!5>ii$bka&oO6u$*%&SwW-X)*kmJLY=3Th8v7 z`%Uk&WJKw(F~9Eqi~8gFm$t4eI^qkzS+Du|BMEJq{B=QPH?$PE1q`G0JE zWmH_-vTlM)uwV&J2=49@2*I7;?oMzTYa~E$f(3`*8r=aXS&E-ZKWvK zN4DJD*M(*R; zZ2|Ny_UsgSq8`(LEIfjBRTixM!=$gX5GS<~`u*{!FktyUOSfmj;>iGWhFI~k9hZ{) z1bE}rdbO~2ns}lrzuZ_JfGrb`-mw_{h`_qX=5&9Y$B+f zL&b#HNg@i{JeB}N&OGcKhHhHDyaOo_^3k;L@6?h7oLw?$TVvoP+*MbY{MA@1^ry(SHOpI zHZx6reH_{X*{s8&qvog6)Es%D8&#&~BE;5a>kbb?`Hpd+(5aR@rHbAo6{^B4zDgAOT61Jwio}EDnOvL_brLMibkhbe6JciVKyJH(* zPi8@z{u3{9A?r{;K|9UbW(S<~C>l@K!fVHL-D#TVMb_WV<5`%!{MGFTqXLy-$dUm} z$E^ib!FtmH1g&@pk9H*LUi4#34<#K7?j3Bm>`pDM)AE8Psk23@erm#oR;BuNSICb) zd_??d+sm-%_^!#h!+q25RB3N@(e0g{4jAcoG7)p%mV&LCr}o+x_OD-h zJA%~qe1?>4AcXAU{FQ*ouqu98T)c-YP#SIYpd(o}lL6LX=93>2vFM|QZ*fCbI%V|2 z!B`F~j1Le#L8OfyIAZ;T85232+4X_g(dx;fX?vP?xR}+u@cMvt)I{G#lW)N;>PB=c zi#@>8xQtj*GVOk1K&ajnpPX?m8>w1nZ}uU^Bv9}4YgFKA=T05ci+llixraC&fFpA~ zPkgiC&kUq5Z=>VsOWY#4Ks&E?z1YtJ+Quu=u+u>-01_dzY4lK$C$&y`6XdKoPdAu^W#7{zoN@GJ{X16jN${=I0+mR*5G2q7lG z&rGMCrfDy-B(0>u3ZJ$sB%3y|k@p*Sc=6-IrxRfc-3t=S=a^vAJ7TpYWg86ox;e#W zpFo#A6#{ki`q&HM6D}6JII-87+E}Q{jPA9XA~7-|bwLU6T)V`#jng*X6hp+4b;C8| zd8v=eDfU&QxGq!)_K^!*tj{qI`+b;j16v~bZ*Hj|g=|m^pX{^a_;R}MP+T48Ev(@* zOVO()67z9Jw$r&UC)nK-?O7c^{S{e1i#dRC%?Tsf^PigO)Q7F?viYX5XZhVpV6vC zpO0>Vj*-}~2^?3h3l)#!B2p$K4V7vxD)7{GS;aM|Nqo~T9%=C_A9=)e6v++Z`3j8= z+FUTly{e$>Nw&t&{QB0EOgiq4DV^pr=6#RQ9?2KEs=97HnOWz0&n50Y=~byrNpEIw z9~nTExZRWXvkQjRFB}E%tb`5ytNdExAOfLX>8TZ_I-Xk$C`LhXCUkj+o7M*N6#kBb z@J+5lxK@KeKQkMH=Wq<->T?xQc~k@$Jz;F%vP{U?ZyDsH&+|uVgFeyd1jQT%vj+a9WE`_KcFRALW%@OZ`4|Av!b zhRl)v8QYpbqyNo$74aqTD<})Mt2O$W{O*yr)i&*UC&)2&b9Ih&n_0V3ELyugvPhBM z{_Ysq!n4NUVfgTCxp`vy#e(m2>lxvLZmDS`9@jin3%Y&paWMciq&InpoU*iJr{1xzi?!Qy) zqLq&bfB%(-nI^snq#Piiyv^=WB%!wrPb@t~#^p7{U)I9&qgcdRS@)O;uGG@lk@g#vHgbt|B?E@&Y=AEcbTL5dTFopVQ5FP#EL!i+L z-CgcP#3qL>Ilo!1uzGVnSJ=;F351eYO{eNfS3G$1I{2keKm!;mqvJt5I)N z-a+a{Of449u&J?!rX*;t#M3qPw&QoO?ip1e$K(R7=MtcB zTZMs0`vyqMHM2hRfXGw+QvGnh7W8coAnqN66R+OSh!rGitZEqWR&ZMfI*A`Vdso7| z9bJ3brjrJDs;eNG<}#M*V3_9XG*G;RFu~1EFsdNXLYfv=d;101E{fedUZP*Zc$dSx zoF**)jG+*}NbWol1xnhI48;?r&vfz6X{N4M-VERdZ*^3U37I^n5uc&V;~Stzga{Sk z-%?7N54EVTRg+sgNWW@+d}Htdea5OjL`<|olQXM7&tO_M!e#Z@SajL}w6rk2A9rVb zmEuo>URi5uU~G-qa6?DD(7ewP;Ab=w~}3U5~p@BM11esf;8-QX_*5;jX3J2jf$7K zKgT7wii!PpkBlZo2i?CzTMw+Y#x{h4+#+q?Fr{61u_J@XJA6F$JKFVcSFR+)%n*6M zDr(iK?oHf%vjBh3Mt}6WvGrx#J!96jFAv4QGJwh)>iQ*=`GQ zTG!Te_)Ae5gUyZ%^m<`f-y~&fBtse+%hzW;TuV1Tg1iIytrdIzg;}Iq&d(TtSLUV< z#UaOi7wX^k0w>(5pHAk)OKN=jlodqo+`O#?>4t^ceww)*_Q!pu(e0dZSH?Rih*npT zYrilwrP|f(N9VQV!(xZld8VLV#sEI*>0;tFeOHw}AF`o_6u!Tv|20ksIFP0P`o6?? za+H3L(5j_dXF3UD6FO(^&1I=ZdR=!A`fmBF`*ja=Hp^G6-1R%G3ePD}?y}q2w{dTs zmUV25gytKdUzmP_0xO(j?VfiNGYwYJb+RLFnO>Vgmgyh+PTU3*nd(|8^`Z*xkzxe? z*P$Z{7if0HV?Ul--fIAH(32@3P$9_tN!gOwdge`qWc#7g5@B169(Z6d4d7r z%kRjKvy6+-+kP&fH1Ib;czaN2>*`uZs*pxK#d)J$%3a+DIa~K+IQomoZPkTCVV#?j zg;6P~uAW)^S_;M$hb8M?>TTbN36O_6QYVxvZpwNbT!d^T?8EaNK>DXtwb?p4J1T^* zi&tG&BbPT@ZL>uPF^eC6BGCU97}70t4vKb#XL98{ckM`lw+d=i zIiSQM)nRC{Qu!Qu8@9ZlkeM6n9v_NP7Z|-MN;4$e@X&x*iL|k@^5J>Gc@ujZ$b<*G z=9lH%7nl||#ziDU{r#&l3iR#8wmw;af6*7TdM7rAYbA&-%hDHx2X~&x&&VzONIQdR z`o*S#nQmi{u_~SGVAxFs*%cX#_pqe7H$g>Q{;2gHhs!GMsI9h?_N2Rc(mu4H!biP+ z?>h zyJTF-deS&$*%nDR_CG6C+*|rk-Ky{@rAoGL2Kj7_z55nx=^d|%gqNSMFVS`M=5=!@ zSApIVvXBX~eIz9)K~(*z`R06HQBXSVwkDJ7i3kGEkafqt*AXs*n^i;H3rse z_N5{z-w^8FbL%S?iv+OJd6r*X#=#eT(6?dtF2i)6a#SU(Bm0beeCD>jl!|c>f8y_O zVJ%dn{b1J-mMVE#7YdD*`zs2{f9;G)kN>c45B~j_0Sg=J$VsYdt$b-;Vfm!z$>8equ9mCTfoG{_DG2${4Qs~ljoQ9(7qL@dft5JrP1{~ldG=$~ z>c=QYI;(W_2LQeq%tf)Hwf88Sd}%jU8_O5ybBNnQof~k62QswzkKhGA?M%&6)AieJL5ZVWfoz8%j(i z{h4FVUu_K{sGxvG|djt38{sS?aSh(9dqTe{gsi{8YsqlUgc&e&>* ze261QFFONn?O_m%C)tdAE}MB@+cHG|mOaX9%XL|wXLyo`LB^RIkjPCclZAnj={s2H z?xFz@i0V2$ZW$7J@rjsC2vGC(Nkk`yCL0`aS%%AFV1-mX9(jl$$^IL+&(B3@);b@@ zPG{$H%pA2Z4f1<%OUX**9xY&}Nk&5FWey3Z-i!Atv+vi6hCqDh zO*YoyEbYhIWrDj+V#O0XpM8@w_hKtUqAWC*lee7xduf3HyBmHx#{(n}>rpYGKKj*L!S+4teT1yUghZ|xpF8wVV_CN&d*I z82*T@WbZS1MOf5g4J(PuT#-jtW{%RAyH_dS>1x1cce1bP4Sj{S6CMorSVqlFE`9om zQfc)hMu7K~UikFKs`Z3{FB+l!CAJ#h``xJY3i^LXD}^ym+E)=n7~k}^`XgcgeDRbF z6lQx#mGBLy)Lm;p(e^3@L20MMBAP{U#v1(FWu4S{UkWOFwW9P3yWcj6<$Gep!;$B# z{CeQc{v!T}xu==>?JLaGpAx^xXVcUi6GGdrO<>a1bo4_^Lu%Cp7%jB`h%AkzHCes)h{ zKjcd4M4EkdzTdBiZf_kYhWUacJs~9$mJpjdGu*XZNRu*G-kwuc<;-B|#Bih$^PKMuVa6Adn6Ouc;h)bX2=;$d^;lpSpxSf2L<@}97D80n>* zY8r9Ti(d!+O)|6Z^zU%ZSK=+2#M?Cnyr!+$pid1BaZx|ov_Bz(t*JVzOs2$J*b)5- z@Y4DIibt8~)`IKKVKW6neoJx)pBp|V6{_Lrz$SKbDv$B%+GGSah}(dLoZ1XaeTsYTbFf?ij)fZvSj7(qZm~Dh zCjmP1n-^RC0m}+Z_EfvU?~8Bb^;h?~+UMOs19m7-45WdHmnUC;j5Tc~{;rFkqsB<* zSAIJ9XhwA9NSElzFIz-ovZHXFy}e%d!uufDy|6wWKt3~GUW-|~Dtay1W@|eqwfb4m z(V4O;Suf8<4hMN-N?ev+b@6wO;R0M~&vNX1sQm9CLbpleD1&aGQ9u*5u+LQ%1r|rx z<@(#vGu{FwC$5*$-|==n$gRCbI(@^YvP~H(U>8YGcSM|z$KF0^ZJMomTr^<^tIxkp z)}wSpj~EUjGE}Y+gn_r>x@l*OW-j=wfHvhDT4_&zav;NRru=W0!Q&?l^C>L&Y{aka zEn|l8L7N?>gjeL3UJD^F$$J{Kr%X)ZkF-4b8SqsKvEPAQn&Fc;(Be@;&WibV}1 zV+W1T6H``pU3U}IAcM=`1pvA^G?quTt8?@!{g4)5?8|)}#x>n3{%77%j*K zX7yJtyJ55?+CM%XeN{u2ZD(N*y`$lW2vRKFl=?`&Nh4J~iL2NXnfs{{LHZMBMiyP^ zP}r`GIV39P@Ikv;4*om^vLf!5TsQQ)IC=#&R;KzTB{;X#lCDHCiaLY)unv*Z`a-1X z71}ypSR|C(_v~#(;qwlnmYHux?-B)2B6_Hxpcjj8g+yVV|A->@i#S<-p5a1$2k+bF zyfVD&_dU_pl3-?G#+0}4Y^C*cN5HP_W4&$%gYSsO$fIRjUU&H27b6HRLUJVQJx8? zF&z*3IT^^}3u#Z!6Slqwt`O>7T;xojghJH!U*DAL8(HI9-=Ay23eU0_3k_y6V;(I1 ziEO?OTP<-2Z?C)bgF>veJv-A-@=}?S)UxR) zM*;^Pr!vZ@?ek`)J6EiygD$GyxS4DmvHB}yGw#|hM1M}3`%wfJQ-z#f{#tGs19CIE zq`?>5f06U5ymTxPuxnmVmMw5SIG=OplB{LJBbttUeOHoFcw~5_o9;6kYX4Qov>_ij zlSu$B>Fk%ib)Q-IQfS`5Z=go;T(L*87>VrIA8x#KFu54@;JBm2Nk#?nwI4o?KvLcT z%J$wqcp8~~JbPReG@B3>t$fwn?{HA2Lwyxd80EbE$`tAG@%o=2^a^z}XuVPOF^Qbo zHuHMrQ>&}fuM|gxOD$+AH~FHgZU-gqv8b|i{!1-W4+~Q0tmls0c=OJ?Jzddsch%ZG zbKQq|_0pgzFP!N{mAqk3C(KoY;i=SZYv?R;^$vjQYxDD-hOOWVmv(Y7BH@QxoF@g} z+OM;@HKeiNHU`x?{cI){n_iqRGRzJPZj~mlSJbAKF{{p81?&-*2sLx(iG&Q5WDgtombWHdLTWF0;0b(}^cmmDeimHRNdp z4jHfW)1aN`=iX;{3}%GT%e;Fwi>c8{s=8+9LIr3UE?CXGS2XrYwOc9^?=^Lnht^WB z;Nkm}paWI~s*D2)p0!2oJ&t^9n0~&0`Zk$3xV!z5;AK9$Tst>i39s^2b-n>{&|!30(Q@U2`YsXV<8 z78y))K(RrE;v*Pw%=^5c_0LDF3{bl0duGHAk%8;If1L%#Y0h}j->sWy){2|ON>WF; zWq3LC0zaAIHvtq*2}Mg2#U{n{0SMKEey(5ftb58_5&e+Ga>Qytlu2yKS7W#7Uq-CW zaCg3Vm}#}P0uGDFzq`{MjKfvBYv$*HCQ-QX!3s%wnjpmvf{OWdiO0z9>`cy4c%K;} zniY9Vn(IqBTG)bB8By#m2sY|#e}oPb_`)hSJafvLUNEyQoN(`^SbB3O#V8pTu|B1a z)_BeA-_|YWy8qy{fj#SbKY^|Zb83M2@uyo2-yK8wAz)3m?;i!YC09%fo{?}0n8wKC z#==n z*Z#`tRgc}2bjl|Y*7pKdvI4}go6Q>FYCn(nfz=v-_$Ib~5#4DqS>3IMHOzcEM+v`b z{A)L?5D!5U7Hvv2JP!i~U)k|cZi--JXU)>k|&wPV3L8# zeUjA$acq0gS#mu!Uvgs`I8>8Fe|oZYtFgAciRR z7=YXMeY;Yl8Q?6hgRz%Bq`CFk964G|c6n}1XlRb&p&$Vx6G?VV|7Pjur(l;Ne#0Xs zUuvbJ+|TvNs5h%-%HHA~GA+x*5Lybx*pVii?iu`vB|hjk^UhrI`*l^!$j8)|;ei{b z8E2rtWW18jpd4jkZtoVXrC+ei8YkSLM*^jT#i)EP@d%N+v4!&CHTd8*i+0RWl0b|l zE%UTn3&S2@(2?|L2bR!4!#Ci_%P3FP;2opwj>rd^YJErfC2?b_9=dafmS&ljA+B0L z6f0VvA$@9!1zNS|@JIYW614Ps5d&kHU-){H3LO@127K}wPfeaWc>S@3Mmc;|GZwGb zh4_+n-QR_4+*SLy#{0#qYSKRG8LA=#Uvn>!!wzdN0$YAl^aCr1ackX~;BdPtuIP0H z;n+g-r3vXGjyCZ_Or6a_jN`mvVGBa%PN3nst^ix?oh4(9!;LNCs29u!4is-v!q`^GKYD65Dw6Ra^ugu?2GdQL_B4pFjO>#IM%1ZEoG4(^?9oO8QD; zkr5dS;yIK?#Y60MOJC&iwvuvb%)Hlp@v07=m_Y^^4JWC+HTJF(qx9DwI?d=`C~N}z zIrl_^zceGq1<1%#wH6va+m=!Fe}77XawDr|>pIhC#f>^k1NM~5BhHL`H~BN_j8Tqv z;*ej{eNqX&RLhW2=D`rtC#fnH)z;3>KMH(x8Z0^9ViwT~2-NY~q0$n4gYdf=^G9G? zLWs0>>g`Y4`wA554gQ7g45*EPY{EWS%(tEn3!l2)E52<}C23+hSPG6oDV%wUt%Fb6 zMq?+)Tf%81>}RG?nuO3)h)eN!Ge^(sx*)-HIb>6sM3vKx4L(WWQAxXtjkr}c8~|^c z08DRkXcz%rRwT(DI6;G6J->*ruH7KKI%g-obTsa+^M6R5o|(r(sokNS{(-D%UJlHW zeFvL&h$zod(C=;ty(58hDAnedDuly(}roV7$n?Tat3i9Yk}*!Hkos)Ex%0a zkuF2B?Vcyob`yTIOL!p@{C@csqY762O#Wc*G)Y6hb-%yw(Afpb*UPFMp7(VeJ}@TS z;v1r_`&6ooIV;~m(88*^jM$53!{fdI{bQX=oOYt0l`i85YLjltaFl40lDH_A z--&kJI6@`TPLM_qHrBX4ykjI_NvM0oo6=+GyuAO+xHCScYYx4rY-)F$`P6g_>PUz; z^!Yw|eA#180_8HO+DqhJ#>sueqK<}EK|F*Ymv6*uaMbLD0zt*beU+4bZ-}$|2V@0m z1A&9JMAw{@j{DX|IyQ9`Jht8f_@hFs7~+X%fKpZzvSlZ-p@=fIJal+#Vq1_2dN25cn#l1_Ou^+&WUZeFH^Umh8xXNAmw!F6WaxV( zSS!GG;(l&bK56*#*$BlmEhQW&>sDk@~lEe3w3ZEY&&~D8`5|g=zn+X#>Qta=?Li`OzVcWoT z=pz3QR!^fB59Ewa3d+tdo$2{*4k7<+>vQ4m!OkJEp#xlSl)O|S4Cd#dp z&P;28VZC+&Pq8yAY&aC^_bYM#tD1z)XtBckV;L3gi*%jolB(CkrufbD4XXdx;SCsWXF#KXeAp-dY`>9 z8wOxEY{eWyP0C!3C&}{0dYm}A8U(-S+s~5KPT)BK)95z&lvqZiNpNon<$s6x97-fH zKJpB>0DS_2h3sqCt76&(kR0uTp^eI=fi+>8qaBD$FH{q1UBf0Ugm`wTAo${fYEFE* z_4*gw4}-wBq(rknd^#j=9ou;&_$C)@Sp^4Dc{Jq=m10)YyW`dPoIWkNA8Y12F4^$! zr2yX?i;7DahizvChh%lojiiTduP+i7x(m4+faVneZey3|%7ydECQ-CKy~(xoG3F!3 zdTuT(lQC|wRwN81Uq)t}eo+r>tG%r(qL~(dRGPqPM&* z_0%+Vf%_+wY0Y9PIybrFsdhc>OC3A5mO!JQN74**QZ|n^o(i+=7abx{k(jy2F%`w_ zC6ngw{+;3Mugv<%p^+OJLNKcVS`pvMaZxoB9QMwqM%5`j4Dv}7*vV)no7e7muWMV8 zZ=e7mY9JyP(2+HNH8wiS9L$R6Y;>5lkP}V)d{-R((D2R3PLPgY@#r#8cp#$6Aj0uT zxw!rm2l`fYV5PpD(j66=lW$TwGzc&Zb8(>Kx&x83615$}zJws`0quxPS&h|(-ESVN zNw~DVy1qGFc+?nP)ve!q_PZd~Jc^iH7*tVu#t$wOfUt#2PxYrGMek5S!zTGI!fqx^ zU%mV)eh@jJhz~5Xp$`^X5LVR;F~tooZ~&XK>_|2l^RX^U8*|><363Gnnb?P!tj7ij zxz{dA8s{u}_T`P|HJ&~74+Ux4*+vem9&`Wq9Vs!rfD@n)16{Nj9sN z=~m@WA)bam?zAq_)D9dUUE7N*ma>|pZQfc6&fVv)CQ$Xgp z4fWC1xe10wuXe#oGFzypT_P7KEY3W|loX#EB%DYzDc-bRvU|Js*jUvIkO$77=z!M3 znS%xCiV$!Y5<)d*A5esrUhWQw(&HuYBnU0Ha^1}Q!kq1O>#?A#5?IiPRspT)pm``O zqtZkV(_@uyd~k}}_4zy1DET6hT?=G&H66*?uZ?W8)ojd}pp{|EI6w!hO!|8Xsa@C_ z7uh&aX4#9pt9P+8r^2S6nC+)+|Dx~j5I88GKpP_xg;MMr1X zWaLAZ?d8P#ZrI?R3A1^P>Ley<>-Pz29WTEh#pQ%}hBHJ@XC(3PO{8fF{E9w(IG=Ua zvE@`avW9n;Bkqz&nteK?(=wbEy}oqnvi`}T3{GzUD7Q-Uo55jD{X-N3Jtlr%_8 z@YIUc6-s4t1BKA_{rOkj1=@n-Pbg-|0fAEYT~{i86R@wY+$Qk9I%uXUu8_6xm6jK_ zp?X`skzc}-eXcuggBXwLrM(YYEMdYAtRk%O3jW>;Wi5xiP2Gbg zFG^(hxaJz};hE=B$}vc7hV;?N=?eUjfSEea=I?c z+meio56up`@hgc~k|BFG6C5WQ2oihDQz%Z!CY~luRylp6o!s z*kv>mSs?~kDmkwx@BK@~cy=g&V~;85+f5#YJ$VeuG+Le_Y`8_O^LF#=t`P>~Z*8?4 zhte(%-d5Wi2fYJ%&3|w@>-{xZ96DJK0^OOZk=os~}@zH&|Dg z6Qz0=LRDS{GM@OvHnf(l8!v8hB5$tHT3&qctD6bN6ljdXpeoY}b~XrL=9)_#K?8}T zV<@D_XUvmtoHykLhRe#5S}6*}_!i`@aJ|~n+7$f&w}F6a*F692fmY0|L|LKqUMldw z7Lm>Ib_f-|hP}dpwW3N@l351a-(%&})1L>5%7pjYwc(IeDMFu(i0QdJa@!^~+-9N9 z@5=^d(NqwFhf?P=qcU_1l-`3v`s1fPXyrP%CQ+Gxqv(HPQVvfgjyO+{dhSIT<;qi#| zTMy})|Bf1&@iJud+1%Lru>8xrjdy_hCgRoNOT)in|6Yde7d~%iB5`4Jb&&fys+I8X zI_Dxr)GtGI=Bd9U6-#~T6|GG7e(1Q0J|7m&dTi0LBdPI8VIqA~G}oud2zQ*soM?`y zNagJ@lyT49lYyoEiwY`qYzmIJVV#Fwl0b)