From 71dc69aafbb1ae3ff6271f6103b0bec43ea4cbb6 Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Fri, 11 Jul 2025 14:41:50 +0200 Subject: [PATCH 1/7] infinity value sanitized , input pattern validation and ISO fallback --- .../stages/InfinitySupport.scala | 134 +++++++++-- .../InfinitySupportIsoTest.scala | 213 ++++++++++++++++++ 2 files changed, 334 insertions(+), 13 deletions(-) create mode 100644 src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala diff --git a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala index 7efd7aa..99298fd 100644 --- a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala +++ b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala @@ -16,9 +16,18 @@ package za.co.absa.standardization.stages -import org.apache.spark.sql.Column -import org.apache.spark.sql.functions.{lit, when} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.{Column, SparkSession, Row, functions => F} +import org.apache.spark.sql.types.{DataType, DateType, StringType, TimestampType, StructType, StructField} +import za.co.absa.standardization.types.{TypeDefaults, TypedStructField} +import za.co.absa.standardization.types.parsers.DateTimeParser +import za.co.absa.standardization.time.DateTimePattern +import za.co.absa.standardization.types.TypedStructField.DateTimeTypeStructField +import java.sql.Timestamp +import scala.collection.JavaConverters._ +import java.text.SimpleDateFormat +import java.util.Date + + trait InfinitySupport { protected def infMinusSymbol: Option[String] @@ -26,19 +35,118 @@ trait InfinitySupport { protected def infPlusSymbol: Option[String] protected def infPlusValue: Option[String] protected val origType: DataType + protected def defaults: TypeDefaults + protected def field: TypedStructField + protected def spark: SparkSession - def replaceInfinitySymbols(column: Column): Column = { - val columnWithNegativeInf: Column = infMinusSymbol.flatMap { minusSymbol => - infMinusValue.map { minusValue => - when(column === lit(minusSymbol).cast(origType), lit(minusValue).cast(origType)).otherwise(column) + + private def sanitizeInput(s: String): String = { + if (s.matches("[a-zA-Z0-9:.-]+")) s + else { + throw new IllegalArgumentException(s"Invalid input '$s': must be alphanumeric , colon, dot or hyphen") + } + } + + private def getPattern(dataType: DataType): Option[String] = { + dataType match { + case DateType | TimestampType => + field match { + case dateField: DateTimeTypeStructField[_] => + dateField.pattern.toOption.flatten.map(_.pattern) + case _ => None + } + case _ => None + } + } + + private def validateAndConvertInfinityValue(value: String, dataType: DataType, patternOpt: Option[String]): String = { + val sanitizedValue = sanitizeInput(value) + val schema = StructType(Seq(StructField("value", StringType, nullable = false))) + val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row(sanitizedValue))), schema) + + val parsedWithPattern = patternOpt.flatMap { pattern => + val parsedCol = dataType match { + case TimestampType => F.to_timestamp(F.col("value"), pattern) + case DateType => F.to_date(F.col("value"), pattern) + case _ => F.col("value").cast(dataType) } - }.getOrElse(column) + val result = df.select(parsedCol.alias("parsed")).first().get(0) + if (result != null) Some(sanitizedValue) else None + } - infPlusSymbol.flatMap { plusSymbol => - infPlusValue.map { plusValue => - when(columnWithNegativeInf === lit(plusSymbol).cast(origType), lit(plusValue).cast(origType)) - .otherwise(columnWithNegativeInf) + if (parsedWithPattern.isDefined) { + parsedWithPattern.get + } else { + val isoPattern = dataType match { + case TimestampType => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS" + case DateType => "yyyy-MM-dd" + case _ => "" + } + val parsedWithISO = dataType match { + case TimestampType => df.select(F.to_timestamp(F.col("value"), isoPattern)).alias("parsed").first().getAs[Timestamp](0) + case DateType => df.select(F.to_date(F.col("value"), isoPattern)).alias("parsed").first().getAs[Date](0) + case _ => null + } + if (parsedWithISO != null) { + patternOpt.getOrElse(isoPattern) match { + case pattern => + val dateFormat = new SimpleDateFormat(pattern) + dateFormat.format(parsedWithISO) + } + } else{ + throw new IllegalArgumentException(s"Invalid infinity value: '$value' for type: $dataType with pattern ${patternOpt.getOrElse("none")} and ISO fallback ($isoPattern") } - }.getOrElse(columnWithNegativeInf) + } + } + + protected val validatedInfMinusValue: Option[String] = if (origType == DateType || origType == TimestampType) { + infMinusValue.map { v => + validateAndConvertInfinityValue(v, origType,getPattern(origType)) + } + } else { + infMinusValue.map(sanitizeInput) + } + + protected val validatedInfPlusValue: Option[String] = if (origType == DateType || origType == TimestampType) { + infPlusValue.map { v => + validateAndConvertInfinityValue(v, origType,getPattern(origType)) + } + } else { + infPlusValue.map(sanitizeInput) + } + + def replaceInfinitySymbols(column: Column): Column = { + var resultCol = column.cast(StringType) + validatedInfMinusValue.foreach { v => + infMinusSymbol.foreach { s => + resultCol = F.when(resultCol === F.lit(s), F.lit(v)).otherwise(resultCol) + } + } + validatedInfPlusValue.foreach { v => + infPlusSymbol.foreach { s => + resultCol = F.when(resultCol === F.lit(s), F.lit(v)).otherwise(resultCol) + } + } + + origType match { + case TimestampType => + val pattern = getPattern(origType).getOrElse( + defaults.defaultTimestampTimeZone.map(_ => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS").getOrElse("yyyy-MM-dd HH:mm:ss") + ) + F.coalesce( + F.to_timestamp(resultCol,pattern), + F.to_timestamp(resultCol,"yyyy-MM-dd'T'HH:mm:ss.SSSSSS") + ).cast(origType) + case DateType => + val pattern = getPattern(origType).getOrElse( + defaults.defaultDateTimeZone.map(_ => "yyyy-MM-dd").getOrElse("yyyy-MM-dd") + ) + F.coalesce( + F.to_date(resultCol,pattern), + F.to_date(resultCol, "yyyy-MM-dd") + ).cast(origType) + case _ => + resultCol.cast(origType) + } } } diff --git a/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala b/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala new file mode 100644 index 0000000..b3a49e5 --- /dev/null +++ b/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala @@ -0,0 +1,213 @@ +package za.co.absa.standardization + + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite +import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} +import org.apache.spark.sql.types.{DataType, DateType, Metadata, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.functions.{col, lit, to_date, to_timestamp, when} +import java.sql +import java.sql.{Date, Timestamp} +import java.text.{SimpleDateFormat,ParseException} +import java.util.TimeZone +import scala.util.Try + + +class InfinitySupportIsoTest extends AnyFunSuite with BeforeAndAfterAll { + var sparkSession: SparkSession = _ + + override def beforeAll(): Unit = { + sparkSession = SparkSession.builder() + .appName("InfinityISOTest") + .master("local[*]") + .config("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "CORRECTED") + .config("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "CORRECTED") + .getOrCreate() + } + + override def afterAll(): Unit ={ + if (sparkSession != null) { + sparkSession.stop() + } + } + + private def createTestDataFrame(data: Seq[String]): DataFrame = { + sparkSession.createDataFrame( + sparkSession.sparkContext.parallelize(data.map(Row(_))), + StructType(Seq(StructField("value", StringType, nullable = false))) + ) + } + + private val configString = + """ + standardization.infinity { + minus.symbol = "-inf" + minus.value = "1970-01-01 00:00:00.000000" + plus.symbol = "inf" + plus.value ="9999-12-31 23:59:59.999999" + } + """ + private def replaceInfinitySymbols(column: Column, dataType: DataType, pattern: Option[String], timezone:String, minusSymbol: String, minusValue: String, plusSymbol:String, plusValue:String): Column ={ + def validateValue(value: String, patternOpt: Option[String], dataType: DataType) : Unit = { + val isoPattern = dataType match { + case TimestampType => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS" + case DateType => "yyyy-MM-dd" + case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType") + } + + val formatsToTry = patternOpt.toSeq ++ Seq(isoPattern) + var lastException: Option[ParseException] = None + + for (fmt <- formatsToTry) { + try { + val sdf = new SimpleDateFormat(fmt) + sdf.setTimeZone(TimeZone.getTimeZone(timezone)) + sdf.setLenient(false) + sdf.parse(value) + return + } catch { + case e: ParseException => lastException = Some(e) + } + } + + val errorMsg = s"Invalid infinity value: '$value' for type: ${dataType.toString.toLowerCase} with pattern ${patternOpt.getOrElse("none")} and ISO fallback ($isoPattern)" + throw new IllegalArgumentException(errorMsg,lastException.orNull) + } + + validateValue(minusValue,pattern,dataType) + validateValue(plusValue, pattern, dataType) + + + dataType match { + case TimestampType => + when(col(column.toString) === minusSymbol, lit(minusValue)) + .when(col(column.toString) === plusSymbol, lit(plusValue)) + .otherwise( + pattern.map(p => to_timestamp(col(column.toString),p)) + .getOrElse(to_timestamp(col(column.toString))) + ) + case DateType => + when(col(column.toString) === minusSymbol, lit(minusValue)) + .when(col(column.toString) === plusSymbol, lit(plusValue)) + .otherwise( + pattern.map( p => to_date(col(column.toString), p)) + .getOrElse(to_date(col(column.toString))) + ) + case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType") + } + } + + + test("Replace infinity symbols for timestamp with valid pattern"){ + val df = createTestDataFrame(Seq("-inf","inf", "2025-07-05 12:34:56", null)) + val result = df.withColumn("result", replaceInfinitySymbols(col("value"), TimestampType,Some("yyyy-MM-dd HH:mm:ss"), "UTC","-inf", "1970-01-01 00:00:00","inf","9999-12-31 23:59:59")) + .select("result") + .collect() + .map(_.getAs[TimestampType](0)) + + val expected = Seq( + Timestamp.valueOf("1970-01-01 00:00:00"), + Timestamp.valueOf("9999-12-31 23:59:59"), + Timestamp.valueOf("2025-07-05 12:34:56"), + null + ) + + assert(result sameElements expected) + } + + test("Convert invalid timestamp pattern to ISO"){ + val df = createTestDataFrame(Seq("-inf","inf")) + val result = df.withColumn("result", replaceInfinitySymbols( + col("value"), + TimestampType, + Some("yyyy-MM-dd HH:mm:ss"), + "UTC", + "-inf", + "1970-01-01 00:00:00", + "inf", + "9999-12-31 23:59:59")) + .select("result") + .collect() + .map(_.getAs[TimestampType] (0)) + + + val expected = Seq( + Timestamp.valueOf("1970-01-01 00:00:00"), + Timestamp.valueOf("9999-12-31 23:59:59") + ) + + assert (result sameElements expected) + } + + + test("Replace infinity symbol for date with valid pattern"){ + val df = createTestDataFrame(Seq("-inf", "inf", "20245-07-05",null)) + val result = df.withColumn("result", replaceInfinitySymbols( + col("value"), + DateType, + Some("yyyy-MM-dd"), + "UTC", + "-inf", + "1970-01-01", + "inf", + "9999-12-31" + )) + .select("result") + .collect() + .map(_.getAs[Date](0)) + + val expected = Seq( + Date.valueOf("1970-01-01"), + Date.valueOf("9999-12-31"), + Date.valueOf("2025-07-05"), + null + ) + + assert (result sameElements expected) + } + + + test("Throw error for unparseable infinity value"){ + val exception = intercept[IllegalArgumentException] { + replaceInfinitySymbols( + col("value"), + TimestampType, + Some("yyyy-MM-dd HH:mm:ss"), + "UTC", + "-inf", + "invalid_date", + "inf", + "9999-12-31 23:59:59" + ) + } + + assert(exception.getMessage.contains("Invalid infinity value: 'invalid_date' for type: timestamp")) + assert(exception.getMessage.contains("pattern yyyy-MM-dd:mm:ss")) + assert(exception.getMessage.contains("ISO fallback (yyyy-MM-dd'T'HH:mm:ss.SSSSSS")) + } + + test("Handle missing pattern with ISO fallback"){ + val df = createTestDataFrame(Seq("-inf","inf")) + val result = df.withColumn("result", replaceInfinitySymbols( + col("value"), + DateType, + None, + "UTC", + "-inf", + "1970-01-01", + "inf", + "9999-12-31" + )) + .select("result") + .collect() + .map(_.getAs[Date](0)) + + + val expected = Seq( + Date.valueOf("1970-01-01"), + Date.valueOf("9999-12-31") + ) + + assert (result sameElements expected) + } +} From c9d99c9d0c9031dda4e8f2ef994687f4f47aa706 Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Sat, 12 Jul 2025 21:34:10 +0200 Subject: [PATCH 2/7] Headers added on test file for ci license-test --- .../InfinitySupportIsoTest.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala b/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala index b3a49e5..919ab63 100644 --- a/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala +++ b/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala @@ -1,5 +1,20 @@ -package za.co.absa.standardization +/* + * Copyright 2021 ABSA Group Limited + * + * Licensed 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 za.co.absa.standardization import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite From 20e9c973c8ca555be243f8970035bfb83949f5d5 Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Mon, 14 Jul 2025 11:55:58 +0200 Subject: [PATCH 3/7] Removed abstract members and modified method replaceInfinitySymbols to handle validation --- .../stages/InfinitySupport.scala | 63 +++++++++++-------- 1 file changed, 38 insertions(+), 25 deletions(-) diff --git a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala index 99298fd..f5d26e6 100644 --- a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala +++ b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala @@ -16,11 +16,10 @@ package za.co.absa.standardization.stages -import org.apache.spark.sql.{Column, SparkSession, Row, functions => F} -import org.apache.spark.sql.types.{DataType, DateType, StringType, TimestampType, StructType, StructField} +import org.apache.spark.sql.functions.{to_timestamp,lit, when,coalesce,to_date} +import org.apache.spark.sql.{Column, Row, SparkSession} +import org.apache.spark.sql.types.{DataType, DateType, StringType, StructField, StructType, TimestampType} import za.co.absa.standardization.types.{TypeDefaults, TypedStructField} -import za.co.absa.standardization.types.parsers.DateTimeParser -import za.co.absa.standardization.time.DateTimePattern import za.co.absa.standardization.types.TypedStructField.DateTimeTypeStructField import java.sql.Timestamp import scala.collection.JavaConverters._ @@ -35,9 +34,7 @@ trait InfinitySupport { protected def infPlusSymbol: Option[String] protected def infPlusValue: Option[String] protected val origType: DataType - protected def defaults: TypeDefaults protected def field: TypedStructField - protected def spark: SparkSession private def sanitizeInput(s: String): String = { @@ -59,16 +56,16 @@ trait InfinitySupport { } } - private def validateAndConvertInfinityValue(value: String, dataType: DataType, patternOpt: Option[String]): String = { + private def validateAndConvertInfinityValue(value: String, dataType: DataType, patternOpt: Option[String], spark:SparkSession): String = { val sanitizedValue = sanitizeInput(value) val schema = StructType(Seq(StructField("value", StringType, nullable = false))) val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row(sanitizedValue))), schema) val parsedWithPattern = patternOpt.flatMap { pattern => val parsedCol = dataType match { - case TimestampType => F.to_timestamp(F.col("value"), pattern) - case DateType => F.to_date(F.col("value"), pattern) - case _ => F.col("value").cast(dataType) + case TimestampType =>to_timestamp(df.col("value"), pattern) + case DateType => to_date(df.col("value"), pattern) + case _ => df.col("value").cast(dataType) } val result = df.select(parsedCol.alias("parsed")).first().get(0) if (result != null) Some(sanitizedValue) else None @@ -83,8 +80,8 @@ trait InfinitySupport { case _ => "" } val parsedWithISO = dataType match { - case TimestampType => df.select(F.to_timestamp(F.col("value"), isoPattern)).alias("parsed").first().getAs[Timestamp](0) - case DateType => df.select(F.to_date(F.col("value"), isoPattern)).alias("parsed").first().getAs[Date](0) + case TimestampType => df.select(to_timestamp(df.col("value"), isoPattern)).alias("parsed").first().getAs[Timestamp](0) + case DateType => df.select(to_date(df.col("value"), isoPattern)).alias("parsed").first().getAs[Date](0) case _ => null } if (parsedWithISO != null) { @@ -101,7 +98,8 @@ trait InfinitySupport { protected val validatedInfMinusValue: Option[String] = if (origType == DateType || origType == TimestampType) { infMinusValue.map { v => - validateAndConvertInfinityValue(v, origType,getPattern(origType)) + //validateAndConvertInfinityValue(v, origType,getPattern(origType)) + v } } else { infMinusValue.map(sanitizeInput) @@ -109,22 +107,37 @@ trait InfinitySupport { protected val validatedInfPlusValue: Option[String] = if (origType == DateType || origType == TimestampType) { infPlusValue.map { v => - validateAndConvertInfinityValue(v, origType,getPattern(origType)) + //validateAndConvertInfinityValue(v, origType,getPattern(origType)) + v } } else { infPlusValue.map(sanitizeInput) } - def replaceInfinitySymbols(column: Column): Column = { + def replaceInfinitySymbols(column: Column, spark:SparkSession, defaults: TypeDefaults): Column = { var resultCol = column.cast(StringType) - validatedInfMinusValue.foreach { v => + + val validatedMinus = if (origType == DateType || origType == TimestampType) { + infMinusValue.map( v => validateAndConvertInfinityValue(v, origType, getPattern(origType),spark)) + } else { + infMinusValue.map(sanitizeInput) + } + + val validatedPlus = if (origType == DateType || origType == TimestampType){ + infPlusValue.map(v => validateAndConvertInfinityValue(v, origType, getPattern(origType),spark)) + } else{ + infPlusValue.map(sanitizeInput) + } + + validatedMinus.foreach { v => infMinusSymbol.foreach { s => - resultCol = F.when(resultCol === F.lit(s), F.lit(v)).otherwise(resultCol) + resultCol = when(resultCol === lit(s), lit(v)).otherwise(resultCol) } } - validatedInfPlusValue.foreach { v => + + validatedPlus.foreach { v => infPlusSymbol.foreach { s => - resultCol = F.when(resultCol === F.lit(s), F.lit(v)).otherwise(resultCol) + resultCol = when(resultCol === lit(s), lit(v)).otherwise(resultCol) } } @@ -133,17 +146,17 @@ trait InfinitySupport { val pattern = getPattern(origType).getOrElse( defaults.defaultTimestampTimeZone.map(_ => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS").getOrElse("yyyy-MM-dd HH:mm:ss") ) - F.coalesce( - F.to_timestamp(resultCol,pattern), - F.to_timestamp(resultCol,"yyyy-MM-dd'T'HH:mm:ss.SSSSSS") + coalesce( + to_timestamp(resultCol,pattern), + to_timestamp(resultCol,"yyyy-MM-dd'T'HH:mm:ss.SSSSSS") ).cast(origType) case DateType => val pattern = getPattern(origType).getOrElse( defaults.defaultDateTimeZone.map(_ => "yyyy-MM-dd").getOrElse("yyyy-MM-dd") ) - F.coalesce( - F.to_date(resultCol,pattern), - F.to_date(resultCol, "yyyy-MM-dd") + coalesce( + to_date(resultCol,pattern), + to_date(resultCol, "yyyy-MM-dd") ).cast(origType) case _ => resultCol.cast(origType) From d9af782ccfa81d9dfbdf6e135b09852c5af6354d Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Tue, 15 Jul 2025 23:11:13 +0200 Subject: [PATCH 4/7] implicit parameters used to match TyperParser's expected single parameter input column --- .../stages/InfinitySupport.scala | 21 ++----------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala index f5d26e6..9d9ae93 100644 --- a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala +++ b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala @@ -91,30 +91,13 @@ trait InfinitySupport { dateFormat.format(parsedWithISO) } } else{ - throw new IllegalArgumentException(s"Invalid infinity value: '$value' for type: $dataType with pattern ${patternOpt.getOrElse("none")} and ISO fallback ($isoPattern") + throw new IllegalArgumentException(s"Invalid infinity value: '$value' for type: $dataType with pattern ${patternOpt.getOrElse("none")} and ISO fallback ($isoPattern)") } } } - protected val validatedInfMinusValue: Option[String] = if (origType == DateType || origType == TimestampType) { - infMinusValue.map { v => - //validateAndConvertInfinityValue(v, origType,getPattern(origType)) - v - } - } else { - infMinusValue.map(sanitizeInput) - } - - protected val validatedInfPlusValue: Option[String] = if (origType == DateType || origType == TimestampType) { - infPlusValue.map { v => - //validateAndConvertInfinityValue(v, origType,getPattern(origType)) - v - } - } else { - infPlusValue.map(sanitizeInput) - } - def replaceInfinitySymbols(column: Column, spark:SparkSession, defaults: TypeDefaults): Column = { + def replaceInfinitySymbols(column: Column)(implicit spark:SparkSession, defaults: TypeDefaults): Column = { var resultCol = column.cast(StringType) val validatedMinus = if (origType == DateType || origType == TimestampType) { From 577ec48c31bedcf727b64ef03f8398fbddb378ec Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Tue, 15 Jul 2025 23:50:04 +0200 Subject: [PATCH 5/7] TyperParser updated to include implicit in apply and parser classes --- .../za/co/absa/standardization/stages/TypeParser.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala b/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala index ab92994..76dbe6b 100644 --- a/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala +++ b/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala @@ -16,7 +16,7 @@ package za.co.absa.standardization.stages -import org.apache.spark.sql.Column +import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -138,7 +138,7 @@ object TypeParser { origSchema: StructType, stdConfig: StandardizationConfig, failOnInputNotPerSchema: Boolean = true) - (implicit defaults: TypeDefaults): ParseOutput = { + (implicit spark: SparkSession, defaults: TypeDefaults): ParseOutput = { // udfLib implicit is present for error column UDF implementation val sourceName = SchemaUtils.appendPath(path, field.sourceName) val origField = origSchema.getField(sourceName) @@ -165,7 +165,7 @@ object TypeParser { origType: DataType, failOnInputNotPerSchema: Boolean, isArrayElement: Boolean = false) - (implicit defaults: TypeDefaults): TypeParser[_] = { + (implicit spark:SparkSession, defaults: TypeDefaults): TypeParser[_] = { val parserClass: (String, Column, DataType, Boolean, Boolean) => TypeParser[_] = field.dataType match { case _: ArrayType => ArrayParser(TypedStructField.asArrayTypeStructField(field), _, _, _, _, _) case _: StructType => StructParser(TypedStructField.asStructTypeStructField(field), _, _, _, _, _) @@ -318,7 +318,7 @@ object TypeParser { } private abstract class NumericParser[N: TypeTag](override val field: NumericTypeStructField[N]) - (implicit defaults: TypeDefaults) extends ScalarParser[N] with InfinitySupport { + (implicit spark: SparkSession, defaults: TypeDefaults) extends ScalarParser[N] with InfinitySupport { override protected val infMinusSymbol: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinitySymbol) override protected val infMinusValue: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityValue) override protected val infPlusSymbol: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinitySymbol) @@ -384,7 +384,7 @@ object TypeParser { failOnInputNotPerSchema: Boolean, isArrayElement: Boolean, overflowableTypes: Set[DataType]) - (implicit defaults: TypeDefaults) extends NumericParser[N](field) { + (implicit spark:SparkSession, defaults: TypeDefaults) extends NumericParser[N](field) { override protected def assemblePrimitiveCastErrorLogic(castedCol: Column): Column = { val basicLogic: Column = super.assemblePrimitiveCastErrorLogic(castedCol) From 42e1af1937129f7baf5525a3a9e5ec2c36aca648 Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Mon, 11 Aug 2025 13:16:21 +0200 Subject: [PATCH 6/7] solution remodelled , objects extended for pattern independence, default ISO pattern used if no pattern is specified and direct casting to DateType & TimestampType --- .../standardization/schema/MetadataKeys.scala | 4 + .../stages/InfinitySupport.scala | 194 ++++++++---------- .../standardization/stages/TypeParser.scala | 16 +- .../time/DateTimePattern.scala | 45 ++-- .../standardization/time/InfinityConfig.scala | 30 +++ .../types/TypedStructField.scala | 30 ++- .../types/parsers/DateTimeParser.scala | 45 +++- 7 files changed, 216 insertions(+), 148 deletions(-) create mode 100644 src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala diff --git a/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala b/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala index 9f5f0a0..0c42fc1 100644 --- a/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala +++ b/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala @@ -42,6 +42,10 @@ object MetadataKeys { val StrictParsing = "strict_parsing" // For nonstandard data inputs like the Mainframe's century pattern val IsNonStandard = "is_non_standard" + // For allowing separate infinity patterns + val PlusInfinityPattern = "plus_infinity_pattern" + val MinusInfinityPattern = "minus_infinity_pattern" + } object MetadataValues { diff --git a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala index 9d9ae93..eaf5497 100644 --- a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala +++ b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala @@ -16,133 +16,101 @@ package za.co.absa.standardization.stages -import org.apache.spark.sql.functions.{to_timestamp,lit, when,coalesce,to_date} -import org.apache.spark.sql.{Column, Row, SparkSession} -import org.apache.spark.sql.types.{DataType, DateType, StringType, StructField, StructType, TimestampType} -import za.co.absa.standardization.types.{TypeDefaults, TypedStructField} -import za.co.absa.standardization.types.TypedStructField.DateTimeTypeStructField -import java.sql.Timestamp -import scala.collection.JavaConverters._ +import org.apache.spark.sql.functions.{lit, when} +import org.apache.spark.sql.Column +import org.apache.spark.sql.types.{DataType, DateType, TimestampType} +import za.co.absa.standardization.types.parsers.DateTimeParser +import za.co.absa.standardization.time.{DateTimePattern, InfinityConfig} + +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.util.Date +import java.util.Locale +import scala.util.Try trait InfinitySupport { protected def infMinusSymbol: Option[String] + protected def infMinusValue: Option[String] + protected def infPlusSymbol: Option[String] + protected def infPlusValue: Option[String] + protected def infMinusPattern: Option[String] + protected def infPlusPattern: Option[String] protected val origType: DataType - protected def field: TypedStructField - - - private def sanitizeInput(s: String): String = { - if (s.matches("[a-zA-Z0-9:.-]+")) s - else { - throw new IllegalArgumentException(s"Invalid input '$s': must be alphanumeric , colon, dot or hyphen") - } - } - - private def getPattern(dataType: DataType): Option[String] = { - dataType match { - case DateType | TimestampType => - field match { - case dateField: DateTimeTypeStructField[_] => - dateField.pattern.toOption.flatten.map(_.pattern) - case _ => None - } - case _ => None - } - } - - private def validateAndConvertInfinityValue(value: String, dataType: DataType, patternOpt: Option[String], spark:SparkSession): String = { - val sanitizedValue = sanitizeInput(value) - val schema = StructType(Seq(StructField("value", StringType, nullable = false))) - val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row(sanitizedValue))), schema) - - val parsedWithPattern = patternOpt.flatMap { pattern => - val parsedCol = dataType match { - case TimestampType =>to_timestamp(df.col("value"), pattern) - case DateType => to_date(df.col("value"), pattern) - case _ => df.col("value").cast(dataType) - } - val result = df.select(parsedCol.alias("parsed")).first().get(0) - if (result != null) Some(sanitizedValue) else None - } - - if (parsedWithPattern.isDefined) { - parsedWithPattern.get - } else { - val isoPattern = dataType match { - case TimestampType => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS" - case DateType => "yyyy-MM-dd" - case _ => "" - } - val parsedWithISO = dataType match { - case TimestampType => df.select(to_timestamp(df.col("value"), isoPattern)).alias("parsed").first().getAs[Timestamp](0) - case DateType => df.select(to_date(df.col("value"), isoPattern)).alias("parsed").first().getAs[Date](0) - case _ => null - } - if (parsedWithISO != null) { - patternOpt.getOrElse(isoPattern) match { - case pattern => - val dateFormat = new SimpleDateFormat(pattern) - dateFormat.format(parsedWithISO) - } - } else{ - throw new IllegalArgumentException(s"Invalid infinity value: '$value' for type: $dataType with pattern ${patternOpt.getOrElse("none")} and ISO fallback ($isoPattern)") - } - } - } - - - def replaceInfinitySymbols(column: Column)(implicit spark:SparkSession, defaults: TypeDefaults): Column = { - var resultCol = column.cast(StringType) + protected val targetType: DataType - val validatedMinus = if (origType == DateType || origType == TimestampType) { - infMinusValue.map( v => validateAndConvertInfinityValue(v, origType, getPattern(origType),spark)) - } else { - infMinusValue.map(sanitizeInput) - } - - val validatedPlus = if (origType == DateType || origType == TimestampType){ - infPlusValue.map(v => validateAndConvertInfinityValue(v, origType, getPattern(origType),spark)) - } else{ - infPlusValue.map(sanitizeInput) - } + def replaceInfinitySymbols(column: Column): Column = { + targetType match { + case DateType => + val defaultDatePattern = "yyyy-MM-dd" + val minusDate = infMinusValue.flatMap { value => + infMinusSymbol.map { symbol => + when( + column === lit(symbol).cast(origType), + lit(parseInfinityValue(value, infMinusPattern.getOrElse(defaultDatePattern)).getTime) + .cast(TimestampType) + .cast(DateType) + ) + } + }.getOrElse(column) + + infPlusValue.flatMap { value => + infPlusSymbol.map { symbol => + when( + minusDate === lit(symbol).cast(origType), + lit(parseInfinityValue(value, infPlusPattern.getOrElse(defaultDatePattern)).getTime) + .cast(TimestampType) + .cast(DateType) + ).otherwise(minusDate) + } + }.getOrElse(minusDate) - validatedMinus.foreach { v => - infMinusSymbol.foreach { s => - resultCol = when(resultCol === lit(s), lit(v)).otherwise(resultCol) - } - } + case TimestampType => + val defaultTimestampPattern = "yyyy-MM-dd HH:mm:ss" + val minusTimestamp = infMinusValue.flatMap { value => + infMinusSymbol.map { symbol => + when( + column === lit(symbol).cast(origType), + lit(parseInfinityValue(value, infMinusPattern.getOrElse(defaultTimestampPattern)).getTime) + .cast(TimestampType) + ) + } + }.getOrElse(column) + + infPlusValue.flatMap { value => + infPlusSymbol.map { symbol => + when( + minusTimestamp === lit(symbol).cast(origType), + lit(parseInfinityValue(value, infPlusPattern.getOrElse(defaultTimestampPattern)).getTime) + .cast(TimestampType) + ).otherwise(minusTimestamp) + } + }.getOrElse(minusTimestamp) - validatedPlus.foreach { v => - infPlusSymbol.foreach { s => - resultCol = when(resultCol === lit(s), lit(v)).otherwise(resultCol) + case _ => + val columnWithNegativeInf: Column = infMinusSymbol.flatMap { minusSymbol => + infMinusValue.map { minusValue => + when(column === lit(minusSymbol).cast(origType), lit(minusValue).cast(origType)).otherwise(column) + } + }.getOrElse(column) + + infPlusSymbol.flatMap { plusSymbol => + infPlusValue.map { plusValue => + when(columnWithNegativeInf === lit(plusSymbol).cast(origType), lit(plusValue).cast(origType)) + .otherwise(columnWithNegativeInf) + } + }.getOrElse(columnWithNegativeInf) } - } + } - origType match { - case TimestampType => - val pattern = getPattern(origType).getOrElse( - defaults.defaultTimestampTimeZone.map(_ => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS").getOrElse("yyyy-MM-dd HH:mm:ss") - ) - coalesce( - to_timestamp(resultCol,pattern), - to_timestamp(resultCol,"yyyy-MM-dd'T'HH:mm:ss.SSSSSS") - ).cast(origType) - case DateType => - val pattern = getPattern(origType).getOrElse( - defaults.defaultDateTimeZone.map(_ => "yyyy-MM-dd").getOrElse("yyyy-MM-dd") - ) - coalesce( - to_date(resultCol,pattern), - to_date(resultCol, "yyyy-MM-dd") - ).cast(origType) - case _ => - resultCol.cast(origType) - } + private def parseInfinityValue(value: String, pattern: String): Date = { + val dateFormat = new SimpleDateFormat(pattern, Locale.US) + dateFormat.setLenient(false) + new Date(dateFormat.parse(value).getTime) } } + + diff --git a/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala b/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala index 76dbe6b..1015632 100644 --- a/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala +++ b/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala @@ -16,7 +16,7 @@ package za.co.absa.standardization.stages -import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.Column import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -138,7 +138,7 @@ object TypeParser { origSchema: StructType, stdConfig: StandardizationConfig, failOnInputNotPerSchema: Boolean = true) - (implicit spark: SparkSession, defaults: TypeDefaults): ParseOutput = { + (implicit defaults: TypeDefaults): ParseOutput = { // udfLib implicit is present for error column UDF implementation val sourceName = SchemaUtils.appendPath(path, field.sourceName) val origField = origSchema.getField(sourceName) @@ -165,7 +165,7 @@ object TypeParser { origType: DataType, failOnInputNotPerSchema: Boolean, isArrayElement: Boolean = false) - (implicit spark:SparkSession, defaults: TypeDefaults): TypeParser[_] = { + (implicit defaults: TypeDefaults): TypeParser[_] = { val parserClass: (String, Column, DataType, Boolean, Boolean) => TypeParser[_] = field.dataType match { case _: ArrayType => ArrayParser(TypedStructField.asArrayTypeStructField(field), _, _, _, _, _) case _: StructType => StructParser(TypedStructField.asStructTypeStructField(field), _, _, _, _, _) @@ -318,11 +318,14 @@ object TypeParser { } private abstract class NumericParser[N: TypeTag](override val field: NumericTypeStructField[N]) - (implicit spark: SparkSession, defaults: TypeDefaults) extends ScalarParser[N] with InfinitySupport { + (implicit defaults: TypeDefaults) extends ScalarParser[N] with InfinitySupport { override protected val infMinusSymbol: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinitySymbol) override protected val infMinusValue: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityValue) override protected val infPlusSymbol: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinitySymbol) override protected val infPlusValue: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityValue) + override protected val infMinusPattern : Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityPattern) + override protected val infPlusPattern : Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityPattern) + override protected val targetType: DataType = field.dataType private val columnWithInfinityReplaced = replaceInfinitySymbols(column) override protected def standardizeAfterCheck(stdConfig: StandardizationConfig)(implicit logger: Logger): ParseOutput = { @@ -384,7 +387,7 @@ object TypeParser { failOnInputNotPerSchema: Boolean, isArrayElement: Boolean, overflowableTypes: Set[DataType]) - (implicit spark:SparkSession, defaults: TypeDefaults) extends NumericParser[N](field) { + (implicit defaults: TypeDefaults) extends NumericParser[N](field) { override protected def assemblePrimitiveCastErrorLogic(castedCol: Column): Column = { val basicLogic: Column = super.assemblePrimitiveCastErrorLogic(castedCol) @@ -510,6 +513,9 @@ object TypeParser { override protected val infMinusValue: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityValue) override protected val infPlusSymbol: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinitySymbol) override protected val infPlusValue: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityValue) + override protected val infMinusPattern : Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityPattern) + override protected val infPlusPattern : Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityPattern) + override protected val targetType: DataType = field.dataType private val columnWithInfinityReplaced: Column = replaceInfinitySymbols(column) protected val replaceCenturyUDF: UserDefinedFunction = udf((inputDate: String, centuryPattern: String) => { diff --git a/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala b/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala index 068f071..4899580 100644 --- a/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala +++ b/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala @@ -26,7 +26,9 @@ import za.co.absa.standardization.types.{Section, TypePattern} * @param pattern actual pattern to format the type conversion * @param isDefault marks if the pattern is actually an assigned value or taken for global defaults */ -abstract sealed class DateTimePattern(pattern: String, isDefault: Boolean = false) + +abstract sealed class DateTimePattern(pattern: String, isDefault: Boolean = false, + val infinityConfig:Option[ InfinityConfig] = None) extends TypePattern(pattern, isDefault){ val isEpoch: Boolean @@ -81,8 +83,9 @@ object DateTimePattern { // scalastyle:on magic.number private final case class EpochDTPattern(override val pattern: String, - override val isDefault: Boolean = false) - extends DateTimePattern(pattern, isDefault) { + override val isDefault: Boolean = false, + override val infinityConfig: Option[InfinityConfig] = None) + extends DateTimePattern(pattern, isDefault, infinityConfig) { override val isEpoch: Boolean = true override val isCentury: Boolean = false @@ -115,12 +118,15 @@ object DateTimePattern { case _ => Seq.empty } override val patternWithoutSecondFractions: String = EpochKeyword + + } private abstract class StandardDTPatternBase(override val pattern: String, assignedDefaultTimeZone: Option[String], - override val isDefault: Boolean = false) - extends DateTimePattern(pattern, isDefault) { + override val isDefault: Boolean = false, + override val infinityConfig: Option[InfinityConfig] = None) + extends DateTimePattern(pattern, isDefault,infinityConfig) { override val isEpoch: Boolean = false override val epochFactor: Long = 0 @@ -151,8 +157,9 @@ object DateTimePattern { private final case class StandardDTPattern(override val pattern: String, assignedDefaultTimeZone: Option[String] = None, - override val isDefault: Boolean = false) - extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault) { + override val isDefault: Boolean = false, + override val infinityConfig: Option[InfinityConfig] = None) + extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault, infinityConfig) { override val isCentury: Boolean = false override val originalPattern: Option[String] = None @@ -161,8 +168,9 @@ object DateTimePattern { private final case class CenturyDTPattern(override val pattern: String, override val originalPattern: Option[String], assignedDefaultTimeZone: Option[String] = None, - override val isDefault: Boolean = false) - extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault) { + override val isDefault: Boolean = false, + override val infinityConfig: Option[InfinityConfig] = None) + extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault, infinityConfig) { override val isCentury: Boolean = true } @@ -170,26 +178,29 @@ object DateTimePattern { private def create(pattern: String, assignedDefaultTimeZone: Option[String], isCenturyPattern: Boolean, - isDefault: Boolean): DateTimePattern = { + isDefault: Boolean, + infinityConfig: Option[InfinityConfig]): DateTimePattern = { if (isEpoch(pattern)) { - EpochDTPattern(pattern, isDefault) + EpochDTPattern(pattern, isDefault,infinityConfig) } else if (isCenturyPattern && isCentury(pattern)) { val patternWithoutCentury = pattern.replaceAll(patternCenturyChar, "yy") - CenturyDTPattern(patternWithoutCentury, Some(pattern), assignedDefaultTimeZone, isDefault) + CenturyDTPattern(patternWithoutCentury, Some(pattern), assignedDefaultTimeZone, isDefault, infinityConfig) } else { - StandardDTPattern(pattern, assignedDefaultTimeZone, isDefault) + StandardDTPattern(pattern, assignedDefaultTimeZone, isDefault, infinityConfig) } } def apply(pattern: String, assignedDefaultTimeZone: Option[String] = None, - isCenturyPattern: Boolean = false): DateTimePattern = { - create(pattern, assignedDefaultTimeZone, isCenturyPattern, isDefault = false) + isCenturyPattern: Boolean = false, + infinityConfig: Option[InfinityConfig] = None): DateTimePattern = { + create(pattern, assignedDefaultTimeZone, isCenturyPattern, isDefault = false , infinityConfig) } def asDefault(pattern: String, - assignedDefaultTimeZone: Option[String] = None): DateTimePattern = { - create(pattern, assignedDefaultTimeZone, isCenturyPattern = false, isDefault = true) + assignedDefaultTimeZone: Option[String] = None, + infinityConfig: Option[InfinityConfig] = None): DateTimePattern = { + create(pattern, assignedDefaultTimeZone, isCenturyPattern = false, isDefault = true, infinityConfig) } def isEpoch(pattern: String): Boolean = { diff --git a/src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala b/src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala new file mode 100644 index 0000000..e3351f4 --- /dev/null +++ b/src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala @@ -0,0 +1,30 @@ +/* + * Copyright 2021 ABSA Group Limited + * + * Licensed 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 za.co.absa.standardization.time + +/** + * Standalone class for infinityConfig to be used to define configuration of infinity values + */ + +case class InfinityConfig ( + plusInfinityPattern: Option[String], + minusInfinityPattern: Option[String], + plusInfinityValue: Option[String], + minusInfinityValue: Option[String], + plusInfinitySymbol: Option[String], + minusInfinitySymbol: Option[String] +) diff --git a/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala b/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala index dee3d58..e1647dc 100644 --- a/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala +++ b/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala @@ -21,7 +21,7 @@ import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetada import za.co.absa.standardization.ValidationIssue import za.co.absa.standardization.numeric.{DecimalSymbols, NumericPattern, Radix} import za.co.absa.standardization.schema.{MetadataKeys, MetadataValues} -import za.co.absa.standardization.time.DateTimePattern +import za.co.absa.standardization.time.{DateTimePattern, InfinityConfig} import za.co.absa.standardization.typeClasses.{DoubleLike, LongLike} import za.co.absa.standardization.types.parsers._ import za.co.absa.standardization.validation.field._ @@ -393,6 +393,28 @@ object TypedStructField { (implicit defaults: TypeDefaults) extends TypedStructFieldTagged[T](structField) { + val infinityConfig: Option[InfinityConfig] = { + val plusPattern = structField.metadata.getOptString(MetadataKeys.PlusInfinityPattern) + val minusPattern = structField.metadata.getOptString(MetadataKeys.MinusInfinityPattern) + val plusValue = structField.metadata.getOptString(MetadataKeys.PlusInfinityValue) + val minusValue = structField.metadata.getOptString(MetadataKeys.MinusInfinityValue) + val plusSymbol = structField.metadata.getOptString(MetadataKeys.PlusInfinitySymbol) + val minusSymbol = structField.metadata.getOptString(MetadataKeys.MinusInfinitySymbol) + + if (Seq(plusPattern,minusPattern,plusValue,minusValue,plusSymbol,minusSymbol).exists(_.isDefined)){ + Some(InfinityConfig ( + plusInfinityPattern = plusPattern, + minusInfinityPattern = minusPattern, + plusInfinityValue = plusValue, + minusInfinityValue = minusValue, + plusInfinitySymbol = plusSymbol, + minusInfinitySymbol= minusSymbol + )) + } else { + None + } + } + override def pattern: Try[Option[DateTimePattern]] = { parser.map(x => Some(x.pattern)) } @@ -400,7 +422,7 @@ object TypedStructField { lazy val parser: Try[DateTimeParser] = { val patternToUse = readDateTimePattern Try{ - DateTimeParser(patternToUse) + DateTimeParser(patternToUse, infinityConfig) } } @@ -416,9 +438,9 @@ object TypedStructField { structField.metadata.getOptString(MetadataKeys.Pattern).map { pattern => val timeZoneOpt = structField.metadata.getOptString(MetadataKeys.DefaultTimeZone) val isCenturyPattern = structField.metadata.getOptStringAsBoolean(MetadataKeys.IsNonStandard).getOrElse(false) - DateTimePattern(pattern, timeZoneOpt, isCenturyPattern) + DateTimePattern(pattern, timeZoneOpt, isCenturyPattern, infinityConfig) }.getOrElse( - DateTimePattern.asDefault(defaults.getStringPattern(structField.dataType), None) + DateTimePattern.asDefault(defaults.getStringPattern(structField.dataType), None, infinityConfig) ) } } diff --git a/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala b/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala index 37a86f3..76500a7 100644 --- a/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala +++ b/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala @@ -19,17 +19,17 @@ package za.co.absa.standardization.types.parsers import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale - -import za.co.absa.standardization.time.DateTimePattern +import za.co.absa.standardization.time.{DateTimePattern, InfinityConfig} import za.co.absa.standardization.types.Section import za.co.absa.standardization.types.parsers.DateTimeParser.{MillisecondsInSecond, NanosecondsInMicrosecond, NanosecondsInMillisecond, SecondsPerDay} - +import scala.util.Try /** * Enables to parse string to date and timestamp based on the provided format * Unlike SimpleDateFormat it also supports keywords to format epoch related values * @param pattern the formatting string, in case it's an epoch format the values wil need to be convertible to Long */ -case class DateTimeParser(pattern: DateTimePattern) { +case class DateTimeParser(pattern: DateTimePattern, + infinityConfig: Option[InfinityConfig] = None) { private val formatter: Option[SimpleDateFormat] = if (pattern.isEpoch) { None } else { @@ -39,15 +39,34 @@ case class DateTimeParser(pattern: DateTimePattern) { Some(sdf) } + private val defaultDatePattern = "yyyy-MM-dd" + private val defaultTimestampPattern = "yyyy-MM-dd HH:mm:ss" + def parseDate(dateValue: String): Date = { - val seconds = extractSeconds(dateValue) - new Date((seconds - (seconds % SecondsPerDay)) * MillisecondsInSecond) + infinityConfig match { + case Some(config) if config.plusInfinitySymbol.contains(dateValue) => + parseInfinityValue(config.plusInfinityValue,config.plusInfinityPattern.getOrElse(defaultDatePattern),Long.MaxValue) + case Some(config) if config.minusInfinitySymbol.contains(dateValue) => + parseInfinityValue(config.minusInfinityValue,config.minusInfinityPattern.getOrElse(defaultDatePattern), Long.MinValue) + case _ => + val seconds = extractSeconds(dateValue) + new Date((seconds - (seconds % SecondsPerDay)) * MillisecondsInSecond) + } } def parseTimestamp(timestampValue: String): Timestamp = { - val seconds = extractSeconds(timestampValue) - val nanoseconds = extractNanoseconds(timestampValue) - makePreciseTimestamp(seconds, nanoseconds) + infinityConfig match { + case Some(config) if config.plusInfinitySymbol.contains(timestampValue) => + val date = parseInfinityValue(config.plusInfinityValue, config.minusInfinityPattern.getOrElse(defaultTimestampPattern), Long.MaxValue) + new Timestamp(date.getTime) + case Some(config) if config.minusInfinitySymbol.contains(timestampValue) => + val date = parseInfinityValue(config.minusInfinityValue,config.minusInfinityPattern.getOrElse(defaultTimestampPattern),Long.MinValue) + new Timestamp(date.getTime) + case _ => + val seconds = extractSeconds(timestampValue) + val nanoseconds = extractNanoseconds(timestampValue) + makePreciseTimestamp(seconds, nanoseconds) + } } def format(time: java.util.Date): String = { @@ -112,6 +131,14 @@ case class DateTimeParser(pattern: DateTimePattern) { pattern.nanosecondsPosition.foreach(result += _.extractFrom(value).toInt) result } + + private def parseInfinityValue(value: Option[String], pattern: String, default: Long): Date = { + value.map { v => + val dateFormatter = new SimpleDateFormat(pattern, Locale.US) + dateFormatter.setLenient(false) + new Date(dateFormatter.parse(v).getTime) + }.getOrElse(new Date(default)) + } } object DateTimeParser { From 31722ed8d6b98f26f8fd58fff456d2c7079049ab Mon Sep 17 00:00:00 2001 From: MatloaItumeleng Date: Wed, 20 Aug 2025 12:53:04 +0200 Subject: [PATCH 7/7] Algorithm logic followed , canParseInfValue used to decide (A) or (B) with methods isocast & origcast , and NumericParser compability retained --- .../InfinitySupport-ISO-fallback.drawio | 89 +++++++ .../InfinitySupport-ISO-fallback.png | Bin 0 -> 428992 bytes .../standardization/schema/MetadataKeys.scala | 4 - .../stages/InfinitySupport.scala | 97 +------- .../standardization/stages/TypeParser.scala | 66 ++++- .../time/DateTimePattern.scala | 45 ++-- .../standardization/time/InfinityConfig.scala | 30 --- .../types/TypedStructField.scala | 30 +-- .../types/parsers/DateTimeParser.scala | 45 +--- .../InfinitySupportIsoTest.scala | 228 ------------------ 10 files changed, 189 insertions(+), 445 deletions(-) create mode 100644 src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.drawio create mode 100644 src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.png delete mode 100644 src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala delete mode 100644 src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala diff --git a/src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.drawio b/src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.drawio new file mode 100644 index 0000000..9a60a03 --- /dev/null +++ b/src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.drawio @@ -0,0 +1,89 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.png b/src/main/scala/za/co/absa/standardization/adr/001-infinity-support-iso-pattern-defaults/InfinitySupport-ISO-fallback.png new file mode 100644 index 0000000000000000000000000000000000000000..866dfe8255a04e2875647d6852639892b21de2ee GIT binary patch literal 428992 zcmeD^1z1#B+c1KH7?@y!ih?52Wxyx`7J`ci1|5PTjR+Dm>zYWdEj6yIC@NsAh*Ab& z5EdAO(u#tBNW=f0JNHg8DEM#q?e{&;uFRadan4)k?X%xtweIkt6NmEf@C;wRZ0S!t zJi`RxeF(+il(49h{x$%TQ38Cyy>g zKe)i`-SJ`ih`Y6=qb+?zo<3sb>}F%@Mjyg%Q`DX*w-|0l|0!zW5P>gFmM|L(QU>r? z8T@t!8~T`vyrQDAoT`P@~U!vm5bZ{1P-^=-@-F4F|wxPX`+U0{Dd< z&d!b=4lX!Y@Ih;5CnsBL9MAZerJEZD{)VB=wr zUn?)if}mx#4x8=qSF0)GKUi5>Z{6(X?CFHR&&k;dNCDw=n^2P z0)z4}xW(4P!yBh5OHU7H7_#@+=7|3a>v8ioLHEy7hF`F+>+#27UzQ>=Mh~ZdLnMs8 z(dUXL2m|ni3xx0?cV|yGYy2v%3&*eXuyorD;tG2j*XtD2(M!LMcY;KK7i~f8z}kQU z+-w~!Jsh^P&(spUHJO9TRM8;Q(W%k;t2v)ZmFe* zr49&k;29N10L>B`hwbojGx|sjSrG$L&TbBy4Xy2Mw^@SRrC$$IW&F-E#`YMrY!;dW zK6F9Tu=aL@eZkgE5#Hrx@8Dr;=wgYX(F=qejIq#=6=HB(n^jf>Q8C$G-_rvggPw;- z(VZ^Y1bi1c#U=4gpvzr-DQ%L5kyu&loc_{8qO zfW4A>Z`do!5pZS&z5)YrKMwR2m3o2rZ;;RQcYamQauq5Bt$b7Z5*+eP@vDgKPv60e zKe+fM0Kmdkgs8;iB#UrVarH!+2a>(aPn^=2%k@loC}ZvH=nQHq(o8+vES=m5X$Q}g zWBJVyDN`~wmTp_6+%{WD0pB3qn)n0q9eqW=C@3q_zp#FoJ^QPZv~vbqKtaxqF3*{Z zS?jP3v`jDqY`x&$pPaW@Ix&BQ5vFiqS$-dYfQ7-$Ty8-xm!WS22nh0OE+DMAK$I!j z<`L+C-HiTN_8kykPVd?E_f>`-mQDa_8?-wyH%|Iv*;?80If9YOKJq_w^_C0_2w84Js-csW zJNl#L?Bwp@=4tJLWD0#BT8-(la=>mdg!EM%2U|xQ#OeTMOv4>8W_x3(_XsW_1}^Xj zOPr8*umutyxe7PdpZfrco2{h>CS@IPC11fgpObB_~I7*f8u?@i(j&OW13X)5+1+j=r~}hke5?`HIT* z9SRa|tq@KVTwj)#@2xE>6UqvkvP=yAK)Nz!s4aisyo^I)lddEf0w&39JV+Aq)8b>D&67=s^j9_HU#I zf+7Awn+t7)-0TcF><2r`Z4QngXz39G6}sMnn=;(ZAht_@fMApOK@@bRr37=*v9$JZ zMqW>RO}H+`qF($3mdbw*tO+=BHA(&g+k=Y%4O95DZ0>&_FL2Vmo}~wP#E}7Fhb)bG zHsDd`FgN*iohtWi#CIp)x$J3^P* z0%arjC`8VzStDU|c4V04gt`46$>@M&2dB*h0REM948c!Zq>BBk7rDg2$9(_|_| ze~V{mhl7U+Vfn*n%nLA20n2!SuS~xX`huX$a1o%pQ0fAUJBfZ9%9~-{hkYY%q{IBL zjM?I7{kA^|9M=Cv|AEp!5{qa07k@N=Qh(_|>3b7ue)JeV4uApplOXNLLk9Q#BQFrc z{oj{$4So|Y*57}R>jRn1Hgw|xi`oG={^x8WaykCee2st5*)(7-=`%e3-@_0nU z1@#ixH>P(V`TrpO-x7_`e{-?)@UV4*JR9^Ao{+K*B~lwZc-ZSYI$K$y z707?47lLj~{uB8q@=6>T+>E4bmfZRO7{9|mlacc8@;lJY$A09o{9E%D2mlPg&IF;- zN4S=|LHUCu6%euzR}2T9I?1SFBbJYz#4RC%_#NJeB~YWCkIl|hUeZ4u;4esQ_%V@z zmr*e(YanT<|Jk|NTs$yLFc1_3f3bbMPh0fIMC-tL?5pX?bZ*jivbJ<__k;pSEDK^l zwpPFG(ms^R|6zt9``I{@jv3HrU7-5W0r1kF{EL1ggS(S}J?#D6QvItG*DvP8>hHg* zyANCm?;XPb2N%dNG=AK5(23#CGlG67_&eF!+PE9p*r9MU^QS-iLvNJ*;ETlzhKMy2*&&dV>IT&8^8MV~8xsE@qxM=Q?uBA36Z;McssrCG zetao~GVqX$vUL1Ur)c`7;lgx5S~@y9do6deb8vF-fNXo@RP+EjrRM+wS!bK&jz0~d zT(R8OAX~*TO8j9-v-!^~ij!AV#bf(;@{_u1@3J^0VvIHUNttLY@dG74;ccH-u=2N- zL9ypCG5Y=dQ)x_eFJXV&iT~2H6(T+FpQ48!ERCt(^q`ETe|<-#?GK*DL;&C~bT1=S zh?^Jo-&YERR1StxIkT4&`7iJm|M$R_&~E<0%p{mXA90mF-M;@l_~G8s=KtwJWKkwr zK;H)vzSHDlcg_qLlPvSXVo6Z@XT8^^tkA?Q9%=n_Ev|JzaMjd1>wA{Jbf87D~IFjrZPUxJs-64gUKa7&BKa)aYwm4oa8_sU`^MhcNk+Y6IS~1Jmt#& zh^L$muD`Hwm)_#`qlX@$`_Awmv4{aHyqv@W zZ58qAYdAs%+a)@4(~t#u%Bhl2}V$mKgCn ztmgXylX?z2u<|OD{V8V$2VTM-ru&h8JeQmlGb81@+EVu08u}(F{f&@;A`>|O4ZM8V zaUam_&{50M-9yLI$r>Uf4(`s(ZwzMpHxY)Lt&1ZZv&4`v-rKC49a(*z{a)<<+?yQk zO%>wU3s$2*QGxJ+vYG{ooL;OSz%0NNRjg`_&$9mu- zP#rf4WivAAj7!Y=>lK3ECthVhe6XZ6p~HjVtTZ?^2gwyL=-&Z<@Lq_Jj)NV6NdU8t z1;8+uHE8|jvI?PjrXmN_W1>_Zo0Tv0MK%6nj z%Kty$S)R&@|Exhbu+(p?^Q8gw+=iYy*eTn{T|i&JSED8P(YD__gpxR|@|z7!mH%Vx zd(7tRX-B91Lrk^eWGTFy9|9GzyqjP<}@#&PEV4QG@8spN^%vz-Gv zxE!6!&Sz_C%flngvwZ2|wL5q}6bs%kwe^<$ViUiD`o_Z^nEiZ$SbueGHA;77pE?Ny#89Hr15B?QpxADs>(V+ z6Q94>Q64mDl4HW+a-(=3BkM@d0*8w!UtR^B(%P$H*4=nawP2HbPG;ydt%*iP%G1j- zjQw-#n_3pm3(0FLF!d_VykOq-J`*PPV#mj==C5~j)tZ;SunLr!xX~%e*80(L`JgFY zMVBu>oOeqpGiXv_YmKhvtK)K(OUhT8xBpb0K4_%lWY>G6f}DNJCj~v#*G!T3eE%$S z=CMsrZnVF@FzKpY&XywAtSLbq^=WAhpP%Y`Wv`fh2!~ewQPP3wILHXtJrFTD|BF^s zkF0;^NH&-Xi|1b^4V8q;soTfxs7tE0>h5fFEe}%79A~^~RCT16S%h!neb<^(dox8+ zKkaVK*3}rjXwjm}l}9&bZZ+?CWwP_*AJ!?&1%a0mOXh2|-X59R-BB-A+M20aR@&25 zYMps-f@{ae#H`aH8WXkljtDADsq;k8&fHnrQSxG2VzE`U-$P8_VNv)g9PbpQ7sD(@~l=BV^&|5Y6r#3!9%AHpFOl z&Aw7l-4k&5bW8Jxt9hAXJ*`>oEeq@3?iC9L0BKg`gc}#SHsz>hH5ZzT8Jl_QH zoO{gw-SO&DkEeQnI2V+}?IfGx)J&z0ksHl{2ZPk~^qG?tT(l zuZJ#ddZgH_-Qqo}``LNJ^5fRQ`%_v<7IZg=^~eLH4b4CITsf99x8%$9#Eht&brU2b zT$XgU1oU_nx#ulv-u=n@QH)0O`PTNrG&5D+irt+fHCiqF)^z)|6pQr0Wya=3J(@jD zny!zvMKaW@BgVu;Zr|WHs=F}t%O7fQD1+YX>?-yuO>t7m4*XEsmg|){V!;Ok)sYqa zEzUkq!>yj3TXVS~b5hpeNshwtzLnC5Y4-n7utB=Jw5KB=tohRA%!zA^p0A9!5F0D| zKCiUt1~5~B-mu!ZTTaRC&*OuKs}?2X-gs#jo>BAobgj8>= z-v?`X%R9fcJ~@^;cE}VDgU*L(ohyq!yR^THDb(YCxydC*x$xeHmsiVk7t~)27kT1& z`)B@!r_tTloMP1)vuD)K+UK}0`{vX7EX{7$;$dOdG0p|YCb)Nh&T9+SXs=OgDRQ5Y zm3P^_=#v;Ob;!4do(tqahLkxiLr_`pnrM*+i)aEq6X#DxqKekVKYrdOQB5GGa?S2? z&G)k?h=#*N%Bpuac!&9!4T~1JT)n%~JFzQSPBWw}F`y&yxboI@6~(PhbsgS?T?=Q` zQ=+}@rNn?3H9k~t?AJu6gyMYg2yhvcSQ&oN6i zyr)GU4)w2&4)Fhd^Ubr)^EaQqotW13aaoVA!rkAr1IylDh>r`GD`|>qd8ThJs=Q@w zm(9;0&7`mFe{)Aj_Mp0Z)jjKwMdKMF*ZwHQRge%KbO}stFLM&Zq=tQu*;2DUpg+WWjX0q-Oa6ziN)LHt?um`VOaCQ?dgJOmC)`s z3F@X0-#c1y&x?}W%af|3E)>p*w)=zt7^cV9_RAi}IoUDO4^C8`Q__;!l7R?0%PXy; zxD^Sa9Tj1^^CxLH({V<=%70LwxdLn{hh_6G17bZsJeeKASR4AVLDoVC5k!p#)luNM zhM!Nwjft@OZWRf)l>cGu_x}LX5ponm>B<8WR+1iO<8)X#bfm~+H7D~n3399DT~RGLh}H> z;<*X82T!_Z-qUQJg&0@u?Jr}>PHv6cuPf$vP5R?6voI_aONz_)k|pGqu$IFqsZZEY zmHuF;S}%wm-ZkD@gL-VJI$N(ZmKIqwm+*}-t7&@$1Je2yg6ZC_Z*j7-K5 zFps)5K1F4OX6L6DmV2j^4R_w572#R1{_@*BlM3`l+AB?Ty*pg>9_;<&cYe66>lf2n z_atn)5nuQF`0hCxUGMa>(z-+YAMxJDfW2!!a1$0?B(X@pz;t+ zjVPd-*2f|KW5-CYz8F6E{jM%8ni%P|bDG~|-%qz)+bWM`AzlqH?XOKv(l%h zQsYtAi;&q7E9lgkyf$5xUF=ZEZKTm7Xp~u0-g`=8+2~-{@080J@1{5M`?uUGeICSv zw2#Ry>AZ-wkS0-H7495t9jrd9yRN?5sclf}k#rt`4>!t#T|Yn8#R&7|z~Mn$Q$Pc{>D@zLFGvz z%|R*;FDYlNTE~a0*0(-h58N#zk_o%pl+BTz>kdv(JbS`$VtD!%LC3VNFG%@nt&|Oz z@bbm49iKa!^2{$?j#!FeJo8z#+QODkrDR9BTKPf6vIzIw{qcSux5|4rRvb~!k;h(T zA;NLtF0~(>AjJ9>gPRmzY-%(fD#$+N z#^wXY$-hyQrh8mS_o=-kQrcdSSa#TO^}BVXB*as+6QoQ!+L*9=AQBpyzGA5TXp`kq zds*p*HiiEg9oRRu-xO?g4vO6V>iovC2kR5kUtG2>Kf2Lrw%RFs^q~At(m%{NJ$GZuacAJkHb}Z4hm^JvRyl@+rjST;fYp=1kBzAvQ#Jv$ojOBz?NtroTyq86nkg}nYvR2yxlWFfv) zi+iJP8uiOqV3?@d-{!Eud%h95m<}KEQ$DZ~L|!{y)$9DQS##^F$%AL*&!rY6CIz^Q zscs!nj#f<;;$KC_8xu8$WyPmaM<8ar9T?-m^6U-L7Ao|w7B2j=^rI#Z5%YENt`UU%z0c^ypf+9}Gf z%;jJj2`C~G6681DAcym2v=X`+ddG_Kp+WR_N$FWsWQVzejg$d))~M6EA;CxsyZn6J z#tiwa$z!~VJ&he)E8mGzivp5H!dCsz$Gochk>AM<<(6$jwAajmr_DDY-)E91HVGta zM~f$10Y+Q5?tv@Iuh*g}2k7&p#*Ig|qjlN6{jLvX67SuYjGo+5>}8%@Yg!*WX)qFJ zt@ri)pFG#Nlc*!!vc4|Saqa!CV;!G>KhC$<*5wt&TsnVmE<9VxgsjAZGW2*P(gcw~ zphp_?z(J1X*TE~&)$sXHjYjRLUQjwACdy~aqi|JO&>ydvSO+P;AN(`Y#=&YCd9NW$ zOso*>)YxI~Oh_xoW&0iOnB8Jj7w^cI-F38MUW<9%-J+0{i(Vc>&-^r+S4kvpD&Y$- z$I!q+0EovZmZd_(E^D;<2A<&3on}+~4Tw?m))w z!PrrTq8vGbLbx z6|XKq7B|<6%XXF^4b45eV^h~Xs#sTzZUd+_vyfcDw8_(AUG^37-b32%AClD7*VkWb zWTY8n0q%xi{ebqTn7iR!O!U-aWVV`TKa_GqmaXnxOX+b)OrT{J7*RK2NwQ;Y8sa^0??kx5lDBdwMI0+iOLe`3-m zh6*}AgXyLu`1{WyOuV@9OVBrJxZXsMifnsC0m!jXJBtEZu3}tL)yd-vyqp$xeSYft z+{id1On2&pdeE}0UnE#I1e$kFZ#*vR_im-U-J|1(_pD%k2QOWJ6{}$f!XRYO2sQCw z^{V5oV5*3SnESeJ0zXQ#H&`q6Wgswy_3VE&ZP61&zbEf*m3BFOJPLk*a}^VKS{Is-hrW`1fsmbOr()VE1=!FEUBiK(E$Yb+P9x6#~MaIy8H zbHF5T%l07E5|h$P-N-g*NvZd{e1BH>J;fD>mf(ywpce;X3eX}u7%y2tiPM;{z7 zt*-apq@rEn?d>0YsJ!xqx_|B=kH!5|%{Hd%%?nO6+ZC@5sLEb=Lb}QJ%cEP5 zo09fwL~BlIyydYx+T%eC-|rjS&sh6_KP4$Fb-lSyj6-|%e48=kxRr7GaZewbeYtYR z?(b3T0Nc$p1jLl!nJD*~^Q>;0L>?wyZ`nus><;-&ddpN>I0ooAt` zYLw6W^EX;bcGlWaq~mT(p9fAgq-0L^`~0}!dAz@JYHNmUrqivDKlL=OD}GM#{M?2Tc6|9Dg3J+TJ}*DU2#W;G=U&6cZl=9|9( zj2>Ot=rRV7Hb`2`;tpL_0YTr7kVSct?}52BXd1X|SSRtutKYZtP|@pi-Y_&X>QDR? zjFsR|MA|-qXM7P-=42$Mg-wuVC+%FhjCdpU%V>k)+#!WOja8Ua(s^Ka616b6%P=B_ z{X&sv@EZLCzErj;5OW%=MLv{naW0(UP|qP<=|Aw%Mn8*s-E`t^;{EgW$r;?`0vq}; zEea+SDFf!k)k1yTSH(kJCwP1>+eAS%#9_s4D4er<itKBYL}9 zkrK&*J7LUeL?1|h1N0+0+3PAsKUAl2q2U9dJ|3zHvUW2APt#4O{Pm>RnXEDn-upG| zlW9+3tM+Ig8XJwh_1cp3$!rq^I~y|s*`LY8h{tX~f1HUO-FKLNg}-wN8<+D5FbI{& zytv#)15`6WU&`oi`sy3`BnBWk;r{eFg1~(<=MBNAkGg%7!H_;7%>XOwiUykj`qRTF zhGD;c)=z&qpydmTN+9mzbKZdPTeF(?{6KgITsXOi`@B7$VcZufK|@XeR{A5v91Boc zWw}Q(fIrxL2`{>d)<&{p0-27nuOgp9_5$Lo3HChtD_Z{1nQwLu?DfJs)&Ted$CjWe zlO0HntW!r*W)ceH7gl;mk45NHmeob*BkvSANB6Qau4RfTCjGHN=IVYW*L-+#NaKp|T*Zx5JpxM+65MMNR zzO}duo3R$wb>p*S;RDzQWnn2zg$5`XptLc)m7NaQ--SPv6}tH|fs$_8oXP-Dq(9;J zUCBZ*XjR{OJzw{9*av0Dm!Rp-<#k|RDGw{r5ldpDt@Js_m#Hnc7f?x?!uN~`aQZ~P z`TmpBVEq$fGtvCt3#k#>2=Sb6VNzh&Nr3<2)2a|0;u<2tR83a)U=;(PggOHwe@xMc zok6A<@;zHVQ1A_feas+z`T?M0cm9K`2YUNW*oMf77q3O|3Cz_?_ZlGb;iY0QeLuP6 znPi#-@A`ovkB4eBE?*)c)+$(%Z1IVIEdg`_T;D{VfD)R&p~>(SfV`nQtS#E5|z9x3$iRV`#IqFM2tV>hmTRULn2^>ap3MxMCbKA8XDQKip{XNt=V7+mY3_Tw|hVL1n$M>K1gr$}M@r`;YwX})vAa@TQ8$Yp$UHGMDD$ri>)q-E=&3?mm z2Vlnt2kC*hvVzr*FaLCEEU!BIrQa5t2*m45kqR2FZtcSVh@0S`TJZ&IcDJH@DZ|D` zCln>Gm-y5|K>PN}%slT4u?J$?A6|%wk(?UP*>G&~tqPHclM?GMdl!MjIpxJ76&rSzaKuj?C)12a{zLV)pZ;dhQRo?~)J4-S}|j z;rj(W)x|xqz+|C-k|0g}2QkBu<19?1(fGM}U1qdbcb+6qMUwX>4Q27Xw6X)zWp~B8 zU5b30axNp!db0z`+qbk3^2S8mt8;@Oy{gc>+9ajP=(&)4mDkGP@KPD?O^edqy5C*e zdEwIa*l4BDX4$O~oid94pLU5|49Q7qy!H8f?3~8)0x5=-Ek@0Eo{nmah=^%X4Pa67 z?87MRdnN57BId5qe*{qHFdfEpNr>&3RO+bgz;;Rfs7` zu2JOO1~2L;@5P;gn)4d$x|$2)LvjK?Ip27@M-&3tqji%%TTguo&M(Lr+1RB-E#CfG z=1a-W`u5p8ZAui!so6=7Lx1Vu>W+R zM$>CA0(F}?Q_<+6@0Ff-{jbqp)#e*aQ$(K0eoUmc)YM7kl+RDH13TP{P&{6Cf`@I$&ingU`pO%W zV}9?H_3o*S%Smnbs58$t(aj>H@R8L?zY>ZcowupaLFGx9Xur$*02bwDQF-Wl7Ht84 z*-R_s<)~tPZkW zw47FdTYBfFQum`@{5>;Y+8#5tYXAIBesgYwPqyHgSxX^&QZZ)qd)ZEAzAWu7-(e1Y z_vB32(AQtNpb@9YB8kwRg72s43jA2!AY_D0MX5|Eu@D8FQF7xL{e|*ZmeXQmF~@(3 zS7%=NRR7kuFCcSc#w4MmlXp}-Iw9}d`srM3tbe6xOHXZD_l2p7^{LVB2&c`?ZHKAy z)|pE+*3jy2s8?MHeptHh%OA;Y>$V?+3{Vx6g!y_~qN%C7%1@hIZu5Ia9hq{MCAhjl z!eSyDfr`tW1Npq+j`L{tVZvkl=_l&`F_M{(BDS$7RHUnV_ebxy0sfC$5<5GCy^?1j z7kZG+jK;*ydFI}(zV$6A%@O(2!%8c6x4@;Tjeb*KcRXlvk4TMAb+2~y3sHZ6?)3D4 zyov4?Co7t_`LEBi9S-S&`lc&`3tNs%*qK$^<-fkXBEs7->QL5Ma?;~ZRdEe({7?zpff7AeYt7`oDd@BVBi2KLKXFjFklh2BYWheiGf zl3#sP>4@fYRK|mP8>UpJ(KQ9Ls7@QK*RxN8oH#VQAL?JiFJ!77X{E$xCV$Z1CS^dD z;65?>20(DmsG5tjsLz+zOy<1!D&JZTO6F#{>4j=a@>3D&4_&@CDg7t2ibLsT3O?OUT_2%;D5p2scu~0g`uEH-sRpI$Ikq z?6A9+V0qUHGIyJ$OXKY+5w#E)_u1t0?u#y2ne17-T zcx+xhV-}aj$c^DpQ1W?6U5=w`D8H=W+-=)uzrk`q?DwQY)M$hG#WdI|Q`CG*lG0+$ zZoCfAkG@q?-Qum=(-{Y>etV?9fu^>&X7%oF@0gy>l-Sr8kjqnRT<@&DLuK&7hI?be z^aS5c8lph>0#G>JyQwMzf>rPBc6D|^4)=vt_$%GdVHVOjI)K(Mg?d_Rbv%a_6@ zTvFUv@2h8hNsCofE0SKCgf@NAh_txx z(=?lBubBT1Q^TnNu(M=B`o*}cDPEgwrg-}ogt?pUc)L3+35pg%vL&b5=z*^B=tQ}l zsrnv<@JQb>R1HsX!NmdcThsKH;Fd1ShQmvmDVU9U9zRNlilkI$)TfNAI>d8jF)h(S ze9{n1J<9FzzU@0T5rVc$OJhbYEZv3U9cUPnS*4%LBGJ z2w`wtv)`km<3Xv0>|_aj<1v%zNp0HRpYda79K}+uEN?P6FRV zMYSht2ad0F@VvUX;MZz9_r={&St*d}9~Y1p;1(Pq9dbWL(58tDYR!?HkuPcaEeq-y z5~zFnn<*T`4O#NBv!3CWIoo=7$q-}U#nw+jFTUOs;twO3eB-}WAP}@c$qvof) zR~n{G5^tY+E4c*%#wkx%FUa_#&}^fot;PNFXb3kii=LU*asn5u`TV5my){0inUFMn zHeupnhWN!kT#TbMA>l`x#MgMIJ-Ibib!&a~IMamEot;m*tF$}3_x7CWD(EnUJ@c{s zGyQSdvvw!9YD^mAQ`#~$@wYVZ!k*+Cnh@}}t~lQIU?M727#G=HR)2MKUa+GQ%H*?# zE!D4~Bh}d_+J+UXk+R~vy@C2><}(So9bfF&vht`-V?yZ;#Lo2=t)JdrhSYp*v$fG8 zUqVFXr-J~+L|1WF(}<3kXn&uBVw(OZMDlvP&t(TT?u^p(D!8~FEB*kXWeplaie_Pe zhACJ+VoXs+qmC>{Qh6X%0UT=?Wwx_k|$L8nfcuE=!-J4pYnvdnqDlF z%rnh=yRvJ3b>1wBV{o>-LesB9z7!-?v}h2r{?$6NB|{?8KY&fTEy~~!Ib1fhkxk!W zxL?x+;%Q1ud^G%=|C=hhOzE3?JZ9vY|Cclc>| zibL5&{gwOT(ep#&eWOKQg0Oe>Gc)OdRQy6cfwqnQ(E&{_CaanUf{OG(kl*`ufPZv+ zYMSlvl%_AiVdWJalf59V&2!Jf_?C`NnNon#<{WjM;3L(ab3u!D_m8fwt_A%uF6&|G z74PZcQpo1Wc5O6a$n7!w!+L8I`N}BoKIbPPcH38W+$Va}S9BpYIj|e$dOkDM|L8U9 zrOT%`yG9mxea>6ZSWWRh*33@d4oQp6xU+oWqz#hui@iW?k(RS^^aY$q3MqT{rrH5 z4x3jce{_l{(py_7b21RFL$_F~zTjIETR zb0@rq>8~q(G=cSg(M`Np`#1e*)L9V4u^)ZWlChHkwiD))a_J<-d}?oPo*v-YH5+t# zx#k=eULmUrX0!80-yw`w5W^$CmJK7Zyx%^6cU=FFH;t+Q#CP~|Fsq7{KbIFvGG;|J zcha3koi`dHu-bt#D2bLJ^FX0idJZ=+(D&?d>waW3^HO0cEi%_kbavrLw`47pVfq&b zpEP8-9~^A%{X`xV1)%xQUC+8+pom;HI)SD2g*?8&M&Q1rHHI^t7UULzW&K5-0A>z; zF`vPLE_*ND=6)vu>;~qMvYD2_oNxYQ-qZcWJ;aJ6fc3-IZDa|&`th?%%dah7U@nBM zEChwcn6!6TT%3KYVh#5*2nzQiLjpZuFZiT86MMs=C?}Xt!%+T}7=Rw|e!kKM7DLd) zh&QX>$mbhu4A^_q9*&i?p-Ip-@>OtJQhnlHz=HzD{0I|?nGAqmOQgJKF6LX)KZCGm zJF}93J^MWr`Tnoo6uEgW9NlO_upTu{UtAg$ld;jgT2J1_kYe-?z2Y74D&mnD2z_?5`3XrsESWN@ z|JdWZ3E(4pT}r%$zY?-RSbgh=D854ad;-Z!0sJeM`4W0hjR$XY|4ARbB#V&VQe8C{ zCm%@0Ievt}qVx}aLVx|Nn*D??0A@-a1E^3MRhUn##b}hvTmr55GdUwq=r$s~hI$(^ zdE{P5Ti`VWU@6)3pAUhQkPcm()`gtv$@?ZdzEO2-Ruu{Yyhkv2ZRn(rhyx%ey zmRzF%pftVyrVOV(S%KC8C1X&JLH@vw^ZT9YbEf8wH&`?P=s^q=DG|zVBk(-7VS6a* z1E54m05}V*VE29^o_+;snYsgDBq*?|0r(I06C6ut+rC96&XpLMFjX>YE=q)(Q2YA| zj283bNX&FtTsVI9ku-6DKflabdT<6Zun*#ZV7%@tVi3qM5p=kk>*J2mk@h=<;e|ou zFn`sXOU`~9p;Rb4+ zU!A+WQ_+nVcH!H~HfwgQFPtw>l0-^!cJHau$n8pwknsPCS&_dM0I650tcJbYWIb;d z8yln_;Nx09HZHQvi?nNAXophE*1FfbMrx?JiH-DGGZe=igA-3mS=ckA4a(f@R5rjI z6hylZ2EK+JPv5N3bOL$ehKXq`gf1Q90RG00PAYFr&r}trJ|AyVcCYAIlBRofm4A+3 zc!$^iyS|(fWBzjTB|;#L7zGqIP3R2f$7Pi8GC*9*uM<3Z0`kzIUK^E0`l5AH3xtjnw!Oixz+S(uGaQc= zFS?A^nhmPta3M!1$&`03yLSVsdCDpd=?6o}xok;uoY(W=gCJaF-&rB4>CnXYVCx{N z|FO`Hxh)%2Vk^2I`t5iyJLXx3Ld)j5r~dj@kXp|GHTpo)B;^tbksgsSg}9AwR3sO~ ze`a?J9iH_m*DF5Cb^U>|5RNl+`a`rwoW z*Hb5TLPS<1vBYO9w3P_h=j>Oz@iOYL(b?8y{owHIhB&Vj={+N+U_EMzZR8xD{Emu{ zv1p@nN%GNW8{SClnm8L(8WqYHD#|wIHRpKdHs^ZgHRt(-c1&y0t_vxWY#V!^NHr;7 zu<|eW8Sum?klZ13j*;_E8xCszwcFFBP*loRXc>o!`fB4}Q9@R7YDn_U+f(FHo-XQg z=d2(*R(rhmMD0naor*w6(HF=@YYLfAwAPNzRm4mBIEyxwAPDMeBtvFPpIHqf*G^MT zvN0Qle4BK4@5qpC*I&-k=xzup)Z^J4fZEexxMUISk7Z&yI(CT+boZcM^c8H357oPhrvhE$!?f6v4CuttID{6w|i-f_&nL-5=_TvH>{q+lD+mWh!AvJzF}oxvuq;G&1)ceFh;pBnC* z>J7xEg5nrA_lwtC^g!TU&MxQVIOqc5Tn?pb`zJXkU-qAPd-0GdW9y*}kkuhQmHiNd zOV+4@()y4Y!D`br+V1OTLcRYN6a%wH6^+n?0P1HP*906BxfxI~Lu~(|mLkP3P_v^} z5vo;o?sV`0Sh;#C)X!xi8W%D3SW{v8sfpAInH=-L_iU&-6$x=yD8AD*bAh%MJsK#2 zy#wMf$On5ad~!E9U^B1gX-o>zu|B`JsiSjI$iu)7P03h=k6RHc@DXZqXRGkJT6a;? z-6rbMkv4^$RmC-JMOx8$(IeVSZ~hJ?P7d@ECk=jazpJUAbFH6f7ETN2K}Cyhk!~qR zCqBVSYYCGE1hWv|QpRRgqXED!POKWusk|ee&L>fm?{;N}%_!gIymMnRRNe(85ArV& z)U4l!T-Vy47407FgH8`;oaUHZvtP3idf&K0=aCF(A2k})1fy;fP|XK<0|%!rnvc3w z?SPD%GRT;@6wVuWqwE0cz+?pi0c!#Q9k!mg0#S;h2;RX-s&P(M*+O-wqQ5|Ic>K=d zTB}{dRrW!Prg;DM#b$@oP`NC$#riEaR>ZtbJq_CY#C3Z6MjflJ)^1TZ3l0}~TfGoH zYLRnpw5QFY^~MFqat5t;E{JU29Q^2z9?uC`ce|2DUD-`-Pp8SY>mQqt*YUfd&l#)+ z@9frgyGt@As|^`C0rr8qeXId93Q9dQblCd$iKz&f)DhzhREP~7r{w(mB=0r;C2KV6 zFGj;;BEK{RcX(HKp-hljdMC%~PVrp(^@_1f)MIEMC1c5fJ%dIr6mCnZH{@$Fy(^%)fT%nEpq0(Qd)R zKa5%MZ_y0q*2UCFMUcAr8+fmB@+V=TvOI%L0jACZ<;z|n0aex|D3F2)3zh)&SCwt# z0{)0nMyIS>JRDqm+`EFXyc#`~Rfh#VRTM~m%GI?)A!E0TZ*9N5g^c-u*Uq58XN49e zoh&FifPyKy``Y6aXd&}9d4vgi+@f@Tb*6L;mK~nyPXPLJi!&0zg>aS<%f9n{-DbxY zK=ns1?;%l!g2MJ^C`s&cxIg#XA?#gGXb==OxlEZc@cXYLYis;NngFSC=&}J>N*G&t zpoH1n_0mU1+pF?63yW*7E-EK{0ZbW(K+O&}>}EDP#4 zbV*RGDR?bt(#KN5wH|UN(QuFnnp-dbQNmo1A`gFU@T(R)hkCfME}&1x6^ltqGY9Oq zXH-`Vtmu*v0;8eY*^|zWdn$!m`|UVmeZL;ki|;QvynfGG*17i*m)wL9tQRV#meUTc zyAZj>XdqRG1fa;SF#Q7?pL7pVAQ59Y6nlZ1Q-Gr>dV$ZXCeHI1UOdQHSbsoIfeb{D zp@+^s^*0!J!2{atyOGN#+I(xS%J;0DhdLf{4sT=Q*-zYhT7458P;Cb?H2H`e!+AXb z&_O*oY+wl+9#-}zUk+=5!yr0Ltisw&-1Phyj(Z&T-`uv*vhvT6OrJ~SpV>N>Ls_bs z!^;-LL{RSiZoZ>31}hGwUm4TBo0H}G7j*R>>H$JWbIg23&;?&x4G%9fkb>{@TsDwv zODtT%`8I^*bJl96zZrU*^O@XeaU0*s0*QPT!Ev0$no4@<>EA>=u-(jr`Ff|;_M2V@Czk`(?VxzuS{Z>FQ7vht3YZ-1CguebOAPLg2@px2T5n=&9Syd>TP(VzMrbG9lD$zwg z9;i!S!<>8?DldX&Mz%6(Sh^xjj@OmYU~qQDFiPeifSZDHSXqTS03GJqejp%~^#ZoIkVZf-I7?yCVO~M~ zz(PG*WPz3p7>F~(1?eotcp;`=fOq~7S@P1+4Jw+xtR!ebH)f7wIIoa$ASB9?0e^yIRJojF8+Q?Acrl7{m4F&6^711=8M^6Se1^HF z=BJQ;<#fq#*^lU&>O;uEN?0{flR0_udz1zUS(6w~{# zuuvU6DrYZ(nEcA<_sm}mext0yL59$IEr@*jv3>Z4u}dgqOHvuj!Z9>skRt0MUO!j) z6EhY}{W8jcl^h}^A!)pN+$Ad#e*`0moUmb;p zEfz0T{F$cez`9O+Hblw~ZzLgZfF;L~s+d5AB?s_N4AKeMi`LgyS4!BcQR|6yQsKLh zjKBz*v;741sNhPxt}^t_4!&voc%0C9uGZ*awIB<~zPTyx;|?io@GZHN->pXV6xMG9 z5nj%CIaVPAkF7}qp|~$Lwp}xH)JX{oMrIZPG$IL=ycOXk^CfEii4pS|Ax>@nt4kqI z0(97)&M+8Cf3m?*iW2@PVnN(Z2Rws!z8;97m`lg;Zb#JZ!Uh~^do^mX_l`0$^4wIs z%Os)i&Mw-BRo&L_##wcpdHRWgOaxlN9U`~HrO*XGt%Fa3xdCJJZ931u$n=v0b)LC3 z_&eDIrLlkjPAPLyR)m{(C!U_{aey+P{aPN-Zy-s&QY@c0Z4MEY@6A879>lKB_IN5c zEs3PlhlFBXBa`N6+K65Gw&5j=77nCTMsG=c>KkUmekUE%(X^?XZ8bnnOy%O*At3ba z8-~+$A?mMuHq8BoKI>O2pj2d-?;@Q(L?FzsMh(Q+lti1MbcpW4Yi3wLNK@pG-BKAX zk$*zah*LgZJV3F=*$laA4kyL^ggF?g&&Rn;C1|W4Ov6huh5ih*KoN7S9IZx$*P9bH zjr>O=6CT^o?a)o*1PP0t1dYvsdrPz8G0-zhJD3~HVOcpI!3(hyDQkvtN+uT59`q+j zHRIX0hYpCGv2q-K5O6e-rAPbxN|rvH!||M>GH*n1Nl(6jIC#Y?x-@-rhLp;QqOW^n z$Quy{U#K(~hJeXcMA|v!CyH2A_pl;7|&Y=kt+*+vOi)9>S|$w8C+=j!VHuTA*xWZyJwEY+dI3P zh1=Q^?vA)n+QH%8xps`Qg+YIVm%Q2CV`m|*qc9kIW*^DCjwjP0^8Y?&aPkLR*$dSu zkZu-W22HAK^SU`mNM?-OC}Mi#hlt(|zFLYv2${QiY@noX)FV>U;be!#vZs8pu?Mt9 zF7o&k_!E1g&)Avb7`{r(`NonRURJQJn@z%i zUqK5xH%u}z@|I5+@_~0}`qX^~s6C7D&zyfAlnyGmAzi2jbOB{rh5Dn;&FP6G8_A!W zGuSAD7Av}T+^$$F1EXQF2Ntso8ciGQ=1~a;Uc`PV?e+7`r{W!h_Mr1RCI;;h83YqZ z7&5eUP1%bN4EPCk{5+`@oKH` zTl(`RfEqMq=#@pq#S_aWVA3hz&=wWe*#)SjttVx&Se_WCvR%wT@DirpdI{yy&)=#D zQr^Oqm$e`R2}*9BV75n+!x1%IT#)_fGnlZ-og5?Y=Qq<{k_ zJE|4z4r6a!`+1c3fa*60?kz@*IByl2`;dxfhz~?5r1NLO64+~{CENh8P)@~>Le^*P z5utzp#4UikhaEf(ZGxd4(SVT;;S5m2?5ABn55yz=hRp*7D`bZ>wqy?NFy9H}{Htx+ z&$MWKyb}nZ5U~9g?Lj;P9cd8u@T@&)m!t)^vpE>@m;P)Ha-wYl@KwfLvf*22l-_{9 zmLd6wBynN@fXq~tFiCa{DdvsC#r+cnvimX~(1mom?*$KVT+gD8o233Td8g1hEH#k@ z0!;0WxV;xX<;VcN!2(a@_)bo<;A|8?n1YPEdovRm`6RFX^m-HYq2p2o{f*{r!0#W? z-cbCj)VU^p3Fb2C>t*9oZj6s*F@P1nN!4iNQ3@CBtS>X0fhT?dztj#_?TyO3STjQ$ z%#lkDi4(p6Y%c6r!>-~q!{HbKXa!eKsmqo{ZMIqP&exfV>H3zAVzQ{T|ry4PFJvqpTV6nViXKcJ+e`uv98N(xqm9 zOBF^#WP*lzA&Q%~!(I!JKPXvh&-cCkJV=ps{*{xzLxO<}u!nayH@0(Vs9*IW;Czbl zMmXO^4l|=)3FhqCr#Lp+Af4 z38HZuZ}daFgD|-6hpN%L@iM7WRE@fQ+i1}nzYu5m5J=&BWrQEINV~u{YjuPuco(bk zqQ$};uPjHWR83*;Ys zYB?PLr^k`4EHLb63Z6VpJu(iRtmb(0-Ptg&?)pWN&_@6rHd9gD19nL5i}p5Xbaffo zEYJeMXJk;SzErmLv1T}_%Q{O(Wfs;201o%Ft}uafE%Ya6ySn9<)I*b$kp=F}!6%fV zRa@2<|6^IzzHLddX06+_%PZ8)=+h+6CiQB4@=;1I0bKUS4fHsclAb)YKT1^Nn*{=29Q+S#;QEsYb=5TUN+<@ z!YM#-@=rmazC<>&Z+=NzN3f#fi-eN+E*uZ9Ov!9D=t#h44ZY}bh@3TRExjUzA&tZn zo{&c1zK_AW7SLhG6?*nBHKRmBvxCy+b+EwWD28XFbsKVBP0fD~Kqr%;W;`eZ85)7a zb;GfIlIXZlRpHN3uRBy_KS!&(+a<%X0U>ZUrCeiKPuL7yXdjY2m{L;i(^KWT((G9p zWEd9&4Si`N2eleE&ri#G=j~r|Q3lQdfCg{ZODx~F9i3dAA(n#MGlR9$f9F63`6cP4 zgu(VD9NE`A)Uls%585jeT75>dpKPbO*?s+m4NHrKLx=NwUk-p4CB@xMaowozjQ+=t zjb?A6^@r7@+z5blUvx~?>^L~&QLXMqxs`X%1ayuEFhKJ(_0>j3VX%mpAq!jI>}pOn zGSY_jGheoiKBwP^pAMTphqsBzY;*`$3SE?g9D{MA)fw`d{^8m!N;{qXsH;V?Z+)sR zZk2@;;u~#M;WL2zA3lUrhqh{))kJ&MxJKidsquc)Hai0k9e!j5RsC6+VD9dfO8rTb=XjO7KO=+6}MNaBn5o zX3*qY&^9LDN%}Xoz`L;2V)9@d_K?%1b5L{?96uD8+~IF#lU6~6J{u zFSI1Xx7x0VEFZcL@($lUHkSctN)JS8vLUNvzT3;O?5@mROoJpnT?pYe^>jdRgjZQ! z8ZJ%KEruTN9W$gA!Q&p7ZoJRfZO{h`BmT)mk6BR(YRRqk>M?`cHD=OU=5=|;bo**9 zt2{w>KOLSa&QW#*8!GvjBml8;5}6SYLQ0-TAcfsC#znKED~mU*V3(IxaYM0R=Dhk} zb?!nzcBb&}mdu-BQM3sp)Onhz4q)))c4)_)b;!;N9>BC~)ehbuNr&8WV!fgqc4xs3qdIU9%9K8%XS&w)ld4bTm{)Xj; z9ftQ)e^p}ZQ#;r!eIE-kAX(GPE}~PUzYS8Qz4Ugo^4oX)IuUtIWW-$MulZz!*e4@%f z4i^cn4cc-xT%0uF>2SrU-;G^84Q_{<%-0gp+c$x>l)7a2f!_j$30Tp@6>ZMuMCMKX z@>X?ouDW|vUj4w&|DE-J6vA2Zud0(<%R7p1`SXq={c+@bY(9o2Q zH<1qTRdvenFj*vhldy0v-*Xy8fMtS+qp0otOP5j7?fJ$E9@JSri3M-%#_5lUHR9-} zU(7y#`jlqrk;Bq|U-O^Mv*3`YiK7bHNtN zAqX&#QXfA_R3dx<8etzc5ET_WWr3r}I9q%T|uJX*)AO}gq?`G_gB(^m`YLUfJKZ@B%K21~KRUUPQQ;Tc9sE@=(_Z zN{tpBcSwhMf4fmScA7YoHgK$SL!`i}D9Uu|{j1Ly$q78v`{VL^0S6#uxtupF zHmWf10B-k3$P2k(wVJVjtLxK?xKb_G`9Io#khlHaDn`c~i}v2GmcAGXNX!ZD_l%FP zVIWUOnl`1sbPQ9{4x=zrA89_kyxar^`_vhvBsU*k{(qcZcOcaN8;_{8lZJ1K>X0N# zk&M%jF5~*PB(oHWqg=?Q`Bfq(TdpCR%I>VYN@SiW;*7G6glz7d`#m3<7Qg=#cc1s? z{p{y?JZfjQnGWfWPHCkY)_ro|Jn);s~%MK%K*Bj70AIr7J& zTs5kxMGk^yK6&Qb@Ztd}J5Ku%?ihOD9}f^a2Q;&d)%1yMJaU7-)uV{QM#J$t=!pSK z29GMG!#zf;4xaZ9{LH}_a}&yQdyr_4w3JUyg?ZrmKY}sdqw#Ri0N2ZMv`>r+|MGMn@w30G3USG@O_y4f0Q1sSshP-{LXs(?TE~~m8>(& zo8t42AmaZi+oTQooRrhZp7Rkd{>ue)!sj2k+*BDTcF=gwPVxK;PXqtQ&UHpOL-{#p z)l^g$!eVgl^1g6N^p$pX%9DTLS#vaLwamrioD=ouAj>|L3yhRr$*uJ@Tjfx zE`=Y@-!eDo>W)J7CvM~@g{S11#@+0Kz;Oj#5fx?8Kb#dMpt85t06Yx{F;$Ik4+0YM>(Aqht8K7Og#w-FJ zWVBIFV3hGjwJUN1_#JQox<^lK;uVr2xZ?jr`XSDpz;jqQz=eO_z;eK?v;P|z);HwJ zj3@WN87A<&;RX<~^!pDbVAlLvLy@UJ-v=Bt!vc&o#Id`%b%(QO!%{Z>We2`}0#ivz z!vT!b%O7#oJ#l?C`A}n+)fnKTC<{t@K3)`{$7wv-n0@$spkrvD<8r!|Y;oZ%Ty6MM z6UuLL4#N@xYF%zMT!7|=7$Kgyl3zyfS3`gBR|bYJ(1aHd{pzFa#JOgw3Kl3Th4@HD zRRsac^|H9)lmqFCh8*Y!1+QgwX=`elx`e!W+SPch|=Cb@u5)(SZ$tBEH zXFi+@7-f3+^9ghxFrgFm18W0#`$VmbGGkCC6Mf!^e+=9-k-q~v5oTiX&NlqU?J;bm zhiUzHUIZII3&eRvD#uFh5 zlem9Z%6EhtnZ;+Htj+%kBJp=-27S642q2(cfA8)2aWkiX!7RTb00$=(IN(D1{YcCE zQ*<^5$wgTX>cnHwGr^$XDI2o^r5%pAEQcO>lCfI zdfouSEbED^-{I?;IjS)S71A%%S%92d1AGB#*3F6Y(bfH3e`-tp1wlJV6cmK~c9+8^ z0vpsXx^A2dM+iw-c`MIbbbVqQ({gQ!HgL??m-w|pS6F#(ja$TO$FYA2-1iT6+QdOX zzho)5sNIkD8g4jNI-dF6`WHzpp49Wz?Jold-#K(`{9z66b4E@i>jX~<`M`cMB?mcx zaoNrBG{hfPE*t-iKWp!SfS|s5))JnM)~bMihMnXAnakP85i^WP8NQldKi}MIzcHo=k4k^U%AjIc+ zmBtC&T_ui1lT5oA@qz{y*?4pO=MA`WCPL7EV-Q0e!G!o&K2aSG-G7>e#BpRbKXy+xAurj6V^{2F+v=eidvuDJt*&*s3k%ET`urRM5bk&ft0?&U=*X?4%7t<4l8$gLO(DvRu>o zoV8QzJO67B%p<9*1c)Db_IXWe8WT2#_&<$IJhT4lio-TUDf zv-akvJ35Ky8@AcojP~`qxzBnvp$p@8yo8dHar<_iEAa<{BZ&q~LmTWn^hMAWiXqhR z^b2_jGb0)@2*PTmy&;ZmnN9xz0A>SLskesGR}`tIV*?vk2$$R9UvFS|*D!0d{rwL= zJrcJmy*tP(Qq19dl&T$!4voKF^DHKBy_nGFZ)qsOZS{9AWg#o?UiSNxv6hzK+)~t8 zY+w*-|Lt)!3@6umGYzKA)alVpPRWKx!5}p`t|6Uzu)XrfRoPhjW-^Q}esvRHo@gz6Wl^9@`o2pcmO;7gr-Q;z;;a;O z>OO4=rao-RmLw;yj4%yaT`W{AlUvt?47MzIcZ;^ysH-$>9aE3o`+a*(cR0WEv)6bk zqjQy6P%iR!)SSnS@|D5H?X78d%#!e<+LQ4wC1e~af2+85a>f`KYA)lc$!xYOb@JPU zZ6m|>CmI*)RJNb*0@&PI8y;-2-qOA>I4c9)-@3(gahV_h`4FihG`4m0!A`5^ivEHa z6uRoawOPeXP&`fH! ze&L{(sk6w6c;3=suORIoaXeMGK()osDA3@9CQ;7+6<(ooF|(~UTP-KrMl5aH_m^+? zHT#DojMyyo7}u|pB?EppLWYw!eGEg>7z&x}`!_ab-IH0wG&5ix!`#2%@TEZzQ>O$2 zbxTtgYi_&8N`^_9`*@pUd>RX>n=%9nMaY6@DI9XT1>vlM_O+Q!cC%dw4L&`gH9t^9 zdIM82Hw#vO7Ue~Bf8Jkd3&^jo=QG9Teun?sSBy=^cwp+-A0w2uE-XA+xBXR14#{>t zzc?N{Hzi}Sme$972ggS(=dwetZZCG&;NF1=X;Cb4!g4yzUJzaR99wtW%*yFfaS6bz^YLHRxb6e#9wygk z7Zi;kXQouBy0FJEZy68DP6@+WjoIwN+us61FRfro>s@Zycf+}W+WeO~ytO`Qceq*< zSW52YdIdcQxeB*5>2MY7;H@u4RMd2x*^3FMLnAUxvUTRo(6nF>MDyuAFxT`j;WSaS z$a1^FcQVRb)KAyH6&V@ZrNK!OYDWx~@}{>(>2O`Z-q`ULI?%!@Ekb(*&P49&EbsKb zcqt}t`%@YY?JuEogis&4=6V-J#yQOO1YqlGd#b%8Wh))=nT`qPGB0DjPUWO|-@nDP zptNvD`hYfB+kocgZOXeV`1Hv0_1gXxH=6E>*bG}9{2}Gh0UCn_=%*CLfRl@0A`T*X zFP}4^9xgC*)`D>)D+0}GXp8nwd1fyD7@gTIwtnwLTi*9RIb_4v7vsGYGSQeXO?lf~ z!jy$9r|o^?NML4^Xkg3(v4q@0)e1i;Z&LXCyQV>eu>Otd0@*8;7q}!md|!2I*L<<0 z@6l@RSX;**CXA!W7tU#e6{DHd$-l}y-7T$IWTew1YLxTAb8DF{vj116E(Fc|9+M42 zi4P@Syit*wA}6#{X3(4(=JCv$I7{$1UMOlXkXFzjihdJa?E-zGweQz@IWp?~1jS~* zs=t4u^b#hX>W|33#t+SZcZBv3jW8Fmycbe!-79-5l3@3=Q!PyY|E?JOuDb#xKTh<6 z%Z(YAZ0u?DLy>ZTp3ASoYcP!yif zf~P9l8qSmMHa9gJu8{*4r+w0)e7&XHlN(%q8}&++-leMXQcXe&LaQ(N3%h^EoJQ9L z7+rO`p&WXE3|@zSxamFu_2B;eCDX&m!fJlEo84KM9*dxG0q}=}St6t)QyByQ__evh z^rVyBvGPAC)3foR?I8jdi(9ZTL!NM%<)gXXRER^VonJt!&K}0($F^&f*)iDBr?N|( z!6=<|5_L-vzx-Z@vn}5BydSC?pbmbhI$4cEPg@ui9lx~vf?g8edQ^CEO0zX?)1ETv z7a`cWfh}|Ne_Z@NAJv($F;p(#GAlN7`$M8Ywza8Gi?-J@HP5XY2aye;ZoB(nX@Hk; z4xd&Ctd4AmF6YI{?r|_I^!ni8Yqr^6BbOGJ6~roi!8*oFk@t%;4KiuIDPTadI^}Lw zrNObMaLaScI%4|}Q@Sb(&JLJ3E$+8@V`Xm6!=P(FY(A=gh@D?HzrwSgQ+4xKV$)?y{@Uu*Mtu-#;YBI%efG ztZ_rOdMkj>Ba96)h&yw*1cb%mbbLPp_89u=#&q(lj`|iIkSrQ;JBC92U*vj*^v?39 zl#q?QpP{EB8}ZnXpht;WLiGc&Ft4?Rl@+%E$$joiPgKS#9WWD{Ubc)*#B}3dD1{oA zWk3=bzsd;!%5a*S*rI?!6;Ao)S4OaZSfk|CIU{Y7RqZXx%$4W&mw@9>6KRT;(1|yH zA3uNe$^%}xt{Q!}f!F!NW#xn*ZxbMAGrAuz2g9*rvVkk)kMT(UI8}KxeNc)lG z$8w_U)5#NXVZaVJfb@TsBb!{xco#x#aNG_2p2whwh~TvC(7qs-QkcqK%3*il%9VbN#8GeW-)yJ>P2U#1e-8E00&V2wqgiA8=%H{ zVuTll5>UU%I}JQhxf1Hh^-ST-I|d1Gm(88?OVIQ%eFr&XR3xmSe)7m@rOMC+sQbD) z55LOal`Z7SuEG1~+b^VvxH(&gq5)jE<~dna#=5Jf5spI@)N_xQkRL7(J{ zPeJ?vCL;{*k2j5bH>g4FxgE#lvB#Ztq(VDZqg(<8n9CV8SiL;6aN!ioXv@7Id)oSl zG??tp332tF3j09?^fpof-F$52fYiE0%|1OvOoK53I?Q3L7%%~K*IlkkiC%W=B@7e(s{B&8Sv16)!W(7I60T{vXQUdxua4d()WN%voA2dRzlUDYS>1A}sM zZo8hF+FgPLuqTa1+1&w70+w01o(X@}9_`HVQ!e z5#zV(1CeeJ24Lw2X)q#DfAFOS)k3 zu4&74lVHgoV(s1*U-Ai7H=!?Dd(0L zZ(cA0N*<=hAI$2&c>{D1&0?M+2q92gR{}ZZJFHZHXc`pVo+s~*`lk{iUR>Lcv+i%5 zmR@b}I3UzWg7mM=bC)kOo$orf`4G##WuZK&L^K&!xJ83WvhV!%y!6xgFDoj0_SjXg zs649F=IwR6#f8muT70YwGO7P;$;~H1nPPddBNC15GQ^j2Dxna;()%EzRZ29?vz1=SCZqwi|(Qw^4#7|Kj>`UmLOSQ0v>P%&*RKZD2h?Uz+T)Iqjwq*8I`;z&guyRuS)!E}T zY@M$Wu{XzL&&ukhD`~^N&ai!A9Ho5xBcB>Na#Bymc(&MUX%LioeJk7i5@Ov`*mQOu zG5#pzaH3(8o4;&IgGMc$$kYw?=)2S6XhPUiW+MwcVoQSGCX}Bg98>SU;IpNP$JR%7 z7P83*S7pbW*nYdGY(dIOS&f58_x92aNB0+`c89bSivM_KqWDw+W|38%$rnI>y}WoI zp(A0tT;R+ZoxX4Gyim{d(avxTZf+puU`cNw0RFslZid~g-j8i}!c1=-auvG1n6)vMg1K@-7U zGpnB1|1PEY^S+c(6OZV!K%~*9Q2eE7AM&cTFW8J1+1UOXtY2?C-Ew<$Cpp|S3-YVk zux6|xw6DLLY!I2s_!dnMb%@V=jppOqXaDL`kCrvcX=qW8uJ>m(0q}AJbvbW>Riv6G-_ISgw*hgg!Eh*ET zDjeH$n`ra(JD+IzY9&`K!Bv{ZII|dpsKu&~?zNWl!sn#t^$WMuUXgS>P67?0 zHhYohiWAAYvLE}b6j<{r4xIjxE}I&q@zFU$zT)4=#BWccL1VZXFHq_+v|}|IR4TcX zO@pk`7OCEL0r@niM(7o8!3p$+UILYIx;r@Bl?RUDQv|GeLxon82a-Q&f6)z^+@yGgLJLzW>XoZkG1?S@ZbWuP0SdX$!*?(6V>;lgU1PJY`f-^%W< zWIHf&>d{F4ts)rS#PT6%XZg|_4lV6i=Qf|(Eb$lSEvqzf~_MMtTdMD<dPd-S{Ww~P=?@O%TODs9?{#Fg!pWIhlf=kGe zQ<=}-h#V>l4CoQ4*!mMh>rDDDW#~YPBv={!v9EOgwVqaQFUPR&@idFt%KoOx(ui;( zK2L7?^8?FUgNl>GHT2oAU-Muhsu5LNXAB{zPOR#}+Z7f0d4NF2)b)SUA|JE*p5WY=kY&m4vh??WC4MeR$u+`~Vl7TB4l`%- zrf)9Jku&4>=fk4>$c(4tGfr06bo(vcCm6kN(S2_Tq<3=NX)tf2yvqmw7Dqc)+~G@_ zryo<+Ql4x-XVZbRx84LnnJ|%+!hrJC;flUG*@P>iVsCn~v{E-q`Sz`X#OtburZS=* zOgx%TT9&1-ygtdWoWSz0Ltd1&^wY^ZF`-0AXX4MO(`x+<^z)!J*|iFL9LeQ>)ORU0 zjfi`Uw;eRnJFJy>D8yWH(2*JS*&+uZsm<!8%Flf5W)QMrCHD}j9k zallqb5E^}??xYf?-MRb&Wgv>#=H82DJU}PDvkp8MdaQD#y4CqeG_3k8jUb>wAaKMq zV!POx_w+Bg4ei-+m@*f@Je00+he8rLnXcH9`PD|vJI$-_y;rOy@B6YUKdyR|o-MW# zDu(#HD|BJG+l z7bbt$zIxTRR}I-0%#)V8bZWW80lU@Ny><=9^dksn|w5T@ize? z&nFC83YgDfH0)Rn^WC^~Fhu}~QS!L4$0+~_`6QB6C(W$>{sfh&#FK2mEG;gLpj$pA zD!8L@J|8RMns^HZLH?~^Fsk*nYkfI!m>L$A-GGi$nVunlS7+XCQ@Ga$@@#;qYPq#c z$>}T%5Rt$LINRGE5)3>@$Zy}$)qD}9FN*jHw%6i&1RDDYx7nFk=V+&0JFbfmVnexl zlbeLFiqsIFp7MEwi|&XgAdLS{)z)zf5jZNYVQnS@?2Er>!*9nj$epr%LUsY8@LeJ_(jj-_vGU zVY@8s@Nf5;711JE#>c^i+qpdcl39ueAMIsa<|EIV23pFznJwL+w4!d*Q+h*#U1>jT zij96=H=RBct(U6Stnr$n{9sXP*m9xFH+JH$Wnsf{G=OLbyIVvEtu9yIN7worRS|MmVy1CEL+hHW>f}4rdbAsJGD6_2Zf6*W=ZleF}i|f zSi&P852wcG34X85WM1J~oOu&pfSDmAOzQ*h$C&i($y0l*Kk5@bcbeTkm)VNE?MvsS zM@fg0>Vgz}(eB>|>@9*@BJ6aFnkz_rMm-wYP+FIma}Sz}L-r4I8~ z&}bs6+JN0@ASuRIPq=~Ze2n#lu~Lx<4V3O2(+%y+2cYWP1lg^EIF|(qKa_h*h^!JT zsJk>(JdJx zwDMG68l#fUbdmH@>^eF>j(fTZ(aFZHX`USiJhfG#GL<;1z6G=_!T@k{T0X@yOWF6k@ zFC@_@rE?Za{W(;N$On4GCXoXDIt&GR2tEs(>yN44Yx+y9EHysLI(dDvflIWcdoMqD zvQWM~pvJjHQ6{LxuKg{)Nyvv(jW2MMY;9*ogiGSe2i>L5R9a|!R@CtS8fqn7?3f9n zDl@<-CKso!Tua_XlEGP_=@mJtBLIxSoTrBY#do|cPK4qbuIG*q&dM`umd8FN_cI#R zc(1nyvfR^{jAC*sVl%~%kU>{-+tt;i3#(NnkY{eDV65@4dzPKh_xXuy&B<0`ypnIJWq|KWAzGJTB~FD z`>(>1b{Gt=U8rU#ih&Y{Ail0X)x0)Wvbe({uo9fSV-V1#5=2AE;Qnr~i+13BUI~lj z<1+jtbc#Z;bH$?H>mnfosmb_7O4oWwKj>&fy>i9dN;a#~pU=4FWRYkl!yO%E>nvBi zMI-nnJA|$fF<@phzW}Zd;Y>+nO?wh?lU4sE!9K%A+7^x8dxDI6gylZ5cY6D>$$e_Q z5Sjg3;KFE%f5t@IRzhQvktrM8vUu(pz3LacOqALK32_?dR0vIlwq1Bw++2y@K)2dp z^iEt!3hvdg^Cwx&X5g0s=x!mxsTEGfBodP&M8WjUYMA^xK21 zinLAQR8}?65vN+MIU;$-YRgykb^eV>xGjxAn--g-H$-NebV*0#NaAy&df3e^{T9q5 zU+`Do@slL2lyp36S7i$j0RbxTkJXANY+Q*=aSd)&gFJaHxlNpWpubg+9o6^kMgJKc zLqaFxbB0n;#RIXky<~Pb+hUV*WOk<)y8i;r_=XR*B^d@MZ0yxRBHvgS%TOhSAKsr2 zEMG)?)`DQV`tw!+g!UAOfMnpoe+WA&<{0>-cW0|ohe5TIp`@@7hn?r z(VQi#$TAMw+J6&5Et!%&#GAoWsxVoEmJfBlPMR9YOeM|r=pHCkbuta{sqDZ+SOp^ zH^0v)0+x2T-e<%9@P?a(xMb!jeX}OXI3KF`{=4ZA||0Mz_cUa%86)?{Inw z?A_@yTc*wL>h}n#HJBE6Ii_|C>Q#pPyCv>hSrB_!iX%eZ5U0WHAaWAYd|AmULS}bR z3_)g;pg6`H%V}RB<$dv@bz@OnJPu)E7LSL@}$$b@vD}SX-U$6mz zPxG5bY80OAk%(7t?_cK9x=gJMl7`Ft(+Ca&$uP`U<=GqJttCb7In`$Q*3zAu=~-3p zZqSshHt37>?FEtfkg)thp=Oa)n6-w-uIJjb=OR)Eo>y)##yk5JS?|lt}FX?1OAl^2WnUmrSh??8QupteHE^lp?E|vh> zgGzOzDh90yrycIkLF5kL;G40&DC0enTi#~r0Lq)Ns3-(Zy%}xEennSXSqS6D^UANhw=znNP3IE$5hd`qi;sJMua$#P~3)cgz+z(2?&D z$qa35lU?l;mTw5LLX&3PyTv4STatG>sj{7!FYoV`o;of<0uaZEztl{ZK#}gz=}lJ- zBq$>Y(zW4|StQv56-wS!m-zxwy(zRdV^`bc>XnEpe!(gHZnNFuofWeVs3>0?_r-2J zvtIqBO)zJxTT|UJN|mtk-P@eU#oaPiIXpw+m?oOgD(9ZBT{jn)c0hVNqVJOd>#*Fn zM-NI9L~1^pmmx5|XmI`FEjhM9V6Aq2Rol}4UPskD=dmk%hw$p_i1EM#iuq!xp`!6f zZFgdemEkOqnq7Cd?KYH|6Ts}@3T2kj{UGO$UPom2(X)|ZsF0;f>dKvG>D(Ti4G8*d zHT^b1t96;ER&KEDi-8=Ls&kEM)-`nBS1ZU|TW*b6B7Wp{F}c697Le}xcD3RL+Mw7{ z4zHLwPNwgoxpKzrn;WPIO>ndqE`Nd4Ub9G^l#V&e&&}qBAVgK*+-?%&|FV$y1n|0c z_xnY`yN+h}GqGRa7Z!E`+WRmVAxYneQ2QrgKkk_ZJ)fZuY4sJ|Q0}5p5ggQ~^pSem zU?IwiWBNFT0sntoNDco!xE{(m`|PYW4P!Y=r&ItR3?>5X;HIU?GmQ5qx%P~^2uQyF zxq>?I1v$L%`IwXU)MKVLL~8g=l{KK)cIKEylzO;p>i_e6Ie0$aB9mKp_Vhn{83R$% z#Ovd_id2DbXU$!yGtx&CUJm1z4G^Zc>aF$nM}TBkjkl&A_ur^@Ec#~4Hy+)%n`+hk z#o;C90xhqI);PO1?CN&?z^tRO^fZ@0E zZIIM64Dx;`k+!^S-Mi(GfT+5?e8H;h3ER*f9aJ8(6m;PJ#=;N=1k|krzzBv{wbr{c zAP=P%Oi2axIae-NAw@|?GZKiDL(JYrksr!yUPo$_f;KW1SHi3~QUddmC~K*o8Kvv93A%uoDmRw;=4{HqD>W6UrYH5a zW#cbAm1x52F)}kP(D+ZwS0JI-*1u@faLTAWy-UG zj?|GWm=)cAj+%Fa|1}7z=|OeS-FqFH;$?0E+Ji*yzU^%~&QX zgnpr0<|3+4bhQ#s7hvMCq%RleWL=%Nq7>EZ;mz~VDIS$fv?(>k8PM<8>XRD6gR>B! z(gdd7{BgxB!k5$wtyS?oLadHvlQN4?q{c#mx?wh?t_nnQ=(>O*K8IJrc9Ws1m<0tZ zouwOf0}eLDo0mk}m?G>F`u7E*4NHuo1K?(y`YDrMGrmPC})#!$93+_EIEfFSzO5lhj%aP}yko zXK{MriwOw=^{cK*e^r@yERq!}R=C2Uy3+a(L{IpAN-^lX$URQl{sKg<4pbWb4eF+7 zi4sq(Moh1Jd$zx%rBA+(A#u$`bGs~-GqXvn=(&!6_SG^1mkY#rah6L@2jx0FtW@A& z!?bjJq%54RvP|y~VU`P<)$N~Sa#j^oX7y4GD}@<|uOpbv-}c4TF&6u!<@z+M)JEwm zbC5m<5~M3=HiezlNL35a0tT77n~m!PD^&%ZVo@Y?MtZch3K!hlE*(SfDv#t%YoZ>$ z5~3&dH>Wb239a^uGaRgOPUE$N8*bLNKNs+jY>HGEu-j_wc&8r_+m+n^@P#Fz>G{$p z?1#d0G^W1Xr(gB7()Y)x5X-VxQBDk}(D2F>ej#^u13O2Wy1@>w+_Jd$+Bb7xBnzLf zg{lb;g@m|gG^|yb4#M0Kn0oD-(PqzDG3K158v;BcOGUdPUZB|Y$E>y1LAzP6=}BSz z_wwdhm`y;ss;9Uv`YR#1(vg1K5F03E_bk<#Ur||ABV&`dt=c1ACA6T@nZLuR^V_FW z(1Uo%(}E;_h|{5B^I7P1^F4@FjH|&-!z5R>J$c?e&~Nee`jWPfOuqRw?hLUrNDo=o zp(V3XfoKe?e8tT8(1v=iu!U>6=QkET7Z?45{R6e$h(#cDR<&_(BpmQIi?SI%jiu{`UI2R$V0~2|VA`{5O{0 z4@N~AKu+pZN%Sclffk!Yt{$W#Hnz#r!1~kST7>D6XiFBB0XDrByg)OrTGi__+Agyk zv5R(GpMGHf*(!HYKVZSN*Jw>`*^-jTF#pW51AMo7E6x{c2wSc^y<%?rgB7*`g*q$q z1PWX<;mz^#ZSU8royZBE)3)alpHf&ZWI2u?AoZn^`XO~q!SmXL+cnY9@A8oH74knD z&+Xfx+(NQI<2LJ8#Ghp}L@eM_?#O$Xv{8k(HOskT%Ppz}y6NEQvtWNMqC`xMhC59L ztp!P3$m*}oikmY6kaIh{_c+C&qCJ#AJ)X$l`IV2OrlC`87Bq8i$*q%MifcuhGclVT z2xUI56-+)hD#76FYi?mo1ITjX%vBfeeg~aW8(`#VdSd~2sSzPo`;bCIzDSnnSF&0v zmMF-`hxm~#?S*vsAn26o_|I^$*~xb8`})<89#%Ti(iIJ<1Nsk&AF3bK{6a4&er5Mf zOGl&N_mxd1Wg9&O96sF8Ka_alDjzp05}IDFPeVW$@ybGX=(sszXZIf8Rf;M?wHtsg z#|10YqBDE#jFjs_&1P7h(?zm&E`{bc0BXYdy=oU0X^_Ta+FSrJfj~0k{P1FF?>}P| zI}gc0w`b{z>#yEz$N;DTl6x)cid?tLZxOsbbSF>*Dlgi+`l@1$m(Zoh1 z9*3?*6B54AuLWtBEvIfRj}oBzt+|YPPruLt@Qx;jgxGAKK8SLW-j9d2Xs3=mv2#dp zf?HNj`pH>^T97Oa6Ct|OwWF&N92Dh?^k^Z3ihmS3sQW7UL+Q+7Yx2YY(t9D z(m*Xw(a1=rIzPE^TzvC|QWG>i(FkdOT%nqP{^y0hzXxtSqdn%!AG;hH(SKthsDs&M zWj%1guO7o@{9Oh8Vv>r~Ozt{e7)7I7ojTtR+^tisRqswUmBt}Jj13^pO^v-l0>j?^ zV8}3^BW4n?HK|x-4G02N>rWfZK^(hNxj|b`fk)gFZ&jO>8u}u0T(m>+m@~2C?H8{) z4FcgD0=OHy>kmv2KtsP-0V3nh`K!AEq~+3p-ZgHIJ@>OGnc^^&ub86up;brrG-8{R0* zF#(eS$VN9@e_wCy*K2|nCL{nbMFXDE4c{AT)RZaK<9$xrxllox7+%uGEO@9EVOVp3 zSF6e_NFu{5D3=+iRVQ2YdS>^EmZ+`|anzo`jNOcS#y6!xtwM9pO|2%W<)lo8Zqx}C z2f>>UTb_Z#7^)R1*aEh`Drm1@^-7xcm|OHXtn4s>Diad}NXDRZjyW=W<+(<9$9tHl zfRttqf%PB`1RKm2GWk3B2u%S<+=es&A$0)}| z+8otRuqZ#eaYabj|4wyraAjd8Yn%s zJb7Bab2MsKzQcKc5`&arh>nLU4ADvVH>52-3QP`p(G7ZuFi4c(WE%-BzL(hw{7Q;- z4NdKIsw8X_5r3-nVq>)(!J!xd2=+9HI@uc8RG-o>NvsA#YXnL?M+_Hij7OXLSJO({ z_P8S9X-#HV!`dd*pu*QP1$9@!AOr$6?M_&~2aIvZz`;xhG^++scDxTd0^>;{I;jyF zEq+}M361F1lbUiBTXwyG=0gT`8#0a{vuH~&jqa!(Y~0vuP2WtkB{R#PyT}?t7Bf&a zZ)G?^p(z%-w9`QT$v-%TuxIvyF#9noQK_T0ONI~;$TXBZ%D$s&+wo@a&Qg1|+R8aq1? zlpP_h1#wRhF`VAyb2s!3(&lQ8NmPOjz*diH8k!K4CGfcFzR>r?-9Nh0mOCRl0C%w; z1YI{;LOVJvWqYO1E9a)TVn2YQCAha>IH{RAYw3_3P9u;+NqriY_6Oy!g=81@BOSvM z*JoLk@23H<_z(ikZ3| zr+!|hdv-_W=LXTHm`WLd9$FrLg~XigfN|Dv&|#}F_g{|S9nJ+nMCiNH=JUNKOR#UX zuun$p?Q;#w1(#Mz`}>M(#m|T!I@`NrJRh>3=@f0vYzJ5r#J;6(u7ywo?B4FQtn6KN z>6ugM$>)}keH5@r@a~HV<#!-Db&2uI7Yo9jG%4DEa$K;y(uRf`fDoDzZEzXE&Zx*| zDv_13NuS6e$U-P&`UQ;{(a456ig;9BoIBfHH1zYgV`Gw|IqNxmCe>o4tg>i5?{fNy zaH%hk9(b-t9Q{m_u8fjTGLn$TRg%3GcBiHr6)XBdk*5W-9szR9cI55!*&B*)*uqoY zn-c{uUf1|49+El_R-!X|8iX=?kRli6js0xyMAQ#d0o{?%(mul#mMo>FG#3*4&oj@n z!SC#@VVFgdL^M>(ppeMv#y306P^k4exT{SMOoQU73MJ_3xu5Pi6vMVn5 zebl`o<1H*?Lhom-*I*y~r$Gg+P{&(KzSm5{vlFq8UY_bF|$iy{e+_7)-WG zCsn0j&9KWZb>q!9$D^F+^EmCEI076uvssUfz^j>umgJpYPOkNjiR=gX3kMWu44Hb=lGn?!I-IZQilHwpHNi z#9XHqR{7|bLqIF1fGN{Us-U7?(w|S4BStU;&!sJ@-pKdf(dYA>uvTPB&nd7>XDZj^ z%j>svZ{$c-I|8*L&(8RgqV88Gb7IwA?^23+IBN~P%}GZm^fEC)B25U}P=SdTGNfM z*y~LaM0)?LHDe8mN82SY?4ig`%JKQ{Y%=)j>0;eav{C?F(L5AGN1gW-r6FONF2Egi zbp1G8EnS#2VNAcAcSIr?QjIyq0QZg9FC=fLLqWh5$S5QU0$Y;Y*G51wbbZVUn~QWT zB_uDro?}%alkk-{3xekNR!A2>W$&Fzu@_EFW(nU+KtoM;uMj)A_r619reBNbo%09e zU+4y0{sDuv)*DenPy`7gy>g&#jwD0DB*c;O$TP)D-eUQIsoS!hG zq(MEVNRBA07q)IKJ=(fh)RQl*=4jD+l|`fsY3j2^4cYQTZF@Y%4AEJ*+}Cwt&itpG zFIYKfRrq=v*(o#4a`tqH(YtRudd3ASXejc0hE$;ij3hHuLti3(>Gyr%cFDnJHJ4B)muwdl~fRNlz^UCu?~xUH=wLGqtc4;$l<(n$8a2nl_7^@DhPt z=~zG;DTgGU+uTfq79u|sQ#Q9QKJZ|b0eCB!bVv;yE-+y@f1P9SStOr+0bsAP{AOfU zm)RWg1C(2Dt~K`|rM76{bBg%)3WZ7=8_*d+;iehx?Ffwis!p)Y%LteAshCIDHfMpV z;Lk^%A3IzCp*xPVU)W#wfRzG$v1%2+ZtRcq2~DadRy16KZ%42kJrTjsnY~Df{|;LA z)H-%r=qBES%Z!@c&i|8Sp}9=|!f_!)3#0OhpwiQVPPlgX zpYZr@@?4)$;9s{t6FR7jv!S=b^1Z_oT>Z#!U}*7MxM9Q@8T{c_5gC-ZwjsNXjL*&^ zoJ|2kj*Oae!<}_gF0H}z>IEyfE4hCyk1b4<_ot!)=w8pbzqUec{hdiS9CnVEl2 zqFH-jHUCmf7ppR;UEE+v+HRVE`trRMaZuQ$! zO`e5d4frLS3T0qKo>OGDbKb+#t22^%>n=oU$dBYnls|D^9G5>BWO0~*GbTtIjEYU% zK`CoT@0V80?QAUeN9>C*_6D(mG`5M~&6!tflB3pNBGa9>KCk0TL*SqVTQo8ke=+Dz zEcR89NL?XvW%`X}k{gja$iP%BT64(Q;As8Ag`B@Q;`ma;KrwS09cj8I2@C2_AaRC7 zfE$n;4-AILU%D*9&=B-nQ*SJWCh^ZmUp%CY?pkuIdkh8izpY{n^@~c~hI_epNMP(W zQob*d!jHegi3q5XiXi0`GFCS@@5SG3@x+QY&o;YXqmk}DWBuoMkr0jixqfr11OIT+(x@l?f&3>a! zSGzqy8bRYiE{APnAA{JvblCR2Kwz3NoB+9K!5d3FPq13@A8|jGGp1{ zk1xOyg05)6Bd-NUlS>~r2n~JPw#xkwfL^fJ}a zPvmHF#7ug)laLe#D5Av_yx53Fackg{S~~Kt!5@BOMUd$;kC8IRr#Fnz8p%r>hr}o~ z1}gx+M^uoFoF_&Zdr+f}>(`wi9x%-pN((;chS%gtF}ZRkD&~pcs&VRVF#^w7 zt+Z_T{=?NM+eQdG_`~m&Is85VS?qMs(wu9E0VwY|^F}q>posm1Eg4qki0#M11$Jca z3>cb8y+6YB(eG*b6a9pwg5mQy;h#2Z4Ibbwkt^rQuofQTus>eFCjlPLxO<$YP!wWYG z*BVY(a{XIZXz-Tc;#D{cUCoE)BpISOCN#$U1*LY^03l%ZzM+*NkN#CwpUV9SNj*gf z_@s98;8gg0G^g>-5zX*7U5fm^6Ct3bhRPrTMb>|@pMov|27Z7Lu>Q@QfjLK0UmZjF zfe+*S@yU?*?)j-CO@4WnZ!1r44^L%2zbwYaN~?rva9rz?jO8w zgn+pR0%jp!_fb@6&6Ms>NGb^-U}WcJ9prQM%Q)Ic4C~Mje^OxZ4oko#e028TD+2^v zy!h-S$E%OL|5Jp3-?DBX1jHXy{y4SQccsjN&+nm}8*t$NpLH>dkvU8M?iffnQ9jOD zKE|;hAkZJx4;(gNDMG+E<_nSgTdqktH?{j$zUzVzka{ct8LWP0@xsxJ=)mKDS*0E8 z9!X^*1dQgE)`JIkTyxbP(XvB7{853yrEf(Dc<%5!AmCIcx(~+-0Wk;xzZc92ndU}X zCuBUu*LS7tMF?1M4jFG0qY8t}$DjWXzU+T%QhK<@hE{N+qrS=*u^t~@z{0e{OL&l0lJug%y>wEigLcg5V%+DhwX)$lj{gJ8!N5S| zhJJpPwD1u|dBYU76vErE-;450XY*nasdi4|5%wDTVc4VnEtQ18wOZr0z4-7Uu5|G) z+o%5iaUB(rDK)G`2 z_s10up)=(?X2ms15vS*PCGj)vAHIL0;;t75Id_T^5IN}Ox_8P#Hn8{+p!}A(ITymY zJ*KQ>z*jy1&-dN<>}9&>m-SPo-@xKZSP`htA}vWe{eRGJ;b&x@2yU93;tH-D?*Y{? zdWjS73fSDW)kSGiP8gsdpwv4bG1ioYVfZz&VT0!SSjjF36s)1EXtTvnR?X69ejOC2Qm>+ zsZl~Tda~KhG1?5A3ne7h;hu9}?f+nY5WhVfA&SoU;amWW$JN>T{O@o53rODn=RgOy zhn-UjGzaehI2o^FXaV-RY3tH)3tPdDSiw1NRS&}Tzwn)3YPe||pkC(A+B)ub;E#<{ zT*vJS0@Ls-?EBYh!W~U&9xC>>Zm9BKVf8U%E+gAh^YEec-c;%fvJ1W2kvMV1qEJ4J=ADn>z;Nzri)z?{ctOKg1S{P9CNfRe zoS$k1r(t|HO+A_n8)4_lY1nZ8zyP2f%4_Iy2=EK6q|F_7&N=v z*EK?#V~&HI>+Ub+xRy!^?c|hRdJ83Gv{rcJ@bH7wEAEbogFunP{_35S(L8&3>qYl| z8h&}?iwuRhK90%052$A3w4+xv@Hl1Ur3{bE-pd={TUt7nPbeh~+1FP4s4U4^$i9Rz3}b%h8AkHWz4d$b&xrXf&vTx$ zywCfbGt(ZgA_%wR<_4hXj zmt-%Pj#JJgLxNJBb3h75;%3ca?Hc~Gpx`dn4-;}X^A^q<*abplG4lX5m)_vcQaWR^4302GiCI!#`g?* zYo;gRrxMH5Nt5S=wJL(x97mQX_%G*e^8J{rJfT|gCP))5_1Q6bKQA4alRSA3=7)cI zc$IJlD6#?Kqq~#eq3NkinI*ye?-`W{{8s8Z$Y9ao)5VjAXlKrMaUOp!)|>v!ws`FW zV`ApIk;W1s-U4>obSL)TE}*Il7w~pFE6@58%t$^s-4UJf2!bc#uwc)UqtyPTERtGy zGJC<~%S^1~%<=;lScA<8U~OQjGaK8lOD}x64sBql&;E%G#OI2H>rHOv_=kU6ewA;%(*H;vty zwH5!V!_nn0#li*djdTu7ATfdE?4PD!D9oGu%l@&ki!TR5(l=L3as0Kz9of}b@4{M? zf7?Gc_D6e`^ombB>n908rQcaTmGA&i;?7);c*oVP6z`mY=~L?eMS;Q&x8gv7(4(Rz zb`4Hs8S5zRMyt^OXz-u74l}T9O)?~|^Orx{p8advR zkHh>hbz8YD-~y_{ZEXtZ6^hi@PX68zntf0bJORpWeNS|0sUvJZIQOtj|7L`kM|_PS zR&YL(uW~fbGmmv}C!Y8}zwImzx34@fAL-s4Gaksr&-|{ey_veE_%ZP1O0&a`BN>EY z%zGD%ALmI`h$63lQ|vj`i@_OOQMv_U5d3Fx!T-tj%@ho=-Y|C~ctTH=_*7qpK4W14 zbdbZGy?>Qj{qEbWhUGBZqpTDPa+DLpQ#yV8)_=VXm>eOlXAHjUXf%qtPnBUY}jXJh`uv(XoFzw_{Gmz`L-@RRJ5^aCIdf|7X5HwvDnis77B|FG=k z#QHunHhcdNYw=!a1qbVLRn%2SN3u`6W;v6*E7_gD_oED_BCTul2#ts04DhUD?I}xc z{^PuX*0lnGr*o4K;0(zndDa(M+c8tzHF+aIi$UwE8VAGavto%dhf%nozx4FD%<{*~!W!~ama=$t9d1Luds|2{0?IySfOQONicOg*10nfmb*L4F0C zpXA51d5h0|V?RF4!ps=GPrM*(rCAa@L7;f&-@;O^b9Vlg+rnJ#oCXvEo+I82`;+%L zJ8e2e;@|w%aY4&j49<(xD}nHCp1>{a2Y-JF%(lWC@Gis|+?XA)zTx)_@Ax{Hfa2<9 zzpx4~JO9e>c`oDY?e-^V&gJ43T-^L@nBRW^PnB#?4Xuv*mo7OM&))r;h7TRboKAQG zDVLh(@<#vL$^E~#U>1J67--AI$}5dm9AtO?{XLw8zb63~c&xmgPF%|V>G!1&*fz*` zi18;1Mv6B;wsk@YjeqzrUCnfHHp5B$n`>l{>&g|D+c$Ae|N9uiZD>MZK5<6#7cFC> z|397|EGF{784<@_&#-s@rt87fsQ4;)0yEj%MP$o=ruXvrg)L-b_d-6PnY+Adlnw3o zH5sg#stJlCJuYwvcrN>GRf<%}Idl;D$xa9N$^SBh^uu1IK`4H9-?J{SbN`?FdqVCJ zlwt9S)4ml8ew3hJL=~iFA&WGN^7=rV$^Y*H>F@+a@=oHv={t_UE`B@m;Y|5&tNp)T zKc^IF(g#+p^8cUnvzT`s`%8JOS)k&PLqEEC_(S4?=YC;1#zadigz$`Atvu83ugiAk zR~BS9es!Qw>Dzfrr>nqA;hqw|;2lUUX7>`snhCBsFqMLjREbo|?15hisjAjV6%uA! z8eaH&`VqDa^1Sp5yTV(;PQN20m9n->Vfu?hs4eu|BUFZxq$!YN^DrirVVNpZFQT1?Q| zGf#NJDVeHDbFMGnJAKsK&sBS%I6MK4cfu+8;Q{vYK%)5v z(=vl%ka~qKz;sG}xIoOGrN3sJco)JOj5{S$rA7+FRyp-&c)ghdJJSiLD#)*O#9Do-Y}{(hL{)%yLT9Jac9(Sn1Lk zZ}IBNpUu#2;r(CY&FAWzTEtV8UeB33`7nuUDhiW7jDMI3`169?PmZAb*0f!o(0qj1 z2M}5xi$H^@a|SStrsG-y`I`qc`X+QxupW~yOWUWS^NTl7HHI7P^*9f?5S82PX!=(b z^!P$E*mO~SF`=>3E~I~3_qSc^CvkG;WvBgrux9gSO8uPHM}~6QZ3FMwmm+f_l_*it zP|jMByMCLXfNeFqXC zVlOL`I>T=LG)LF!1J_hKBccH@I8Ppl&93O%C?g6ks4C7$XC-+C#2TiT8qRbNaFWis zH(4_(qfv^6dpP4bWMGsy3)oP92FZ!05aVC-EN8p}fcr>O^-k`mokf8B{a*ZcVa6wMU17-&OC!(dRpYbJ_k#nbCSti4vSm(HBk0mI2DfIjz=`6X zMcO#!1}bs+7}JZV#22(89Us-Fw_VjvRvUdz7sWx|Ke3Hkm$) zUx;h8^2Pe^y1r}4;^}<{W;&;^JG2epirzN6_Fag1x!Gx~&1dc9j7vBRk4Iv-nAH9~ z5e4MP?(aYS=Nnj~)4xh}o(etJ<$$O)Ql<4rmLKiBR$`&!DO^CRv3vB*$e z`NHz`bKenx0BcW4C57F7LW-!9`zGZ7O6hpnV_RrNULr;C#a5A=vr3*-(jy{D;FCr> zIhD^Zn$%LLSMEEacvNY*)j4^HL#Gxu>2V6@0+?5@ybzaP?&-Q%ER?KHd z1LuN3VFYbGbJzf;nsrE65gS9u6k3xtO{U(p-|t)F+W_codW^(xu0p%C;*@?;m1VH} zr2#o&G-)H$=yxAzmf5RRePFreyQ|Ma#fxw9$wYM9@l-+=6vI;~pD)pjzT{OHsSg|4 zWIiA53&<98thLuOmS~Ac7LR|mr1_zb)?Jf3(wOR!6~VS`rnI0FZHBau)~QLY_pg85 zpM45@aA+V>8S|J&iMGYe?++|n=yZ>ILU_5CBMhz?d);^sy@>{LZfqjz}AkkE1(UjK;6v4+IdFxLir6qB5l5?mJZpo9BUWoRN98qh%&ee!I7CJ{LMK`_^= zW6;)A%ks1vXGLV_owd9Dee;LKlWP*X>vi;ub*Rzs#kKKGbtF>3<9!bb(4<3WQysD? z`MGZtspa;ENZyAf*2f}c!UDInxExn*w#yvxm-iV!?KJF-3pUPf)7HV%JJ8-6X<$OT z9869WoUM=4(7~AMeQC{48o_*!*D^+}C~gNF*{efN*4ELvYDzsHYO!SZ7i>@nIkBKc zFA@3I&C0l2*=i=w`>uX^s zWwyE-KJ-^nyORY&yj5>AyXac(dKwdGOhJ~9~>Uv zKV(0KhroXa8vfvV?`L+)(rn6j(VI(*q_8a54f(W2p!Z4JH{e# zaF}#48{7|ewYYYlObUDg%`WW|R5R}OS6MZCcDv;dN7NqM7NJ3XlAKiA)8QU6L?V%h zhSU?-vwQk-hrTtQxV_?3%Dq?0(d-ZD5w4t#kztAq5DREZoMxWm-r^`h7OM9NQ4cB^ zx!a>3=Hhcf%CabsAg?V;whca+Bm!-Z+!v}ZStFml$Vpd3pb^?P#R{c9Iu@CgOWeLb zd@2|12OlR92kv}lVx{!Newd|jT?t_68!*J%1y92OluC|PnLs*HEiS&fHPE62*QV5f z;dmi)#AUvPr^6uXv8m4ayv42?hcUE{|0mSCG-jT+bZT9vOsS$*vbKjdF4plBUuh_w z5!%w{1cM2Bj!V9k`LxXB+-K)47mB_RbH~&rQqA-I^)1iZ@ZdhwAEVeiQyzwL`;yIu zxUcn?Bt>Rz?$RoZcUiTq`})ZO+=;>l8h2?=epYHrY}n+oK-zW&}EbNAYmm#zGrE1!x*t0Am#oczE<17gD3JN3jC?tqM%f zeZFWB{UoVicXj8VrL5xG38ZwKnFmHEM-V|tc>>A=&;achE|&SZ3vIh6C+G0ygD-AX8%~%1XbHtZpe3eEiZR-c zqxzRPxY!n1kJXB{W<-uW55j20VDnltls7f;y%F^41HBox+bT#gxWuj)Ba_hMZfX^3 zutl&wzQk@=sY&5uS-qmqfJpS7hl{XDM>7a2rB2yox^OuEOnOrm!nC6=wf|y)7G{ET z%BDcGDjl<>U0mVq(Fo*x&#t0b%8B@hEN&s2ByKT*O^0{JMUNC&DapJp`o`DbhB`!t zS$-kzd&F0y^gueX-T4bI#`$bqa7X?ouP-jfx!1)~KwnjGEekCdTD|(}a_l*a^qlA#Z3W7_AigFaQmg6MJ5^pAxU^_SfF;B|PIo&2rAoKB7od_V&T> z3Ehv6niDc5Lt%Dq{@H~wS*Ydr!4prmlsEKtO9xe95}#XnpR+tm<{mDT%yu;vOMD|I zlTx{U9iuHnY4~&+qXuT6Hpe8E+`X^C`BqMpzvvoBJYAEZ%J~-e?jD~sW8o~6oYLwJly;;A)26#<9v)W=^`Ctr>OPY1z=iQe7L@ zT?LLuhf7}z*x8Lo-rg#H>#UAJy@WIG<~*aJVFiBO`CP>+{b___quAOuZ3za9JrqaN z+Cqiiz4F=qtqm6&Y5gU?rP~nK>v~NYBp7tN8+b)_tq)j>a>`=CtirdK)nu(rB6f`O*qCT1d~{EcGZ!%2((9&IDJ76Lv_W-zm`ge~J-Urb@6BrGDMk*-Z+z!-AtzsLX8am<8tE zZkEGjL8@;azz4*}k|%unp?qH=w>*}Q<`hdTk_MOAIfB8>X~P*G8N2Do8((JoNp;S0 zJslnCepeYYg-X;&%XuAbU@b?AT(xx~!JuBIVFY@K$Ml@MC37?b+>K0M0XPEro_!G> z#W{U7&tQs(|0Dl{8wCncm!;%8UNL)0F?BVL(2c9fHh+Xz-Oj=S+}e&L&fjt4J=I9V zBlz5YoQ@_Rt{p3&ElYVw2d#L&+oqt!l`}IA?32YnQ-LjqV9eZd z$E4t_1LYpop>>$d_$MozKjUaV?vFg$th6HQA+rw-e)H*TCt+@o*om0JChku~&1K4`6dL^pUZa1{5i_j!o=E*i%DEd}P<7L7{;vnE4UOcXK@_Sd0| zI*N@!Q))AbN;kFiZF_3L3Lfh6YwR{`&!;9vrH@eVImJ}YDCV5NQq9RpA^A<1ENH6J zJ(Zx7bA54Mcn8*JonNzKB9(j!zjQyh~Vp@F-< zkl^M@nTGV#uo20WhV$<;i&|hXGcBy!o>JBma-v;*n0r4ivpBU=JK|V&mE+*rF2}kL z21@o3w=C7GO3v+1O7%~uy4zUd){d*cJD7;WM8%0v!ET=Y<6(~puHX@PQr&G~C4uiv za7o9KtKJiRPqeq!MA9RS5_?RPsIqQtoi0?{C^@$i?O5A?V5i#m41YZsX%!2%B7zj} z-`#$^yDO{S%)7H1oXmq2MaANPr~ zw4uRkkpw)}5LhW~YYwZ^sOudX_b98viJ(-zD^KDWVI_j?TPpM)XmO?vyjL#_+;k*@ z^aYJBK$CncnGU`Ao55HfrNz8L>~PFAYkk0V&CKX9?%CL1ENAMbXy;M$KWrBkv%+i$ z87L+4!H45Zs^ZA@7JZVko&&S6seg1-4HVldOC{FG#agBbqy%*rpH{xd{}HoyA;+e| zO!>C~5aw1^FV!>J$9cG;BfCiEit}iblIGpbQK)4qrY;Ni)DPyH+<&N%D4ZJm_J!6G z=&FLXg_W(ZbhU}BTDp8j!CD-+1CHHaD;JLc1wLusmG6ymf3&kK{|OxZ$@mZ$HmJfN zD>!xHTY;JE2WQ;C2|NS5R#w;st}5pqcz{{CqzUXVkdKY}J;EH9&VQ%*Bx)nSx2$rU zk;JRTe)!iRB=m-yh?TdUapf#xK>>#4&W8DQ1P6WP6RMF|Mc(%pb9>FgXP5md_&UC% z+-pIuU$!1vFs}MEL>hiGyDx>vxgwaAOg|RyFz?Fr30Evv>E~N0D8&OvoF#j)`1lJr z%soSrZma8%%Kwz3=eQha3O+E!~gi-$#Ihkn)N_G93nu!`CtWQ9(-jw z`hjlj2WNnyrk<@Srx5hJ6kLE~SM*`#3;cSW7<0>!U?u9Ib4aN~nfXiCJ-7Zc5InI) zmpowwR`+_JMb*7thAKT%;eEbaYmniME@EPL%gD*?nbYh7?<%4r*yu^9^8CM5;BM*yZTl2m2)0S1(s z?7GcwOd{R=>~AZgEb1(OswSmv&`NjkQFdeKK5KBd5_uIItUctOD6xN7e0utz%Kwm8 zB;;^2-mTSa(ZhLbZ;Sb?)7*$GHiB4cwFbA93>(GaO71Hkd2=IbBoW%=my4fz_HI`* z%rxp~ufM>>Q;>IOZQ)J1I(<_CHIKBnvOO97{s{NDC09{(H)TX2R%nBHM~`#4+?6?J zf0L;V4`+?C6Tt#1ropBjbP5^?h^ei%#WY749$jW+boi=p0UD8-Jb>H@9-LzQSWS{L z3kjudiGhG7xigWhL_Zi=U>WCG@w>7(k9!VmrRryq%+j~RUxVXTLTh2+k>y&HltN0K zZ^yTx%eo`cvdJDh;Z=1W=D}oCssf-fl*;V>unPGChw;Et{sN;8>=89 z>4CwhxXMZrX`|+F$w8S7St0;x^2wbtYH8+I#)WK+H(!c3S_wWPCR~Y_Ug>ut2%}sn zy+?_e3&0US=<4V^5~-w*4T!ebdL+4)qynmctA^O08Kux{`Rq{`om!u*2=RgT(yls( z4=p9krIZ#QN_nPB4+16Xpgq_EA^b&77-NV_ zcdC65cN3a+UHA~|(>}6JBXI8^$P(=!0O~_1VNjueLMyD`N~5dUPAHnRc)FQ;5?3hH1&wdKlr#msja>c zrf3re`$7;lPspVKaCMMw-B+}K@cwm)jnKk=XZ~-?bGu1UwZt$(i-yXd;6_oOBs|L`3__^wE&SwRh9NFQw^42EnSZu05CQ}{l*!*rzz zyoJ;-bT8IkBp-*%=w0vi1}1pb(~bf-8ATn(2P0YpsgYsgNgwZP$?6|_6zA;y%9A_BrX8zr zb(2>kiqK*MY40n$?IX#Uv-dgX0w30i;wgGarw_Q|!Vh3WOl zfEra`S8UQEY^Sbpz9w$c<``ztUWoZX(`x#(w|3w+{TF0if>0{X?v(16Sne=#_}E!l zuVI-61_J#IPd?Yt(zGj78Rou3R_QA`OWfYn`1W>OAsUf^FoLD{kh}&)(HN0j?7zL0 zZzoZQE5Sm?>+8Wmq&@W(54{u^+neC5g5#*sj;-2t^KYH?>zcBjCBK+2S|-ZX0si76)?#Nvi=?@-uu_IZL zBl3kcuKmTKmnkh_ht;#KzAEg~+q!kEbq>?yOd85b)RT3Rka3!(eK;T;YBuZ4T-EZR z*Ju`;IA1R#mFyO4;Iuqqp+C%E-#iTi* z7Sd0ZxVSZ~qvNuik>atsLvRjux2)GTbs@Q2Q;V>BiXxR5&eF_1riVM+Z9MpKqDd*6 zPtm`Hwq`3u|F*rIg|9;SPX}c@+bLz4#vT~nXD^BxOdFhd^1R((xP#}zkWVfZyD{%L zyMd0N*D!>c!)a$N`+Bl$Row0^@{qowBs$AH+?kU~{ahh(ZC%np3!FI;+3NX)vF;Pb zX9UKIHYAmd2_IW6I(s03_UaR6_3G#?qkw)ru_5a2P;rZ|CYF^~_UugFNONyph&-q7 zD?T;Inf(80Zrr5{$mjVMXkxTJIH*w$R7e6~a#Vc-0y}ErPCLk1-SSel+FD!M&|P57}r)YVqvNk zD3pzIb?1&iQUN+(d*l0{-8(g{W4`GtVhQa%xjWu|u!o7f`|q*)kWlm-&BS>a>qf0j z7CN)m{_EEKl_-qIb~?QgsZO;YeAMr%6zSR>T<5r^**Q@}b()(qO~s%EnWl+UT#&E) z&T1PZoCyYFkXKl3j`F%^S}KZGX1`u$sA0Dh&#h!?8DycRqk}NmjeLrALL3j77@Xys zF>dQP5-A2yVquy0w^zu;c4Ya6nwZ*R^5<4X)@+Q9*}v@I(R(Wa0P2LP*%Z$qlzD7& zmUyUK+o)y=Dc;Jo+E<|lgNqb}~P`~aKhIO?5Q7AlB(*L*J^0YPMmO;+P0Gvut%2VjINJ(E>l{uV%2}wB&qj6EGR%sWFLb})`BRHJ0 z+=jI>0+VW{NAX1&o(ezS{(oZ<4iQjDg7#KNYEjun3g3-|3_Rc_z;Qfw)2?;Gd-P~Kjxqtnh4x6S`H*qx@hXIml=rfO^+ zX9Nd=qxh8FT@i0$8+b`B*5=R4bw!aIVZ5$6VHM0{!1&wuy`IN?%f$t5xDr!7jYlr^ z*PH=~2=}M|D0H;J7e9G(MN&4Hs)ophyJb8tNa;Gr4hg=8=GLx*K*OE!AA0B&y zVBrMdMpW={YbNe}dl9nTrp)6DV_PiqU#YGNNK;z4K07wtZ9+?4iuWv?x}s8>k!HBr z@-Se=0(ZPSPoCKqAS-5PKsGmSZZLShixji_3Pd~bc`yxp(a00=r4(Wg%Et|UIuU0J z(;EX|yv*Yp{FpRX7z-|f@)mJ_Bl}d@A4@uZm#7`9w{+0%UOi)siUzGq1{gaSIKkla zwBsI-%q~DvT))EQ(#QzV##W&;aWe>M*G4%7<%a?ws-zJ;9ZdCCCk#++WXg*dNM=0k zgrETFgO{6oZewxoA2M?1Ojw5;v!$hyQ!T|a0KinP^GEERu_hmC`(V__*uJ{Kq+~w? zJgx-CLs-VwN@Y+kd$`#41=o6CL~cNMW{$!@qW~kte8uBSODg}|y>g$W!`?v^O2_Lm z@P=FbVZOSuMcbnQiR)qsd$9N4lHCUr6LQ!W45WQR)2L*T*rdW1w&SUCHK$cG|IS6o zKN{RSm=-%Gkpk-+U^@C`xdr$KT_kz>Xs4C3q1Lu$Ut3VC<@p@-LvQ2?&?u{0Fd;YA zcK9?(36g=PI=odqsbtV!|L|4Yx2qhPdP(cAD<^4T;tYBkOp7gqa5A0t{N1h<>`n~N&48gD81gIB4lr$CplJ4Syx% z_HgJhIv>kz$_#XnF~f-dc>^M@x5o+Az45o*VzhW$<8!RH4g=`y3&CbOlcS3!XI46H zxylYcNX1{6w6Awh6qqt-l_Dj7G;OC20#|@%tI1$OfaW`E9UTiwxt=7_6mfcPf%n;2 zJYoF3)!_kC-%_yD3r;xy-jKg%?3=*71&`=ZXaHMAk6Vu@WOUvfs3pSDD?mvb@_yy= z*Y1jE&HslGZ(I! z@Fl0In$GeNAcK(hqfV7UeL*anYvelSJe%YgO#ocuHu2ehvznAM!d|g*RpvgDgG&aOIVEgDMRj667-bfmgafLAF4v}1ozxN+c5@lc*~ zmnlIETZm#1Q}P%LR)1pkSQ%&#%xoNWt=BtXjrXm|RevcJWU zNc^+vV_Lnp$|$&T-i-`>>HH0ZVS+uaxe8(>j1n5We9V1C3Y4Jj*me6DgpQY<;O%ow=Xyal? zSM!=MXP94$rLuXE(Tm zjm#&6S$h%E`xzN6m>!Yr(Pqhx+fJ|dVdPu|eb0+rfENlNZDgt6 zf;4i%(3k50FxoB6T#$0;@~Vm>^A?Cd+nWkOYXJf+-q!;yDyDlkEg_e4g|A%Jx0}rh z^}||0Hd({%;`!WYx>F&r5A?>U9*pFbjx}eRso4y-R0Lz7-=&ckVFvbAoq4-TeXy+u z>3Ea=yOm`!BUU+%ht&@6;4GnjjLKi%104Ro*OKn>uk63Py4tLlP16~MA2$uFMRnY? z=e+s0=kAHvsN&m7Fw)c66)0o*{$zutYrKZ7A|&k}LF9xVa#D%|lUay{F!p&cs2=wD z+;5b@$Q7PS@6u(?ng$0uoxe4#=dEIYk@gbpC8E-e9=Z?m8YmxEp3Y%wruTSw&}-14 z>1)@^P7~r<>(zB+_$tozzPZI!A?#>Bks$^TaP*|9ocu6Z9a!?Vi0}o6H`e_0SaAL5 z*My3LC(0dCJ#6pvp{ZC#fCf_fOYF$QADdH~($9gt4v0}a-J3hN^6KiVJ!sC~sVB*= zMqvlz5&4mFX6#{4lxTguYs-ghuz-<5n^rfcB|Cy)|C$v01nYT>R&Rf(?=mt2odEby z9Nqg4(Q!y3jUQJNUu|2PeJXkDL_X+Jkmu>C?uv@p;DVw}BkAFD#g{MUpXT(A^Nyu7 zW=pG&!R%ENa%^3-w@mm}j#3xHF>xCwbqs{Y8_1nhL?_Ij)eAG`pELav1Y6L-~4THjKjDFA@)@?UTrpyX-Cf z8Qe8wvvx=s#;*g=e?5KYq|&^|dkJ{HASPThUZ#;b>{0JBNVfCt-lO7p<~Lz%oe^b70HssAu3UdSs2c3kR)U-slc9 z3oE^c9DU_;ikMh#}ZBy&Yy)wr5W20WC(#z!Rrg)_Qpx?(96BGC~7Ts z0hW}xV%oM1 zb6CA{CLw_dOujb*ND4M-Z4ts89}>ym`t>3havY%n5LP;ek8vzgl%l_=jdh4sEQP?b zs#Yd1%dSwPhc6^_=Xq}DlA;cfVk^u7*W`_nTRSyF0WM;z!)km4C9cj{W_5`!vhi;2 z_U`=7q4goBcVK2QZYgv>_TiMRJJ(1nA(7g1yD8h>61hK;XHjFsyj{K$J;t2Bj+T7v zA3h>5qMvzx2cW>Dv2+;085`C$<}uddQGm3*$;hY5isr1|i;`v0eCGZmLV`kJMem7A z7fJ8Qe6RrH9vT04eF!p`k6V1b_Iy&|{RNDrLLi2?Vt#P|v ziR^Mh-*#7dI9YGFOH5CtHm4oOzW;}l;uw1FGyJ)QkEU#to)dtA3)*+yiedi-hztGV zk`Hyw3PH}1#)sQQ4OXqq=fV>5%Sj})C4vb{c9ygl*(ALRXt`UyEJDwHmoB6WmS=}` zJ5F>P1SSOS#{~V!C4mn#bFVxSm~-IAU(B925stTdnFnOPr}WQ@JW|Y62GI~Trfq@4D%jUnJMumt>zR3jGiT!ZK_O*x+0nMz5A;uf z=;%pg^uBN|rGr(vEsa?EPEh>x$^-bd<VL93mqNi)Jw(k@ z|C2>TWd*bol^D>@LAoI&v84NLXl0s5iApnKUW5QZ+)V3;>v2z28iN2vKZ{l#=g$o} zsIwzJx!pFdjVshG?Z`##$n+Vn;qYGscszk^01dcrA3+0%32e18f9oqfkVh1=G#t2s zM)?h6;P9kpG>hAa0LC6rh_Y(Q_0`HuuHJEWBFrmomq%JH>Qr$7=hev6(`bm@v(O9Y zm+I=@6MLG&P7!24Ml42gq%+Yz9{FzOOFe}SzlyahOyzc9z=knrQ#+TAqHQL)NPoo0 zyyyXcTet#2jYd!u5TN_Cd*DankW>j>I z1G_{YY8(m?OTVu(sn(%de$(1@B6cWhBhovqm{FUPf6#N~7KP+DTUrPYz5TfQQmbrx zg6jq%52mMFpjE6;ei1>VuY zb-fCgbH~(nOl)N-gZ=n4;sb`OXy~K|&T*TZ{+xm9Q=wxBPA0UOl;Lb?pD{2+CctNV zN^ra7QhkTJj+vlYb6spdc|IZwW}2@XsUf=Ex((g~VS6ob&yaV--&w!SOvwU5h?WGT ze^C#LS?A3* zY``Vxrf=b=V+;8MuL_S|FM%?yS>RsP7c2)omjJ$PKxM|)EzRX|?)F_ixmA3d8H9k0 zhv4CxlEWz?8`~XvA=Z1~+I7M-nr9*5Xnc`$VVOHbdf956)sfkJDf|yUHW}$E?CEq` zR`unh;_c;-61s%~uJ#rJE#3aP(v1;`^v?YWeXlx9sA&lhart#Kf}pMdIg0H)Sly#r zKVRS}K)zPr4T5((xbyx(3u7|+3hcjtl5B>;iw#(%J@3_YFMLD{PU)+dXjKhJ^Z_bN z6cHG0nA`Cc%>s8Wzxd4WV`8W~t*tpynSKn*06fQO;o4m{9%mnMvc%?HIgzp7=F1b& z;L-J15$w~vX2l5tHh zwv)FZXS?maH`hKnjn1nEMX z&7V5ZO5LiVU`mZs$I6?=xx#wy+cIARFbXHq7r8cYJc`s=sB3IrZu{~NxiWqOj9^{R zXXTR?g)J`^D#Ucuafg!|zI6u6I)C!`!b5yK_u+c{YWg)j&dZU?JuoI!?Xb(s6?J># z4tPJj5Ocu0=@LZp5nAi=!Uu&Gzp{l$kCHPK)TYjvV3p&+*>7d$GP?UVe{Ca)+kk5g z`73yIrlH*LFtM&SC0iSB$g>G4mC24fMg?VlF7%Z*!=!2CF35dig6bD_IE3v{nR=P+ zI`-141djJDK#GmkI+#Zftp-3;hXI*|goCPM>HhK`$j1z}P&boBsRI>pi6@B@ZdW(^ ztsfk5U^AfpcKY>o<*|SL)Ded)1ZEz_>uC=J^8|cB6%lO@$L0;@=468V={@)tRjYVbPX)7Ne69L_R0bk+$d5a zHy6NJTxgB=m@=qod)pmW(%Y5RA6i{`mUytHU?;51=kAX;AIx5iYXBPDKIa*w)ogl%Mn{(zE+SW>@84H#Q|JsXf;&Z?34C! zcYGP4p}S&eFge?}(l2mHBh787k1r+8QZ7PG37DDtP*bkY7-09zunCRrgNoSpGeyiRa@oe@bQ0>luDU4a&And3;L+Dk>AN z?Ib9piojSMo=aP@p5S%p+-x$kM%z#yxQ-pcNn0~Vz zwPR9lWkedL&25BA6)ZbONo%6q9-lQ2MS3C#ksPtTo-pyuP{$L#fce?1VGg+F@zOxc z^^?o^{6{l@T4Df4-@X41##3ix3>Vu-fH;_>qVe>k=-7kZ5Uq<8a0hr<_!~rw`yYj< zJP+Aftv{WOKt}x#qT!FSM%J)$!tUCC*|OAepjr;skLv@)s+UX&IlV4#){L{wIM#VL zd4l1Lvxb}la_0n|Ivkk9auPjPFoD|~!9RK7$YJ{F2+ipGRGH;KGARbH)M9k#8_IW4 z91wq77{FqZ?aR-v+()|fpiVH=7DbCNo!6D*-MEDKX5*=E!KCnsYWkesKNo)=>Nu_e zN*}01L;JWJr4tZRc>SEaO`a!{sv<@K1&_1ZIST$;Sx?tv774G^$J6dd$(WwHWV%2_ z8995gATRT3Ckkx`f^z@@+nZpk;k1g2Qfc=bvR!R!T);?)2xHfl~KA(&`%u zMB~JOjGWR=ioW)Q@dt{S53k_$LaLqnZ5OzV8_0_N8uT7156)1}g{-CMNvCw%05(*Urx*vm8O_siA=TN6BPR#`@QzL|PI#G3d?13jpr z<_H#&;$2fVcU)Y;>t~lJGe!yJvnv4PkOuZkB?Sst6PpDXL%WBOMnaXV?zOZN|FBbT z5D)v!jg}c;v~%A)#>rnz^YG)G__6J@@x9Lo33Ga@P`8iaQbcrLvUj4~8}@n$Itaym zrK*c4=W6qFMSB6<^`ipvNAV|TwzVPl`; z!4CWx(<+DRlj;!Kf(*XpVWrc80PF-H1eMVZ zpqQ<2dG^rjKFCodg@6;6NLnCcy}n0+bo`U$coj$S)3C7lOfU`N9hYq#F^L6$@z-kK zp&99F6saA*Po;?|dCb}RLzI-}_Rc%qUH(bWR3}HCTYz`z%tzN0GyB$BA&(e#wg1cx1{v`Z(vT=+AUF~d`-k6b&C2#78ad#VyD z*6W{~@yHbd4GD0;TS@JZNX2C))IlN52m4dH9($@$1*m~dCZ*kABt042Ta3Q4#r~ub z3Vv%ABwo~X`D$R%3JW2NMA%M&B6MWJzXmpDE329dGBGZE>yF#@gJmXRHn<+izK{E=Q7BU{*d1rqV(RM=cNUo@t4r9aAkA}KX~o7kvo8N+~u{&eKL$_FAKH3bi??M?|& zvAa;dO&4JwxOYVmp%@}NMl-VPP;U0sE zS1QVO@qaZDw;A+=;#{m({^RCNX||uZy8$7@PQ@oov=;*Vqp#aBJKa3T#hI{u(7;%b zi1J4X;GO$Ikz%ZMYO_BD9Q{8=doL@Cp%ixJ3zKdlETy_-MK&mnLKHwSlR8C587LhU z(V23lu>g=bJ-|!3N5Ke>{*})KWhsUzfa;Ky&Kvu3T{7$o5WXIqL>@dHvMAB3YXJJB zh$Hn;Ib784QDIL+B*Qq?(Lnn5h8e=KH`=OXEL~h4jf)p7qS!A2W#pB+sa5WO@r+ps!7u^^%1IUvJyF zD^IIzLv_R`rxT!le#epp35V-ZcE~n&&T?O{H*^a~W^aIV20^c0K=3;vB1o;Y&0F0p zEa`*|eTG`Q3hL0Gj=d9>FSh?ie&v4qz^tuQ8RIQ<^G-VcEP=4J%N>m{hnUj?w1y)wZJBuUP< zsE+9HVGN}|KlS(!N6k+%VjVN1&zi=6z9CccxI@FZ*kQOiAOHj@`ZQLKiyNm*KkFvpVcFy@U7P$W z2>}m)M?L9oYyn+`+%N~pugpQepVkU4nXn8jWdcP7s%v%tf9cDnHq^KG`#I${VZ%)H zDdF_W=>a4M<-f|jvJ%N>Oli*xHb(n$=O}{!PpGy*(U_+@Fe8j-?M{V;b^E16ae;}corH!)YgzZC@S2^f2Gk8ii3Hj^n z&*vg|chMj&GibH4OrMo7KIZP})_a#LL>CD;t*LbRM0o1`@Wv}fT4iSJt0q7XHTi(p zZE*J8KanZGDNQ35=hxU3ZF=jA@s%mAIB-GxWiBsOj$5BcKQ91Fh<78k2v2>8|HbMs6I%U9fw${;xYf z`UH6QWqT(kB}kdimhNYuT2<**g^`ui=ly*843xpEfzBktUSDLS6gkK;D@Jd$|GnjS z*^O%&{mz^jk;Q-Q5lA4Xb|zZP8ew|*y)PHFJgqb&utDH`ak+_poSFCb4uegrmH3{9 zT_arac56v`wM4y)V-S*{U!AvHjIk|^30qumS$Knv-FURSSdASNbo6RefGUjT0BrS$ z6LGN>!~Ff$7i=?EURaR^B~9&m;?u3MppJ|o;Y}@|2RHwocsOhLld%eNkZu%+m)N0q>)XXd`Sorzyw%;E=Xt?s5kXxPEbXC~~}o)qU!i5BLdj(i#b zC-C4PkeLWWpqX%fknc{8O->9$`0gXS=JP_k%qbU;SrJRfTO*9+e7n+4V;eMJDTHc| zef4*xFR!%SlYe{xPHEU2)oga?tZcvT!vO0j`_ascu#_G3qsX_Sjo}DOzcEr=Q`G3V-|#fhG>S^O9ae4oFJowmc)8c9R(lb zj}#(!W|0(Ga0qdEnHYb^lVlUO4+ue!?p2WKV|-bjy#W#13gO9(18g{>D(9?cH2KmJCEjLglco)bx6Bkzjlif&j1yJKR*0mHEOMQ0wE1 z(7kK;vC!8jis=LIF<}%)9pb34wdE1b&hiUx$qrh8yHNgNWcU#MeO1p~<)Pd(BLX+! z6#{^dJ_VLfMWx+0gh3MlrcnP<{0*_32z>@iS4Rsr84*@~L7k%f5wchVHp`yqqQ;35 z9O{-sxW4T)#AEIQdO)^N9cqkH1JO}4(Bw~(#1S+Z1YTB$6;nByLqi@dQfpluO5Vgp zc(?dJi4DK3oVx7xD;^(24T&?k#eA zZ0_l5=}B%Ffc5~7&1J{Ro+Dz}$>iE3*MSEDPv*v0&2nBcPx^{0n1~IiAMoO+$Xdl z#BH!gWo#=83#hWM`id(+;Sr+tO0-!O+<+`@YRz+;7+=g-MRL}2NN2>g$=AE)xdr(t ze{1eXr%|454c8J88E{aX_cEeQD5RDC7ndD;Y;J)s3*reXXMl1O$L`uw1*q0F(O7+5 z5C>;}M(WqQqOa9-Ezk7_*BeRaF%W+o6>(#0z5%*RkZ3>e!-#qIUJVswivj)eC|3Ff zFv|7npuJ3`Jv1!@_H=6O`gI0R%=*ObQzczLFA#RiJA>qyRLEPj@4U<#Vcp^{ zbP1OIx%n&)!e2>{<)Q^e)N%LrtlB_~i?)0aI z$xACRz+h?H+O!9nxQGzNF9G8pwL%jni&Z8z8+;AYEq}Hu!X8HpQhF_BFKi%qMmXv) z^0rdzs*=cHA!Zt_rm70Xp{3`voWJ*`1GAF5G2C?02GKX1hB%{=YYuj2UU;8nVf$=Q z0#?xS)QMK}36bsBX@%%Ra1^Rk-@(b;%HH#y6HW>XqhvzDNi0I`ch?hgN!jeosUOUJ z4)kMZQmBx^01)ae(o_{PeUbFYFlQ2=8Cxt+=b@lg$}N`9YMnctSOCw2Vm$%{PzKoZ z+i_IOi|30v82})aJXT}Q!wG-R&OX_KZ-Vaur5-On!{jux6wH}#Q)^KL*(ZTt{Py)v z^VVGVMW($OowP!+M2(;$x3(2Ph;Lj6tj2(S4|e&hy+_FV1e9RbHE}5JqUiaMEJ<$8jpWq)?i^ z*T@(E1Ks8yM-d?8)NKk-JGu3I2XwZS(@}Sk+Bc|_kB)jYY;2s3wsT5cF}dT+I-5y) zsVbg32X$e=@Me)n2yjlS@nFzDWb}Z^(sP0W%1=5<)QDTh>ZEvBm< zf>g*5C4#qH04>MVIbSfjuTY;N0{XLR^FC<>-B{Ewuo-ftq#n&xA2L2)T!59>3X!h! z^Nl%7L=0kFIMvywN5ray`5K_FmIxw7IOVIG><7l5V_Edvhsu?v9&RWqvHBQL;|L-^ ze#*dJ7|LDbF7=Wozu*QK8zcL29g7DyW!n`oFUfA-Y&I6E&B4yXetCtue#FI~c(p9i zPuQsuk|s9VWnmK*RqDp<3%^?B4?H_a(9j|6j>((PjvOQd>-g8G1h}k`dn{-*!?2#F zYjE;;c>((ZvAxn}naNVj)*`d|>->qv&4~8ehk#>;Ey`wHgXPLs*^ur;ZBcH!74$Pc zzE#p)&pn_&$u<3zvJ&(fOes^{3942MGixB-F&5scp=^KHax@ue0P=k1&c=HZ_7rHw z-C+6X_}n$&0+W(4xyVuXnMEFmkp&=7GU9ZPcG-AD^KhU}F?}D1W6z+r_g)d$Qq)8gsRv1vqR2eH zT~`t@qPBP;w@v~vws*Gp+Z=-pCM}U?OGoy zP(URN_SMA{OXAo=Q>0ii3htcQP|We><_f*{V=QP*>x_+65i(F!Hrez7@crK%sA^Oa zUAj3Qm46_($JAM@(n)#{dAX1b1zJSlky|2&{bP0KeN_N}jy9u__T;~o;#wM@qf3k) zFr{0F!{B;%y#EFf>j66EN*f@?2W(FQRp7A?u1^sbf_O}^J$N1CYVB@i+B<1T@>Wt~ z!9TM%@{t^7H3|N)&pG!tyatJ>9yJQ2o8;@?FI1+KX*XkfoN$@Vb~Z0E06j>rbJL1; z4g215d@wC(#r5f=$`pg5BDMc zqxUWrpnfvYXfu+DyCag#4FTmWexoq95#+}WgKKWe;2nf0Gec_!0^9GB3cHf)O%`=4 z(+56uur&rPRmQT3c<1)-UU_CG6H(;{gi1c@5GtX;s^#SjK+Q$A>$mHs|3}$ZhDFtN ze@m(es3-=Cpb`QCN=m09-JK$xDhxO@q9O(&>Ci~G(lCS~1|dTuLzhS^-Ms6daNp1W zdY%vO7cb#bm~+lPYp?aIecAf+aFLtJ$KXtLjLA)0K#gaJfH5b$B-g8F7iN#9&}aa-*XWcV}OYLVxR$uL+xCfj%o zVjU;1ZNW#6$XYqlpVHH?f_&J;mxdxD`6+#;r}Kl~gGW?8+M3nutIA9<&)@h+@nfK^D;`07ua*g&l3jNx}TgZ1FLo;3|ju$vNH)WEvc6 zKcU?ja9Wlxoo(sw`Z)ENP;xhegPg**e4d6zgf_CGVPG7AtB%Q=Z zO*_ac;v-+?4Ou`W)iIwYaJ|P4-FSXVwaWHek{~+56Li=eqZ{Y1@Q^J0_iTxuvL=DC zN`f-<`SWxj@Ls&UxtacoU>k-YD}fz5`EvR?K#ZKH4~POIe^#mR8;I$53@@v%W)PQ* zE(AxVp3>;wd`zBK1r&6hrRY4y?eWI-6S_%eOIe}yKEJ2(v)@*enZwJS6GFzdB1|+A z^9FY);w%bi^HsOUlQpQTSzz@2C|Ffg;BB7ZLzu= zAtOy_W73%~%mEYun-q@sr7Bp(gW0&Os`BmT1;hfAU(&6A|1wd@QMEC*vu!(-e%XZS z!Z0{K$m~f;BS44du=8+$TW9F_Xsdta%GeJ~Df2SQVZ0TAMc0I0Gdwq_pBI8=1|)-1 zYT-t1Vsg%gY2_=nNkQ@*>=x~88ZOGmzhXZED3)mUb`hD> z-7dU@ygG6P|Fwh;awDAhuX6usMHEX!0DoS!Riz~+h;jBZ_ch_Myr(vG3n@8UmqJ|K zkjAkKB#H7&(z3xf`e>Q0=|hs8i(V~zoF3>yJJ5@{7Dc!=qT|W|4t7yAJ(h1LoA#A~ zNzr(YV+FA!nm2`_A4tdYb zEI=t_mlZ^GZWEf0PBV|{U5H6@ePaJR^UaO8y;H{V?zg>;*B?G}`Zy>G*V*u4T6*b7 z_}8KhA?BSZV6D792m&X<_lho#%gNizxxV4|8N0V1j%OZOaBfAmtFkxnAh`8MxCeyBO~dLywP0mgCRp>!z0V)7bqgn{ep?T(n$Aw=A{v#5H3Z@8$iO$-Q3Yk3vc}TxaibC zejBkLR~$C_dJ5q)Lb;Xh>5;9YI(iN;`$k6hp)vJAWDu{>q_@1 zXj%Ohftbb`(>)n(@Sz(9o-9B12&3ObNI_7oCfC86trnz1uF5~HawuvlQble_&>*PRQqp7WGmNJI#8^^SK%>)_8Rxd!f|idwGO85#4zIAV)uJ~wtz zv%js*AAy0dFZJu&CYjpnD#uG8H8b_6%knS_$I@>YB7*5$5kAaqz}4c5F@o45^YU#) z2&nB^;@&5kUmFJ?B4Yrd$cdS2m=Qk*P1UKX!+!+e4WjnqQF4WZm8| zSt1&N;IBcLL^sy{Zl?f#dqm}(Tr5_`ySz~JBES`d_T~~AKvTaL=Kp7 z1!Tp(4qy*NVoXLA*bXi1YgbO?67tBHus(unEVi$gW~1BuDdVni#L5)zRFXm+-x%q= zxMlQ?SQS}wERl%naB|jL=?Coyg3JPg8aI09eQ{+$glhxFhk3SOp2Oq4P^Uj>aN94? zC6${=Mgp;bDq~w(O3=H0a6wZXy*sJxOvt^kDZx89=8=S!lN(%0{f6=GI5 zB0I}dcI;4qt9pSHTdK~zZ$8#~>i{p}0S2d11hQ?9mJA27VbU|5iqc~7xCX`3(pv~~ zz@SPmHjvbVpkEx|(ea2LEJ6^o4U^yx+nD64(+b`YEf|(%6+|!w2dPBVA&|tJz-E0R zLhn3ADd&m@AMOLAY52n8MY`3SXy~sehE{J~*3uo17j6E{OfhsXH~YZlz?|kS@W`3 zc{1+XBjII+gO{;WdFF_T`ffL(;n1SKX~5zNj9FtI-O<+AKh=82rG3hS5c3Ls$!;fe zkyW4c9c9=wVwhn7I=6V+{9KH!#sQFt*bX=IL z7@Bzr2=z>!T0#N-@1*pHE4_!8D`YUL+qTB*c=G+Kmgda>0xn0cMu*SMPGzjkmX@mn zKTLqdU@5yFg4_6im$peR_S8U7sJSV|@pM zeRm#ZLAq6*iHybdP|O>HF|+lDwq*^`IEV!@>|_R#$w*mEjcJ>CRyK2D^*flFU{<)c zYL~vf9h%{`Yp|XSueZD{R`nE@V zr;tPoQTW6WGIX9isqz*j%3}9|yg8X>PFYYtSzwVRiJ69We+f}gv7m8!kj+jd-8$Q* zuBjd?UxB1hZbGtZ#yV;uJ+)(2$K573oelP3;GGQvnb82oBd|Kdf}{K zJ<6*w-vIoUHr>LlNR5t(uv`&f+x)S8xQ@F{XyLQxV}4fgcx^(m|Gx1^U561*1+A9< zHV#?V^@!S?t6OzR!0YNa%YilM0DBe_)h$HpFJJFMcxhm$(#6CaHsRjpdN0(92t?3{ z&VTGg08p+2!bJFOXsbh`>tBjO&umrG-P2X+h!5D_is#!$`DVGSrH)j7Xg zq=S|w+lZ}eZrCmd)jvn1vlcxcz_M;<&E_dv{%sLqhY{w9QA|7;a1lAL5&2Sb562nn zwYW-8pxiI7-u;P~f?{-zL?9d3vG`xMC^Wv6=gYH7B&T92(s(9iC(c9FdBQf-dR-fS&eMmOXVp0xb?ir<>y)Uy_d3oEFAtZien{Exdg4p^B zJeWoc+^QpLqRbwa>o>drvA=6fP>`$Zs;h0^Fl68=E{>c zz|am7u@>V<#X_|nz#RL!esR6CJWk=wd!#d~a#9>pjh-M3*{;>t*u1X}}6r37sBiOBV+-b{bRc7wZWrg(qfLYx-9wCW@hdiB1ktSQp)dFmfs@o zb!|sukDH038=oVR^ZF?lv+B!2JC3x^}Xs@PXS=G z_U5qB1TH@4N9;yZ7*{Ft^iIA#Xk}3wSJ5CNGh#Lw@vrScjP4!z#>tI#0h5qGeHp~~ z-4k>Nhw0YaN;{3*+hxhnTb_%``rKvA)2aQ?h#71Sn~@%2IDLCzWLCcXUWHSAgXP^? z*OZKuEdmb+FJQHdzI!8P-;rk<^ig!u>M~lEU>v|CXF|@rd$4W?*E8L^?|&Z)a&AwP zh4038>}o%=pXV7??)lWX(+UC_MGd6Fp+y_B9}@B~7;h7cb=<)Bggo>3Srt5G)W(b@ z?0BvruEwV?#V!lzYI1RZ8r#$)btqo#@AP~zjX-v_?jRhSfIeSOrVdV)@k^Wc_T#1( z>J%(4SFRs92WJpGj~w3EK7+n1b1#=EOA#Zk26Ya(Z?5DP{W&Ir=?s`ks>0Gt@?drZ+#g%YL$dZ1x911PR0l`A~b-lSfZvFj!QI*_tXF>|XDn~`Em>kic z*o;(nM2Bp*5DgCFc168Ac7gdF^9TB4n2}A(iEreqnD>lq2N}jM&1Qs{y>fk1+SeNX z@DYP1gD!gqx;T2|DL@4g3eqiTWoXcAE`KZiBMDJ($OnF`V>auUa-S^It$Y^#Y|mgG zDyedB;C#anv@g?&=|QcNJm!a)7V-ECDRt5H*JBkBkwztDS`7GH$E>SU1*d+WfH%N1 zS^eA0m~d33gaw>D|1BzdXiYJ0!W7ufSvDuNPu930GfP!s=rbHFWhI&z~JJ2p%(r<|g*t@$@`FO?Cr+CGAge12Hjbizp63Au0L+YKsV zWk{O&92k?TYV43atly1?12NXDe(^0giWqdwKKWi_4BYT}jY*yRc*~-z?y!gF;_NpO zYoW*E-fP`HY?2IdD9Ib>r_*ApJ;4D*++aR!yeJsIbDW^l1s{{J*Yl8Iv zmSg^A)u9Tkt*5m`EkAFS6WPxwQr*xxxuejWtK|HnwW;;qAIT*$9nu75q}UwANdqS3 zT5ManlzrWZlNQU@r}^LDfROxq4#6xb+I+LcAVq2mF-yowiIjDqnKgK(zU0|EC!8Ny zgYe6O0vAFu778gx{U7=${hJp_%;lM(cMB6j)He@zg>_Q z+o?HlCU+C}?#KeM)n<}U><}rR$Q8YN?-%0iEDCXC7?v>4zs%(HM89&e5c);@O0-w_MK#h?j$4!9tdGb27+2^CKE26(Ukt=vP2Nm*F+V6m zIL*Y}NoNply?h~=lRbbP$8=YzQn+0VY1`y;_)i9#1sxR6BU7QG@Pd-e7qR9UC$Tc# z|1AC5Z;SRvu}o6d6W{9St(osC)hDoYPer*$4sU&Du4U+-RLIK%Yfo5~N;BPJ#%-;G zR=WW6E~&X|v_#Hfx9Dq(ftwO9u4N7AJK}-dQPI~P4(g4nf0?9fJqU=(bTOUq<1e>& zu>b{Z@!*bH?*KU~LCoIC7^k!d<=G205cTx45uZMhB?PHLe7LAVM9WnEMC>%sdvsfG z&59s#K`8f2$7sH5VR(m9Ey-OE_m zoNC4wT9rAwOk?ipR?x#NQz0;u%YGFkBJTRSie^Z{-43(r!2i+guCEuPa7f7%{Bt?R!+Feb&cJhP!ox(-8YFy zPZyt3^zMJitV%zdie%ZbH~^NIT#ss)c2x58K&sS220=}x=)Em2gtF>$mO>g{Z$*fj8Egi{5hYNcE*s;rt7e?9y+4 z&Fv;v4m4anC76H!JU=&<57Y{ z(}%%=y(3p8@7ZpC>7@aBs|dGtU&KDYSQmjHytVNQ*B;wN4M6f9Tk4wXBykKX|TB%m{--#*6-jKdzM7=r+f^OI8s4yZeBlKulEE3^mR`{SNM=el%d{8 zk*gOu)pP3r+dWCJf$qdmtIubtImVhRQ^o*Dlw4{F>J?7FgG~j-lR_q36Cmcypw>wT znC8jZTOs1jU0$MN0zXR4c%PR9o)dA)IH*0eQnkHaMMXl$Hy`!Ck_DxZV^M<88iIRV z`IQyog?}9o;G4h^^kEDWCUQ*GtiI+rCjO1P!D;CV==f1Fz$hUkRjy?s*b$HjG?8B$ zU;yAmjWVQQ^oERTMe-P4pn8ECZ5I zF7j8;bPia=vf0z|mB3k`5VFgx$vwK|##+d!$Pe3B8{?N5R2PnWtE+0As{nDKoZUSf za*RiDgy6hzA2440*X1WfVSAC+dVaGiSOL<7_J_8`x%>cKr5njpI2E$(Ro80B%~$T~ z%?o(>RlBPSU5)Jh$->VJaN(ArmYSNaF0VU8#x2*5=eLEK`PPIbJw-I9BW{-65bE+W z)Uhb#@VCV&sE}w%*$ZFaWn+DA@Aen}0f|C_C-_kdCecs#l3(Jhh}-_%HTzQf0vrPp zp5D6__r1Va%Z3O=>LE&zTbMxaEGn2hGU>?)e*u(LxtP(Hshjde~jlj)RDM2Br1|PH{w1H~PjNgFc{-3+&}(fqs;NX}D$FgEKt; z>#TTCTQVFhKOkI8N#_ecK6^`I1a>PrX9=-?>ykG8z9G6KMBO3LiNj1cElfGt$DlgW zhm8<*$U)}lD^_u>4WDO4k&D|slVK~aLddr`y>I>6mcC?41LhOU^&E5==ECWEXZ4h) zNf+8*XiV^_M4U7GX%oa0O#bNk>8F_*a-|89ROh+P&vA)VZqEq1#@Ww0EeQk;M|+A^ zKoG6oq*!IA=rY=9`zQhN5%GP<9|MvHj-WjYeg5#OKJ1-b6Q^g!Py7NPW z8N?ZOfBeqxK>m2*7Kp`@oYT(b0<<&S@|*T99_XKH2V!8G3H zwmdUUnrF+HIFj}o4$I8)-rjQgHSl5NCI{aOm2o2|tq$!;9EN2zkL7>Lr#VwS4f#H% zU+ho_sd1kmn2H8Y=1Jsr?Caq%*#baP-!w$7aPVW7t>k<7&F9=lkz_(v`&4jKy_1S zmi1dWkLyfNIIpc1zv=)e*X4@btmZy|j3LBseDxrsc3Ev$pmC1M*mQSsca9-eQsn8H z``n8)IW>vB=jGeRDSq(ipSAJN2cTagNjv|q+x+KKNLPu6wg<@)sxC>k%A=O1yDdG} z7T#63FEiPOg|PLG$T26>ZGuE5h4v5?*-3lc%X1<&Sl8LWA{$k@e}16`lla}eOKiLR z3vzCCDm_*e#@Y;&>P}#M@L1cnv_Y4rFBm=-*?0HMRAlQFl3}}NU@h7;yuPZHuEwZe zWQUR0`YjiF=~nDmsZ>7-C-QtiN@%?!*SP=FqA)Jc5_c}jg}C|P1GsHTas*_L_HN(q zEpO%wI%oTp!FOS$LtP@KeY^?l>Zrc|Ggyqw?7NFlF=Uz<8t))(>T~#=#%KYpcyWT_ zLmzrdou~=T8dVWQghIT%|f%q=loNpK7{!t8=}LVp18v#u*|99 zNsSXBUuhC)F9n??yqDItU3tm!jz1bO9HA3EIfg>5jTEtC0>{F%4=cwCTW9!BaHA$P z_`5DR{b=9=hFwH-@_=@VIPTN4lTi~x=-Bow-PVorP#sBB$9#m5TWy6UZr(t!K7vzQ z0!h zz`Z2*lOL?poN&vUr=jc8st-Evhnnr{{rr;7G2>-R+j;)&8};PlQLLI7W;Ru7LW*`t^S?eh z&d-;ZnExs->9jei@qJ$9VUR2(4bRs?5jUfrd-wacbJX49OI&moyF(WP+X9y$jjcQA zyp&|epk&YUS=UjtH@=pS#a!s_(8DM=;OhGo^{Cq=JP>_+8=9zybPCaH4ex!44Bb=k z!<&3sxb=L4*pq8n{jn{&1y-+~oyk$uy>$)Ntgpre@vFOQcdzUqFUze{1WX4X`VKCw z9&3uXK;hGfhM~d~f}iFqc&P0&wR8DUGgiL}@AbdC?Joizw zP~s@HEvt^*uv9!@)%@mBlv#6uwK~6qw0x=y?nM(tF9-y7`~G5SPSE7S|_jOOei+!xgua^Wpugj}xn6 z*=mdtQVco#U|LhA!l}ZYb}eU_^W(tVgUNI{+EZ5-Om?vV&%+IledXvJB1M!Zq{vT$ zC-NPfsmJpA1+7pA>Y5h_G+b7G-Lh>>dBO60BDd_}yeEinvQ!czQ~GUkjViK?A$(87 z^Jgyv;Z{RCMzk!X|IYMLl=v@L(@}L)9EzVt|EX~Yjt>a!7Pp{l1cm(RBJZG{YbD)0 z+W9%zg|WAFVp7j~6He}UYtx=l%GXenl=Dd0S(-*hh$!ua+;ou5vf6qCUPAL+FT50q z3kwR<)`F939<+MDOyBzuECJ_H;yXxV;00q&DW86ryuBX<4bL^>uDGzAiLC4D@L)zr zzuL~-$hY`r;QkX@{*%n*<<2#w=yRN*5tQZ${b3D2i0sR6mnw7&a+azfE4BN}YG6E zdM@dzkEN4NGiz8(vWq>NTET5rVBet7Z_F>rfpqRI3^DnJkm`Di22d7-=MrHC2ap2v?hRMpytz2s?@)M_DQ#O`);1_%fuASw-GF(qA{tG2|`AQ)@CeQk!DPvhB{+7gF7xuKuOq zyZ;#4@)2xoZAGb9$lSGj=AET78*CGf7KS{klbkM#lumebRZ>KG#+~M7uFDrNwtQX4 z(cmbotpLeGii?Fe5S~9oy783u44!yBzXB=qOv(hP!Wb>9ZIJWahBJ6k$4Dd}iFCTl*2>3d(`e-Q z{z#ZphR2mW*e`^hOG zD)N@zE#rRNmsbYkwh#PWD4QtB@wXBbqxc+7k}H~?h7=y@>zxd~kXYCVCL%ZWv(rQI zvP5p$d23LeE*j}_ydgjMaQ6a}C<)|z_r5?Auzzi?^mNxgJT0BhspskWvne5kpaa%Q zzRR43{!5p^sip)!*kkLatH;r?`o!0NcjOq_0xpq_jvd2Kxh+kNpHO@!y^$Kj+P;+!BuUu4)E{shd!)4wRm}HZ<4!}3A=E_*^CmrXbb;kDJd{>vpTfY?Ewuw-O4&^y(?$G{@ zFIB1QtPyx35uDSk5=y01YV_nO9N|`Xp>2e|3FAOG^s?Vzys^i>(>U~D1s7_3ZNHep zFJ1cxq{*F?IL2I{Z++)f4h#Uk^9 zh%Q(8@)ZzLPAuFCL*`mQb8qXVPlA?#0+Z=NQ391maquz484IZFIma%W4LFr!Ldy%D zYC`erHS zADE;dr#-$NkUlII6Nj&$PY|?i^6bdcMLLDM$4J_IGn`Wtqby$Kd`OL zo^nDS0b&)$SpxAd3!IYc768wG?UHH$T_{q@k!(P0%n{ggNAS8px09%^r&qJ&;49C0 zE~sMR-JJ#lL6PgM5f-o=Lu`cDKI6e5A&o7_cKcZPr4l4&DP^qXq-~k(_P~_70D`Im z{pBo<+!mg`A}G0P zSsVNbpw$x*RVUb+A3lYCRTu89{b2Q@yDp0D#DrDCa>`#O+A|fg@fkff#p+QrP|v5~ zY6?_VS4p2@ty`JY^^>7i1ki9bdD3zYhr^z-@pFa=xhw&7TwX_`X?W zMte5U7Iq~1--I=LJL_&2zSGh)GXoFV0Ux(4NV8LlSn4Hy`@- zq$M-o+rBR*we+udZA(*Kf&>*8_?TVTB~>_%Y`Z63hsFW8eq_!gjc8Dh9S?@UQIJo3+1=bek1vj4o{Z^)|6F2 z(!ZrW-o22-rl}mSK`mo#<+A}2A`>|8Fviv~JdCv4HmJ(QsswhIo*6=#70#-g&o$`6 z6#W*$`oQ*QqxV)?h~!egivNkJrU&eiB?(1(({t&c4BCFAelje0CTSv^JZYS1s5G1H zGL6rfR=0nh5#mC*NW1wqp%L_B82jF`cEuCTy7p+VYlRdxk`M4J8}5QKL-BWf{qa1S z4%2Dz`Cf50-W22?U13!ZTqbbXD=xeB7r-i&kIpPX=M6g|<(MzU2M~C!FY(44BR{LW z4@I7yfBhKoV@g_^2|HC81!zkZGE=<4-=dW)N4<{h)Dz#E_u0a*C9$a|A?KIGJ5YJ+ z<6}OP_xp;6Li03I14ykVy?V<~iqzL4d&1rA6XpFX=XlO|y_|)mH z`W;#JE$girx9eoE`+xE$x@9CP_*eR8<_i)4PU)a0VjS~^)lcv%yqO2jv)aC`m`QJ) zhsKd-Q_C#7A2YMxU-6RX=*gL&S6=EFes%LOz?RjXy*JZ~EzUM5ZQ-a`P*aylB3;Qm zt0$rm6Z`Z`T`_cADOGHnH`--04_?funrmV>n5~ntmZ`vsW%BsRFJ{}3^>$Vvgg7)} z=+52~dPR1Ga<|)ia)Th7DR+s3;%A3$XRWR>JN=u3`)Ho$(2DSu7!XYr+jW`tVqN(Q zn?FUwo!HPRv(@oFkJ<<4kibZQT#M6 zuu`4!MCDk=mo`q^Z%LSq+_6kYk(y=B+asks@!R~?)WdYG>`3dm53p7C5EG&5KxNpR zjCEVIS%+XG(}!H@yI6n?q!d1|ZmMKDBpsY8X!W4g9xw`d_0g8*RAsu(S}G3Cs^QDm zXniw=<@{5`KJ8M6nHE$f6T*I1u>ryYaS{^aE0ot3}EqhlO$#@U!{(3X)k`&5dh z?_%@Q<+=7v6&VKe7CCL2OSv8zobAes3$3lLy{+6~%x@dX5+#=1l%M8~(X8Z3EV-S~ z(Gm&q{3X)cIeU4fvYPe9N=xVb9sj;KFBv)S@s?zV2R?LKvpQ95VY(qVj%$VJEkvnU zz8(5`V5tK5-=3N??W)+@e#6rW@fur)Dr69Dk6Q)!U@^L2E-W<1_2jC%IKy zc%Gw-d{-;~E;DgA>iE9BTVF)s`pFcnenBHJA-O#m340j#%bccbI`^h+DkDcZ0k$yvOt&MAgEhD0Y2c%>6QA_?;nBhQl2+SId0WZJqUdT?5E-rU4=^C2%*@O2Ev^vc>To1q`iim-HJ-*M1?xL9@YJqvj^yd zB1tZt*}Kl&@kF|6AFuSl#-S^+0~=>ThzM0#;x!ivGb#iZoCCwila1Pl z{?-_^(tV$#_U`XBOJ4x*@X(Mo%nQ4ApppVIvgO(OZK>T0U0GshK*;UdQHHW@%?abR z?IEhq^bR+Uw5W7BK>-*`&$t_T_C$icV3Fr@_UuHH=A}&^n;pn8O3wzk*wNNem2jiHI; zGGRb>QhGHu4dtX(>u=h7SK{K5ubxysMh;magIR?if5Ij{an5$E>9E*F6Az2@tP;l> z#r;@I60+l0rXR#ls;s7aZOzKfwlba?k{WGxZyxaSCX|$<^g6qFBeWu|IbGtX zoH|?m@=R}ccDXkM7T}@x?j`mCJPZ#xP+4|ed0H77EUgn8klT_6q?+FF=NH-cM(a}A zR|_X9TAJ(DMckJP2#82agi*@=_~e3&O41qIfilwg%xJe1DO|f(GQsS(NZx!n-K~sI zu&^B__itcYHhXr0Jr8gS4C!K~2b>S&&+>wlg61`o)TPRn=6$ZyGx&`_^(RO=D-oR4kw>*S$feu4H0s6SYL-DRUjXwiUa4?Ux&+$^U`KjzQkErSSLAa z50{tX(bho|=f^~~a(CfO^Q`P$o-Qfu<)6K_RC0i}Bi>2pD(k{g)<8BpVY)TNn&s|i z%&aeu+*~Hyx7RN0);^@6lOk>jhLYIQ()}juKoS)J>TE`$Y)p_KpF@w;qeCZ^7?b7Q zuId$74GO-}WTw?S&oQ2U^w{4z(BivKQ+jVfJR$TQ;5%PU} zSgOdkbVtdT4g5Z|-x%}P>^9t7TU7L`-E@Z~Tf{=Kt!Fu!$irJI;kT%HvKNtPO+#>V zOX7E_V@!mt>erG_pP?UkHjzrbGJ52W&{AerfnHUMiVmhmRCxY+XBmL6H!`GPm&%fS zM8;>?cy=u9p6wBj1j$E@H6YR`C)zB$_^_olftG})b{H4*gUPA~+(9YS3#gx%Ih|sM zm_oo(e#+;buD?f9g;5XZp5|urMoX1k1MCg>Rr~sI_QvZW z`Wl>Jx{KPUaT*u*%d={Ba%}Kse?DmCwL+Z|N3K9j_G)S~xsPM+#0~Dvj%dosN-DM4x?7#ppkN71|epiya_I7J?T^}fvo`=5ggBzj+T5URW z?1Aln2iw7SK(Gku=0zTnKosuPCbCfEnzeX1$x0>_>67L@(x?WY;@>W*dr#7Qm-B6OU2)-XuQY zw|94TDMkbW1%#V~7#;}WSF=}cBudfOZqW)l-|??ZeH?6aogFvdBc~9%Tm_pSCL1u3 zB0%@sR21rVf5HKV#C@{=>pJ(I`=8s7VCGNu5vE^4zj3d!c^tBBH)vC;U3w+JCY)?Qj6ItO+qzaKdGyr z@O*rV?gb5R9;lH!mS%cwA={59Y};Uto@4i2xRXOz^WQn8|MR>5;D)ds-zdYiH!mxt zt3D)zA@CXWYRY_OUyrbUM1mCwx!<6<_$w)a!ei~t@o*2rzy=)5{Uh4$`KKjq3@ z5C@!u)&#acp(P*$Phs%bm2EyZZUM`qFyMg3aH2;-vfOW|>1le8Ab9`;Q zB`(%MP13`9?XJMzSxemJWW2}!xsQXyAjz`<_suh*1rdXi_v?UMC8B0}x_}*wecyez zau#4p(!=NM5U~Ovx}VUWNeIVed^G<3qWxJpe+3;!jNpL`+TW-8L`cK14IDH!@SJ8o zfc%18pjWbm72?vrIOx9$A^G;JbE7$lsYVrUOXI0=WhEb$U8nKp2nm2a<^hCk3c;p5 zzTh2rS6Tpwy$#*g+X@hEJKWg%hUVX~{s z6Yce%raQ8u%t6!9S6&@vXzuZs(gl9ts(K_;7j>5zNKc@%t9f#C2^?+Tp;fVi0_i~L zTPF%A#M?!eSF3;DIp0HU;xqrgdqI~8Z6YQC>K|`fHj`aAdraW|ulRP*hpuJI0Os06He8<+iK=}=T^x}@ft$nf41q}&-Pcx^Iibf<-l2j zfY>=gNM);N&(`lm365t?)`Xh^czWX4vD@bL;V4x$P(C1F6|^ArMGn0aGviVr`%iEh zLEmPcPe?*m@(WHOdPQ-DuR*eR7GIe8rnKPMaTZhEe2YR@+!(FsmWKq{b-C2Z_#%5- zNMcH{&U{60+Xo=mf9-ScW$)4du?&jersbDsk-+kJ(&cz{}!qN z;e+r1{`EYPqJ8j_KN@2u@zrgC;r9f-#HoYk#{g4M@&(a39ExsAbQbW<89Ef27iy zwx7s!(Ztj$SqSnZ3_+T&yPYb$nWoS)RKiQNB=MRv^3-nx9dbR&^aQ&JF)dv{?2J{; z`@%3P{LifQe;%H=+yN*mjG?s`bx8__Dz$#8?GJ1$Q&J$MIh8B%3xO>3!}{li`45U` zjyU#|G^6q!$>mB1P$f9%|MH}$MpBKDzNDkV-4&(_!|3~Xf_T8%9eA?^oPLy8!j9(> zLI6GZXwK&0?4>G`4}=JzHu?`6rKkRp%I6ie_Ski4&tK%7WI^pfPIE#&hHR` zf*}PAQD%sm4en*xhS?MyLRO~x%_`iRF}@Qy!$kM@Ndz4y82XP!hLuXi{q>0!l_(5; z1z4#VRyo&=mAjehse!Qj2sKJx^cn>I%5M?gl+E?Ad0~(iwN_uU+28(vg+?P+_Rw4_ zA%j;R&6C`vo7-z1aZ;lZ+?GnMrfeV3P~dtsEGi}kEDbIOH4rlR|BuZWG&L7NkDxKn zdg*2oA?UI()#}qzj*c$5yCd&e|K^%k?&#hGzFX?gyJ+;O0fmfhgqWbpAa)E>eJRU()+4t4@d`|_c=+r`nWOXyZAxQwQR?=Ax#e?D*$qlm zBE~jvDtPJfB!g(hp;%MV`P%HGi+-0*#M}Jqh57TC{&DWyLec_lrJTSO>43qR^R3alpRn zz!GV!ld(3h3#|p&GeL3Gdl11ofFLa05d2|o$A2HSHcFuKfSWdr%nLE((4*KXM)JIa zOKB#_R!Mk(OnMNgB+(73Ol)dnCAe_$F0*~pIlG0JfSZWm-*@kgnz+z@t5+|9=qfXm z1oymt@w$bHdEtC&b$goX#}uU)8$?9dfZ@iv6sjy24prj6BhWU@`&o3X5ICD!rhf`p zHPQ30z}w{-E8vt;uVjf^pO39Bi8|=v40~bnAUf7MYy(PX>8Yo)Xz!F{5-E^H*|Ju- za3KtjVKmgt?~u-CfH6qorUS2f0Mxuj5nMQXR61owJx0c1j4-@Q?8)LDxi9lWed=ZSZ) z15{SOb{kaC`G1iZR5z`hj1W1f{A!>LoSY%?zVv+V31_Z~xrQJ-zo_mP66oVlfZ>!F z8#aTIx8^_vmN}1`0=UFi`U4svflNPjT4?A-okSCaLpk>U*$0|0h>QHkSmH@G)m@Ae z-CDVgg%h35`kd%&iU2Vaf8%?!SOX$y!XCo62I{=;zT2d z=|rboE+SenWKv%Ol~*?ymv15S3{d&sV2Vz#QzWV{ycBRk6b*4cy%i5djXW1MTOZKb zIJS*E1aWiOC4kG~^xLF`@-qy7Hww-2DEc&!ZGZGXTny7qnTHT+!i+5OCbHK;GxHUg z4YpRwq{w~0!NvR?GA|dtvufwqu|_eGm4k&oWvK>foZG^G5=O5=@GG;zXcMs??7p81 z+^Xg@w;)U3Z@yKM6%?7S)A`U|h#s5t-bNINa`WD<4HXxG*H6_d>_$$Rd<&k4{@%l3 zagOmkJm0rLUwU^s*5+g{$ZQ<2Quf($+Ou-@nKM5yFCU(5Bo>&Ao9F;F4i+S*JvrG# zq~vd5jmA(1o;j)I)AK(&!5|}KEH&!)pvftQwgJ$k$XCN#|_CE`L(}4(}|P7NCQ@lmvKM6Oc_iG*xJQ zpy79j4rABXYVJ*lA#eMT>Rb>RibemBvd$!O1o|(fQRb%xF9JE3 zvi>QZs;s}jHHQ5p0XvtaA#+iN-U#5gOhvT0YRPiQ`EVHgCupl^qN14&r%|X9^%t&H z;(zf3d3=9+%OSO8U>yXIsJme>~ zD89`Y7Q#Y%;$3>>f2v@I;p8GSXa@rJ)AZzCqgjbV6rDhB*dS1t^=~S6%6pLCikXOC zx}m72g8czc*^xDhbPz$Sz%DU(LcqEy-q-qpugj%&6Yu2A0CF4SHYGwEeNAj4-i{Y7 ziIbv@d<$nv)0Mb%BBzF@7@k3N{D9%|vujYD`Z6zEEbr&7Y+q}q@@~kg|;{H z)T)@A(kS%~j6#dgT^P1zZ5#X%lxvE&Hr#;%8bpPc;`Q5?GK&lgu^{gIv~X@Q`r>us zum7-BOgUqK49?c34DOB1Hd#|0EuBhisS*q*kDr#_JAjzAxCkR}K;euh45_D|2dP zUK;~keIP7#I7x(Ziv_(7>c9zBjZ1v?sI+vur@4kl+7gnLs^904q@F&5;r#RX4XzMy z{R?E-=euw@odySy5N7?>6vg2Jx(P2yfBc!kye-go zDPn1?n#x?v!=BBldazTzaT-YIy}@hZ-JYT8V5y;~*2J-oi$b^{^|>Hk=et5B;chu6-nE>BuYRp z%dZBo9QgxU`8qR*YyQXo_9Z4RP^%>EVtd_K*0Fr^aJEOTA%9_>_khm|tT$}?k1x+p z4{KGv)s8Npe8Vfz%DYB&a^NTR&ax3!a035P? zsS8K*Fn2CSzB(h<#jr`~;9yJh1-3ewMe?yDXFFdW^N5=~Kz!i8>)v)QO-&kRUzTbG zrUM%+@zhBLR1srn7Z58V8Pxo2x_KjUhgbg$Z}c%_D)#?@InWCb15$7sY2oDMEDf>G z4v6xsxbJpeq_z8RsID7=C7@Rc6b0v2F1O4gLYis2c#i_C<=?i;8nq1>tM8|oet~-} zN~^01L^_L?nq^=Rnu0+EYt8SA_-C~;wb(&C?H%o0@GQ+m<(Ff|<{Suw-ol8g2O4R` z5c0?5r%eI>ERgc2EHxpU2TUbuBIzKt-hu0j_YUJcfa`#s0A-E~Va8#E0=82Zc$!2t z8sIRf8dFnKJ0MwRu6b>g96~hQ^RACk{K4Z=sGDEUU(39#o2TPTNTP7;G@&Q8l2Fq` z)AE9git&Hk2?S2qL!5%#h`Ewt->6Uls4AZ6NDUR7%j~NOqCN-Q05gVeHol8sV8DkN zk)-ket4hOnObG#EDnOc=B|fC7`9bqm1ezNDczwTazv26_V<*-snfEEekE1mO(=C?qj9fKV!BnC3cs~L6d+C z9M+_AY7D7N6GmRkj_<3zs?W-X^0hm?Lo^y@7ErlT?&vG=9Wx-XW26n*!P6f%l0Y~i zoNy6@Q)NP<-tph3i5>ZflXQ!|zV}dvBVZmgpmQo&`obc)s}JI)uQ_&pAqyXeyyzzY zU_=w$pa{Gp6k1>zdBVO!9M)x^ft<#dz-%5o{9UpcTeGEU90;4#rD;zGZqIbt<}V2g zX5L;0fS_76G`n;Px*#-jry4Q|)ykBgNFM_?%(2m(KY)?H*?1EFG3mpfTKMyKzs3j* z=N$wG-l;qq6jMae{$K{?;AnA*$JhI*+8T{<z_6j2p=w zQBX<`pOA3+W~O(l8f4-kfR>8Ab?AX6q$@SeA(-WewW`!~7y>Y99l9_p;M^$_qh=#| zf&m8aJpR!-VACkw(d*xhM_N;Y_3>yku6B{7E{hQdXrv4`huzhl>Zz~qF-Qkk+s_sma7c4+7C8aO{hngC1@=R*PaU9zN2$elFzWLKnt(efu9&J_R`M1 z(1Uev;ar75=Nf@^jYKGVSx&0yQyd>F{Z=IzENru@&oZKcS9tPqdsGNOamZd1BSgI=4vl#FK zDGLdaa14{xVZq{SP-@Sk@a+lkMxwV)&Q)=6mb~T-o|t*OXo|1T4J3#1<&-tKI7`+X zRD5kvMTY}x&#n9P%D2nSlga}ZfWC19*`RJo~F^D^kJgfStP__n=+=ezg(qx(WFrCOSh*n1HOv|Rin^7wUnqdzq z9acTCD=m&Y+l8yS&+}B!?c%lUI|nkh_wamMzGLRs`$9SAcHw$}t(~SyxZ>4D)ldq} zBiPJzt*nQ`F2)~OzDPuEyh8B2{NyY^5bMAPm(*dAm?8G_+)9=wAtG_+(0InYX4|mBGu@ToZjJz5C_bZT)_cI!YVBE$3z)r1qHy!*#{H|e8 z3Q44HGvA(|0h(zd7ltkf@>sd*!?$oKRS)68NK2Rkv-JLvkCo*(ljS}kpi%m@XL^Bl z|Gqy*qj*!w|3tOm!#v7IGL5MFzYlqzNq)@2`p<~&uEVx~`04{e!7qDrYbX3SM?eo8e^(Iv2;r{X5L)e^E$B;0`xN@RcL1?u1GkM%i2OR+&oW!?AKGyJ5U;W( ziuf0$HbKmcVAGf1Rosuc?>GJMF9(AuV*yYyzypwBr-?6$ZZK*c)<{)W6yed%j>MmT zQb)&e(h{wtc=3`FAq#D3s?Ybp0HXP0D_{X1i6(Ow^;U8 z^#Fi6BK{m{*AXH9*lXn-l->|Qid&p4_zwdvaePUG%j4M-fRcPbw8AM~@>A$@l(dk3 zxwr8?LDz7Nd|LaQe|}kKw8UYdJ;9CMIrs4z12Vl9TE1(=*8}x=Qo>>AJ_tHq!!{Ff zPyvtkCrdex8MJL5{0buM3*uq^+GFD2Y=ffz29bIoXcjSSj+1(ifb2qxt|HKP@p7+? z;Whr-Idy)2)4xuN-{t(@EBy1F|9QW+lz_jJ8?Z=G|3og7=ZOYx$ZpnM6LR|?=`^Aa zK&XB1PAbVfAelv46B`5BF6l($Rl6y#BiT;?MP3 zZ;DMeg1432>-<>2BByq4Gu`ey3L576UB*T^1}6YR*UWwi=d~5IJ5Ra>7r`>YXH-=I zDA5}c%kDlW8L?iI{T^^BV1!BEy79H)78`0yyNB?pf)O+Bt)nN!ufE<14wMkUUJq;1 zj~(=_fjIyxkw!Ee%03|5)3!W!7w8vV=-ct2M@8mgXi%jCuV+A`;HYxwf-;e97GT~Op?_XO1Z zHmuUEBiM$|@PrL9@07frm(uBL|HFEY_B&gjXpIeLYi|v(Nc}6TLqjXog!X^+l|JqJ zYj5ptApNvl?E)rDL19=z!F>o{O%dZ&^#EKrZzEm~-p|q{AvVtX4Pjz8&PVMBFuK3kjt>B(*pA&yzJHS(j_Iq6 zT*`S68o6df@S|xqiyv>uT;AI+UkW% zZPT9*&KRJM5P&NS2^HlJFiV^V)5kw!e)PQjqnh!tjP2IhPL?M;0UA%W;HOWb+L_RQ z`g0{S??T62M~_I+>a;AvsDyQ~?QwXNXz3dSH-Ljt%+Lifc^{9Qklg{w&N?NOfhc zl>k>=56(?&Rvg7Mt+JIuagz(4^Cn&ITwo|D^;s7EVC(xn#TcYpo_Tqu{y0KGMsQjz zeOQsUkU31!mX9JMck-zx-x}716GfTN3|EJ7AL#TWb9|9W210M>P1#~T0!ycC(rHaW zz{LX!TjaUy8YAsh1ud>{_GwE}_fwmE(wSN?L0_aOOWDsWFmCoFTcSkIqQc8wZ|&%g z)m*m1)y*+M{~kW~sVNHbQwezvGxtC~^$x;lEWk1+>}C2|^Snb)*kRHG43E8?+FCZP z&ef5h)QGtT=C{MpulN*SuFiV1kPxs(&_B3xSA)?>09AnAmksLrE;A6*&vX#s+kehH z&PybuKRYFiIRkV>=B{jE2ltWSmwFRqH+R)0fbcD7C%uC2Fo(;R?eD8R%5fAPw3*eg zSVEa^Xfrv@d?B!Hd&dLHU4~lrw#iY__5&zS@hY4zLjR63-%2^#4|mX-L4_z8&FIKT zDnR0PE=BHWF5dLxm;NwwvDS|r$s@|4P9QU2TVFa-MFPF4oK==lK@4LRvyDHl9k=oW z%m*>r|4cA$Q)#Pi-USD?E$JVR(9cSC-`krxt)*@^YhRXA2KkLiF=1-8)N2eanXAy4 z*oB(x!zx&n>%Bf$_1!hc_J@DqO;a{e>jGGM@%k=@bX;ObsoH$1*}<1vf7R)=%eBtD zoO}Wmz*3zvKGKWdj<@kDhP3E*E$n%D?AljlH<%|b`w@{T)K|UZn=F+ zi9Sn&l@oP++c&q#1psUlS=4R~!^oXo$GH+qh@7o=sR4Yi3{L9VE`-A{k|Ko+L?D?2 zIcLjh+LG?|#$?jR)Bjzo4+MBKSGC-Y)O)cHnlbC*W{?ARLm|SFr4{-P-rjZGKpVD! zW$AOuN^A#w0@>~GPyH4=W;8Qn&IdHqaRA;mwm`hVjAcSksxTIH}o7cUyO8V+M7`LJx+l2FK zDU;n>=Jo-vsVSn=Ut*g7NjV1q3(HqDk3-!cbjS~=&+kwTpw!sLF5V8-+s|=)kLpHr zzi%gs8+`PfAJD7z^oS^GD%0N052}H<`B|A1_OT1@%szH7VC(=#P8M9AgWrGX_BMCO zP-RLvwk9j0ya=7ay|OovW?_nZW{ZyJXyRLhDeODux*2V3%T_HBdlJa?-2_N|>ogKz zv}SutubHy!pEGm^D@Up+YF7t#7q1#ZF+Ja!L*P#S$W10{|gVx zN@^-|V%hGg{P?7l%4Xk0Ef}E-E_s1Q%P}fhmR%`fpZTjnIJYzcZiw@BBqZ1pceky{ zr+EULFfA(7#9Mk#ufRFkE6EZsr01DEN*bU8mC29M{=LI<%x#Amm+MAMRvZ2z(P_kRZ+ZJ2* zD3ev#fP-xjUJ=_Y)QteEB8x5o(d$*Xwp8o*3PL@^iHtX%hd6{{iFB?VW_>>ZJhldY z#a7}fueVdvBRExYoDSf>$%Y{ay2?B;Uw^P74-=jVoxi}yEL7qJcwNvQH?z&H+xMR- z={Y65$sOre_SJ#N%`&qM#HpVNEcbx2Bev%nRd<*87->3HZ{=+RvCI{qh>C#t02c+| zrF9mk7FjuK zFA&wA3G*-uUIL0FJ!%FiPmS--p?i}k4p2{xs%ip3b z>!|ZSi-;eO_D=K}3*y+>wCb6|>?w|{?~ox@huFGB;Ec187qD!hIozvc0y>xXXt_-} zh4GZWG{ADhSdc193ueEpL@%ZJ|dI-b(G3FxweQKP{n!JnAqya4v*7_I|D zJtJO5Q?p5m65_DEFZ}@a(;g@)8b&@V&a$C9X2yGM%nvz}G`ZJDJcFEo2g&Zu2VO~U zELSNMFSh+Wms_7;ZaNq^XU3hOfMjqSbbq#4q)67Ic{-MkXpATa15cxEk2{`EmmP;Z7{`0y+MqNN|-_ItdZ{vvVZ1L6&N0hwP+0_cue2blz%rO8b7$Q zU2%RnZ{-A!-BQVlE{F+sE%&TISgu|1G+T%FJW{C_23pA6X6`?CepEZ5*FUGw`*uG( z5luphr@c(0ugBXDz+)zK?99ytw`)|-CljjKmhomst5=)~7OE-dxo6~#RL)?=C02^@OuDmn1xj95J* z`>v_?lc)W4^wUReO89jz_{>oV)~`mUZAm^!;y9F%uNG^98sL=nMqPy2v6bY~(K?D{ z=BxQG32(Z9gWB7;6n+x~9z^Z4t>mPR@#*pF zWQwIWiQx%aKyjTu=9p;4l!`cT!+=Ir_Ps>+b|r8Mw$xLyi1o?FvcPFj1S13ikWzC! zMP;%j(0q?*`w$9np)qsxnI9m*%rQ26jl+JCE_;?et5Ju&fQCgt$dQ*6Vd3W-GK`u_l2dFQHXYR}O7o!%&zq0TWwQ&l-^__U+D?~4#ZRpP(OR(| zdgCZt%Y(hQVRy8|yPn8c)+HM)RJ*h4_M@5gI|n`gyK1h3Hp^1T1d!V6O_-RTO)9dl zwnegApo`6vA+bQ`~67_4Qe$`u;bZUiplIG zj+uqHwr2&rt?--~q{WFJD7+CV&b=I+?u+5uk0KlJT*>$+%PQOOiMc-Wp6B5t6nUjr zs2mW-vkjk@j4um+SZ5x7z)^vR-O4Ci<#ctv!o|9QwC~~t*r%;jC)=s=0Zh}7Dwud6 z&gqOuXmMS>ES{XZefGUTG$XB8j1!xhl<>>FEf_|Mg2spa<}sF7PBj;kbT{!EZCeG{ zx&W(^Bu)MePyU(?&FRiE5i+mH{;H$X4d@i~QD0n%Q z%2}T+j%|~oapabweMC3?x_U5R9?rO)OZ!%qYx23M%uYKN7DGG!tY9dBi2{tv+X;vyITYpJR{XiM_Cg}2o!ywQBDmM1XU0X z(^SRBvq~R>CPA?n;}p@d__pp;^x-con$64OpPHx{jaU=<4<1xWcPQB)>3hqM`8o4L z!xe@GI3>lz_FvY>SycBom6S4`eTI$EGb z56j-XX47;;D>Doi;RkNB7$>!sP3GYiKl&RW&TdNT#I}?p&#x&S&tOKzG~}Ovgzaj- z*C$ULhMD1V^sRXR^EZyaB`58W;u%m?depZ>mRs0z+llcyG%=&_@ADuQVDm{(R~IJ> zmIoi_AlP9-$%RU)K)^ZX{ISSNzpZg4vnz_zIw2a&K*SLjiI9+Du8yJcGogN(gMF*C z+|W7HBx|z4kYv z440jjy7p>RExz`v_>_T5mOF?KL}@>Ca!!FPA!ZDBzqm1XPJBrS)RA;42@b-Fr*IkW z_LV@jW63IoF?wVTT3-J~GdzW`mG>y0OyZasW*)ws@Os^VR_#6FVQHk0(~twPf~r|1 z7A)knygIv(P>W!7imVUC-AO=`%R}36u8Q|@KSi&GL5IX6e`2b0qSk~dSP-4=tHnjv zyoEf$a@c7Q_D$F2V1Pemo+voEQ3u#YzP*x7bvxmR@lw%t4@;qSV>6L8a9DN7sl zkxw3o$ZE`~?M4Aw;Hf+h{R;@Z%u{QLNP0fm=>g$e#tz}E`161 zXC@Z6x~oPw=-xr6t+&&7g$OHgafK`u*W zw#nWH^D+f+>1X`xoZMMK`TUyN1?kc6uDE*RX<_ctbqRJ1_gI62P4Q;q0_ zoJSLyo&X)&Gje=*<9$!zRj%O<%~g<#MoE52)S7i%hDZTDu!8BE7!6EpqgC3rL_1V9 zh6`V|W!5Xb(9kAP27G&u%9fxsaokg%!Q~S>PimG0ISVic<;wat6i)hih6;9gdv#30 zqge#Qy#9fKt}j&BugJuSyCi?WF0A;`wA;DZW!doy_W(L;5So`8-u!X4+M<}G zLw49~dI<^RV4=Xi)hF*`Wf?s*RsG-gCa!QXVZ7KtH^_*C?R*S9LD$G5G)%TmFK(TU zyRuur#w59#Y8h)DLA!j@aj4u#g?6QjoV-`ZPN=l5_rxi$Mbud{@o9Gh?lyA8Ea&&BF%cZfDMuYk&&)V z3D7|rpE1^g9IZJ(gKQSoL}`RK#*k2D?Yw1FfgV>e#Y~qrHS;Xv$Zxn0(@JqG@HcaP ze>ZX+(4B5wJJ`j%NR$6kKm261HT=CPuSAe z=T!5CScP8v~UwAAalV1*Y)W#WYWmy)CY z2RD%C3X`OozrGeAk{uC6oDElzcLh#$UaZ#H_ZLv6xQIpPVRZa~LXsP-NB`J zKCuLfIS0q<00kU2ZF)2VC|!9V9rC3h8UW#Oz^dOVDM7Ymuqz+J@mWcSVm|LX9n=lX zwQ`otOGAKLNd|rKi@$qAfM_*ET#eZA_zo7M%${%myCvd8}1lW%!#SI*o# zleKeNDW8;Z3PV;Pd|prga4+QfLon%Vg|tq>y}-a@{;K)mwC8L)p65nXjhMBkKahge z%qX;t4RV#)B=_ty&n!#n7%~F40utGPS%;^{5+sgA*}D1BT-52-PqyuwON`x71v>OC zGS4{V<{QOO9PGpmnUM?7euF6!2RaNS4T|x53KOrC$^%&y2rix1bs1iP3vCHfiMihV z$*EBYD0>Rp9WgSjnwaXljgGywM@?=YX6N6R;*a`s?P0v11NHNMjuEt%?l2e_4A?;F z0s&UiV%!C3A#W*ACl*1@8E9D4GEH|PTL1t<@R>pnn{*g^EbkFIw@;U0Yv?Wb9+Kp@ zwZ$)i%J4k^0Cj$nUd6r|o#fj`_r+iPp8ut4MzWaz1{~z%TZVc0Wvt|XI?ijnD8zoM zCjLC;za?EiZ+!+dp32Bb-c#~{TX4-df(&yAvJ_I2)I1HhAP$gG*j0*BWK;UVMXdxg6b2ZYBDXoV^Kc@hDo z*-28jPIw2mHdHd^GYpEsqzM0A?pp@CtU)4^d7;LcS9c}V z?LXfAB*_CXA!{}FQhQi{oHFteWfCBN4bnuQQwdPWBIq}{oYd>U{%j2UDDogYvIdks zCV&#y;<%R}B;wmceQUd60i&u9jxs>#G=ulVJX!%l$n&}#G#|pB=tK8C3feAv$G1U? zDBT+Z{59@fz6ZT?$smZNL{i>o=^PQwtA+zCCk{AhB5gra`_)jK$2 zw{$^V4T6#{uNQR~S;i@5q_y70_Ej4J*?Wb!sm*wD^&_>Lxa*Wzb{h!gERe(L4nzii zNxrK|>k1CB;$>i9ir`a3S#H}cW{gr@fH3mI&Khupai6Pe&nUO<5f|OH_as|oR!7!; zx7-I|XFX2`?XY7!D0K03wXXC9_i>S`UcdfVLLt2u0F=Q4UDRao14XF+u0CtOt6}Ne z{f0D}GhRf3|I&j<*UV+>x$mFg_^&Q4I?aDRu|WWIMe{6>G_RR!! z5^|)pOP$F#knP8szJ#(1dH1d3(afP7s|xAjw&k15zE?8r6gCg51ZBH5lNY|m2(lIR zA=3TLW9i;ZZM=E3cf5med^u90#FuLicO9UhwbWr1Z@-xCP#}AQOWG`a8J4-%>J?ey zh(hW+S6dum{_64D1a2imA#ao+=t&<86^xdv>-ea4JjpEYHHbBmd8k-^%xujrj^$HB ziQH>4lnd&_457=6bK@-7-D*9WMd+SuguJtPREkPtw!}NfJKI6N_&Yc{m8~x_Dg@C-dheIcY3AbwiWzNlt#`lJ`{zqBn zBvPasKcz+#m;*UirUkptta{rl0xJxX5NX4zIe)q>HHm!T&IKxqc8eT~s6q;89 zhEEF=d_qe?8~+!evl(%37uYoR%uBj}P_4Ziag(ofeh9hLPOjWetDPsR zy$Amy+7bZ;a<(_pPPNvg)z1$(*;VpEWK(ii#E~o0@}-t479$qX@r2YOej&rsXEtQg zt)u`&Bt_ev18KkL+k+?BMnUw!_|lCw!l-1C$V2bQ^a~5v*)e{yeX`G)t=C58=7WcC zCndWQ@M0_GxP*zpJ%6{lxYOSDsoCBHMwDIYL*wCB7V$Ji+^f_Il)nXL&*oo!^5O}w zM^#7)a!af?RWZw!LeA0(%cs|(h8CnXWZNW-S$BZrxadfnOW38=iLY-gqWuzPiq`C- zQq9Nt`&t3lMW^#9C=PpOr3<2~3Dxa!!!8N0vS-fuPkxqj?WEgke%9@4lqpKN_*(qK z9Kwnh5*2sqDeP~Z+Oh#+&ye+2|2+UfiDO2M7jv(k_>oDlo?eA@Ikwc|BLu?(KV*G} zd)+6oZAvOA^A2|qe}F4I-7K0uzNr}82!blluO>23QD-;tRUbb*bgn*v=&pGvS{x6D z%~id5-jEkWO?X_!mC2f!2s zp#GdkH{H#SH2G67 z5I11iWbW1~Knk?jHHoeDC&eaBK#|`GdUy@}&rz@PS8xyRfu!p2TdSf87syTd2FX=V z5#h>D&+zCv!qK3mdm=$aEIg5S~uo=R>s>(0j@+@hIPKAx2b83oc2 zvV8Kmm_elW1R?YeK_1%gbn~2nqJ>KCP=%G>D@ub*C6yl^9yalo+K|ws-}PWOs{=bk zeIUXN&g~Zk>gD~oI&UWi&~8z&6Ta$^Jr~aN0-0zhc$D*EO=@wKu=Fgoc9D3yO@#2< zjDI%A6)j&(Wv3T1>pA%78^_?0Q&Hw=K~P?^viWoXG%2i7mm zOi5@S$`ZFKbS0FA1(&REmry5vz|cFAhd+L1cVQ*Yb_!{?XdKO$7NA zDtEa|e6b_TcH9G_ioV$?nj;RtAs0&Ho=^j(4l6zj``q+i8crj)A4S~=un%;AN~`NC zv%$lY9t)AoDUqPunK#j7HQ#?&E#4cVs~y^lk{tF+wR@(7f0Js?2bka;bfe8dtU0&M zWitTEVtbL}zPylW@MG6->7Ywa(?ZVH4(%ZqyYXSqzU_^j_i}4zz<^S;x-=*aJ?)Je zvs8A$=?Z8##@KJVIos`9o|PGfjWYKBd=_G9t6T%xBVB!NmH*U;XknkK?F7rW4EsWk z8$j6jMXMpbQ5zI+LpB@1uYVQI1Tj-=$@A(uBjB7VP`qPTN>@6q_2<=#w4MIQeIT~{z| z;>@Mx73f;32cX?q&VIS)nb*3nL=L}+X`O9@(*rUiIX$XZg>>BV0fYAQd>@ccMjvfO5|CYx&MTJp8cT^%>8Hs9;!Z_*yAKz>1hGac$S&V`6gI z@N(uJ*CC0Q#O_^ZSL29DvdxaF<%;Q_wWmAlDXTB5HZRGBa<=niH>p%Iu6p+A&S$O= z9W>)yQ{9N1z*Q7xwCS?n0@miC+acFIkMDRgQdM@7PcCFWZSdnM(`&FU}11 z_4+y0DWu=_yc%7RNOb@)T3*4&|1KE_AIwlU2577E$z^#Y9kT+7mlOQ!((v}el7{{F zB>O3)v-()Wq#)&L5jLYLTkV8_{Fp!3W7f)cAr^PZhcMiJ^7!@defeVuEAgEBw3!V+ zV-K))TdUWCELqvN6*NB=BUWK;Qh(!9_j!BKO%(v&3aD1!W);e1ngaP6k!xRiB`grw zwf{u>Oge>7`^i*SZkrB?qt@;wnr;LDTMXQi`{$;$disyIP6DMEO_|6SM^`A1qc5!r zkdS%D%P(C)$VYjUAk>ex;FuoI>QS@0(0HWEe*P0#vyybk(`SRpB4Q~SONHHmAK$`B zpQNAjD7rr0U;Iib1h_jJU2MPVipD>ScV-=414(Rpa#ulWmD(!@6R*MhZ*X7Z<;Pm` zkyr!aY3aI%JfTY(45zAfp+`Sk(J4se8SJ3K(w? zLbT3cZ|xE`*ECj}UDSDBNU)wh>EHk@1eP!d{V$eL3*b)^-Y%4|6=3vHjLL2unv2n6 zjIUVtL~-7a$J`{Br3Zx*SA6)rc9NvK7-T!$-myIt%1E(zl=wcl6LCaIs@qu|uRmV@ zOj{1(vCB4DZRtRb&LH6o<(L=vj;zbg9CzaKip8yvh*(@ts865o!f=bTW7SZUIO%xF zXF84NDU?r;GF&qbSjHIfe=>D)$0lAui!}{)2+FNHjR`Y)St_``a&d1VF+-nrp5)QI zLcxdPg_>78WKDZCHmBAh!OzOdZ1sCcGZ+^?)uEfJa$vv+e3D%xnGhu|o{9E45Qkbb zmw9uE4@7M)6sMGkx5?8k@HbMLH0N<<%*nZD^8BccpSOOz>%zU~Fy&O9vx_5TY(~URD%gjE&^D|%+6E=1m759Lmf(|^@&cZ`1EQJ?7IDxSUgjR%5>r;{3?rm)md_Mcu+R2*wIj z**8#9&oLn%HIAq z?d~D`(Ue%m9Xj&HQDHa5sR4m@Ep~;?NNFFEsqvRmOvM@WRU3}w=c_cz&qQ;6lGZfT zheGMvOLq@`iVCio9Y`C0 zJbj^KKAVb;cwd51vZ>lu#53M@T&p=$UBHFMx^MgWm47FT*m|WyRf5FQ!&7XnN*)lb z)Xq=$cZ2Ir(lm_0jPk?{He_TFn1rCUkR-f&0@+`wwQYcjK7nMw7(!yd+^s&DQ5jZF zywkBldGbB1Vi)p^Yr8ceX>bhyRbqJYHCFZctl1^5+&hFL6{k*<|YT z+~Aj^6KzVbbw~B|G{3-9YZshiR_D!SWX4(5dV*sh#3We-&{WaF%>w81Y6?PqJ>^+3 zxedBgal+d%p+^eDf}GB%MTL&EjBDYv($!8KYLhu=F7LoX!eP!5mN4jI$DBd6rN7EN zf)soA?G;kz&#wareqd3s)T8B-f%80zhws+lkNINKx ztw-PXkzUy#nd0HTzn5ZiA0K(EpZA~(a|J;=n)`Y*-<4MC?C|f8K_pS!`cV8{=tHH} zOE_n8=+StuDW3hbRIh{|pw7l~)n$Xt69rgJN*?*Xy?0%gj)sgxJ@3Z7^wl*7BMkb% zCQQ8o@nj7r3PzHc`FBD<`oS4#EOpDKQDnSjPG}8CC$S9R#~iWg|J)LRAZ5 zPRB{Qd;`D#WN_?Z)9O3w5`dfOwS&{{viW9-hA+f6m zbf0LY-^hp3{Il4s&XKd18dJX4%9lLl>KulR+*vWU697*~_h+&g#}y@2OE;ux#?mJsF`hkcf_`l z3Y-L=j3`w7L(By3z9UK&f$t!dWUWi+h|kz?@e=Gm zX+t|k;D2#4^>155@_Xe_`KiTod#H0|=SQ5=dINI`#gu>y#?G7Tkfz{csXltGdUSC` zLtc~U+f(I`m045gPt(&I%1i`tw`ex3|4Z#H{g~UBJ#!-b)F&Zb8ON3@q?L<(72(o9 z)6L)+#Vjzy7p;F*!HcphvDh5>Q5bmcr`~}5Us>Kg5H^X+{4k8q(@XdUM5JwO+c18t zon^y)p^!qlgT{x3H{PcWGHUI8AIaZg9iV(|L0?R1f@0l~u^X|Yw0k7Gg)ox9R|7yM z)<#W)B{@%vtlsN^PiQUK7IF}1994&L7zaoxF*Gb3l0`f??5IS{vgmbv-@3xf$&`zH zEs(y@!#l}3g?ERf+biNQCZ;{N_O8qk9s>5u4a6$^ALt@LDp>#u()Tc1d^q5b{LIi$ zngKzHc6CM(IgEA*a~k^o&^l?7Bltqzcal-Cl{gRJbv}w>q21Bt=ZA0?6vys{MBiYU z7w{S_$b<1F^a0mJKYPuOJ^@@5YfFG?=Q8iDDw_o78XQy#+Y)6eW74sBr}^dN$|Lf= z2wAf6Lb3*!XRU$;dv#7CyIM@QEiBudFYeJD*BsaPor?h|Yk~a=MAPvzC(fqz#&GS1 z*s*^=@T4y~FY{?ic^+oNj!KObukU@Qife8;F`}f|Y)HCrLY`bIUPzKVI_ePLI|Dy` z)TvpGWNB%-T*D*0+A}x$_CAXa`RP^xI%vMwwV6~++IvrfC=BIAkBuu<+l=}V1J1nz z2FNTrGI8RiT(uNdq(y~>Fs{h?{nxjq4oS zL~z73JL31x#kFdbDH+Y1h0g%97(u>5DR0FiVyZW5`w^ih#UCzhY7sfcsw6D`< zdahY-s+mORKrcr7;Uhq|ug)?uT30VS>5K!>U+(iIh%x6jYYqO?kY|;+MLz~QVE(=P zfn#yOg0^b^UG@cvSDjD_+&dbA)%l6%=Vt^Mc zt^SQ|ll@1sKH>$b29r%|0~*sv7W7K$yuP}`RYJa=$i+Hx(a4Y0x%nS6+?Qpm$$Gjl zZp+rlM4zRe67=%rwxlvJQT2`r&|g;_$J#lXsRw%&6WD9ECG|d~T(Hm~^MAWtz-w81 zI!0^Pj#Y3MIES8>9QR_}zvF6OjgNHzx&J&!+Iz0Brew)V2&XUkGz(Fks-_sF-Y#IX zetE3IC~4R)=g+@Kxx#q}ZlGe@PU_Rnj~QNnystr?KO-KJS$*Ebh{ka7n)xqiYH)8{ z(D-&QhJVBV=yL^e)Pko%wvrCYdJFy4;y|yymghWt3X?Tsae5NOK*_^Iph&wQf6)eD z{iDr`y_sXUGW8uBbe52SrM&TF!gj#RD4B<{kghWJ{f~Vmn|zLwLhLn?YrKc`BiWGI zCSukSf84q4YxTOKhs!4o+kl`p1#H4?yn{1Pyne=5xLSMxcMGI$BQfT3DTAI~Hi^mK zgWx8gdMbMOdHa|UIg4WXy_#$hnr%1Cf32=_Db5 zq!XaXMWAM^KmtXT7JXxHL0vQ;iogw4=Qn-;i<*9+xfb=qo7o#K5gg14@}!#UdV`-w z_3tpe-95Y&a4^e-y%Sj-t zoq>Hs1W++peR=T!wv!LjoD-V(JNBBhkV*}@)W6>Af4(VBdLQ`0_F?uZm@_LFyc3#R zg{}u7xnjN};NCWck`{!ugclKUof_&MX`e}f;T}ocP*(hV+P#J0_a=US13$|z=rxmX z{?UJIka?|Lv0uN0oP`_m(sKfF96s=Q1k%lPR93sIw20|bNzPQ|50E~iW z@MPry2ciphagesPCHsILOCKsUs6hVWqrMoe&qm3rqt}l@lcpgBK(VP; zgip-`d4vZ5%}!82^&pHU8=zK1T53csOOm}xi`|O$mvSAfk4WW+_z(1QI~Dzw4Sfds za{lu+-jbXS--vox%xiZa8b<-J9bf@(MrC^umR2I`rxPer%Y0TmqmUj4Ff%*8X$yuL zXbpNWOZwKO4OJ@(7AUFPJj{fB>yWIPo;tQk&zH3EvlA~lm2Nwp`5iO!=K2?Y;n^m|Gg;f;m`#7 z|Ls&D???c%{QtUKtW0pZSUv1mQoA$u>--z|&_=xq=E5fj+c+2H_qfy-+4Q0nfMi4j zuC@{9sZz8O6r)>L5tlgm9;@8bs@C0na`81BZJ(5~ z^K*JmlXIX_i*k@fuAqP=;ym(+K!HDhsWk|DAkYY9!e!~?#-WrJq(MTl3Wzz-29NwJ zV8RmBJL-K=tb>yf%9vY=b(jMf$&d-HQf85tIAGT5%|-r5%&x9#Y2vH|93FDG?SGvM zoU~ii^96;_$Tl7I9O$Rn>nnl1stWL8@ z>O9zrJ_t}EEhgW*#c5CWTrg?Pmv5B|33fWwVolB3T@8CQETEgL{}J8=x4k7?1(Z~2 z0bweYfX55yW3>^BxaR*S^|yv>-k)1~I}iRD#^A^gpy9Xsg$2NL#>)`Tz2PxMRpECq z;;(`LH!Q4<0YgBbI*~S1;4GS`V~$?DfmVN%scfx>zR?EJjcdB~ybBUr&^9s%;XQ^@ zk%%k)Lou+{F% z_oB3WoA2-SW#U$=K{+DFuf9G?>#6eyAXr@xLgICDFNio2HH#;`x#%I{5-y7+C>^P< z!r@o|tbCOA2P2@(P!1OgbVn*DoO%2*$ljIZ%bi9IcE>ta0!kGlXcEL0266Fhe(($K zMjg}#D=5Yfk6k{)26`x9XkRrx1SFVeo#(EMQ;;PMwR#|5$^%L7H2anvn+q6u(Ztwco8@ z7W%sp`Y#2S^Zrw$Pt+ZkzV>+6Lq5f<%Fg}l|216d3pPS1n0ui+%p-NHgeh>ZHQ2u+OmRFAZSAq4-R10y8x8W_f% zL|XjFq$|8&!B+8Szv&}L^8rxE@s87s5HsW?5{X7f^&1@0dEbKE-dh#F1EgpiT(O3| zZsSrkA6|kn7H#MzCiTz>`CvjBzU&mhrXU;j`j2jWm2=~TfWQ2&H<5Y*bQ&S^Dd04C zK#d|lu-q+u?p`&OO@!EZXl>wOdZ-Y(Z}*0u?v0_QxzE1wkKpzH#0dU*m^txi70`sJ zyaIU$))=slPGl_tqzd$~Z}9E8^^jI%4ySsluw{1(+`}pQe}9EMd>HHXA8qaX{jbMH zI@m?L-74MDmkCHfq1W<=Q4ySFC;<#*qoEi=WH^#Bv&LQB&OT zC+LIxmyQ|U!&_-ZZ|B<>-W@QW=Ty$pz|yoZmS-WVFX zaiQeToANIq`_Ioy{{RymcYyvJEtTKKuNPh)r{`Vw|2rq>6;ArJiz{-@n=2^&$A!GF zN5ReV|N163`RrgWL2YTt%_|#6Ky;_EV7w3ij~V&z*GxX+^CkAceO3#FD0+p@q)3JX zsW8nBDxb9y=GOmXdVbR1YdHV(G5_u9^~OXq-MSuviN^1xkngKT!?IBGq|6=&_AP?m z?%mlVl~>?wTd=u3c}@J^HAK=^P5}5Yps~|mN;>6M3=xKgY?NR1N)y5h0jQrvU#8uD zJOA}p{u~b`<o=gUbbzZn^jTgIK637UKJ8tgKDFkrp3n@Hc3H;BTcy+A~7=?v3&AL+I_) zlqEb*0=ILTpZ|5==A!dF5t6JjAsJGMab6{z&`AiD2eK_l9G zxi7QhXGZ0Af30{)mx=R$+{&oXw_IZf1Hw`2tPvU)vwjR@L|Fl~Py`9qG`c$`)Yh6H|~l)Hkqu@20`Yd?SY2{UhTLk0@AVd)MXT#4U5?q>4-sKFYo;3T6! zCIy7c3eXN*L#D~PVO`^Sp||^Bv*p?tYX1V<5|#pE07`!gJQSz4!?|9k2>Jw;%`8#d z)}5xU>6;qXUATX*)&Ds7mPs@4H=!U~+4xNu*!qAqEt2wSE~K%*JU>J^ybaa>A>dI0 zHou|7ieDOhwn)^wvUmarVBL=a`tKVU1@a*JsT;xx`nc#3Z7Jl^2S9UC{Z`8^?^Ohc zryPnzgBkQ_22o(?VM7QN)^`*(6&bprP$QKsNX`%S>K!tC|8tY}xW9%0Rt&`oV96Fh zID8fAWFhq3t>6bXLDK;2*`>}3hOZ{F-x5GqxaCcc9Xkdj5d08F3HEdWm?%WqqmKdu zMrWEBNIUTgB(5=A@*ukz?H$C%ig$P}3?EvAD8p|MSQ_|7XP)P=nh6Z_F5N=m)r~LL zRePvF@f(a6aL@nuwi+P~oY>BzYb=^d*Y6Ez_)9PrH;dk)e9pq7UjOv!8zN-~hm>nO zcITD|h+*O8TEs0^`s4G7Z;Xq9`S|YJiwY+$5(MlxP!cen!#-fjPqz0 zN8>7dOK5NI=<3S&>Z4N9f!9%OsEKsqYq6$k)DNYDInWG}&YetJtSNYtC|8jtL+Bia zkz6wld_v_f$sj&xs-G{0OVaP`allA|n$};o|91z20&j19WEE88bnUfj@B9WTq(WHK zQ=+v9x@~)H?X0gB&%NP8mikP731G)Hl{G${!^m^}0J{rplI(7sjjOB*97ao% z)5gq?7DNjlJKWKGp2Cy6o)s<$?SMaw>lwVB+4>_94+MigBl6 z16;o~Ni2Yvp~$p?g-zInT!QV{x@{<-6fI10+x>r>9d|s`{lB{qNeQ7M7e&)(A;mSR zXefz{D-{*lGvh8T5h_Jev=k+Ytjr=pC}n13M~LkCdw=fSxhs0k@1N&&dhUDQ&*#1O z`((+v0xQ~LJ```l$+N0(G81?rTZxZ;qubscnezE)7n*C>j7CAkqZxA_ewo8|vO8d* z9+nwF%g>jdK|`9sQij6&Rb0U$qx;XNHt(SfEquI`x&r+Jr}k~wWqEKts>{@Pe(2Ng zdmB!o04Ee2u1H<`_2$0b%grG?Q%dE#w#N@MX5f-wONt*m0=yiL#fk|@( zE&by_MVm*&Ha}WLqB_Srk#n3b?`&i?0ztqQsO^e@U1k2CK`%UoQddKuln9{J^cPP> zB;SBcJp|lu3|H)i&|4`)f_3(Cop*tjE)&QDwyuAy1brP%9cLiMeX`E^>yG33kVt{) zA*Z)yK;9QdgB|IH%@So?y+uvK*S6|J8{bxgu?H|fpjI{NFiBkS>52~;(JEP+#+v*{ zPx*q>=~eYV8Jajx9Bp~O&T;*i`fdV5gHf6$rZhP>-gAv@m}4?D{y`@)REgStvTB3K zcO+=j!_X!89D4S<3lE#Z3YO*)Ai-}{4tLj9xjgdCB<@P=his+Z@iEz+qGS6Ts`S}j;_*j%pp z0`10y<}bdquRH9ahJW=s;T~Momg5?w(;p75O(j6m@5Ny0*G+jo;T}V;dOx8d4&Yx$ zz#s`NXmfmyCt+_vFmcyY<17gsf9KMGF&p)P2GN18Rf9rfp4EGy=@}hFk$=aZBOd!1(ZQXnj4Tqk;C>+YL^``zbgvkgyBVj{>Do;5&C9IH*qPw9riTUuaUgu#>y*!~T%n&S}PehtG=4%cPfRC8?r$FcSEr=VV- zJpY4v(CJZLaOvRJqMd{DVqpN>jDqu1Z|={tYK(*FLNA>__NqSXJ#97wZVrs7LcOhM zbJjB0XX(E+sG}Sv`2?>a&s`;N1b;N`uZ`fVzy@CY99-GlE@+5*b$#)!_#IQH3)nq< z0j>L13M+M7hPqz7Sa!*Vke|!xf9(DT-mGR=X(_En_*H`p7-%mxgigeW54+k##l_#N zW$x`YJHCJD?c*O$K8}@M;JBvmJw|65P%P<*j3zwS{=3NWC|`ab->@r1WJ)-uC-u_7 zkfK^5?9llHnUM!_+rVoLyBALCoM)^L?sh>|-@GsPpqwCPS(42RIq=wj#EG*k!Km>a zinj`xZxv3reETlvx$FlP7tf+8B*=Z=wZDDMrXYw>EvPPvB@I zPg$#a2SqRLv|UwabO{A^5*^1V{@?xz{EWv@6c%mIeoPy|Kz4rc)hz5+>D{1z`U+2v zI0!&E*lbL3s^qG%KYiwb3;Kg}OekDPLxt}OpQtjI6i zq*|26wb*feCWG4W9>hLkSX%b)J1xX6T?VN|zB=*M^moET`{dspw-b3$PZwfC&^qRj zw!3>p>3>G08S5i58PI?!4Jp<}_lL#wio)y&OC~c}Qh0_U=N@RmX+90wgdBdwjy9{D z`ttM$lx*#|)u}SJdJOk<`d(@J(~?m!F|W>6=|RNUqeSYQDh+dxp+vA2lGZcPYy~Hn z7aB5ek&ZBgVt!^N!{n_4y3W&S(F7eij0N)^0$R5&IvbF?XyZrs+RCG0I06{7`T+w1)uIQ44?KZfVi^>R zhr!y67taMA7}wGURs6ZujEt)ZT^Dq)(ctWk%_lciJ#d$Ng$8hHUCIhrx?h|eM~7S( zH99K1(cyrXdfk1CX@#4LAeTwM|J&+@i2CvGOZlj@>#c%0$H?L|Wd0eK7{$<1v_r+G zOFZVl3bPN3eGkD(0d;eZ8YyiURG`}@Yg|kK6ZMLc^d735AaP3JIsW{y@v>|n5hxKq z(gC#k=0LCd67{U<0RVHbF%0)?Xg}XlnG9jl<`Xbn*8D>8=uBzkB%$y!T|Yl8CsDn# z=Hzu0rEXD`cI^FCJZI2LMClByJ9rH~IShVk{TcpC#P2;cMTLPCeG6uFpt?oQW>`_W z+-c5cw2-s{!qQgDYa@2jKM`@n4E#RVc8H0M-7ngy+qoYnhfJ@ zTD0GEm!8(0``UdI#8S#1oj2=J_jlIG9c3$TJ(i~uRM^w3(DCA&)U_Di2U%A=`g(5v z8J65_1c+uIe55j4Hec8Edw4>}P81_V*%Y)K!nYIcg`HY;iEDm6TW)hb#O}kO^Qjx6 zy=Bm|ibi1}YvL?;nPDy?!_hD~i!~CJOoCnX6KR>LJ!N2bqa^FuVQ9rDL!Bz04?)iN zj?i9+6rmNujN8D?PNEeit%*H)vWN8hV(%C#qKtq)d(e)VJD)%zK=OsS{4p33u;X~~ zc-h*!A6oKUsM`q?d#k3xdQKm-?nMn}2c=u64QFBV<2555AE^-AdSS{3ceF;AaU_-F@j%&TO!N<);U zM=4r~-y_eUV+{rg&A}y8Wd(5eI_03 z*IeLl9JD5ia}(HMEofmB8h;dZ2M-C77MRxIan~=%ZJPfVGnnKOyMEvY7;$C(>r9$D z!l7DWsmS937`>}>IepQq8OZ-rlPdMdiko^l;mvTug*cdSadUmO3Sjkq2 zkR4bU*JsikO?Ouy=HpU=C(j;LY~gu3>fPy+nn%qsi~ zoj5D{bne{?;=9(sPGB3+P3qY}cll7zQ5>Nh7Z|M1*;HN!-V78Xb*E> z^EDWud>RbilZGV7Sp@92@L!P+!^;XJS)j-!5?aBx22_B5VE{E?wm0U$uy6ueeTmj0 zR`)umJi4{%1`FCE;rk>j?CwnZ!TXO-K4Dc7p^5~v`>dIh($(O7_o229$6JqlHe*o; zR8!ABy$%LFlD$v~B8t`B>|K7~sdw!cPsoOpmg8NU)|Lc8MEDkP?B#-jTs9OTE>o)l zK97M|_Lmn49a)!amRH63dZ_gAN+MtdpgX*2x=*b%qTB7Pa2)cIkqU8 zbc_$TbTWz^GP8VWBG+`+8|87F9D)5 z>I7teIic=6DB(rxM#@)n5jA6>(Y_c08Q(U-R6~r5cfy%d=SVIG7odcyYrXZsSQxYu zFFYN>@hAM{%uwTF(oX&Mb6#l8`prR<18auygRq@NX5Hn_n!-K}P-SJf!qEt|WhuMit;VNhwqewi$C*P;a1+N3#IeBpTT zRE?2m=$#0jq3l~15fJtTT+t$s12utaS2PfwEzS_aut#AHB&-l5XPZ$prEdfxU8rRm z92cm3P;zmtJGLPB(fDF!#&SQ91H@lGx+PkJK<8bs%y#%WVl{xmQwSo?@MTT)gQ=G zg$a)?jqxDEp)N(S;L0#~w|6m7^GV`3BTX+FwEszU z;^xWOUF=(9z(#*CFr@xw&9b1ooX9bjt*|qg7ls*)f=|{7CL3rTJFtW7+$xKy8|k2a zl755gv1Lnz(W zA4Z+7C@8k2T8%XCWnw3lT%V}29bC*sV&Xd%nn_|2c42H!J_-dQdl3rRf9PBn8{Idz z4~L&awHlcI`ID%?GD@y*OpE#*p&4$pl=we9G-OYQqMF=L$CZ53sqa;JgY_AkgvW&Q z3I``eA=5i*pmm8n)r9o^jIE`4JItQO2gNY6^rgSj++I;$9L*!3UGBd)5ez$`J_dV) zMX|4z7bP5i2#K@DQ1GWRAL}zI-<*sWQ3R&*ds%i3V39!&0slJ<_U#L%KYV#I`Td8( zu8{4}0+V8MDO7~zLfCvGOe8thabhPWy1+4=n+nGpJ8{ix<8WQTT$9$sPU z3cZA4H0lX-$4j72EogfF3DTjp0%DZqt(efu_mJ|t@Tu#Q^X8CN->Fx4s8UA#ORZWk1C$H0 zT#|{M>Sv+D+ZX~zt%?+(+s^Up&kZZ%R*_QOd@|(J(cT#KhO_q%G;G2_PhBAd7QJ~{ z27f|b>k+=fWZQ*#+HoTh84Du6tU0;R=aTqLpOuFT;n2syV4OAQ{8_{p{j-CDM_nTt z-HltqMSmXei>N~FqJG6lr0qzmjZfE5N?_e3(S^jfns^_dh?`3d4wvFFb;?Hf_pk4G zcPrw^`*Vf2LaGRJgEz0VITp0?J})O<+VFSPEpr;#`LFL5{!++t#dA(@=Ox^guf)x% zd(#%-upf)^f*UL>bK>xqCfJj31obadaeAV9?#nz(<;&Cu2BpU{|=v55X%lbmKsIOB0 zL#1Vv@K^Q+i9nsC_N%5}EJH#=7bxYHfZI?W9UAJ8P}v_=xeU7ZtlUO&Z12_N4dfUq z?l**QCDaJFEm^JGD`MaRD@P?^jIZHbzC$@So-&0nTT`QAprWVn#0{;jFN%XL1LQ_F z1;Qxe%>k#x@Wd$?EDJj~4o}1zvZ@Bq{-S)yl)^2f{(s897^&m0;cxhV?0WoU8~N)0q4Gu!2S>mfbGpT#|DKi!NB3p zC(TERuy;AT&UComv}wrHv1WdD@?dtd50nij1y~nqExy75EU~}(Xm6~UQqr7-lEh>f z6urwBW+7PKZrhL%*mE=7@$t{-vNLsnxs@=ETeI(5gQ8|G%ns?t4ri_|7S-I~((yrJ zSFOA0#@>&WgC8xRR&$R+2BbL>B%fav^?+?BPr)%)wBDBn4IXYTWQ`*M8sOBG+d*UCFZq3U4T99LO&$6 zLZtk>&bi9jhOTTac4@qSsjyPi3_88OT3)Vp?aN)A`vw-dWyxd+?RV}rbcqfyd&^;c z(QXLFA0!XfC=DC9arS5AH9+5{)Y+dA%5Ih;-$jP+>)6-btjinznb&0PX0!jJIhfwh z+-E)LU6`b`a6?V?2<+y1z4boc(emvv{sQ^!=G*z!s1_RXZAf!8rjREEgo0tjPh@Hx zgD7bb%fkcoOZP7})0glVS?G4Ia#z<3HmeK(Z)Za_fc*7}0|dp2qeSIrs@+UjYv zrc8c&YD!groauV^+d?&i#p2y_2N*79bO3Kie^JDc&KM33#F@04;V-fBapYLEuQ~0N zqBQbb$rlCKRVLywkY*~@+yq0meDJc((X5anBs8= zdesTH*(tld`JNHOo^}9Qqw72NjCAg)0~25+B&N=4L6?s`V##m!l_UqfqCcIfMpJTq zyc}!A>Nxd#H|rkaw#^MJKCv(yiCI0^CKVwGQZZKy!Mb7(;_9rb>Jg*g4@HKaCRHhw3R!>I zR|k@0g?WV2w;`7&cWpu&V2PctuWWMP$4`$Od*6vj2KMC`WU+xfeI^ZkVG*WoL} z?LVH*l6>=19Wt)!E}W^hNx$z}Kn1h!lY0)K_ue7NbacT!qhC<4rdOwNh-j2O++C#+T=uF-Q z!Hl;{uFMwimbsa(>M4_Zz-f0tU=baW9^>`CM59g0hrbmDPWmwE?uoLO)-9?}W3c8& zhG3zXhQ1GC#utL-MZ>X;8zsN#!{W9Qr=Id`hylNkcdKc0?s?NJvm5MB9^3a`?mZ*W zMf@7MVs(#h>|O?SB~q7!Z4SAKo`YJNR|(C!ZJ`6bkM;W>_wODwckYaY8Dc)Yg+$9- z`6pXR_!$qQ5Za7SKZd>@x5} zwTAn{N8<0D=95LGZ*@H`BR$QVFMu$(xqr9(V;i@(q`+Rcz^6SK_BAnn%~qBh`Wqi+ z`zCchF0CtaZYkQyCo>~AFze|043rkta}Vhk>TJ0-24K?6>3*viwK2|HAHR82e^b90 zw2yRxUPBh~*_SyhYi4%W#KiVG}m71ez~fm>#6jFQ}K2l}JQbj6{}4F(BdJw0*kxDoi= zGF=uk*xELHro_JCZZ>9`&`Z}%9wG+m!*ZsB{J#S}3W=FIx^Cv1lGe9cOSMkW@^V~S zfcPf;r{dIDK=<6CQabTt^n|EIFP)IY>|r-p6~vAnTi486w-p}msQazOn9G$h90;X z7AkmPj%(=Ay&5w@uWZt|S{P(9vd(`N2CK&2=sNZo`B;swQhr`xi4VyC#=-5kuL|t! z<%rn(KKmvefa#~v7N2)W+=Os!j@>;rI%ZtN%S~p+^%5_^n>uPxWlK1cJhgpV;PAOet#YHPc4s>Q3&rc9+kv|w+wGj`NkWSK!5hC_KpkP0EK>LR@jJ%z;mk^vF`)N%M5U{SmsvkhyaGsjYtj`#7ttT}fX z@W9SJK0My$g5}uw^Yd*v`qOWhB5Y;u@RpoQcrp`{KGPqT5RGLpoe&hqKfFrN5}seX z=ng)G)QQD1MXo9;1RH?+iH1jW=ucj{lyHzKdgI|S+6#`It6w|=;BH|1>HkJB9G~F4 zQ=Wop>Y@G3j{bE^E@0?S0~9p@&9P`N4N?iZnSE59?r|&4lpH3^Gz80hT|j%euEJ`G zG3XJ#VBa$k(Zq+xTfpNV1P*PW!}1A91PCTFd0Cr;xOOlB_{ceo!ck` z-hNBc<0MU1S?y1GPWvqi6egrPBFI!e4lDbvF~Lgk!MV4SSzJ^ zAO{N)1(-%Vo-mt*k7ETkej)YlL;T)M85I;$-{=`2F8YOsJ@xhRO9;8lavVezo*=|I z^r=!`DZL&eM*S%&?N6YeDD=q3QLAr2i%hKMklW2X=(v%Z_%g$piuG61Gnq3hC}Bvy zM7Vp&;geJo*2tGwLj5Tw&dxHAfA6Av>8GvK?~Y@B|AXjoF9kuz@^2|0wc+rAtjESA z7e(_-pdQHt9d!^?>}lWkX`{^5%Hb!&Jk{cZ%HZwsiOwhXP~h#cGewShyn|1|4z^9I zsQakBs47*pWD;*DrXXosCOkN5fm%#gnGEJ|($nB^%aqSP6y7$vL5!OW=8)m95_EFk z&?u>L3gl0n)af6#fY@KO#jWrox4hp4rpSg=?4P;nc6@V-H;^cH8XKI_4@PO|IE_-iM) z&HN~oNQ-LHaeh!b{HAt-TrgQTm-9#;o*cY&(S^I_tZ_k)30@$|ObXlVotr^WNZry> zL67@sr_7$*+gaG)^lUr}kbc1h61PqcS?oDGz$^-gT_X+>jTFu9H;qhmG%!tQ!&C40 z5oq10;(!-=?MrElA@A6uWEuGnuvU;wAji_=NanlBbW-0@&Tr%=iXx&tHtO zTs-CxkSX9_0vg~I^c&v3KKVQrJ`;Me+y6NGMD7ojl!8Ry@&1Thbs%TLGRy!~#gAU% z-&_tSmKF!^*W)cOAe_1ADx*VX$qUB3tW+4hHh(52a*~H9rf`F0G6SRG3&%ZlnP@rL z)4x1~P`9fhmW91@AO6H7sg^xT8&LPmH=`C1&uo~9F8UEP0B?^^zQ_Y$G{y`~lI-w7 z67aagtNf!pIO#ZI;v~tA8~5uv2ua@+8^rK9U*eicKQ66|^dn0_z!cItcE3sKHZt+e z*aBQDeV}UxF=dc7fNCI7DPY_X8$;a69Z}w=MlRJL7c9?YVMoNj0lO~htW%CBUXvx~ z29+H~WiFj$rAlE&x@F-3?X2?hvM9IU3u47&$qpYRg}Cn3XA2%b)f75!kc@r}^j{gUZApPzlY8NCH;@>K&PR8I|CIFzb=&v=b(edG=4op)-u>^Jj*}1$Y5~-h zCT-UB3zTQHpC%5P(3AI4K;5H5@4sKjU;Tf8K3@b-H>B0X8~hGakLTzD>b`yC@0sy%(&7po_TnOVy~@kqKMLg;sBRoM1ET&4zmd`b zb<*>k9x`H`Il!}MOTo)icRiZ=e_)-6p9z332zzMof1P^)M`yH3kn-sj%jhu%DmRQ+ z#l#N^v##Rj`01Cw{Lhm=B&ss;xIMeoa}~k@j=O(ytCebs6%&I09goqTYbh^zn#O{5 z6v{L5BHm^K75o~6Mcc6M{`r62HARJG#-C>zM|K+}r)+j-DGkh@@#kSLOqUB}V1n_J z3|k5QqNGI}>3`CE@NBa`QeESRU8^3Szrv#T>X!%OyJ~s7nF@RoJRvcaGh5Y?5qUO(nYj8-fc(WWh&T9GLSAj#Ls@Bt=R5b{k1`%;eECPR zkvOm@zqLz-F6IhE*La0!&y@UIr1De+mnwRNQCz*7ldr&VoC)$2czvE_JdN2qvl*?c z8yv;TjN*@d9Krz`=(`;7ZH@E>2A71Sihacd-2yV?G~sc}IR(_f9pw`I`YAJ*(>y{~ zz(BqKvS~B|Gm7r|fRs*f?_K15WX^GVDov4Q?NsYtVSR=g3t$QjM@&KOb)E=erN=``HtWxY zjASv98cMN4Xk5F2-d?~j!0ccS3yY);5gNrUvGj1@Kb+Fb92|HAJpqm1W1DW$qv1EN z85AWj{K7=m6aPLyqiFSf|FP#u&KxR~*v{A;#wX#0q-cc3f$6tJ=w7GGEv?M>^6}Xh z(jJcx8jsGsK!-+x9oB(K0|M`YP-hsnm=1M>2Soa!B*aU9SI&(p@RzOtJ7Kv5y0_!p zi9!r;`^V$bwg`2ymTYu)cVAAhV)E|Xg2+W&nA&uU4)lHOa!eFaQIN^MA5eF+Y5rJI z0X~rB7#%Fg^gpiLh6)Q(Awt~{&utv-_@r)5KT2eM3?dWJ4SOv@oq_PrMQj^8?#qWnO=dxlCm{b=!NG1U=#0BW!ti*KupN+z>kD;aCZ)z9*tCvaeI@-Sbz-Ci6O`B=LL`VJ(Z`i389_EgIrxjhy4k} z75eQNYQQ!5M%J`Pq%|=uOc^WW3Nljq9PrU%hqqJZedjj(ZYml^sbONe!3RwR!dw4R zC74>KV-!OynS_(C1#ELnYll2lGQZ=_|HJnB2i|e{%YkeYwa*-(-dBplZY&AG}v-NKM* zh-dmg<~vV$#z+rjAQg?HzxY>)_n4;%N*TDW$YIy0W}d;aVKQfDPf!KeWvl!gy>GlE zVR#a6$6$Y>ShA9=;V7-TixCbmPxS!%g;3WbDEi_Bh4M&o>_$wQ7!;m@hz!J%&$4<@ zvYq`n93}ZRhQ5i#9ef8b0eoY1^tS4CGVr6ImWUDbV}KszVMG;U#Lxi&$;`-G*U6?i zDVVjBITL?WnL<2e8+PMiGjsCgA#r=C4#J(%SMDdCk_hRPOR1F>5)zW z1kH!X<0E#TAp2P+bEKsvcY8NL{UU8ua*8!b3o}-yK}IG6phy3O-qdfr12QWqES4Nf z<;Jd?+`FOX8bHr+MHk&I|2W2ysx`)W6P1+d1u!8A?q0vVf$WnBCKAW0dnn{fCLF<6 zDX^*`8%Azr!DgkzcPi};(0n9i?s#dc9`!xsuj7AX9cUq4L1cp5?`R@7d7W4uBs?BX zm~hKA1XyOl$OnCBDE$0tWO$%x|CJ28AsE@}Bd6Ea$!PM_FRofd1rRA6jFgwXrwOPm zIa3lhu6c5IHjLFe8TacgXEyXXXG8OrUbQteuoR1gEAYPGO3x+?rC?TDL6$1>XRU#6 z%?hTHbi>?ovU=pFb@$Ryy~Zs`yJC_Zl)6b%9j zxt*?nGcAVgXH-GA*AD20+2eBYA?-x)!6=Y8cg#HSQzZ<%sk|W^;f2K5gq1!SpE(X_XoQ~nRxdv*d1}~zFlvp-4vVSK zu8s&Dv=3XxrjZ^LT6)^M)nx_yXw70RpZ67t#zb!>z2`i`si^1vwv*zEtxoQuTGA<$P z9=jEVd;X!F$z-Ck7|iOg7Sp#7@s?PJKd=5@at}WOSL-zWxcE=!7he8v`HWt!x(wj1 z&ac41-55%=r(TbWrZGiE{TF%>2XKWXArvoG=H6yu_c)khG})Wuq>q7y+rn135n3T8 zQ*%ibPo>DT;(A7y$+C(wb21hwMQGkdh`arG)jsT`N-n9UUnW9ak;Bd{ld8C+iU#~F zfL*8Ar){kaoWLB-fIEb$jc>du-W7BQd*UBOF0{k`(S*y7@+KI&-HWAnVhqMa1qCqw ziFi=%5C#y8emTsMvm%IU&?Z-Hm?Ou&_U@~4ulO^!M2@IqB42?G16F=xx4HDzyhN~HSh)5`5 zriwAJ6=2O_h=RgB=`JkPtOtMh+coS1R|-W`1XM}_n1KZWgoNog|mtWW`a{9NyL>sRLg z$IUU=cVHPwJJ$SAHe(FFGe$lhb=Y+$E#}eb<58m1`B@#E-ea0%i1_vJYN^BO z3jf!=hxi&bui=%74xPd#@*RTCOPKGsBM!FR*k+1^!{VeKo(x;LDtaMg&$5UKwpN85 ztty19^E1J6+eJcsp5*4|&%30^BL+;kz1sU>cIp!Igo+>ZLcxKoM57byo9XUTnO;8~^S{yni)W)GL4?N+T zJIi+Od~T?bI#}f@GDtv8kj;bpgt+%qTmSTGpm#X7FD3~6_j&P$SuScW%D;8KeiIRI z%3589m5i-A_X^e>7qWpdXyB6gZWi6I@Jj|wqx%yfc0B3vgr#0%QGUpI@m)lGIBWIc zsbC8Z=v%`Np5+qVK$9ONd3GaOTyRunjmOUtFNd;9d@bxj*OLxl+ru#JEtoeyU+r?s zu)Dh~NO}xt^%7M~Q7UN%^vL=7q(dxj6$i~$U0rv5tb!+{7Te8K%p^w%vXXseYi;Tu z`|4?jhQ5;dr6B#NlC-hsTjV#`e!Y0Xv#h|^l>W_7d)MBc=MgSK)}!{Ssw817DolaP zUy>hzIK>+dEV$~I@tcufmy~qS+%o9W1gjNUs&@y4FS}%%7%;tOLjeWTXOCJ4XJ7aH zoIKUEB6BL!oh<8b?7fCJyp_e^df|hduqT*mA@W`hZ`WyF%0(jx^kL0n2(^u&bRrUS zCM|$2&HQ)U1Q@iblYlofnRI>p+YexfskR60Mi+s6i}Cma zoP$fqkt%-M&*0;eZnH*FY_Tx_j3WquM_$eWt)v|>L!4e8k`l1jnKSB&g+VlzYb{?* zA5q1ZvErvVZaR7`*-Z%BKuH1V-EOQe(%aP}2BbP&cFz@gkbK^i`hyD{zufqMD9@yZBP@i&^FL@{um(JKLlFo0%w(mF_+g?; z{yy(~nr(zWFyG2TvVriRQO0^ddex=HTFazShra}Nq?J*GDjngGY&iTFnP_4!@D)Uc zGov4L&~beJd7>iY)hD!Q^LGK=B5=Zw6*9$r*!_~ud6i2j$*bCrz8`4T zL6mZeD#JJb1w6bWyuGjRfG8F5Bqoc$DAPJR4}j?0BcMmkKI8RSYsU;piVZcs#vljS zgCVIU{rajHjcCg!)t^EjA<@a+NUK^P?PC{YjOzZ=ErE6xQPRF8^J$L|PGOXp1|3u( zAkq0z-H&}#Q%tTy1s{V%=jp^8Y7GTY!Wa>ZQo}!h#{*oW68lt?s$%)YtEgz8Dvb%0 zJz}k(aDZKZe22xTni}HsrYoZ%U+6E#m5vUjuL9^!nh87|9i}-Jj>m~I(R(*F zP?h9GDFVG1L`e9@;fbIfp-2CG32u~fNFgjg8B7|je?gA>&`ZEPD{s@NxD|{h4o?R0EZb3?`I=m+fE5>b$-JmW zbh{1cRy#^Cm{Q?t!J#sVc4yG~213AO4AWoU|ywPVC&e8Dz?te1D1wGBSd{Qp}3XML13h1=7wik%3K-$fR6UMDXqU zZF0TsiX`IPF@;Nk{l7RC^Z*Qaz%|70Bv)Pio@2&Pya})PkGC&?P;+B@wLA3Ml9D(w zCshCdIf_o8o8TUEa?FhRN>Q3lUxRWCn60|#%H&c}M!fx3*fHfP5|=wm$qv#lm>EX_ zDGmM;8&?$aFuDnJdvN8Y4@fL~5f@R!9*uM+a!p2Ag53&gU+_JJIUYc`|3`ZY4$XIX z#JL=mg)$pbp_@e14`XH&xkEsqSA&j{Kd~pyd59nMc4tvf7j)G&zn1 zE47w78&$C&h;qmdyx0AT#j*NdUrl~t#vf|j35c_{UIFjhd6WD|C&1PFW^ekRq* z8^!#;R4f_0R%n#YG=Yyk9NdR=Fq7_1!99bwKVQBaV)Vafo6V%r8|A~lIS#(#fEwkC zvZXda*B z{iN?fX{@Dy8GVs30^@%oj=l3R&`olYe*ZAL%H$Epasn0oh+Si2DU~ERPa1Lfpt*4P zk8({HQp{L7|Bu&iN2P-2i!X{wPpUYShPid{dXvMme}BN?Cg1(a-S`MFw_Ebg4yk*@ zYk!HG4}t4FsIh=Qr^WtN4}?dv9_)(_zKWY}G&*X7rZmTkY%}r?ivuL-f*O*iIJrqp zqEM46dT`Q_aFN?rqb6Am#QYeill{f9;^DDJAU*3kE7$E6afZ#?O0doMIPKhZ1=VSM&)DJ@M^r z@!Bc8*8Ca+$U6dOdgnMn6#~af5+}~P+n+h zg?PW0(LYwNmWZK+LN5X=Y#!_0%M*afK>ObN>V&H2@ ztGA{?pWnDxASjcTzleQc4D&kB4Ai$l{{-qY03I0K`CeP4TV))|~FAP)#HowF^k3)U_zD4DriC}(*g z?uyLO5!k=nZ2#V|#_p#AY=c7;YRyBg_EP#u3C88|uUB-p?aj%fUYwznj7%qxG=b5F zdc4#bH8d)sD2@~%R!{kQKZ^?c2DP&Xg}iJE)@eU-Ez+@6+t>k%r1h2Fd&TxDO6tMp zn#7vAVt(5~m&$<(*=~m~d+JvYTNmkSdMfreI)q*lvQUw)zAVH+`l6m{l0z>C%OQ36j$O{FmQk z!gBAruJYO^&1a)qf}0KeGVa;e-?sRELEsipqX!qS?aPs``a@qMip+nK$85hB^lwa@kx8a({D`@0z@stMHjh>iJtR ze~50#V!_NI%_6AJq6#7Mhfx#EC@}?5dT8jpGBo^L-v{jpbf`pikrImSZaiKZ8ehG9 zbu4x`!Qh!yMoNlCI_yeD`WhNqFfF?GT~~8xsp5%%k&skzpB( zwwG+9gyah^&Sryvq3*KqGrU*#Z0!2t;Ed+RZw!YAw;yw}d=|Pa`_{_kP<*LBbV#Gj zDDJLW}+Dq7S zu8@cGdoP9I;-HUd8R+4GD?&F)c5lcyO}#RSm9vowArJ;gv~=nPO@k1=zQGhmus~i{ z6uckYr4;|Qr?a#S?`m8kPgRCu(PmRzY~us>ELZ}i-F)+u&|sqZ(%JmnswrT@~@G2iCq-cwWECAR>w>%g+hycGM5 zHdZt!_Qc6Wz2mQ$Rlin_yFugYu5HuW!>}^Rezs9sXs&a-^=WzF=xjCF@&iXbid|sG z(=9KT!J0Z4P%QVZLSj_lxs)^373Q>-^Kp za@t`(VVJyb9kIoZt2@Ks+ijO>ouea!Zy`btKc<;PNAu?Wrp4mM8&6)sZuLfmu#Q8a zP`p)djx7k$7F*xY&=8S_x=|wA(t7dFLQ7?QkJNHy`~qn=x26vbFn%YT6MAYOBGjSJ z;9J-V#U(;65)r)w3qec923Q9P9CEN7#Lb{e^s7LAM*2_0*LM|L%|^=GoMTiIsy0ey z!lHrS&Fc&U%(oBOWv`6!u?t_Om)7?w5AAypm_H+FHm)?SJ=GvVgkK<2dBoMZy0>@b zbiwMkBkcp#eXwOM{ zz@~+b)}OZWEmJjCDA8f7*j7{I>re+HJH};%#IQ#Lm-hTgz>gt@}7}&w_=I2H)lXy!YlA28F~X1SUvlK;C3%9iW3 zUxA0AGy)#hwJCYLCb0#$0lo&)*O#f8YIJqXS2^&tb|8ABc}-7XdxN6ncCXbnn}5ev zjigQ!Spg&89e(;-%-8w0U>l}ANtcg5Z9vETEo{Tm2@MTR-iFgNh4I>HYOB~X3@VMI zQwPr1R4#XKR~+ff%d78<{|KuAnvyFYg!?J;Nqm`hqE|*SFX&|F?6yzova@D%f9#eC zsJSz*x~=U;?p&qTW1i+yue&vPDH&GFgi|%C6QB2=?yz83%I=54FBW|Cn#)4dFlEa; zmEWA4f3=F>NVIKvT~sKReu-qR9M-?>y2!pM+#+js3rJkSKG=ISew`* zG;P6>0u%j#(xRO@#wE4gUEwZ?uyCW&$jaRv4W~D~B_0Dqzeko;!T|JnmDQqo9EY_U zo0S*XKU$NRt{Z!&qIz?FaLmYX+lZu>(y-mbGBlMQM&q}wP1@L3d*ktsmXIg+0N!Yq z$w>$I`Aguj^!Mhx=fq>|^6EjTuT(puH28g1Hjr{rV>g5+uL>g6bqugU52WiLhSOE7Pi5@i{<9Chhc5Qsy4}o$?dV*TC+4|2MLG#N-bpnsn-!^Bd)aicUyI-k`Rx&@#(71aqx|8$7f-8j zA2sWOE?~I?IHIJFWCZhxEYU>@@@`b{oqppowIb#H8Em#o+=r{N-o4Ta_iUZqv6p+N z*aQnjn#k@~@}zSq^F8q2R-O$uWjE?SGxABlz;?iE*gvVvrBk;2?tr0-aoek&^QK%z z{hshY`5Xa0amk0L2GLVBE}cc4!XpopgWGzQ8l{F8oOU?f z|APFhL+JlEyL4)5+^DQe4y{j4zU>kb+_t#Fay@KYY2J?3EXa3ciWxfQNhfW@U722; zXP96p;8H2<_$$>~Y?66(Ak-rG6-!h8>fox+m+ogvqshz+lFzgj*$|vt3rCogF zC|9x|qSzZ;vcmx%Jj<&77KTC{-NsI<|YBbQP^SG;a`vPrim!NPlW;dSDg;YM}*6*>#dTkpIf8( zwcGq?uFOFVjpO-es{5{qb*IZkr2j6~1nqkHx50>Uzo!j-l@-ib+nOSY^sUiV{I}PN zNou&(LYKE35~}ONmrNIQeA56MQ`KYSYdlu({I*a@vN{>s7n+^suQOg!q-(1?w4vgm zyD6KXi=){mr-X)ejS{(Trfb6X^tXG#_SM+5v)7fJwMS(8`x=#%w1e+dW+-+?@U#_u zl^Cq_f`vFEBlpyXxScWr9&B@*y4&$|MsMHtPn&ZjJeA8bdGjRB+_Ac%J*k~M_HtoV}nbw)n zu>GnZJLE1I{}e4~uY*lRE6|ReY*=3^q_)$M(3>gd-<2I>yMVF7E9Q$FdN1EDnkzr}>s_|UNV|yF zRRNK2cfnfhx^%8RwG^rU>Hs5}`tO={1u6eo8$0*9&61vG>DCuu^?ApqsEXh_&8kir z(`>9IXk$Xr&4LWE)rgW(syWlPb({N+*oCh+!Ryt&zB{A2y{xK?rTWZJ`@N!G1J{Q3 zq^kK9zb^K_$C*DFi6yw0%|06i&cXgHuM1%$7QOkN5X``*yvfLSDa*db*l!y8QcStNDt0g-qU|17AJu9|XG9xP0f-f7&n~ zR>#@}uBmwA5w~XrL2$&`FiIEgndlMC@R|=hpM2lnZ4D?XelNG*1gt2-9NBu|mTIT+ z-r#ew?K=~qsnS|}y4_gzonjc)3ivIBdaT>$wVgL8{&Jr+Ts`$cs_?t&Eg@*5MpyaU z_j{BSrI93U)^(kgRsDj{zEJb0$M1)Msh2!qyOGgP$xp$~t6DRvT%Y#hq!k~zAH8sO zzU-~uDhr)R9$OywK95S-lHZ!i{a+U+b>S4xe}f{2Js=;rEo?{2Tq>r^Nf z@cMK@Iytqb*FLScCU0(~V!wwX+u$4{0>CKB27hQ4hH()cDanboj_BRXmC~dS+i%Y-Iv2P9cHjKIQ%UEwEMdK^H zY_@N&DA=So_Y#-49-5Ok?J%EsTKUj4k)7$&OkEAy<$6dP^RX4u6_*fY- z8OX$x5IFcD@OknMTOWY)MQD%1j(*#c}3n8g_c9fdOcq!*> zLZhi!yUT{{?RY-1oY(JQ_3TY#gb55N2Nv8_>McCiZt{SDn=Z&!G80xnzdBPM->$Z^ zbH1gJ@r{qxuaayV(>SS~4N|8oZV^YDBw0vyO2)7fXz`gx5wJ|o0#-2XHj1dpw%a3} zTVftQHYp&CM|v*(Pk?N|CF8SUHj7coxg5jw=7xh{HN+@(!4^*`FiDO%X%0JN@F`Wh zzU!SL1HwH?11DLp)LZ4+T-c7Ot&%+6->+0gGr}f#04wz3zTKtSe+~Pa0P@ZB$vg4ZZgmu!qlRGfszYa zwsuLe^L~F{uLv^MvLL3B(N_AsZ>XnIse8R*{uR!l3>(#sV~zuILJa}V{XZ5fe!EiC z{{0YaL1Vpay4H`(&qEAj(O*u%JRvJ}(tVD!v-<5Ox7$D7hGn$3tpe zpkj9 zyJ>6&jz@|hfptZebr{m+clTtx1`9+pLNETF>KBxMOo_WPZuK7D1 zACeTJE?gYz49AOOP*35gvrhgne)zHfF4W$-7b&&!@!NX($EFW|2u>{0J9`t1W%E~C z3k5DU=fG$%c^V(4 zKo4noM)4|Sn@*y9bS`QjQ~6TC2Cmm3Er=$%Dg4SPZ#4*pjiFxCDB{TILW-f%KuJeo zOwwz|s;MGN+E+@YyHHrio5hN`^w5#jD6D3#MS+^^+WbgKvLd>Kg{7$UxcMsqfGmFqR?BUM z;FDO|8HcA^X39SjGEbX*{M|*aIX7;JP=i6L#qzpOU1wUMgF>Aa9~S#A+9OHM zt>^4X(P9khOjN@f4P!0vWe1PV{O#s0Sro3Xj{-Qb0de(@+kgS7!!c*V%C^2KmW*$F zu%oQozQHbAdU{c@tiTPo!hnVcPgY{ES}Yaa(J@~ug}Bo(uBzlHOw~Egof;bJ*qo;g z&g>O#ENDv+J-)Ht&me1lW$*h`r7eF2bdOAL9GgZ%ahV`Hdqe_=!^)yWl@gJO+F#WM zzF_v@ijG?d9}yfV1XU+BTrFj@^ZRQi@}JT{h#yL za+U;PMtv}<+<>gw+XSGGju&Ca$Z#XnVv{?TC#$!@igpW#vUtqlG=}|aI^d;=1-fI9 z8%@x|?xvPtQQ36-PMjk7wSpH@(vFf~ExhmQybYFuTYiuSot{`3wD9j! zp(07@kE0LdkmU(fx)*tkpYp#_Hn6g#V`0bp@wgwscj`;r3(yw{+cESDmk_CrFFj!G zi8thmF5;|5i+NO6{(>cpE={hU=*1~f*qMKNUK3<6VRge5mhB-X41E^~i>jF2sh|y& ztU=bhj&7#AvXwp_J01uafI^{dD;R5pmo558zkTA+3QP@?r^iJA6w=(6&!aDI!}{TNDg3P2c-%eMF3g>`$2V!=*h*sW*_h9wV}DXU;RiYM zw@mcgeR{B1x^H;Tlqs}tkOpyhk-ztw53@Ed#QwPiIW2k0Mzr7j=xkVty<+F1(d(e~ z;kcgZ`KxF@N?j;^`5#xBLRx{=u;1FJ&Q13st(AEDKVOUE9>I2UQ-2dr-KSlxPh5bm zFL4kqB~f^%l)M3&@*frcDuD?fVEBD1yl?{3VF9^-BA@?x8V487=8oYzO%lJ84#M7V z^UhOa0hFWs6WSOSzeOpPw5t)R_og0)sh>E?1D8`(U9_3@RAv}9igR*zn?iby5MQXJ z&O?JOzHs{9d9=T%r~D_R0r7_r;spbm#xSu6Pa_ZNKT$@>8ysA97FK_%_EPbRYWMV%}Q(u=SbArZU&l>5u35>~_*fGWr3@)Dt(C zYpxT1ST50O$CBy(571p!1C&L`Uz<(^Qs!N36zvc4f#r6S;1^4C?<#~rLT?=*Wr~7h zui2ISP>tnfFOR%4FTB3GWG!Li(xpocm(6(m7E-MTLl1;xKJV-I&lRJH_A$89G4JjuINt6kxh|TZI3Y+Oaq2_%oAFb#YyqVb zrT%lS2COsjzbdS-ZuynzH}^f*%JEG5btu<@4aIL)zIJup?7dfE!E@a=>tb^{9yS$j zz0gt?^F%9Je3{YP&Hiq&bDvH5L4^VhU^Et(f{jxL?8bFO~X9&sV!xfK?47IroeUdl4t6-}re$_HJz*137x=Kg6&bqbutoHXLSWzFm zsehCI`Sa)VgZgo%5l7P<{Rz0U-UnOM#lG-2c$(a3W!+>QBGhobZ1$Ij31Ql=S8;s_ z-+O=aQ~gclxr?7@&Uby~lOb;ys${g^OiJ!l&lIlQKn2?uC*Kl3v-Mcb$SSI3VQmO# zd-yrT1>B({>CbYu1xm-MAhcKNF{ zN8Z((>v%g5X4!B(TXX(R@eO-w<4UUnR)2kS?8EiUGW$;d6aPY9#|uDSM{DH6x=}4) z_JH;2EE;GLRc6*uw8Zc83) zb2o1K60+{(=S7@s0n0z;++J zWi7kj)NT>`+S&UR&N(RdR!!YsftkwYm$kplIxMwKc)iuHE|wo(#k(|H4z9dXw9o02 zjCQ^!CiI!nrw2(=xr_I(?A7k7$(Blvf*3@9^s@vg4C=LNGJL%ArsHA#vn^{YXTILk z$M@y^ui?M&^^1zU!B@oA9n>udAIy)E_(Ya4iv0a!wFQ3;+7t+U2AFTIXMhJVy98=h%uh zpB?-dUL0=66;yO=Dpye82K!e_1KD1l#wD{of2Q+z{sz03GlS)B)<~`DeQ2{Zx&J|G zvwy%#!@PS3c6~}2iL-3*@Bb07`f<^*i&~K<`Zmm45kCBMLzjk0kL-iNr+elFgeNsc zv(=wgzmO~Dn#KA2odjlUuzSM6v%EY!s=@J3EDk(rDzrYe$>j6uqS~2tvp$C?ITSk7 zUn$r+W#hE3H37dBgcdDc+{YoT?=tlI`XlyD4r;%3+H(iiqjIH&r>ezd zLp8&lYi0W?7i@IsB{i0o2OWwyn$fjjuWP@?MYi5&A5*9F*7|!6eZ@I(1m9Btd>c z8|41r&V4|Z(f2l-owrUTh(7xefb|_?;THd|A&ufk1{GFO0*D32^m?bWXr5$Rme)pCXQ7p6$;5d zvgawzu}4~@;@BZ0k(Ip5KqknAg z-$x}E$s!lhBQmXgun4nN>LzjevVVw#+@BYvu)e9A8|Lw=os#{_IRitAxNJ(Ku>*N$?y`oE>5Pr zQzeyb7G+^dF{C#RNbCq7Ra(&L`)qC8s|!j8rnl3IKH}jwD510p@OfcYg@Qor#YxIC zDQm}hRMfKi%|OV-<8507YG@d8Tko}JH?N89Y!u2I1Wo*&jR++ z{ktvQrO@QDTR5S=TZcjeO$Kw|L#8{U?3>v>7m zNK-^wnh)re_}L=LlT&Uytk5p~*h+ZgQ|`4C74)J7Q`yBSFY7vH)YxM`5Yw&DRMeN) zHEBIj#8yl2V#sLqaqVvmT;ZHB21QmQp8A8WR-t}~Ywe=KviIT#(=33(I*Oik)?)N>xyNXda8S$VZn3ZuF={fj03EbU|#>~3d;Z`rI{`#^vZ{BEC=-fAu?Kg-N zFdSkb&X14BC%32P(*ASyLu{$6cKp|P^2eTb(opv3)d}|fsG|{6Febnqu zbGBhCR!aOv?7RKYd7${G}E&xfMEK~>)Pbu2(HFJ_69#@bN6LbNphw2mBqG+)614ujk+9*v>z|F4&W661rwc(P#x*5&5)IO|5A1|u zhV+%m`}y%Mx7YRzULYlnrmmxD?sWV1F#r13lY%H*8n-8w^%?F}^XC_4qeH3j`JlV3 zp8L^e7G+K-akV!ra!CFZFV_vJ!b|gj2T|c8Y(7l;_$wtgVtb<-M|KiO_Sg z@S=Q+x~$tktgDkCOGAbwq+yrW496EdaHtCqUy*Jx-@1`k75f9SR3!ZIp(tV7K~afiOM&_qUc)A5w$X6R?zunOo-EaEsT zsJm=m$)7ywR|cM@=*zymAYhq%i^h;lSBVB4u#i^GEA@#=OQlFmmdhxOzE(W+)P|Ki zdpG2*Ta@Y-2^v(*nFa;W-PBzx85M^$+Iu?isf)y5Q@Tzd5vm0nC}Zbu+-DcghpgIJ z>@?K-LANjMT;kQw&oAkN>#3j?#>d$0>+O_IrY^5}V_^30KW0W2=<0UN#eWtk&xeh@ z{?$(UcK1B}Y*Z2amv1M!<d8B| z$k#*%(a)LSUPf~G+|D$?V! zkTf1?3r{1w0Hd4<@O@e{u2Q}Qo{cY7XBIgV%*4h z>0;cbjm~|uh%kQgjO$VB!E0xV+@?Pr59XLKN!8#2T|i8@u&JyHucBjldw@C z0SQIz4|!~Qb@$$9Id~bVAOUgdfI2p8Z+MN2)1X$$$i3trEUb5o)5ak8MDvoDE>C@6 zD_2yQ?0q}>?Ec|g$OfMwfDG_y^JnsIIRT=+eD~=q#@k@&<>&|of1j}bcdXjvYb2n> z6On)aibbCMw|5WiAY8OyP363yRm)he^IkCnydfVXY+X#mZaB@eLyc8%M9h(UaGF&J z)g(Hx0{Y%Oz{Wmb`S|pbwCmW6f3LG{98~BX=%vx)OOsXH>f32tetiClb*^J~=(B_d zS_%Eh=@3L0XvmG{K$iBIg6TvL%@_O5;^y(In5C#66L5T!4eOxt@hlP3>Sjmx@wMcJ zD~&eozuj5l%KhG2+ z9YZxFDTf|9c$n7l7JRhC+$hzk^U(NfMCRQ_F=dYM;7+wpmz0qkaaX5X9<4sH5} zC&H%J5JH=QQ?!~XP7Y}7AK?ZjFQCqCXEiYT`k@{}gp(X~a%pbVYZ`jyP+oO4X$}A? zs+bDLo6eapow{wLn-0|?+7tlDO$sUvvtc9$eg;3cg)BgRh|2&6p%HX#5 zu57jRq7kF5P>&XOsXxu$2A^+q;_K@ay=l1ZMnw(@^kU2=f&<)APSs3vt8 z$d*wrx8@jk;P$TGqT8+(x=B}`S+vJwPhrcs)>=v1`e(OE6R(-d$A%hHJ3!KT+0<(` z+!)4yo;>ovCHn9lrhMpfcr;h6$24?%6WeVu+m@#;!dl!;TpT#XSCRuU-|n}b@`X#e zByKx@Ymefcr?NO=^fJT+#?4y)P#lW}^5aXmX-kj{*uC^2TOoz#;CoK{Q!`AxIpurQ z?A_Co?V68D){oM!Vqs&y&nz!buC^_NoD4aRJZP#xj(;$0^Ya+D$=(Dm3B2KzJDaP& zZ}B3|92FvL&A7Au|GvyT%z3cRms9qxF;w!`wqs=ec`qSeHM|uBy6~am$=KE_$LMUJps>z%()=-0jO=Z77-ha@6?~iRyWp3sp z)lkZ}$)*H_xiXL0T$!75=R5N4)cuFVBurbOKg@d_z|;{V>39_iEq$zs%*~a>0k0Td zHl(9P;jAt}(UQCQaN*1EwXwF!%F5-tlCc1&7z28M)+*;EcN!#AvI(L3riG1*n~Gh> zuR#GFBy12$mZA|mJPvWQBSka*8uUBQL6QR!#N8$?nx-@` z0E9QMFu&V-`37`mal+Hkog)R_Qda+mi1`xU2OP)VnNIuv`t<8dGemct``2dk6d-j* zfh24o{IGR3}siId{0n z_FfTS=lqB@CilbmlG69j)(h%qO;k<90|v98n%y(5xfktJ0Xmq3&l<5(i>#~uQqCv3 ze3zm`F~*xAWZ@Kjcv-CiDAhHt*Io!0&|40oPI!0S@2@46mu1G>|9y5RgV%XaJyrP z%s69b1d@|-8f4CxuHyE_foN=T1c@BVWH#NU?$bYD?r(l^aRULr7Poj@Hxr~b1kcOt zu=PLmlO9m`B2@%jkSq|jt~}*3mfGrUu2k|ZLCsDPYguCPfvua%Yq^Kp896me-(n+Cn>vjQNeT1ZheZCZhaTTWeYKI(tq&vw4geKgR20N zwO|w|8s*WQ$Pg;TwrWv=sfEMP5hTc}p;>6tT z)V8vlu7SqnVfyZVBJUn3ODf3s8$oxF;;E8oOfDK?I!TYJmn&rd(#o?k@!>``(>fL) zf|t0}AV4`3J}~t`c)o;@KmMeat9i8UlU`kH13$JwFwI`TU_M-9-dX)Ol>GA*k02#K z#jmIdjW9DDC5<#4%OE~y;nS;zqiWAD{!9xxn`%+2*G9^<&_Pz8=AR9Rt=Kk z7yGx6CgLsAyzDDT`3KGUb)@di-k5;`Iu8((MgfDIt1yQ`>&36XoL47nOGUq{dzJlh zv$Iu$AFp+ofL;wf+o8dNkk(rdor*o4+3Rrsj|2Hrkg}oug6sO96k|Q+^r3ve?fe>T zTXOm}b2W*a^CK{dDmOu*{i8`oQ%s%B3ebfz@O6eU+!s%!F|5;guB8>yl3DF`+mraA z7;tFjE$W4AXbUaCiSH(7_UY=ov_7gAw~&>Q{9ijbcqxlqZe<}6VpLsAjxmxCQ>b=d z{-nC`X0E+s2YsLR4L_l}1IgAMb-^tIm&h2#T6vZLt6O3v`sXyIoqLUo^|&6}H3>}x z4qE-xh(oI5$gnw@os26F*Yo_9ZoZ2E;pic$0gZHy-~0C8D+GwRk}khE)o%ykYNfLk zx-g-mM~|w0NIkM4UaY+<7EH}v)re@e~-xv)sT9>8@+Jp zzLB*lH!q(?jFm-A*rfHHEgZda=iBSOb}Yr46<{oAS}@kpK0suWs7`X1c2_?jF20?Ydmv^=;_{ojgkz00 zPDm*P0kI>;xFqPH&eq`)#|mkF4e?si?g$gZX1-YjR^wc8ec`p_QUWT-Wy(PKm+gf& zzMZUlxj^&JCOGK&)NI)9|L)nkf@J-;?nkJ{Hs5ZEZ3_SqWC4!t0+Xssm1Ee_sX(N& zG6Cp^rA)V!ddMD%PtX;g8|kj7$Tli!NL_?$ZZ}9sK**2N#eS0-po>tOcU0!CvBte-DEI+Wnu zXrl^;-wswPrIN&X9ea-nTH-aTw?fk0WITSU`OlC(UTaeKe$85mnY!6Xts-RauBXb64xalH3Brbm<| z&FGALDasSO`k5fg@sjVvRHz9T{Cqt0b?sm!u&*P}GT8d^?e|w3_9gTymDstuqw)%UASU77vw3mbE6%W^WEMR@QZ{`lZ<}VKI=I_KA%>L|Nj|b ze7DfT_E4PPFLQ(3)N}GSGC_&fDh?_RN_Js*%#9ST*O3oDPE!LNS~EJF1d`iS{@y2s z*N#F7c+rDPj9}zn7j6YtI>kXH;_#@!ZDb@*ouV6}sIiV@F}o_4fbCF2atiAhNV5-S z8kgjodYMepLraGP-s_9zWeFGuyV_U%2wDZPikgZs!K7xy(18E_(XU8E%WmREEB4d0 z($UeSR89mo-lA96<=0IMaUSpL96yYCzj=zb0r;CX(a4kKQZ4B^e1-kNiAurZ;mNU- zx*Cu=)uoSB>tvEl-TT8+?$Jek#3)m51P{UOVW>GlRKOsVRo4J|1JW(Z=5tABx5~Xp z&`H%$d2;1!T?!Cu>G@)1mf+KibPrJLw1cgME!;o#JfScD%57#S3xn4$z^|RzfIR=K zZZeVd-25yo;s817Tm^)M6onzFVW}s6aMJ9ev@pJXwUY7uxJHd>!CBj8ckV|u!PG9R za3@kkAh=&b_>|d9289I}uE~bh^JwcNpfQWS^Qf;Tf*+MKQ50QxkJWb%cd854*|vxj5AV7hWqL$SI_fgB)@> zg^s-Pp&V&Qa}Gw2+6>dV6+GdF`8DbW_wO6lI;qaT0G<4305|$Mgo`NN{rs}hif)F@ z0l1aJGlu|Iw!l`b6k9teM;ta@PGTN435m5!y1F4EuSMyptW6C6c5{EFKAw<$spUs`t`#YJy(O1WvS8J+fML!8?!FLak5SWovlywt1!Uy=@27LJ#4FE0U*zdF`xn z6H>EMGwvF6rP5+k-$BqIhjWyTi6w`TgT2ur7J(3xywESp8*E>_ z1cQmoQD>QLxd7tI0~O2y9ZkECyhU0TPwC?K=qj~SY0&Cgw~{fug0ljRDPv%u$Ac_k z)H5z=ms`)83y>NghU|a1s?m0y{kGrgTkqTZq#QB@?dI&}`Pm;cMt%z)ci~I?cp0oP zE3a;E&UPMYN#WWdqN5+8G`hLgOp=g@W?SLmu;Ub5c<10%j`O0+s>H^c1 zZtqnuey@00uPp6k6@5^ZiM?;IQ3b7yn&1liLG3r==O^H^FDcX2VDC{waGDL}EGG^N z?v!*$62`(0mwi|vF_oB~ZCcTV+wRl80CQVyV7l;qDXS17UI%g|pp8+|R$46ptqszw zr~=5CY#=+U9%obGK*jsWe>z;>9d}dF#+m;U^k?c6lg?oyh%mt;h$x?Sk3ztpZU9mi zy*eXZFYdmgmdhdWbiAI|L0Fg5OC^N7&&AG}-DHZSM^#WM+eiIK7}!Ta{&yBCGW2QL z<0G%JdC)k5Z8sQ`WZMVw(&DV@UkL~H?rT`Ar}RxqO48Li{UqMzm0A};o&=hMD*bcA;?d8(0aAO44^&IEec{w8zLG3*as}MRjt2tw#5); zaY>(6;5ikaD^^MN2(8+LHSLZPx3hr|h&z{fCbA(z;{w$UB5kV2%p**)5RU}id##~5JI zqB8M7r!S%pjsXWgTc^#fAa-fL&`$M3cU$!F28XRl#=2SO>xpxtwh$O_fatf4^Bw`- z*FJOkFLeYNrGuJJAgKN|X{{>nK?QiBzb4%4CVue!%uplA$Bi4p7~3Ra6S3(rY&oj< zqJV? zAi%|X_$dPxwh;uhQ8K2PDkLOxGMRB&b>&*9>KHN4)T^z*(G3Kj2?R=>LA@~XFVnj- zcF;U2$ZA#~X9MExC4)NZ-TUUAevXl(-d1{R{ORpGaJi%e#en5oG{({cKahebI}VE> zo6At87_cv$JfjnZnE>R|@M7{x#^Fw#rM|7ZVRz=A-9}O=w3DSQBtiyBWv9s#{UL*! zd%yX6XlEuu21(%}Gfz@xvng<4KnQrfG_A_-A0xkw=t}}`d{o`$?t?}pt$~L8B5+?( z?7vl@H>wvB9HXn**ZmfnsjI*%ijQ8!3?wLXKC08UKuFC$@=LV}Sq-9hpL*uwqD;7X zvTRxvO9C(_7xbvQq1IFK05s{8)iG>X++GDA*RGX2_^!rY6zB z-y_BmeS`6B)i&JA+wD_8nzt6b2A!*tyK`Mp_ei74zTJU&>sSDBIKr+S6DiFW19GR|jI!56|L3VoGcZqnqxY| zqPbx`d7`QK5ft+GA5}XbD7)lAs=e1s$@zFI^xwDIEC2u^d6ZTAUjXVB-St5EmF){a zeEvWX4T@IhCg(%+oVs104sQ&K1mmKIh?p7Dhz&rnFAxNkJ;C=o=a=kxW560FZa@n$ zOVOb7!4Ra+wjf^|{ZiD3K({K`2b}ylT>JDiwYq?;@w!kPuTQ`@;4OgVLLQ&FQC{1) zY2nSAQ#+4blux704#9Rxs)h@&2C9yq>ff?c{Nv6vF9FTbM)^e^5S?6$D8>?-kK=_3 zxzS;7DQQM0R;YI*0Ng2059vg*vuq*6IVKmZ~CvGCSbMsdJBMFUdbu zjZ!_fVY}8guG)|s$3{l^jQ$JYKIn-eYD?RTMLCJQ5AHVk*fTf@gY3gvzKpuX0B$05 zI9w`{h>(qNaU6QhNb2_^S5WSHDPw}MuL4;GohJOh;OV_#l}Wd#o0F$RL%R6Re&c&e ziK!fq;=xpo)Wknh-ek=(ZlAtt6C8wMl>UXvx zx2S8S2Ryy?;&?B5l4e_L;;zv0T0248lN#EFaA0+_iN#`WrrSkC?*!^|5{Ft#Kr%Rm$M0EI=cD1^rFy%nkPs(nVum^1M>K!E~0H9 zXQN_hQOoD>pXz`jWHhB{;VS`GdLSeMl^N_v+_CdkPvUu#VG6rH<1Wx+Kj(u)grT1% zZw#CkP*5@+IU(q$Uys#J4zxbtk*$TT?5WbpnX(P=?IA@5Mcul7!s3dBi4gHCNOSjC ze%<2+fVO38`2Sflc2Ey zKW!71XZRzG$(tZpvx&DKZuJzc7R|=mVl5Q@oaKRPwD3pnhpP({U(4B$J|t*bDf6qX zCAs+Gxc5r-C52Xw$i1|^C)tl>2RhU6-+5bcer|L&NZp-YAno=0oaIgj2yI*orP*wWmF2h%?#PpV1t-NI&%y)ePkO&OCSy z;0a(++|IG#qx96pz_*8Q2e2pLVxNcNM>!K)uy+t zo0p*2zW`8~n&6P2#-2ftZN9?3p0<@Q9SaDUSc=lch(duyt9e;Kc!zF$SeUsA`+hCc z=7An5ycxotJ0GlQNF!y~{`mYNg^QAfeM@OXKePv|{rKzP#{UbciLo44DI> zknb{ziao=iQ1s~d>@uKUW}32#NqCq7n94`gz<%ZFu{Qow{DUV-;+Bpw?D?%E;MqyZ zzwM8(Ezg6j4o~IL70SV_|EIk^u}>@U3`4Y>cR81==VxwbF4b>hZe>s-YSJ}f@5=(mT11|-^o@JfinFk*@P?UARMXTP=$H|b5n;|2#hx2e42VT4 z_!GoJIw99-`d5n)WMF7X(@9HuZwEGfS`SNIAQ;?WHBrG!)&v{?(;EXx&Nn%>12M7A zEf-`zMjx*e&`s+Z(ME?)gN+||qinyRaWQUaFlJg^iwH4lQJMPQy?b+DCUk488ZDdQPjOy{!6BQn3CQBtQ2f#(H|Urn&gK0z-PJom-Fbl2}-?x5w9 z@%yW_m@Tqji$^hC`GNf5t<|HSK9}7rfX8C1>gQciU%hl#up=6@_X`G~y$3ays=j_i zdhXnK_qbdyqJHt#MTTJPz7!tSvEb57`HwwPJ1n(Z^Fn&R%vR?l{o|8=UYOF&1UX3U z^5n4C_o(PSj;D+OX0`C1Ihqe&Dz-`&)DgsniEBu;;LQvgL|E~2t(0q z%Ry-|m?lDcxK9q=V2vK3s5j_KFCs1fSt|NoWx-he`fBZ>_+}yDuq9f%carYS>1*oD zh2X?F>b<&T`edF@`h!O13mV2<&!G@lz)Uwa41a42*`s(Bpt5|kD#nWTr?T%38xx-#~gJZ?oE;TofS&s`JR(?c@dPZIygZbllsqv;1nC2ynblsiZf*c`Bq7YF*`z zXr88CPrpBBCGZPky1bo-_0Nkg&qxPQXh6@jCB7)J^(Vz+X^XJ?NHw3Y`v}zrdIwR4 z^TI$3<~Nl9>oSheMyuzJ;8k#e?NDZ^bE|A-J7fhk-bBex@TE8?>us|rvJFc|Yhdby z?*!-EClK>M-9H<;$ppPXJ%s;m?(z8{XoIM&uY5wZm*v{>t^JD9R43Cw&L zO83;}2mE%|gZKUVyz*D5ssnyi>udE?NHhXQM#pQI*ck4%&eVSyFKYdGw7Wts7$@N> zs#8MofK#l-rZvY7oP_+X!&jP6`8U3u=>WZXV8?}_9Lv^B17W#6y`cQai?%Os3k2O< ztt6uTRF9FU*W7&`8Icpe{#(lN^OsVQv;6!c`NOP)K8CWYFL0@?9TeS;iCwx_8zY&r zM6a?F9q;(-F#6Jx9sVmSOqC+7{qnw#D~xY+80^azQy%V?k%}vJI!T;@uDWA&WnS(k z+*QO}G7rg}`S#tr@nc}7`xg6y8+}Z$bM~lardOv^qq;r>xCxtO_k@$Tvy_5&yS(GD zT(Ctb^Nq>k-;)C?`9DT`tGN;Si@Qz|Z}QHflp1Th$sG}%*|z2&`bf}4MyWqNQVE>+ z#Q;XGTmZip!v|Qg;X^bq6@jR|f*WrjvN*kT1bEb&hRZbafgmsoywnSXtwN}&M#{S= zg$jUb%VBqHJ_E+b{$;A_{759sTOHu#Mg2zy$=Z8ZtU_u?X`Tfu5WxE&`B`3&;bozH@-zcNW@5fM6@+C``t4K4U!!r3@<1E?fDR=e7>5qNc+bmlyJ%G} zm3Q5`MQPiPOLwJv083n<*t@&FNA!&OMc+36i1>P{2_ac3Z#&@fH6Ex`?bpL%A4?d|1t4Yyr8 z?gi3#K#yX#vb@;x!b}N?Me*hsy_w?xJaH~`L5Pn(a~mcK>kzb38eVvDaPD7$m^`S@ zpQKj@LfjvBc3zv9MUq1c;E%SSpC`c2ecvFs)N1N2#xUj$jJbB!n0LE%YMUR`=4BtR zS!a+Y0{5hFbEaNt6bxhWK=Cvnnm>p;cj6@`0mWOq#|*t6hD+i)Pt!H^V%ql9v$Hx? z5LB8YvK?;&02{5p$iyo?nC3ylz{%1qlTEZ>|6~TA8|~m1cL~sXUAvD8CES#Cn$%zG zgtPOjdo!`+xgcTs47%VkKI*hXvm=1%K0sCQ>L56v-}$H`At50bLDE_te;zm+&gE?1Id9U4sn=p<1}Tx{xP`+y7N8V7=g#fc zL*}jP-u{Uv_wrk%OeKnOu{{d@-FO`ZD9c8XxC91GpBe;9&OX|k!a61GJRJ5+YV?^F z1SakRx5>+LFUw$vsa)-J5Cl4ytL@<+#nKPz}#cCDQHi+iwx30OL3Jr zGn7KNJ@A9t*-SJ;M)3CpSsMcS%5X{l6T@WKY>$F7H@T9 zw%K5*lix{m@%3xymPt8`;AkC&C_Kicwr-4fNZ{eQLbmPy-zCUZfHtko$K%B5$Hy=9 zjFpV%BdxjqGg$KXt_`bpz&Mw!5J%7xCDd-<-cDhcV5Z zww+z!sRx%>&KROu*6b~82)Aupxt}f?E`b@U*#wsQdKm~M%0EG9cn*`ZZf24+Uja47 zgJB-Ap-D-7zZ+ya(kXHnZ9S3@zM_{yJ+*X;WBFXg<#f}xcW$x>>VFi_&l+^KB`|I- zj&H-1BUs%K3K3pu*Y`0F#9%e*E=V7-s-ok*F*5*6R_<75NwkYI;ToG#qU4|5OcgEj zUug!(mTR^nf5dTkpb{bwmvFbK@p%bXF$-d&Y(P-)9Q)B_tH&<~;=QjSGj+&#CyubAVgfHCbi%JqM@$IR47*y3f)K#JWUTlDJaSt#$ zke)8o4w~ySXV_=;etz_;TXb_^x4jm;r|+yv4E%t(!(;eG(4iNA_F$=lGmN_lTfS7i zPp2w{AuQC&@?$VOwh0qdH^{$cUu7`DT^srs0;& zl?(7&8D3Y*6w0kXwJmZ6=rz&wmy(u9p|kCP%7U$ATDC_{E=)h;to`1?c+AIBWWL)s z6lK+$$#HA5N<7!7L?wITSC=gVR}h6Apzk_wZu|}gAxxBmdw&(d|JA>Xr+7>?w^j4i zNocy{oq5QK4)2{{wtuI{|{^zKG z=tUz60;mmGR}&_{q4zm-4GV7B`>_PLx81wAoI8`0x8H;*6gwzz(FVStJ8ctdH8MjO zD5O|M(R(F3x{Fh-ZmT8%a@m{92=K9m0k{8 z_!%DK{%wS|J)?pv+_gcOZ;^{;W>@p>w+Fe`l=bTY+Tn~3YOTzlD)*r#BI=Hc`}IMQ zwUVl5&(ea|30Q(G3q_9hjnB4brNrSeeD9I3T)B`dUfTlv^{kA0!956R8&c z-O2_UF7JZD)6BP0y57j>GJ4?gbo!S5J--tN3=7%(E92;|DY*RZ%FJ$oxil3?4* zTy|jQG`r>fxlGy-nZpN7-jyw25l1DCyhd3wDLC#!*8t(@Mrw4W108 ztw2ZPkBQ1 zn}7K&ePhXPp_v3%8IOoWDPHxczG=Z6NX+P?TZA4Y+&%-dmf!d|2jjX(LuFsT6giSk z`SZlrG(OvkG&Qpu#PT5)4#rU5pG;`eEX(f$2@nIAwt|G`{s43l<= z0b%$4?9-u^DN^-mSv$FXs)z{Yss5nB0GzJcV>@5b^xHm@hjL#{+9?NxuMzc;OYFB1 z@!>r}x##nNV~PDCzuBsuv`sX>+UoWGw)&{=c5!8%Dvz^o|EVUfzoJZ|UHgZ=O^7># z$WB?$!isF9a^Uqw^hkh&;>oG#7r5v}C*`F#hw-1=t8*ZuRLx@l_Vu6}!bCMFvi`*y z1c3an@0xEGfDa`%&yH*e>p#2h-@0AjZ`*dj*zrSdxvm(;m9&uuEQNr(Mk55qvsenz zdUikj@?WliSz)Ag3vSeejv@;L^-}mp`;>@@E?k87P?KJT1P4-}q@?{w8~)5| z$MI`1>ASrvnbzN0h&Scn?~#M`9eCp_F9icpm;k5W+qWLE8n*U`|NQPY;Xh&G*hi@Q zY(h4h*$>ZvUU0%xng(~ZtpSmEgOsiSlnun2$ece3atez=c zzI-{&5}?CZfK%AS@1~RDhAa=0lwA<7zD-+;_=r#q4FG4&$8f9aV7y-ge_1W-y2Z^< zxaJM|3`J|_989rpYR&#%F_;fD0{?FR3Y7@qH95ywRqcNd>uGXPs`Zom3#0j4JGHiX zKyR2kGexec?fwG3d^UEpyI7yXfq=MKgA8>881fuQCa$vAzxKO&n%rr9%YVPwUQ-?= zkLvxtyDp_RWU-u8Q^UceV>=X)+AJk7$BJO#2nZeHh!j4EN71ts^+%63|M7`<45|3n zy`@C<5&*&mpRnglC9dQ7h=y*zwDmgf<-2Pe2KC4HBJb#~x9^t@MP9nTF3ZqNLj;_Y z8|;#_{9DkUV^tu_$;ct{2JDh-q+Na-$rcxf`E9TwQ_QjAe)ww{;lue#uwo4VJ`n%2 zsX!*fd4lJk%kQ?A`&36pmi@UK2y8(pJnXv5J8>QcP}_q&!OFjfKRQ7*vD4#N1NQ+?lu@^OcXhu!;ga*l_A?AqU@_ZoBca%quc9Sith zW|6+j1Y~6bQe)nFHTf)qe6}41Suet@bVmFk0-`&Eemom0o(o2X;1Z-k17V@Y=!(O} zUnY;m}?vNw3?M%^fzO*e&qg7hnw*F0ixsGhf10yNAyv#u3hd=o<6lgAT6XwUjPXY zB9^`XAMt^V%^;JxH(}y@G$~Xg7kZI)P%GK3u8>r|zy1#iJTefY*$6fZ9RSKwtq!7L z7ZvCA^g~vW5gr&jx&S?*9RwcP{rmTyM`lZL}$DAYT{Ehx_17dz%jAUa?*g-R7;I<6N1whLCzUk!< z4@pY`3El}!U2`^zmrluLA)UopwKKuHM?ZYsph@>VaX{YezZ=5SO+UzGh@`=B(&f|V zp`PxhYnHUE$rQQ#X=kLyY>NJ!E}dbnWXIYHX{TCb5J&O!sV;*H3DH~mYEpB<_Su-4y36tz+# zhf079Aj!{Sra9BP{zE0prfa1UA$LQFVsWq^Lq?-3 zTF#Ylfppazb>M8KXD1Vrjknf2c2mzw2s!{$ifdRYH1RIb&MYO?+5>$fr*P71O%RNz z#ztHEXfC9G0?Zo^&oOBOk=RfN_B|T{b|GVCh@20x%NzX`EH|gF#f{jP@L9rh+1kb* zUx#Otwc}e&Y`MK=9?jQTMV4ROh{RVXlZ*Z`>O8A1Pw{G`In&ak;t49M+O5C-@;=}m zrpCLRyrF@XGMBLGDQ4qS7BZU%a)1e7*?U2#>u*hWE!LA@{&?xOCL*u_$@z(GH-A)c z7UrBj6;u{FxuZFu7--_nIJq!FIQwTrvwn+g&H7C8*i;v zi$57iAX@-6syNs?u5OvB)k%PltexND(eZEo1I&wFw|PAzmY9!7^7+ z_kO^(PPR2gK)byZyA@F!mw=*)bjkZsZ2ui_j1cv-AOULr;T-C!c;)oe~K znJKS+$_+-UNVn)ZLv6vh3Z6n~mk-k}YT@Y)AaAm|IrrlQSOg3{{#7dS_`JNwjA2}l zls$!%qFTn|Z(#Onm!l*n5B3c*(@N1P)?@77#~M+OA)Bu3Ax@NfC}p_bQi$myOKEwB z04=TwkmKneb35uTiMp*m-5L2n%a}8y0^YC|9WumGwy%jXmGcev7eY~&?=YH~-Tg;n zYOtvUZ_7Y}>uV^<=EjeO^YLrOdGK+bA2K-;XMd)VW|q1t%Z0I~e~rSp$o#9YSx=#e z#^SZnA%NeEmapgLSx4<4020tB@5tpGTTg_~Q9=&$m-P{e3gZCfWLmp!3QzW(s+Ekr z^U69k_B$Vd4<-6;4KxS61{-`x6Nse;ku?){xkCSu!{MS`{jFnE?T>hLt#~zY8Qy0; zfTPrV`bT;;+vLGvfmBNl*;YiIjZh^t>UkYPL~6uJnU4 z<}Ueap%gQRg-aP3Ovb^S%~aBK{N(F#>jpnIX1ZYMtpic}F))!kXVto?EVwD?IT+kH zr_o8Z2IUrW2X2wBzedl|c27YT6d@w+)d#Zpy)z0S<$Q_%+24h*Q(FDzGJh54VCMXB zgq&*1gT~7HJ0;jCgzPoza}JE&%1+!g11I~=We7XEBOL`;fOTv0ueMzt3Qw*a<6OWh zQv-QDA*h7Vow_D{?}3@JLKO*!$Y$o`JzIv;-G=y5fNfL0b>$sH;I3J!%$X4xjN^N|>$Qt-6<;+{ z4`hIQs6w~sgvtF7kjIF`I#%Uh7}Y7Bn6o0k?Xt7E_IYtCMwtTKG3(QrGNcD1+B1#y zA!X-_9u=RPpTtE|?KvCWevZ2c9+|gN z2XSuqDw)pWKwZ2&6hO!mcO!Yt1{GW2(Xf`l)y}*fEO!x{U_Xa$5ICmd@XJ;g6t?M` zI)8Qc8n=90W}7Z^!XPlH&Ra5F13?9OoZ(vQUM}$5I?hneG-d1Oqo2_#LzR8L8yrHs z4Fv~mk2eAn6-E7M1SQzdv{U?$Zrm7R{Mij70>uZj>pc7T5MR#KYU?I%XK#ST^>J3` zOyg==d%0AMzzVH#m;?nMqtLBqt}Y&}siHOo4K&hsU51<9ziP~Qz>^j?;gWW>OuDHy zMS-(Sskvwii)fn3wLl$0i6O(Yi13_dHX-e2^LUUX&e%2GG+4CR%94C857xH7F?&8&-Ue+0Y9al zc^rz{I?$17%X9(gB96t=A}!A;M0V z4m)&s&Ib1x#1o1;m32|b%4CM&yWTnOb`jhxi&_dg)~a zn^qR_a`J2J5rI~{LC|#Z8P)99hsV1n{AZgBykVLng-HX!bK}2Yh{vye8FbxrzVLbi z;wV<`h14?$B@A@>z;s6T*Zx&-#LP#CP4p{YUM&QP?dpezvL=GhOf8PH4FFhcfO%&0 zGo~M_9-g$Yz^OHtSk>=ET)@=o&bIz0GJFfSV-$a7o1CqL5($Pct7EwFbr2O0IUP`8VS^%XujlPHmMFP!jw)l6G4%Om?sG6Jwjl zFCk5g`ZLK+SlEy~xwxs^Wt(x{m51Kl>6J^v{+eO;xEYdKPtnZ2Ei$JGPgSNZW@kxn zUiOz@$gddA$W*J_H}N>KYkHWJIdw`yL(9lUA(SS}#9eGb4)3H;$}5z%S_4B7a9c%W zLVM8Hb>-G=C(hNR(orZx!)*p}Wa4s$srUA%v(N4wxG?0I#JPefVjL1T+5Pry|F3WT zIV){`7OE$X-azadNU;U>rj17vWg1m~+))QiDdw4WNnAu774+;E2+vfe03>{t zqC@E+r-gml>=WyYC1XQK+Kbi?Z*WGGATU8O_h!51La;lanzPajRug4owkH+9Dr4DZ ziMWvJfZtvT5lY>9cB?XX;LJJIAu#u}M|r6Tjwz)y2ORA`w_?>V=xy^AT$=JsS=F|4 z;%@qE!Q1*!kFUbhJn=iP6;>jSaqK2;3EgD!{)>1DA6~}6F32=?P?foWLl7}M%goL04CTjLoU%GgIWC$p^q+h14%L! zwQ2O1j%ezMPgj8TZErjZ5$2X*=G{1(cE{zMyiJe3@o++-EV_yE7LU~)CWakMn%mJi ztW&X$=w$DHhKh*r0T0vkSh|_Rn&YDQ!Lvueoy_X7pNnxboTeuRMps^gC*D}MAvLv> z4RbklGmXV?nb{FH6aOEx*Ys}`W0ffsVsi2X(mY3TJpN5JP5UW7S0o`=ay_RmD zz0Fs+_W6(5?up`&6^!*4kwgGyKA-}ABQ5{E`f1XA0rA<$I{t8J8-7@O&1kH%E~d0J zF?}5ikZhbuPzKY|5RW2<{-7a5mOTo>hp8bzA9h9YFz;_VMfNr`rW?KV36ay#%+bux zk_lC>f`H_yY?yUVg}#47l|sJ`p(+Q${sp&~-r1uQfQoBw#o{UhUNqG(4F|g%(oVXI z%rRqoDgcjcG{1jwqMUj`6SO_^2BRwa$xp2wP$ zsadQ*MW3R~SS#608IJXE(j;k{;AI*VCIwU@ozybb`VHKTi=ECn{6^p9uWiL}{krD6 z=#5<3EKfTtw9p{2fs0(pk%$bkm4J zLPEw$9ijEw2dVA*G|4<3uK&Y61K+Lt&U{{&{6=DY1{4z8W`;B{511hPE`@k!Am|+8 z(OofdMqB&wZ6lWb zNbawd+CL*(-xZK7^nFhat?B?3n`}kY3V@7LZklNOau+qN3NUZGB(;JLIJVv&L7dz| z{_6{0QA(Spv`dJI>-ZL!G~s-&^F;*)02x?XlQs>+n39=N)Pe%erkze+K{|wb9@m*pUpO9>wdM@)Uu7fW3FW1u>T z55sui6jP}O1j}liYZs?L-eJYb!retSx{&UIv+EFliMP?Y|xjdyr*S1 zy6!tW)+ya|4Q%OIMF)*TxA?ilLq<$FV&8SbG$%$^={or0*0(2mglkq|Vx@u~G|ISB zp%tW?Oz_TMu7_&NwbpENny3o@HDg*U zN&poGsnc}u5my&^L6>Hj&I3r#7Ow44uN^h9LJ!OIh#}3u9Y#ysc;*HCDKbbc`BugJ zMZ~G!k3jJ)<&(d^j`2c%L&;N=L$UXRj{|uaLkihI)GlV6Dn+0F5_rk_*NR;?YE{rG zr4ASLW@Zlq1n8RFXOqmv5hL(FnN8mno+`zP#fZXqS=<RM?>cpU*E=h|&+}XVd>(mJ=bZb#-`91$uGgUMPmLrP zG{?Hh-uk#BKHvlUeigGS0bT)f--eVndVp)DYImKCoBnoL&8_)LXgpy_wzYFhNb1s1 zVup{KB4)pl1Ftk~a&O}@p@?7eGVpa=z=Sr?m}=nY+9!Q=Dqdwva^VqViI?nWZa?h- zV{I50CB|08SuQ*4n8&*YnskEdqFOwos@@qYd4$8d=o27kA(nMrlwG7LB){M(T>7ti zUSf5sRZQK`KS)xK|C_(Pj$BU=d;32hLS&Rb=z_nEoGQ2pr^w5W^ZWbCrAnRmqQX;% zEgkiQM$S0P#0#};6;m=6J--X+k1F)elw9~P@$l+j|A1(*<_qGd%{} z6LeC;KG=`BVdEc9qw_3We~kCAEM8?kG+*FJkNdjwhuYUmk`EjA-`-RYGdetjX`*5Z zpdC&zSH+?I~!{tBRSP*v_wWd!#g5Yu3cn6**3j|8i&HD-mm>v(;Rmbrl zM;&b#@^_5^?)U*ZpuYD9U|%959%lPAXguGaEo_f3!T7T_?t$=qa1H1brF`f2t3dun zmR1iRSk!a?cYC~m>e4k{oB8o`+X3=JR6>xJ(=>7szcy4OzA);n#i?TUBT)m#9T3M*Y>q8H!EI|)I-pxw#&nX_~>05e`34DT{91Gw!6@u|MkA)%!U|L}_*a+ADy&-?^UEUFZ{4GR#VzO}0 z_f{bsL3+IkWY{2_+-WWhEB9NbWcfG$>!tjTgh-A_8W&-Ft0B`59sY-TeX_n7@xeK% z!P?9US-7X-SPvL{N>B>ld;|vw0P@Wx)z1M52>54%ThfS`+J%rn1DinD%1q4+4j9-L zHq5`{m1&(Y0WSbF_mYCd((^T}3R;sYnU1#E2O}~OvQUNzdws5aGY)zij;PnAVjCiV{wFW(2r>r1lOB^6 z==ZMQ0NpYoGJl63CXVjkFAfgOSmBeOa9yGMhnpia6p#)X7`WY)^V*YbOR`$5cK}0czkzLscEAN|p9y&;8R%y~!)kx(+;N)DD>D);CPfHN4z(-3P#daGfj( z=+)A}1G{+NGE&R{eAnkBPsDL%m^3C%fQh(69~aPvm2EOsl9e(}5ps8rTKlzA?tP zWu5aGbA!>LFkr8;rLQ)C`WbORdYLS^H6#_gBAtu%>srN8S2UQ(y{A(IiTc6pEQyZI z$(j{{(vPj;^MI8;5~Q$W!Xb6X+u?@~A0}y%lTYwrGyhcCe;O2U&{W!d%g1+1fMK2A z(xPZ0<3$)1xtE(tojndV`sy=y!~VngZ1K@wSiq@u0R0GDwiw6fOy+3l?3>tjo%GEY z(+Z*1CUD_%0gg+2m%zDh`rI|Jl@qb+Q^Koab{*N;iTsyZ$^vwjVqg`A*s^$~#)^7@ zjF-D06QwBM+jiHN48}y{aTG&K2zUgj-E6_O@8AK8I8loOydA5X8f3oivul6$hdKsi z#1AvwKi)H5=H&2FvMj<*?xi{2C?=vZHb0%AIu>W&H1jms|6aXH)Hde~d1>L7hIb6i zWf{^FGMlyy$bge<33nnq!@A&QFd0lMz0kufV{YWL6m{BV42suANZMG33ypU&?~kvY8zugD^Q$1CMa)*U=~PGUz*(2Xx^I+Mfl-fe&pec*d?&# z@mG2<>afp2Uf_5zCCVi*PFzgoWEHl+vMoBN2jG_G#sVIM{5sJy`Qu;Yputd zKfX>qO*2vNKNPb!*IWpMv4Wc)&&r>EWE4s4iKAmu+%9H;y+%oC8?;VY_7@%G*VFOG zBJvzO6|}EAh39AyDuhqkIgb1JAC4w>gxnMEISg0UBMbCAiK za%yC9G!TaLVY?rYtPsk^uk~)UXwInd3c0@aD4d@;GQb?qXSKVPn_*5>3^s45ij@SH z0ow^y;TLB?ytMFy-OxTrIb7&1$WNqEu0lZLbuWb-E^qLkW;cODH|l%>`>6)jc0iv6 z2xT9YHr{GGvwG8u*#n#L8UK@+V!>QD{WEgw-OZVBIgaLPMbz`dh6J_^^&6?mj*~G= zgLUjPKqICfrnSbKfQl%AxVSwTN|=gwoLKqdNoX) zeG}KxF9-g{3Sngq(TFzumc~)C%%a%{2k`3w&(JMjg+k*P1EFn2B#IkduT?`%I6TqT z%@Ikt3p~KZwk=w|I-8LgEgjBd9aJ+K;_rm+(`)F_hxTsF;HAyEFHcQaIND*X-rf=m$Q?krC=`Z}^PHCYvrs(d^VWuu-k(T!9Kq^xo-rdOz>I#rR^YgQTC< zz^Xx#xpb_2n>q8;wq_ebWZt0RWAD9O9C>d`XfH&q+q0IZwR&Iujsx(rMt&gAvor=K z=#lGa=4>Y8Y5f`FmZu>L&?BEk%zhNyhgL=NG$Oq%xza?7@uDiJ;k0LI;LV9lxEf77 z7<%9eXWd^ctnWfn#l`t}PVyXyQe+E7SUOxI`q~KS{*6PX#h%&YFAy17HBgdK{|1Ym zr(o{bfq$c=KjMnIZnv)6JVZtDYLb(m@m^ohiDhqZ87o!xT;JYQ|4`#UX@9=u8KKYMAB$3 zXR-x4faNRe7O@Ed7Bb7z!5wc=>DRokb_Sf1bFX%^zv_Pde&f)N!xxo!Rp(FXJ0ySQ z!e7`9tN)9Q2dG;=cY^pLp=|?{NOpdGW~wH*nGVoKUGYpJ2CVyBF6F5*hM7vawImC^s(AlS%@X=1dBt2Fauahy=(#>R#vX%)AJ zuySYz3~6rTY<7&YqR|Em1IlG>LrdQVej{06iQj0Av%2waG1Pd$UR8FaSwdl?FJ*W< z3pR}J7Eyd=4>%zyn*FG+!e;Qnq`bK@9P$?tFkZJ6O9 zg8s=HR)zxhg2dw!`7+SUspU+bc)*_T(sKdYd7V64v&M;cW;aR7nZyn356d>xFGiN_ z5{i<{9($S-53JFIz(K{z|9L~LkxO$OKzs)FfFPBVT=TR)ji`-)4t0m9fH^v&-jLTJ z<2q!je<8onIUOfD=T;JVS{gEnvJj0E28^C*iOV-d5v>jrmn!Yv#89XnLdo#jQfR}a9<7AG( zVt|g}SvZE_;_$bzrC^2v7PVm@fT=cWtOH^Itl^!ZQmv^s$aWkpy#7ptOGcb(=AVlO)1*%VW41dEXX7j0(^1NwZ)5Y*I9JPkg)g4 zg(rf@>YrKae03a^E)-*CtI4lX{@`W#>v{Nz%>Hga;K@IUs7)*!1`x12!ledcwE|%7 zjzFzI>|M;^gMiGTj1~AS7ddGrU1MH<$y}jXZpb?Zc-;u3T0TR>*;|{qzk|QXH$73b z80^yV^#A8~UQohHQ1L4CSWdO>F#*mCih>(*^rsq;90%6H;fkT!tE@afB$0Mth0XP! zS2912Ag{aV>FK`M^NyPwjDw2ffe!u*0G9HY{+1YoV}ON%B@lX^1B0nwmwFKvw-el0 z)xYjOGRkaOYylk*!#{z4lQJ`fH$MvSvjXOV#fxMvBqy_L;zOYoNs_hYOHHI}^wnbozcD5ho(W4>;@BJ0u^o@6C1 z-k%0f+KN#Ru+qW|RMZseq_++jq>iLUj*DwW+f^<^p4P*wDFl8u*_81M3#iTOv6o%p z$^>V?IJ-)V$U8<48u%#9sLgIvh_M&N)K@W9)nC{|)sM?O8gYB5G6DPae|wN+pMr(P zXUfx0#RQR^z*0-eiWzVU(h<$*Ihar+-D?0Q(S4bmL4(_`WE}a{TD+5zM6dnGO++y6 zO-9TW*kR{@|HdL@;$$)=^Mh90ed-?SRY1fm{8ZKqw4=?_hcaOG z4U+6(7L1v;4kGEBNfF0i0}{MKNJ*-|;O5Jangzv}A!6?|;K$FA9w02vH0wQ9|MLBB z%17HSAaXGD-^lL0P(dxZ^+7$3+GUl2QAtkxXTie}Q2PDzvyplQ3?yr5ssIy4iDFI$ z;Z2Upo9hmcM{FR>X$MG7-s(uV1u&}O34@uq?PDiv{d+6#EC!5=tQwbcP%I*HIs;wQ zhPKnG?nOr%B@XYu9_tFiAyY_ddiHKoJF&fb&?CyRi36No(b~V{b%RuV%g{VBxIqCN zT73}er9`nJ#~|kVSX{dWOeU;Bc|8}r8|WHlu;+dOO{E@MGN8O!oiDniCXpG{87n91 zB45o8f3{)9h``tz%J#;W!Oki^KsV-s3~2f~`p#5ooG3bd*ZR)T_qtdF>{ZKT0$J=~ zm5%F;rdECMT3iAa2uf&7m`%o5#t=hPTj(P=>LJ!b-fUOk|5lWQRY8G0JTu7>U#K!l zR$Sd+AH1wkkLdf1{sC!Pn3qVn3z3c{0}rIvS?l+w_cED0Y>tWv6}76j&x16k`oR{4Qb9>t zFeQ=Matni#d0?k}p{|FzS9eL6j4Rz;pY{nn?YPJC^Sf`GyG;8-eZ3sZx^m>H9L0)H z&Q?Bwcg!3!cfIX6Db@or!QE?hRAX6cK+24%XIpza!C;MPtO22Fz@_tTnZJ*0jk;meULUq5Az%Yaj|@s*43-=8doGWrE32_+0g zbJ!|Y93jm{-8*P%X0u}?DO*_Y7)B+^PP!6l0GxR14y=>z8UQr9h2||<1Qzqh>=hIZ zi&XWQ0%5saW6GtUU^b>Ye3I#N0xSkT!!@}QFS6ZMA0ShdIG2$OJGI>Xdleh&%u_`VDIUwGa}0L zqBfnLsERxuF$sLya9`8PrbK7|>Ef!dV<f4aaiyb%79b7f!~TCHEBzfR%oPt0fQGA0=xNHFs?W%J*b30x z5>bDQM^gl3MXv@d-)81Hv7axMdXnbGuxYD2yWs^OB|U|6sTQ5wSCgX=UOXLCqwr>2 z6PB20%U2EVfH_rRI%B#b9$ZbUeEuSI*hDy9tSepl30kir+k%fuNUzoleggHI5{yt2 z&wnA)ZhFnC##>;=YC|Ls`KM(M6r>V9L;xTT7v$SbR<~~IS&g{BTR3?hv~ZC?CvI1> z&^!jF=+^pYy1-2K1*{t8opj5%?2BG&=qv9|0(!eeq9-&2dMI3dl(I%llan4=vBO1H zis|m5@u$ck%HsvlSrW2$y;lX!-N;a~h!})>GPXN+GnAy1RGk!Wm?cO>oFpu{*2cGy z*+)f$OWLb~TbAsydWlwamG**Sq7%+C)o~MyOizy1ckNNkbPw8T(|n9aoN+W!`@!Aj z3dlK}moA3+g6;DI5DkYrdZ_95M*`>JJ#veW2q8pqHSil>ZYJMZmFhkjgigMq61y4_ z!+_WTr+MAQ|8TZ3khA=DAh>W0`rL3V_vzX7Kk@G|8DY4?&sa%0j}s859S7;wjoA5d z_XytcDuK+0(ls|KbW^k{k$?jzmboZ61w4Ctb-6Vo0B(DYkFIh%bE;y$O0AqijO$`whz zs((nyVSu|I#K~j2_XU8?YxX>bZ(9Ww*!u0p?)n#+$v%sv_;6(2uwp!4_@%*EDGRu> zn}YA@#MpBf2~G2L3cVXF)beUrLsTd_XH*0vquT9mw8Xh9W)keH6753ciDeGRd+r*D zY@FM4&%W-SG1@;M=Umv3x}i-MM`b90&mjoC`jRZ1qT0)w>0PCpPf+5_b-p#q7X?y- z90E?@ddKP1eGJpOdGsj?%=@%_GgzD0LgS?e^<|?iRn=7WdsEZ-x!)!BCrpOuUXxW@ z@Z9P-5ij|8XG<9@^~F4{QTY6QILfb+z)u$4Balftm+)^qJw*qb0u*3&zC9VLSoG z&#EL#(aoCZrf=~bvTAZ=eW({sp(vo~vsaCOrrR#DHfWt&!@K{4jKG0Ffzyx$;}!RU z&=brXa#F&iTTw(*(?y2P7WHdLYjJ;}lIB->gBBi0OP8VVDaNHwvs!N-WyUPm`%~&K zTLDnu2o3MzYzpR5~*PsH|ulhwQz_> zFYweU7tot$`#fLY;!o9zGmzln%{=D(NUQvJcz}ovZ1rT4k4YRHh(2~=DOr+|ax!3@ z)SBh!Pn_WYfGh&|0r_;$;4atRV!P-A4=Kp#A{q8*?uNGSG@1Bs!uRjb_BYbH7EAcS z_5XPRn0Uykx}IR{e*na=-65#``+WYQ8UAz0*8XfOo1JXGumCS2Y8;5uIfDY|Dg)Zu z|Ic^+XCt*Wko4;7KaQB1%e99 zEbRX(k5BlW1&SSbju?gIi6miymMe4b#h?4Kd^pGLWn3wNyg#CJb-H-hb zZZnzdze0dt@=ZD5_%odjHIB6GrZYPL!JtPQdORU10cCCf`V@cs>DFPg{6Exi zCPBFAZ-w!kl3{u$Pma(VWOIyLnt`(P3H12GFN`r#uJ$+elWlV@yPcRaOShS&3q3TzNvj;gQ zs)v;e+YJC6Ep)tx`v)fM;UQc64bXE1W+Kc!bH~xBY6~wI*B4}4Yx3%@fyOlUqQxUy z20y3Kxuq&*#lUJFxKJB&-~WIr0Z2DX(n)^;Rr+XS=?d^y&j3)4%xuAJG9=bETYLiI z7?;;wQ`U#jT#!?L2gKq95RuGY2I#dt|GrOkd56%2TafwGj06jdZV?Cv25Z*rXYKE@ zXPlmt@bu$IUy4x#VTZ3>jf(i-ZZ|kI_fN){lde97tt{V)O4L%3Qvdzh*QAcnst_NY zWn@Q4?8t>6k4QVmU0-6K|Bv7k(I9d?ZQXe{?x@tSoh5TK9YpaKH=3d^4e9_qu5aKk z@?Lq=me)f%(Bc*cGZde=C2jlqqX|YKarB97TH*3=KI=`i>-#X#D;}2Z-mAW~X3yMfV9K?|ISxwy zJE_gl)vfJLb^%q6iSN}B^3S};KIRM1wUAq)<;S$ueddj;%0|d|kjpLIF@wAf-FK!A8KXrMUV~(k8x4`Sn3p*%?t8CfOwYxa9F}61@&yC2Phf+4 zx%C^D?PJfCjyJFzzY0j-3&hu+elY{wIB-o|z6iD_iO6*xEJw-J)CwIlc-6RiNri|% zxAz$+zgkI#{aO+#*N-^-t~o!BjGpaYNu14iD;X3VZrVf{ez(aj4)!FABsL6ykAm`~ zB2p;@BdXBWU~&XzwE{qAPDkM^C36~!$i>NPxQbBLFFsWAjaAqg8r@Mpziq;zB_%-9!{Sb*GM2aAtEUqh7OYmU1?p%p-#Hym}pJwO;tlI;b} zZQ?4RuuEli5v1>sVEQ$6havth2)3}4? zKBLJVu2(0(@_&9ddBoLe5-6?nZ!sadV;g2Nlt*;k>{ZQ%^%cU0w|tK2)L+QV>)S?( zeD$7jQWD7F3C^J{5Ti)?FmrdNF!bg>lakFbs-z9SH=66bl<1yt$cHEIdIy5tt()p9 z{^Bs2ut4YCA9=+c#={9t$P}Y0%v!0OeFyW7+_@LoCm^9Hce$ znF!Sw%gyyb>a(AA@?558M#kUu_B*>D4_(7o*_(qWHF&#UK=Lu+ZW6Psv z#<=hP#Y6se>GgJF53MP!7h=%3Q2XHirgu){h(J3Sch1ql9WE8F{|4OLW=ZrO1I_f) z$ns2rnKJwc!2qM+Gt41V{>k&MC<1hFRLfiL)YM;Cyp>Lkj1~E!gS*Z%=ntwC&<8PMy z)-YJ~o*Ie%sssANvJ3Ps$=^*Hu$Bvd`SZf_a*L^RjflsK1Ett+lv$Uez{N(!KwkpI zPRS<*6cUJbKlSAfjY|o5yjs-}yo|Rhr#_hj1J4}S;?cw~U5keGi@d3qYhd|#8A->1 zr|ejqAwycIjV=4C|5-`k4I0641x_A?$|p}AY?k>MCOnvwf1Ym)OrE4Y0#%tQrt9?t zIxY5yhLT`i)qEAwuqJ6PG$-!=%xXUxdDug%9m+?CJoa*n<)nVRa&PMX*2jja2{-!H zjSN$xs>ULpjlV2C5AvA=P7G9BK*=JaeX6vI-wD^mE5+Gc`m}053o@3>*sFi-3iLMl zBf9nL1rwp0b9~FHJ70WVg7WfIxc^GhvfouOksp+Mxi}cPez^^vwf3>cgcA1tNYWBr zYo_C%UL|g_rxJJm1Tv%*Xy)86eRdMT2aCw$x%on$WiG`vHQXyJ%mEY zsP8Ib9Fi^nZmFOKp1gl`Y+m!M?*^j{fgs5j!GuuGy@JBG4D@`#^xL&0u)XsdgHX#7 zD}!sxdFpM`8N_e0sqI$Tp6%y&2f%RjQk;`PRb2Dv%*tsB2#2TcSBxu+*aunnv2)zw zjEvjvX3CDc8nf&GVOvTgly$j_@983&G61SVQ9eu#Jl^KLR+Bd!F>0h4_nfn@0U3?k zqkL3l6eVeO8s6V~ePB6s=;YDEbbr5Q_g&bdYd8yFdJB`)dez$^eQsXJy8=Nhzu1P5 zdBrZYHar=DVXIb96lFkBlx-YZu;^uAyR&H?|r6~s;7D|uq3gJq`EUUT-O&8FUyt;_B5@mdy7QIX>D*u82_koqB6 z75Q-2CX`J;6ADI>Z8onNR>G4>(CyCFYWa+Nys{?NE7;6v26Oc=DLp)yuK1Y})Ab%C z1z$MAbv%$5APr8TpcfiNHU5rI^tT@U>zV;$$Qv7NzG_uLGR$;R0EYTA1|Y54HOY>- z8b6lpQI(vyzT7K_pBOv;YPb%cYNa6a)&g6i^Q6TsoT~#+*N5j7Jdtzd9G4EY%i`fAgvprzY(04D}K+)~JkBrgKhHb~-VExpb4?qg z1db0ER5dOgXk$FalsIeg^d^e96kG(X@kJvz45}@$u(y&PCc3LvrCPgHuh>$i1Dh6@^=}n86-+@@%>llRMM}qU8*NO)2*Jf9hDPO*Cx*g^47cmRGdkPUBJZf_|ky5`1OgF&gfMmvBjbBrECd z#$E5}GBUwCBu3>9B}x+Y&bBwcqZqRP?8lB46UXLwX9mlb(Ob|iP0mHyCA4j8-qF^0 z-3OKr*GDa$dcv@{bmORZu~iFs5hlnN%)ki6tz3*ZxKzf%al{FQlA)3Y{(E*ciXpt4 z95&x*O$>|(ZcVg>nX5*yE;h%q1;tl)C=7m1@^I~s&1Qk)i5(pCr)y48tb_`l%$$?? z){>t&n|jqxx1?#s=4k)@m|(UbHK{-DD5z)U8l$hXT~iDu!?0e&W<%FI*pGD6U?sE2 z$)KleIemXRXcYMHN0liDfJsdXOkw?&+m=rq&j@gu@D_1NcL1yt0-hxI-ZRoHH^l>) zs3hN`-0nneQ26Au*9)Lp&BAFLK~2QN-I;KT0-8qggv|`jEu$2miphqfCPLbE=yl$A z|5F@$UWu_g$SDqcZ3 z@@;9EbJUAt<6Pz@_?%-0N{F_N_&W*(M{&gYx&r>9*Yv7!Z#Q{oJKQbAMC(W#v|Lxv zz<2z4FNC$ujK$llj_54;5F?ZbP8|7(OgQ6_k62frw;EitypzEiqU)h0ZAc?~S&cTs zr!eIxw}<9p6AUEk%PDd*7U~-~S(~J1)O(ZKz`aNBoq@}RSm$_cwMC$c$%LF}$;X7G z3lD|0$D5OFF!ZG<3f_R+2Y#umel)Szj7@}{YuiYhyU@h?@7zi5^_{)0vl~oc!gX#1 z=sco6BA{%eO0g^hdJ!3OeGJ8sI>TV<>DW;y)CtYC8fPy{TI3zrqc{Xu@@iDT5aA} z&7o13Buq8K&#xlJo_vd$pKh0EJaJ*4`}0TDV!n2wKK+vY-R~V{aUToYYiDQk-S920 zsClp^=NgFWbp5E{DK6t(41l{q-?~}oWA@CKg<#yo_TXU^k!Ni|N_*MmSH?^u^2Pd@ zYe7<~%7};Qe*4m!WJYHNAPB{6LE+nDQi#UBg74Wb6vq|$q=)by&zRnuZ_dQAfs>emYU1w$+4kM!R|BmF_b59j7uK6zb{&X&ru(dxZYnuGZ z<}${dng~L8`=WRzotM0A&=hhRC*a{Z;c=NRGSP!ZkwGOFxKd?}l(^|QOz#o!0kR2N zB+XaIO!6%>hf{h!J7ogje=k%=KUqU-JJy&l$@R5`l_2f7&HI z`!wLKwyq(o^)zp5mh6SNdm?`6&s$U8S=Q^S1a%*V24!wvSDF69F`!!PkTf%~pou5J z(zB0&*82R;YsxrC1LmWKL?TmnsN(^U`JA0xJ;ect$!OBb)#S>kedcvvUG@#U!n3Mj z$*5R<|7V+^Nwp*YL3rZa$B!xUU962RI#Y^bzB!>oTd%Ux@3xKn{aI7K>C>dUubuCH z+18z-6Z0X=ka6lSM@3N&l0j z#}H<|)KSKG&3a#_JrPcC@t6lHCK3UQMt0RlxG=SrW%hJ?iM0X{;TnkLw0UUB7CPFZ<^1G1~38#b$I$?&bzMX$3jojSSM7)3q0=hQ?hUZ|rV}c6;k^dUyFZySJk5 zHGa}}h|t)J(af?h6mBMvzI3dJd^E6AJ|J*g+|>1IMY6Yti@g?l}Cp4O!GW<2KtEto<$KjPTT& zGdh)t*^7ELsz~g&)YN2?^_&9-*GHfn3`bMu`Tl?Rs! zA<{!q$KcxAP789_;~8C<8dSxLU&-rzFaXJl8OjpgrL~e%9}3SAVxQ;9HphVHLj{~{ z!AEI9R>Q9Yp0lLE&k zcs_;nLf_oJM7d%o4r6y)Hs>RXPrI?>c=b)DDvgO(%lnfVVfA<`&R?7#td5CiF6TDe zYI7h_zV*A#L9)BhZuj@$>~2uux1|JY_+f~hG>2?^-p#kq6^AXIGo?KNxoWmdUsYow z3HS<5C(0?4(e?S&B#|C0RWwWei>n>H(MdLtT+_!x`4u|krwe_LD&Q$m$|Xi0A&y+d z-$CQQteP|r1#b3PO!Xsq^V&;0b+43T%6q4d!}4WHtb0m5s&@I4^|R{7u2CB*gV*5} z-W&JTO1FFnAQ|Dnv5satnKM6yhq_MGr|)HWOw<0!)C#Lq7Z%f{l;3AOI_*RBOW*SS zIre}23A;dL5WBV<*l&Ro!~Wsf-sA80VoQa+#tvWnK&#TTOKR%FVc9_E%brUu#k09w zCXt>*fO2JGZUT%ntb3C7dNCkTwla~0wbb5#G51XF$*m5Ro`>&#_?5Fs9$TZnwQ|^`+02^ky+I-kdaPl?1=VwBBma=v(C z&W)Q^xi|~O5z`&%B^nC9%MY9YUMjmfh5o5ZENTQId4}5B5+WX3Dimc?eL&Ak<$zj7 zuiY!WBHx3vSMpx=MObwKEN$IPmA%!gaR!`UXF78;%C>1HW;pi!dtcSrlFvT*KfWsh zwhhjnq%Us`sKgVnJdrc5!|CimrF6H36s2phII2i^bfwjj9>sWmhiirW#>>I~ohfGF z!=9!g8g`+p&Z0@H^$doC7f%hV@=-bPGw6R~ zpfR%%RZv{Nf6Exi2rWWqL@q9JG}S}8tg8(3TaQ@5zN)W!6fF#UhSb7DW||?Kz$S1= z;OLqTz^k10jKbQh$zpyPH2}}EmIbmdkXilRzrhFe^YPyozs~nKz}-BqbEBNx3jQB_ zkFxlR+DKV{$|BmIA};owQlQku8MtCDol!L#Fg4%8Z#wO}a zu{bKyL7I=15nG>qJXc4M-7nhOWw`Ge-~*UcO3X{G-AEA*_2`@<_2 zuPL%k7*Q@=_-|J$$Bj(IkH_Xwk(y}kF%1LBDk)W zXIhiYnp9lv{D(O4%sm}o^L{f(!w=F$OoFaFJ!dU+O$P80=P#DYhfK33d>FXWMd&1S ze!nE%OizJ7FBiQ)#l4+-z3#WEg*6J1Uv7I3B>wXwxh)%wzPMUf{QWQCXn98e@(L9T zZ$K#&eP(ES?to9`PQf9#Q);|M;IOJXLIp%v|CQw>8r2yTq%D zX`^>=v{~R>eRlNOmbj#|)5=95Y_oEli-fxj^iWR>+Rp?vbWNQnLZa0hpeOn0E=C

-V2`!UWihTZ}UANo?{ffgi!wwjFzi}(yRd1|#r^ZKeF=L01gGPz?HPWT(2<<4= zoDm4t(rV!mjK5*p^k9F3a~ajf4`T5*=4@m??jqS5^Ch`zU($8OhaGPQPH79!#6)& zJe*OyUx6Q&rf4ydXAKE76TULUo$Y`DGg>bF{X5_R_hqx0r01>MNVUVICkP-gDO-YQ zrooFQ;m^CV#z0&>!(<~3lKO{2W{QyaAOqI)TfqlFeIpT)VU&vZnnZirW-O=#B{Hdn z9mf)3lxC;nPaIp^Jo4FDEri)q8jgsawn&|rwsc_zoL<_@PR?Bmkcz~)mCj6f|YGMBo{nOSAcW2Bo>n(@3~CJ6B*bq zM}mR(>Otz>wAh^#V105utdw=wB9X)%*TuR@N5;FFAAJrmtrGV_tnUAGr19)n)7 z@YEe5fIu8i%0;>=*FgbtDXYv8;sl54FlMkZ-e8n><=z`cAL5y} zj5xk{RFfwSqFslQ=V20##=q>QQN(?>yf-nT`BxEu;{RY%7CRNoKoWVy4mWEio9a2$ zLfbYCt8;eiJf7-#cG{6wxb%k1wQ^k>V!h2}Q_Q?w!V%IO0yXG|n6V_-xw18HbMKX? z+~pbVN14OCq4or&_sW8i{}?}A7Y}c=i+X_8hes1U$`K(YjY<~hvLq7wfId^Lb@u$@ zS^$j8LFf=ToSgwJ;(_O1ZQm24_@=mTk$`ttUG}{RBkk_YWGw@(Q6Ny|Ke|uQFfM zdy_|g9xwa)iI}xGlq&enaQ^BwP!!0w^S-*d>0?;KtK94>99v3~bPL%YiTWv1f-h9{ zQ9(bCKZV3xn@mFY_vyFVJei{tIU?@Tm*>ppuTH){P+*OKdeji{;+S?_g=-uD1}q2A z?9fRgEwcGTv{Zq*)U!&@;GA|S^e37V+Yyowv;6(iRDfmfJq^VPaAuQEl51<7{8-pm zkqai#VhBJSh`C`Ul1X))c(eDt38)r!zgTnxz*+m2!&)MafTA z9T_{jYm>DqlVsWabcMwPUp&{zsC6!r+Z3C2zAHQ_c{}H6*$BhTmyql(&!F=g3pe+) zf08~U&~eW841L)Mz(}WibC&ZLP2}Vqa30i@weUiELsLH79%G`bx#UB&eKY@!YG=6^ z6chcZUam)Z6GEAMJkoMM>Mz`;nPuseJuE?O*K-&s25EXZob*~K0*w8 zA-7rQee=tjt3&fsXs$@`oQ!>b2 zA(LpdDJw>U-Uw`byE=C#qh|CFv{24a2jlRh6|wOc%r=X8a2n@E#x0Q*jr72K-5e=& zLeI1je#TbftvsNT(mAUhmlT=JtdShUmFh?1=D-#l;@Jjd@S;YvBj@P+C(xr!?Sy zx9usNJE3cEO@5{Lwrk~3}JbPY!Kf}duUD*H}s_yap9juef`g`Jrlnw2; zu+h2WhTLOxJ(-r9Q6ti|r1r7=1pScHNUOtb>JM>16A8UXF24j%>`|F?&vhgiHY6CG zD))pnEphVAHmd*uNLDFES_)rL)#oYgvG@i`6w|corLS?2=dvf@tOT`LQ}SrQaQ^1! zzyYIE`t-GQE^VI8A>6JW<8osRiZkbQZNHCu>$uCShhtYO5C0yRqE42^|5HTlxlm_7 zc+^~+(E9Y!GawjHdwmCvWwp$~iMSe7++^@exV%#VX2uIuy55)=$Opa6@emK%%HBob zAiUpBBkJ`XB%alYwkVtK!**IQAde-=b>T^Kcm+-JFLd)VLrHPoD{8$8iBRH%u5zIA zPaHNT(F4xYHil!@HFVI%ZGHvr{xNu7WKh!$HyXoG{R0M?j-~cFZ_gDdR1#p@%NjHP zhUYxom;rFM`}2n$H#`6t#9MQ|J;|11loA$|y6d?tw^3}j%%ErPcpUF-24+A6-tB$( zts($#P3a6Qvu?Qm+zf#CF@f~X2uXeUJ^MX>x6qF6rXw$CFz#9H(pzI(BL2CHr0K~j z&?Ba=;wiVsIiH8dZoJCD)Ro6Mrg2oP$=hTkM5En2P@}4B+5J+*+$GBLMq^OBo(J>C zN}j>QkavDCn29|9bpDR}Nt&`gcbXS~r&9BbO&NB)3*r`_x&9=?wo{y}H*dmQ#`*+- zbNQezT=t$ZR(6)&G0M+LSSD#2zexLrQ!YkLBQUER;E6GD?mK}MUkGVr_c+Yfk-Et< zTBB;1+cReIu55~9XMtF8_df4IKAxG8FGun0Cv7BJns1NEEkqxVXGyzZ;_^|2(`dWU zJX|jhj_g-Ay+fp}Wx%^SqJISnV&vHRh3KHAx00zogW;IIG=X@QN7mV=RN#p`JI-?J zkZC#P^ACb>Ce6W4AwGN4&UnTOlg4A;RPdsGm0HU?8BNm;RooGqYNBg>l2hjjsCSj> zdcsBr3%Rp>Z=4>damR-o)kHLdvV@I&RcdLO8E&%Sr+%GT1ESblAwkB6H#2yR`-!FJ^+kxO#Z+$u8@7Ym{DVMwd?%A}s6A$7I#Cqd z)4VZD8j5)}uuj+!`TeaZ@mA$0Yw=EBEbfijM#*$fvCTw0*qu;rr~1TI8juFms!G*O zCUO*pt(DGvE9Qrb;L zCmKu^ZM{9WyS?#2>9rx2+g6iz-Z-e2Tg@a;mh%K0kBRiJb=y2eh;+|5Rl#AW6bN5= z9NuV`oS`d4-8(I}ysMtZDn3p8>B4zX8l=^v6w1JIduk`|wEEq%EVjzHpqB}E4{ZeD zAjcbHsglQqNw3|_-NwskXzco>=!q2QLG{Ht_h3z%(={rckP z2ot4Y60VXrq*6HGpFIsjg$VUVIDxXdLWdtjO31X@WV0vwPP}e{Lx_{|DfC8DWdZh) z_RV$9#4X6gmMYO|qFc?Os|Qa$xZSct-z>666-$Z~v&%M|Dg0!uCN&C`CtOt`t_B1Z zY#qy^PCg8qo4zR=_g+R#QXliHPf29~4k6k+AN6u}#xPJAJPSGpul^n!{gOWV@3ntn z0VspbZ&H(b+u;|j(srh#KLgq7pM+19R;|9XD9uhNhWEO9mTUMrQ+~BDUNw6j6ko1&2(b@jP?9e|6;qA&3Vi0@s9;f4{)sgNcw6bx4 zT%Z1aKk#cF?-tYZAe!u^^-O4$i@p`{r&-G&Lg7c{~ga%dfp|(u@tWvL) zlY%0z_Nu>G|J(hM&!^Fi{nkAC(IeQoR8vM%W&W15dU0k3;-H3LNs6o4es71p+b<=6^t}iP~iEJ!KU@iAzdzGQGtx?Rv!hwaEb3fjZwL@Yk3!|?4+wY*? zt`Q=x|K}@Zg2UA2ggAy?-*M1vn1XmUcOEUn=FMIqw|P}06YyZ0eFR!{$78h9qtP6ra85n4*8lCw zdd%<SXes0-!?*=J0>)f{GZvu+ns0y~qJ*qBQMy%0Orf?=Pmtjdfow1q2orNX zjRJR-Ahv+NCPl>d@g=+X{D;5KARd7y2xoq~ROm7N7Zrjo%zK7BWrJ)TvtPmIZ-p^b zpRn~Gjr&hL{r6)gRx`oYa`RBkZ2P%?%&UA&`B>fbZtmIsOM ze;)zge|0)j-c*dC`Sl+g{`~+Q#vDAp_Fj9;>_gmi1>ys+#etlXhX5wq#$l1A5J!5J zny~}D84z7cbEOa`#P;$HH|Y?R!68UZ;Ian&@fqGqZZQk(v(|y@r(mVt?DHU{H?{uj%QtPp#ytLZS9YE^ z03g5*vX#Xp+&l>!X-ze^u{Ny2cz4-o0Q$QC{_jk7pN5KaWMO7Z$M_XCn0fo5Ck@YA z8~*Go2l%cR+D@y2>_sBVr33oWGA_i-;E2K3pYRBgX^|T*%X;qNR`5bjb4cNC# z@kHh}&|j%Ieo}PlGm`{p36JKtpvSIO2VFWHyHy#FN$Cb2TR})R%&OUx^Tzfw?$G7&YDH5v=4*6uY z^lVqp$KwFp5V7-KOj3O6Aos2jR&<|Qnx}Xj=a~a*XM8Q_-?y#t1rZ{9zy%)WHEXND zx|*fzz7N~Lda_{-`mDx2NcPQ1mO@?|J9MKN&Bm^2 zKC35uO5TwwYEQjw4NuPE#8Qf7s@lplvB4%j|&c4=Zu7+Em0nA%yIVkgIHd%uJr+ zJQX3l%k9KlqkDNGTvy7rId5y%&Vt@&FuTfsd)bZsK8rbHFF%rUPrtG$CxQuVW6iK% zz;^#eb}w4S8uJY6uQ&AC1|Jmr=4vD@L}gd^aUGfgpqg-nE8cNS{Lxl*>P+A$2GZ9C z;e-GhPW69xGq}^Ql@pu=ZYch%h@2K6=6-)gns*5smfvNcRCx9{ui`oVA2^nH_leyO zF+?-Z7KSu<_%xRptZ+fmhiHv$sWSUpXzzsdM3j@LMzSv$w4BpoT-YrfoiGWM6_$E2 zv@I;65w0THXNf;CVFfs>K8X9zNL?^=ObeHvc$APw3U%|GxY=o%)HN$Ln#S7vxG-6# zj^h>h7-R!uHEO0AB5>Lz$+bstQ%Rwo#05k%Fp77vq=k#{4TgnGO+MY3)T!xRfNjxK zmo){;oPzrzEr$gQ=r%SKwx6bcRT4PXFLOFVpD?~vZLK-;I^4)KdB1lNBRL^!+cGMdyd zcF!YH+%Y>9i2ytJLbEG@>%JN4_-fFRd~6&ByjZupBp6ZSgydYIO7o)5CdL7;ry6MF z_T&D^4{ot(`TieGaWwR@IQm%Pbf^YPvR>WI^xW1O>aLhCdc2r?hCf>s$R#_auc6#?y%5&W0ZukeQ8PWLM;ny#;2XdOQnHLdo0A(Xb;ev-#<%uE!jw zDp+^4J1EGs`s-(#x=V+kCsN)B`;SQp75D)*?S-ueUrec4!WC58N<5qqx-D)$@RC}t zLHidzSM7EEqHKEkf(pNhX6P%DfZV}rhG$i{@5R3N9&ZC2S>j7pSucX}CcMyQ7ORFs*Q0~S=*{{CX|ojy<9_G4xn({LZZJEvz2 zXKd-LMQ#YfGYc=$zxja$MBxjC$n(Xw$J^=a_RYl@Sk{1e5tK|((CiY-Firfprd#mQ zHe!~{M~~d7pEdPY$`W6mbm{k?<6=j{l$Yd@=9hftRtZk;@V0vM-M(rV)em&EG0lN{ zl7_{fGLKvxXB78fEuUWK3d=zl%*>4krM z({~D-79<2JFFEAG%0evl`z0z(mzr%j&&jenpi3h0EYUDHHa#r}Y;|f3REEB*29+vgDCdeDI#sNV0A` zcdSH!--r6SP4`AY=vM{H7~-Nca^9<4gJLyMz8KT7-YpMbQS=euiay|28)d_FuPf`q zggKet=yEXcX28yc=d}lQz9@ASkav94S8Xms+dG)lnRP&WVf3P2%nxq2s4LjS20yIm z7;aoJcwCLZymrW&F#-T^gDXovT@4&F8eJLhm0a+hTknK?7gBhfXS1RDez49B(VT#x zb&u>xwOB@vS?SOz4wZPA-PDk_!YhfMCBY4176zSH zgp)unOt#NHJEfx|pCHpi*Fq8}MX?TnS%(We*>yeK@9)$zIV)A&dVhZ*^C0eYtl0G4 z)&`kBH-6paZEllAi`OUe_(L9%;Ps{rlmsi6<(6=L?Q3rHoLP}xjEv1yb5O_Xdmdl= z6k2|>Ak9N8rYn$plo)lZ`et0HW?KKH z^Ymi6Olx%#>V}cx=||D~iF%xN^_a;*kCgXoBBxXsRq5PP(+l8uupZ-fg03LHx`*_@ z4(X|dM=10xpx|ur;?7^AFSpprfCIY^5I;sShK<~^(iufFow}_Na!V5B%>Dr>sykh` z^m+@E2zHi+u08V&H_6N=bFZX3Ug~;{#CIAh9*sz~i$v$W@vBS5ve(i@`Y4Y-9%<8k zak8a=ye*}PmM!LZ6|+~m%oaJS#u&*Lx!$Jst7D(z4&T~j{|&Bj$BZcwezdFdbSx6! zXMQAhC+h%{*w1q00TLsJb^9W65)xVo|G@XDaoWf?4Dvd7_WK!F*PMhcpD#PjL`>g5;c<`wy~#=b0duB4sJKIcira6T0Kpb{JWEg$1qq&V_%`2Kj9@83e9WtBKPu;k_XLf%&n>j{zPYMv|mR@ z7`IT7kl}dRXWCG=xlfrI+eDE}{>qlRrjUN(vhWDsDNH*(oLZa%02Jw%f`-a`P>7+| zTi~x^<9!Kp&bU)Dt;H?pb!cXzRigzguPJm{!ItK;P>k@vpv`Hi*V(YhP4$(Z%dp%{ebEPXMSt1X>e zazzBO&3(3UF08B%I)Nq5V*}<_%OzaTi5kZA^t>SHRhCiJxifjXggIY zbIGf&>#B(tFFA(25p$-%%TfVM#QTvw&y^Ee=uAmeV*_!AZMTbk=lMOvHrcTJ_*lea z)Wt7)d1#Au_%z07sYh6n^+PE)R4@&Gdhn}B6&|T8&O%B&VvfAUwtGsX&V)Zlu>Z89 zz6=+|P&OyYja1`wvM@TWNzq`VY#(FmWnxhOlX-2dTT?qu9`vT|L>q0QhG=gly$ z;t4Fs^>OvJFMQ6_sSzmFAI#1%+qpEd5ZV%kz{UibWn!6AvXYykY$k&8i`mdg8wgS8H!X$2ee^N9WN zW~~p^PtX;(5h9=eH7S)ph00?lM!S(g6|onnw3qU0R$N~N`U;z*jNYiP-W#{ycboH} zSv`<4=DA;OAzZUeBFxWy4m_*-UZr!%C2j#bUA-J{wg?Pr)RTbA3C7Pq8PgBM4BlnufNQ7N;Rng`noZQdSO!9dPoq z_JRw*7Z}GUhe!zQ{J7OEq72e}!)r~eI3vx^=(P!K>wn?nspWdMkFH_sdF@@ZWv15t zY8@AhW4A5|Bb=6)b13csA%xzHGN*8VVY)uaDRz#_zWwrKcA(3Fxn1cQ(=WEU4n1xx z*o<>ZhaX;g(Ij4@CZqXE+8T1-tCt;z$o18`N{-Vq2C>@Qx4vXgLP_)PRp~t|qQ0(A zR5ylHx!{+3$bNbdIba#Tvh5V6BUQLq);UG4vwi5|GW$AZHYEZ_5%~_r%i}iLME+W= zn|d+Q58WAsa(CM1t?vVOpM0Pf%60�(n+IM?{MgmYHfg=D4mD@x_BYV|O~wOPRjmz$`Je5r;!AIXb>=fyJ*g})}CullHH2USOtpkp8g6s zTm%e5g%3T64kWdY04VTB^gt47?3fj3u_XY`cp)^P=mcDZXTUlzL+i2Jm4*3M8xZu~ zi8AKKbCAy*xA?hS&>6{(h_1ptYf64lv+|g@!KN`s(JJV-*lPQDGw+bfq_lCf+~Y_W zkqx>}EC=;lmh4wF*sCbupM z^e~fUEL8ZppWlb(z1WC!w0!2U)*vVYcLHSjzn%}>%RhD4JGQ_UW3eOc36Zz49<0sk z!ZOr@pr^l>sD#O7ID|VHi(^FM?8#1E_?6v z#IZNGjX{643BeNJ&lEz6DQ%!ByaJeE(_c;W`bWX5Syx-DY5#gu4Ni?<@lyb+Qt>rvmwKl=87~fGgv=39B&xwypNsDR_yM?Iu4E# z+H&TQx_N7647%82+|=v89KH6DMT zirM1XqJ(-Cf%MSY@F{P#9f$&3qY~2QPJPMiyI1;QgF^v9yA1x+0CmUKm>o~^n#NM?oxO~^{G1IsPMUc%D zxwHPQDf)&S%6pJEcZ_Cq!rxN=%vrNYYsAS4mLZHD9EXV_*ou<;)g5oj-<#08C;uI zW8^Gq*mjt8nq9FlGoWCSmP7+bZh(4@u5_}3he` zC>@@IY;2RI#?>W!Vp_KKj-%XS+G$WQ14d?m&Z@<&ia#UMg=uqeY+xVsN;6=s0Exi7 z^XBb%RX*~3;kh;ea`@B&gi)3QT^1S8!E?HKus;i+3Sqg5gtw1ZD>Gi1X!e+c;bLJy z?1u>cqAJ1l{|VXu#zg7p0GXREXtBD!c~=%BAkcZd!|J!4;e@4dPN$Rj_B5eQbN8T= z6*yQl^`0}RL3s@O+;;t9e|5;?V$G3JbDgPA&02A%c`n8EWyaN9Xdb4 zJaCt!01V@FZA`q|$~+HW$wMd03o~Fnr=t|Wyg`nN5}iT4*Btaa(eWw-h9zmcM=8s> z{mxNR_LeTP#&tt@OgQd9qn#CqW{&gOs`lA;NNEtQ>Ta~K%0_BR^-!sghR1;6lX1_Q3OTBCg zK{H543mKV{w}qT{A*0ej?{AjP2#7OicpM1fVO4GW=?f6Q{^{vbBoIEFE>rI$ zBUa$O*r0AVm_pq%S1H~L@2ek}EyW99EUG>5VPe&^!_ZIA5ENg5U=yP=cYz{KK5PZ0 z#u?*QY|I7GgWF1#;)>irt*-ES-nBcL|G)wczK;nwAT4tlW=$PlH`9Z@x#SDI{H%)Hh*a%f zdJZ52+XcayPL0`=ihX}7eLH$XbqNFiWXFFvDE?7a5jkYf>2XD>;cRRQW=s{CFXx}k zZQ7t0E%gGOZxTMMsEc7?#|KIKB+-EJmghVWz3ngy?f_JCQRqTP%HighhvZwLVfk@CZaWKa~+mq=#pxp>Au2!cNj@Y@+?FHcVG#c^ad<08rYU0a@U*l`pNEz38Yt* z2#`(OLdWG`fXZCM@-{b#3oCGf-2+MvH+!z1Efc(+_Hja#s|Wj8snlII<~?QR86V?_S15KBCpoYeAQ-$>~sWk>2#MU zXGCKNUN%zvPFB9>2Z_9zU5#{D}!)}ikonP6T1&tFC+;XHDweZUo`CYKPK}90XtPl zWrDKVyTAMqi%zxql7@^`gKp&Cx7*tnc$hbocGvBN*6?3*R4{P`0p}XXvwnt0IvI)lAb@R=x zp4E}!9Ju6F1o2b`a&s$jUOSFlcax}sBS!;b0|7R{IbhaV4tr%!;7nHgxK|}U4cbxu zq>hWT0X;{RS!()&c;^X7#-@!v^0szs>Pi-MFUZgDfh$ZpGlX>xjVpz&o?je&F|%f% zx~+-cjj$ME&Po9p&Ud6aq-$V(+bD6$OR^=`^((i>EYe=TV6aPN9gFygm`W`@$15uq z!BvAkNoP2(m2b)hx&7c$rt5U-H4uFE&-dNok3QxK7U-(*;;BTwuXwxF!lbV-ZE%6O z*C|fiwgVgoUQ5t@$^a%MN5jF=9#QJHXR6E%S7>}t?ZWu|uj0{LACP3TnbzP|cTB7j zA?$Ce`0xq$@pzc5U+0BtZ0Wh#^k7_k;lNyjVB+lon9iQJp3JpXHvOCjAmG5!2d(Vk zvcpFcZ}&aPnKK7ZN-yXmYu8BLFnhr;n+oy7iYk)NQ^6zZIy0Isau+zvEovuss)Cis z&wccNKa350!1Xscv|X}&&7!@A;50~Fi~I0SIp)0^SQJQ_1F5+A+)Nii`u#YTXwBx; z32(V_P=e~GygA_kepuX)7we}Ec|xGoS#1rSkswgv5*-(k1&|q2#H=}a2@=rQe5kBE z%SBz4QDqYFb-8kA1Cqo)^Zhr`Q_5 za?VHrJF{B;qaccTaqLXI{2~Y(&ypS9P|GEsd6itoJ{6|1o>V8z-}mP&Of^F&B6~rx ze6n98a3mC-dA>a6w6VWf)#=4tZR(ew3e~j>%X8uH!o$xxZ%)xL>vCovgmdD09IdUP zmEyBYuV=A4_^$1{P_&n7PPidHyR59(!8gDO&vc`g`FlriV~%50g+TB&$xBqVQN zyi=Fk!oK+IInK!*hAVZqoi#LDb$XUb!z}HhOv^`RWJTD%-}popn@xX@ZG!sa9jDItG8Z^5bYJi6DW7PYw$u<&v>k~2?`X(=_1m|&yS+wttW;crx zA0>mpazksi!)>K&^)V7TUepr8k0m6$Mjxo;_0~poZ6FbJ^SHpZaU&-HW06c8&S;$TS%tY9B>(bocf?F}9B68vlnC|0DI82$0sHn2nC;VF?4Vu(zkJGzv|P~+ ziP8{tM_+!AS=AkZxz9->PSWRIh*+k6JUqJ$e&`tHZKd}DwsPJny%_b;+P3o*TQn)H z^yT_cWkT*hj@BO{i~3E_zt~P$r2YgiQPYzX>%QlgGaAHeo^{cih2=o>M%=PexzP#H zEA=pyo;cf>f-g4gA; zMW0Y_Gh2V-PwK&p-+mtYaBF5=n7yO17h~J0e`)DKKeCrW`|rO1tS)TK`f`%L0>a!L z&x9^KI*_;p+gbvYW#k>rE?D9I0b*-ZSqk-Ub-H9U-<8XyreI4y{7RG4MMMG2P_zG~OugNuu(B81*jyEUQ1k%Z_@L*$mCGSiYVK$fCE+0Ac04N`!(2kS_yyBu>Wl^0&|I2*9?Ys zE(se0h$6_>h@a2ix1g)B`)zaponldMl;#T1<3fXWTWygq^f|LS$Xha{-?>Y=jhXA$ z+KCuiAv;~S#F;6bpJ|yU%tQ`N4H`7qf$z>L zbzguZ^U1~LBP%Ge1a^b@p==9S`E)jty8%29wX>t(yL)RtWW2HDXj?P z`Ad5na@}?T?%cDH%GT%;V%NBAA-7avJBQRz^r9$G>B+9w*I8NYCwP_zO_i}DTqaIc z=c6#4EHm6JS+7u|s;KD2{3M|e448JP{;py_Iv`}GmZ#ecFCksQ1@+&gf1Tm(Fa9>R z@`V#%3z}g$UCI%y6D*PnO)y96Zz=7<5?RyKPv21Cj~~Gci+c=*ig1h3O{j2r^c7m0 z?KX)Ov)HB$x>B!V*ohS7^+obXyt`%1s7jqTkAr$YctAf)OI}%pWBEJx%`*es_da?w z{?{suO%;Lkx1Z=-<5y5oc!zyo=wvY!U zlmb#!2ez>tl#4xcQ+UGzVs1@x%a?0UV8wJ;3LE!PkzPW(rr9<0mrV3tkorV_Y+^2@ zSd2u-+_LX>Ca@!O>FEao=aw&zRV=l$6AJmRS~=JI7fxwbh>* zNX4iM7##iI?IQ^5tv2(0+-WEC>lJU>jG6gcsG1<3O3!jvjYYC_6u!jiB6bttrJu#v zH1d!0s)yl*79qu?CTYp`jVm+^B2= zmQv-Cny%kV1OAH=a)dy3;j9ws4Z6D`U}f(}x*H#G0Qb*2cLm z3>$~p%R(S z2}#SZgT)M-=LKWkH~hzU-11!`9sxidcRrPzMw~Ob8nu~i^L=KT`_`OWa}Q@Aen-t} z2Dp_9_=@a^+;OdsJmNO@@BwRq#Lltg$M>G`TjtjyDjZZjTVse1G!U}Qi>T!X9E7-U zS9j*{ibH6gCqZYx$Rn>#}yl&zrI65_~!MQ%^gLn448lWD-()eM61Wk$kJ!iTTO0e?9 z2i!9cUP*3}Xy6c|5oRvN?MYaIUJu-~q;lou`9~!DJ8L!jLJ+S{Tir=c*SYlxcFm;3mwT)7AF;Vc>nRF4Ko1%k|S^AU9w?w88^0|qOI;y~jW zXko2DtsVVTFbesaBw)9n*N0cIC2m=25h#=JzEu5XGqmo#^r@$mz!6c_aA4wvR+h)z z=;XH9maNXaDyaZvVM|?{<+ry(ejcm)D>0MKZv#3XMN6|9hZq%n&@y5;6^o4?2f{l% zp`i?Z7@I_8P#_~6VDBl-13Z4Pj+G{wVeVkiHToSWn(47|W1PY=jL%~9Y0y`c zVADnQNlN=Wvpd{HhU3Sv{6?-*G9W!9D$_d)op(`lL~^bZG{3N%rC%)a489^8VAchX zHI@@QF8T|{$!gIEzI-;3HWc4wz0tr3YqWLr4Nu~h=0Uw1W1UAvI7;j@ZZvS(7;Gd} z@3^oCx+2?nUCqps&fK-L&+9xE-|k59l8HX{tWAwWRww77b)~_9$|q^XyUp zJz5%eD&4`*cNxS`}Qg@9lN@I;xX2zf+44`ETLqx5S*BV$|>Tv~SvyOh0k# zGk8}FA?bs`RXZA*3|YanWCF8H43iyD=CpDSlSbPteaWrP14=xuJpJAgp&Co%@RE&~ z=}1kHgK`?1>xzN!sss`FCglHCN8)lsrCbAOrV{D`E4H{X>WF|VM2=M!hpjE_5;lLF&mqVmKyuwju5&^fp|1j&zeMDNUP+X?F5OAI@vLF4ag?x6we z_~f94SX$PebYWBIYVmvAfUZg!VOMA=5Pk~&{bDPO8Iwgio6v9^|-3rR`8#j!cv z+<76<>Ye_XKv{M_^UZC|0veV*Eih5~ikMNhzZlBQ&=T7vY`j1Un`wrhPF9|36rfl# zaUWHoSa+VMKsFTircYM<9?4&`q}OCHK}8}c$?I%{`xQ^jj3Q(HBZYvQA9n0 zOm7x2ZqocXwu6e7Gsa6Ssx1rIeQWD=w>$ae&xtjn;wugWnF<-G5+z zKV{)I{U;9f2aJFxp}!b9d9yZue-ky080@ej=HSGbV*tn#0ULyq)+7*;74RrgBuEiP zo*^kC>DrGxAx}#X34u1JzfD~0%5UtaX8V2c;)4|;@UI0dZTWz@RM?s3h3l}t>vObv zW*70mZ;b$YlD>=WRd9esy#J)ZcJ>4{W)xwSm4|8%gsdA9f+aX8Lgz{f3i&excFg}< z?xDz^Z-XL^-n5!%yx*yjl=}f8OX%oC0`kc^(Z*)UE|A9d0l|0|W%M_YQF>pJz>`q& z#bIx5wCE|MtzMuFtks6dhvCUYQ&#}BWLoDXDNt$Mw` zErbYZG&-~bB`wW_cOh0CQQ3}y5$*wpE{AYxVl_RGyn|l(XY_b8A}Poj;@?r?w%Vn2 zO+OxB6A1t!Da$aAMddfHgJ0RfLk ziKP-7yMs#Mt$!|ve~i*Xk^5Pv?vCuKwNs9}$X)b17xd!=r`J zh@KmI!E$cIban8sE2e&Rv1Iiq@S3Z~k68-D=8&U76(T4G1To6plV1Gt!q5Likb?@p zwYvoQ?H%1(KM;MpOetMTALvPQ%$X zkq5VC^h=4xwNj3_KPSLf&SOW<5Z%GNAU`6?_x8onb z|J^#};Y8ELds--fW_`1e&BAb!lI61!dx|PO7VuvurjtG3mg)nLIrWCh=|v#5jgaNp z1IYdo7pDCP3*G@iBL0ZepQrEf2hRrmpT@_^z*SURc&Bq1RGdtc$n_KdY&8Y%vR``~ zM0{)zorpgGZ)7$K)_DJyw0f)1A<67jAa#AtoxhOy_8DcX5oU?unK9>rB=p2xw~2BK zzNqsM(EDrP9qtdH!WzvEz1xF;)M+I6;KPhDycBwmrk0`3T=V9(m$`$wpX59Su-=s8 z9?(XeV>{O^1h3nCzq4nzK;(4cJd~U5&9^{BIGU1uqeD7{h?FdN=U&z@CVJ0tz=Z~) zcpO?45^i0-ROmW*v?ryxthnG(q5*?7s=FxKrs+Yp-$>d#0_BC3zTOH(EZ#QjQD@7q z&gN~UbJEohos4LySM?X(hkFKWMxekIu3Sq*K+=u2!}>?|o3E-+(P0V6IJI@yBwLIeuDt}9m$R1+w6xi0`*Q!uGNDG0RhWPvF!CD->!RuN zvA7i>jqH(L*o#t^v@#A|jpWI~XC}cYJi`$zC=S7e1qUU9SK$5k0c%iNMsH$685xSa zd6tYF{r$neR%ivpUIaPXxcOND2(P@pD?U`?@JxJblu*3ah~z{hDQbT5qL!g30hFuU z(mq5;-?&-aW?>%*gZ`zKdX#b{73S{ucWTTpR6HE8|18GeIiXHJXcs$cV{N!%RSZC! z#i*{EIm+wonQ09bupKs63G9OcB-D!(Sq!ZEP&QQ`5-+SN;RfOtnv;5=F#1`$jOvI$ zdlsX_{lh#vVqWVroqxSP9vS5kor)Zfo~~xT6y+Jzj-&y5{Vu`{IkO5GxA2o3m(1i73++E<-^~*I}!H!vHS8OZgU$D4J zlodG`229_~93)rPe!RLDGn}7SducjP-H_n zGLA|BlfcpP^2q*kjkv@LOFm=W;tf}dD~>oqmfo3od%G5JBQ1L3o|0i1wPUOXLGi!3 ze!ixMsPB7d6&_-V$AEU~?k4UhPa8&2ERlN4DoMN~((4I8E0Z}8C`MbpULcJHh3q8i zs9(Yn;sR(uq#uA3v|`|G)HY2FLy3z1AaNB%)?%bqA_v~c!I?zMuRwvOO?EoQA;z9H z>CQdp8$cH`({RTN)+W%{rpSu8Lf~Af12&trPW@){D$0Mx7Ro_Hr=#Y&q)N>b`#TFE zX|%MnI#nS1DIx(Nbv=A^8750+(oJy5YQ?EQp+8yMh%8)b zIwtd{K}2EJ$NYtwW$l+)9#6f8lBigA%z`=_KhgZ4EE@}!70VlvUYhr&6hWJ zvUT^jgvkQjQ2xFyTx7eKRwe(!RaOGP%Ishew_Ab>Tg{;iet~>;SipvIH%dS3G||-3 zSy*#eBM6rAlueccrDCjfXM`uijoTTQE_)HOE|-m@ z!zF}7;uqHqRExF>@&&uA{tcM^qo5=>BS^4E#!k_a=f}98wIpE&!F#lFQ`wi@z#``aXF@}u7xp=Vp4*Z>1?@W43ypFe2kj2CXpZJd zZ}(XgJoFX5f;Lpx>v906Tj?T)J#@wAFU|Puti#J9M6eI&7FD?p?m9reeWMJX*c5hXF4>p1B(0R)`;FhM4^xwz5sm0Z3vdE%;w9F32=>B zr^>45EO-zpbx>d9qizZbufi|E=%S+75JtBg^+91QH5DfNN(u|!QhEvJ?FK`$zM z*EGQ$EiHEl*Iji5M$VQz9|lps`rY1~Rn+}kV^8v*dv}kaQ8osavI9;+StS zPqGX)8-0(@Ekl-ZV$}(9KAnYhJ9I?0l#zT}L%GuV{De3;wOH%1F0-~$nag!(m~|uN zHCmk42I%ea!@9J+<;Q|`J49)t zkG7x6`fKk6-RsohNx%M^Yg63MxZi0MH{r^m`uB4r^}zkbuf2wcI`72LL-*tFKZ$8>>f*Du*k7q)`dJFhdw$pa97VWN-r`L^cP6VEer*Qxxm@#} z`0~p;ASzUS0*i!w?j0n2Sda1+IFlO%+SD$w zCcRC{5l&u&(0IE~e0S0E7AABi~?E{X9>^q(A|JgI_A!xJU0%Lp(H6EXkz;VYzoG)zBLkKHk)mL+BT z^c~=Vj{qCJ27t9qq&Qcz_$%Fq3~mzd++g?nd`d)JB<{@nm|2S`BE!SFc%swhD`|dY zIFJv}J&4Upf(=lcY8#gVNp55ZS;%!-s0awC6SZuf12s3Esy~NPG05g9T5^x1QkvF7Rdl zCZbw`ICYP%RuT}NN9c2m`{aZ1i%=ZsIOyN*IUUDH7jsAsP%0j(B8WLCjTw%Xbugp0 zQ});C>EhFmD(~!IFRh-*!Oghl1+ zi~4&3@aIFpv`aOKz^mSxC{(LM*^^e)IpnfgL%`PT2^2lU$h4An?5f+URuqfbJ{KXJnx zHtNA&SWT#khm{Q=Kq=7gU;*E&&tap^*`aO{g2fq{tEsP#LizAAm;rg?am!DZFFW>y zZ#t#_>1p=3DSkpip})nFo(p%kP%1NHma#rPV1t1zr^AXkyAue^(8 z2g?KqUFpmpGL2XFxmfl-afv&1Iu@TGgWr5Ys|=i-|=u!cEg1w%GjV&%Qk zqJGm2^`8n)_ee|07i<1cS=G1Fv>BDAzrP1qW& zQ3Dw_cA<$3%?B|(3{vsJm;bo(7K#7kS>zy;SEYMyvd`;y)-pKeqbMjc0ZsX?jq!#| z*te|G>ZfI0H0=>3hMR4(I+>7`aki(+w*WoW&9i4cA6cGx<#1&Qx$yzF>uU$v$?r?e zC<5y7SawhfP4H58205T`ZobyCkaZ58^6&Bh(CA3Ydnz-2YicN1d{49SiP+s+@j`r@ zlUFp?xlI0fo75d{7*n91NPd6ah8dUU+tI~tQ~0_Q1%gOC;} zy47=r;XpjZQnH*>=aDKMZLv%_MzsiN(IY9FXm?x8m?R^ttjbtRQd+iv$x@*k#R7Yk zXsS3;-awGf^&mMH+eb2e*$f6y{qI3I!2ndCI`xS;5O;eSTqlKVJh}j;jeT^Vk)Ic- z4A~*BWTKg>v+2R{rg^0Io~{p&vP<{2d*Qn9#JZKxNg;4UgTK!9>PFI>g8jhl0IE8{ z#L)OJgi1!Z()vYQBiWAbm1s!m&XSi0$M>ztuAXCZROFd_#5HK*T?wgcq>>HY?e zc^q1g!touAsG~QWL{R{qg{f^9FB!x3Qw5^fK;d<@qk6Dote{KlosylTorH26k9{X> zl|J0IH;|FGP{aGU@KL)6I+tLjXF~DCkUHm6jEn{b>bjJRGU6CgHSSEM^yrSj>#gb+ zETx7Lyv$yZk3?_um88_{xL)ch2g*oS9+SxMDd^tA9$OZY3X8H=8 z7@Lh6`a_ayJ0US>^JpUD0Pje%ojyu+uVK%f9`Y__*k0!$ z{T{&MU~%^*B|H6`fr8wS<^P=gp?cSE)qa}4vt4hkA6ctzEi|0}k~y4Uu(`hiNXR9} z-G4mZaUrFMUV{0z#~J;Nzd4!&HH0Yat7G~Z-QA2cDE{?OMfaZTO zq$9z}So+P+-#!53uDAW?!VQ7h#pNGZ!0%o9uOF5X9c8j7pkHQO{g0%IG*nD)vz28?+ZTjuC#IDb?K$gbGu zq0xPw?ORX>;r)j8U&I98KTOm|1~7MqH-1!n)6#S-yQ zQyuP8we^bz_R6?iug?P-*lnQrkWkIZwCL-VMh5}MWrQQYavTb9vA(~4&SS1CkoAhU z+`j;Ob}fQT+{!m8on08Alp3cdD>;gI-;wo(S$rfU;;2iI{u5+kLp^;(+&+SC$5IwR z5?jv+WKLc9)ONqi1noGl1pL^6qt0U{XOGCMH{Qkh`zB?@wI{siPkTFST-M0JG23=d z#FbBbK3)0QPp{qoUzzz>ErbBgcE7%ks*^_pj_CWZ_cMXZ1C}(%m}$zKW2@drpAP{D z$Z3mm7z{xHCUT`Flt+FQv|Y?Ki?ie0YhE36G+;9u(Y_dXXOYD-Xci{raKGo ztbQy+`s4MP$0rnBaP69z&{u{@es~~nug4J*u}8fwSkj+E{NPYXezy7-v5r4O5DjaH zj9w(P6qhrvA%)Vv4ExINqTma1r^s;01UQIGIMZ&+=5x2F7mPgfkjt2=bs_H1A)T43 za?Yzn8!X<=hUH(6lBFKOzO-HCOJbxLs$+*g%y#A5_JIY7zH^JCV$rzs;L+)Zc&RiB zO1dCb8B--80?REU>#|y!mlKdhv$nEhI~F)K?lDr5Mj>Olzi z`Nu(AzZF5L{&PG>UJ5}rI2Ok;4~rQXF|Qcdaqf{uInaIUN?Psy1nLWsf5bB1k;5ZPM9Dmb54eB@mZ*x9+9Pi#-M z>8G5!MeZQH8T4%2VR_Qx(dBj219brf78(Y67H=Lhr9`lf9tBREa%Ur%@9)_6wR^Dh z7~#TCUr;?8paB%dv2W$sqjhW6&42Y2LpGQg!|tXhh+$UrW7FOAS21y~9D}|3?R(r!@$^wg9oU>XNW z$G8Ka(Ozmayk8mOS!<%Ag^4dd#kgwA#cLjL*snNtl~If)tJ)nWGx5ZO)y#3Y7v^-ZJR`zqS_!jnt$zQ`%GUPpt2PoV&8xms1kelCE zlQgQM^UiooU|NJvdjdKF@iblZ@_>{7e%H zR!-+;5?OJ?cW^Q&L2j_oKU9MgFh#Ks^3%SSUF0WG=M>{bzG-C2e}f(CmE;j<_ox_E zU4W{}E5r(!n@K%sDP;Mk@>A?h-?#M_{jXx3n6Sz&B83|oO+NtOAHr@)opm^qb||nz znL~PKnH+9AQ3+-x(Dd>2h5Q^Yc4J7*1ihng7kP8*38a!|DH2lb? zXA&VI>QUuTJ4Q{_dAsmbh6L(1LFy2gkx7Fo_^|;e`^h^l7p|VfL?mzYco+Ee9rt2P zhb`f;XQhmwUNagsss1HrBzEKR3}^n1>W~ z{QgWQws8D!6A=q)8rZ3FzyUeo8<2gep@A z22e6Z`-OyoB-7sGE=uK9*^~hq`v`53%uknvW|Fl8riEu0zN6h&0i__97C_HCbaUHw z8&xL?J-_8OK*`0@+Dx`*yT!TDm9>b&`+hWn*_G(2Hx&?a2gW;2((hBSkmhPx# z2tIYAu!gR4Z}w?ty& zv9u*J58$Rp%eMzIEv%Q?HAJrqoo%CT$$sJl2+S9GG@?%mZh#8Z3?4my_l;lgP_OIy z&ig6j(dej=?Nd#+9+xJZII7Ib8*B_C{q642bgS_{O;}y%uys`P7X(+UJL?F9>n;vg z*9#_P2S{4Ca^kD7MAQJH@QE^XA1e0A=4nse+oiIG(J`N8XJeq4$~rXKfac#w+8+Mm z&&yBAop8DhHEOaSE!`Xjc+y0?^}ZcvT5$c`KsdjFav+YdrDns21}~ume#XM;V?2#t z?pk_7GUU6Z2FqhP*<@5*uG3!aKM=d6@uF9lN4DWs#x})oS)RQcA{tzWUvo2bsX!m1 zSi@kki%>B6{S08tPho6wt#ZnM^3NwK11C~TOBz#AS@DNOHyRzvW8{my>n|m9!jxq1 z*Zk+*q-u|Rt22M2o$&a=LDyromN1<_pfdOC7GADjhnq5)L~ab%$5@@u)ORTMVSUg8 z-Ra_%85|r@aUM7vyu9~@DKt`YfA7FrNYO$yXixg735yw|4nqi%t$azf@%cGe?TD$a z4fHI6_6Pavs3V~A2-O*$IvW^DXdu2)VS9SS=w~jJVyu=rc;BiDw@_*86gP?|hH>5H zaV+5HzvC_y%&l1ohWee%UD7%%mQoo`cd2SvjC{ASLHTMIbhfR{4PZe?C~CytViv3% z)ltN61jF~y7&aYk1vW0xE=p5P%q=PcVqtrSH|EPC9NE($F!zOdhp~4 z{`tLQi=DDWqiqQMSoLZlN@%>o=*|X@&a#+PblRz{u5c|7|B~kQvZ;h2!$ZkVn+(Mx ztyv-zo%T;9a~Rt3`|}d9#6QbBtdwji0BBB`aeNA$wnp2Frb9UVBZeqakt4-drjT;< zc~hSfq=`;@peMv zcNBRSvi<&nAM5WS;l9_~^1%O9)&FMU(AnKq2hwazO;FPEfAzl^IJ5mY)(l1~{I7xyTPjAyf5snlmexN1 zUnLS2*n(zV@j0qAt56uR8Tj-MY7>|Mzicz;2kug_=l>7ls-6boZcR=a{vXXONvD6M zv-lTG_V7A7(wUI`gX5vUczuqC=D|>36o6eFv#V)}IRZx`YvskBIX;GuJO2XM+M*>b zd#BO1INXJ4>NIWglqQ*I8EN6YPWa$TBJw|z{qLw;{(L}x@xUB;$ci`rEr3M7mIU3J z#xJo^E@g9c1uTx>Gff!lpKh@zpGJp#8>(}2WdThN6GqJYc-4&Lai zZ|mtRLciMdTbH+vHT#_)bSRNdZ1)@~(?J)TjoPPjx?4?T=*O-8IohHri9@85W*7`q zeBlcCXp-j?$YVORB>jU_k<$)ux@jAgy8OJO_1t(XtKET~l(fN#W3{hP{?{yvk*Gv_ zVrcU>r;PQO`*^9@v?d7kH|zUI=Y9ZopZz(Vdz5d6fota}OS@*$ci{8j^HQMYxJBz= zTO)-I$LB#$$X-oBP2Awh zi+=*(X2b(xZz9`E^W_3Lc_E;pk5Zx7B;uqQ`DYh(@gZb;pS4i7*H3rFq|Q;Plb#(~ zhH&GuGDaQ>1Z>70zV|65a9sBDg^*t@)D{p!F6el#ed(t25Umxw(LkDg4(2-kOSyGw>1%D>x^M9u=1+G9C#vr+K|^KJ;Zax(th1jX$}9{98r@Or8>@e9a~X zs(@=DB#Uq3nenEMB!>XXGkcy=viq3$&9qTtDin>^*XpGA&kJ$EEk@YJ;uKS5G95nmad+ctBlHu9C2wb$r7_v2x67p`lKncV z3T7f#egk5?23ZVd%w7UnEf!G~&h&XQhm4$$w#&mYc^rI6Ztq&4GshfVl5um)hr-mA z4`R895Im(+ke~Aqr?MK1ssRNn!6MzwN)b{WMG`xP;)B^{@KroK>7N@=(H~ zhvO&P}On?BJ zA}$K_c}3b-@m_t%(xPHQsp5CbWt+cZUtV@*Ho~QI&XWXSFjSGjxOV0kNe$*Q{oI5> z{fHkiiO}_J$1ia;k+|=&pWMvOL4;ws;YdvuRC(4`YXid(*NYx4Z@}!FoZTa`8j^gq(+m zP-T1FLeq@3>gmtsWpL2j^G(qPOxuFCp`+Tj@i%k@kxH~`S&i;F3_+gt2DB5AL6>(? zPni?YhS2n@lIWeYDg2KwEfF)G_c83_2TgvfLZ5Vz&Vm_#E+Td)(j&w&8@H!Xf)oO(^RjM*013x)qU(MwGU16Uo&!vc_zBbWhgV>6GsIl~1+% z?x_?%a50}656Q7!!bm zIyYx6I78z^nex+YKFcU~_n;m1*y;t|8TAa;s-@e`n}tGgi;)dloNBAAGx}Jtv)iXF zYIYq3U*ttd-UlaFosiMRcrZkfyA7l6%eY5C;mlCjf+&<)rK2A@Ho6vJy4ND}UyEvJ zG}1#aZg-E;*2o!?Hy!JlF;1H}6QhGjig}*|jhdePLztfbCFL5Mg6u_kDFex@1LR5W zgU`j;Anl^~8G1^Chp8rsdTS4-Ys~+!!GNGt>4>boHO8t*oqySo)2sMTR^HL)wGOfE zdwluj^@j&?;F|iTeU`Uxa6hy7&w0(Uy?M_LY|J>=yk`~XvW>!lyTqb%g_oDS?jLTo z*3vuYoNDX*PLnW@kX)hJHYoFzXkY%wJfbO~;z^Jpp@^2AL-i?6P3?+2_f`y+hMoiG zvH)+&lrKaz{@dses;MqC@LBll#2@gQq4gW^61$wyw?g@-6B84wl)~4HV*$lxrHhot zKQ7pVvD-lYWq8U57W(xQ4;%eK&<1a%#K`J!c^@OitEez7oApb_mN)(#Tt9|$>NDll zL#chR@?yJk8Kyjdk5_;193GIlTmQTdHj`_Y0W_ zt+|h%x&kil(Die8KntJXy$$#t&;0tS^nOFbyL+6VW3;?2z_DxEGx0Y!#*%1^n#p>VO@-+*U$ z|BZS4phmyNsz_x#l_Y;&zzeS18w3j~2e>F1hGFR_drmwS-$A`+?=*ts--7iU<(v`X zrP#lb9u*f4r zlU!DEZ}eGs5f(uQie-W1{DFypE`qymxjZ4gPxDad^XJbuW}Q+Ur8Q1#3G4O=B8>j< zC-DU8i0!3P#9X?3$erTgG!LfVn-eKH-g-0N+#q<%znGJpa(Hj1e8xF0pcZorn;EF> zI)zOr)WnM_&E zpHvB`r{fK<`h6kC+sXM>Ul8+wQV~*N{>aC$C0!ujZ_a<6$vH{&rAyJ=sJ{zyX$6p9 zeNzqj=kx6_@_WeWfwuNXH2}Ja^&k+C>%%i4a(cPncCjLVARO(VB?*|@K!8J0zd5&? zOw4dk=&ps#?ff=Cn}}OjO^Xvt-6p?)?_m}^OTHdtJ^m~MH*u15dl`-Der&UXFEDQg zS+cOST(LSR3auZ;PA?6bd@kvTXJDias>v=J@Rd*0UC0i8>DEvzIbXmtXAq;kF1RIo z8TV2zCi{5Jl53PwH6s^q!0G1$85Su$;voBeIJdqO*De?!SLV2ToIV?N1FieQME*%ew1{(=W?=rPPmzk#RKwlEJa;*S+ z`-4RvrKr(4;6r0+$3-d=8_;jYcFmQI(Yb8EchsqqS(`%NU=AZ4UczP;F`0jh$*jPd6`8`!2L-cu@AOdD+g9 zx=X5)$ByL25$k>%cESft6?*DPW;#42KLMkrCU@-Eu{zAPzy)}bm4X)IPp86+FlLWb z6ns`28R=savsomy@RanzTJz-EPQ8&WO=~Erz-mm@0&YfV3evp2L#w36|D3K)TR%zv zQ9{lnt#DwAY4$SKyC|;Y9REIU1k*az*P2P~!iEI3aJSOWFP_xAADHP^j6ZAa7-v2( zD$H(rI5F&2xe3Z?DyFNCl6&GQGipsZ7uGXK^YVpBkCPFfz><%d`0D5nvn`;C6GUFS z$a3G9#MEn3dC$pz8?~hQiAeKWUvM8+Qu!A!nfoWCZ}f+m%m&g$aF?CYsFQa4xVw^G zL{mR!eFGLn83>q98?#j&gGI0gb1@E0w!9Xf(qbI_X3-kd|1EzJm&hK>6&jjL6LR%8 zzJR|A(C%flp?wRwxMqnITs|>vc;tI`js9`(G8oMD`TO@3JV6$Rj(gL(ruWg*-K zHI3CX!GE9kqxwJ;SlO*p7&!#5V*EDf^WcTBA#M>6Zq@!57ltj~lP);HrDF90xQUg}N>|1NNss);BN*kvIG^Qk zLCeZomo;)w-Jgd(1C|m4#%JZRDFZ&Rf{n3Co|9%^Rmahe zYnU`f)FcgtJS!TmC)3O3D?JY z@SaXsY(<(&(!96<7YddDCklM%Wwbnb83n{on%*cnY)X4q#0vk5LSLRr?_AGkCP!)=37fw8$otZ*5jd z!P2<6m>akAUn!v)D-F_}>N3B>&4-gNyLM%edAJsFj7c z(Y`eX9I9&HS-XT+>lpdWZ(Xrzu&$nv-3(xaUCnpkO$}6LMa6E+J^yn?~JUN=7{oo0tZFS(ng{o3X(lSd#FLZr0XUBhR?}B?kY<6Q0 zgKyNs1WOtGq;c$6A;;q9tYoA{hc}x^M}HSwja0&JwE$2Fw{hIG2kr^pf0TJraNhC( z=M?~iHhoYCP@Ta>L6`y$9gr&3EnyPh{A?B4q9ziOa|Ex?3><kLU+ zYTvKDdbFN`GeJM@p@vy~l#D?x&v*<39L$2(cY!PDj4xHxhpWm&7PI(y^8h#!V3qSX zZR1nMtlbK1?!;ZeH)S!iYdqx4p#5+!)uW~7Jsf)2nvvIBL~g1!+D_M}Jj;f_BSf+I z5uqCJ4kYmyYeu6aan@#+q{PHUT1UhwSEQQg(l;xs>mNb!*plb+@UV^eo?^w&JJs$} zy{o@!;`Bab5oGP9nK%UHF?=yT>8u&g*HA76LO*JY2^e-vU6G#@!~y)EX)}1Tf78(4 zv&Tp8WxO|d@a+Bd5NuaHSD}0w0PKrqhafkC$nqXvnE^yk!=SFHkO=M4dK?mPVKQ_a zRfe-GCg4d4dEm*c1Fvwj77@4#N%;3VB_!v9tE0L07p=J+8K;8ZK@0F^SbxrF9~_gs zn{!KJDiQ1%v8cU!t5FXmQ#@GmP_>PVnc;Fg_MH9IyoUtOflas z4r|7H(9Mq_n&w{$10uf@=kvFDo0(eI3`zePx-D_pN3ePzn`b;<& zGr%oqF&;R&>K^=uoKEztJXGMiQ$B8*4wOgEic!v4>w3&tde)8LuedY5I87ej39?aH zPAV-D<)`BiRhtob&@cJK+?$bA1w93rg6t{v8>piHx)x165W6PA!RKE2eKfJjW6t4S z_9s+%E1!m)?O~Y&S0D)7NM*2|a&y!1EM~woLT_GY3lfao7rGRGfW!x@-Y~C2$kQ)| zB`9Mu$^p17OIVr4e;NDVX|7eM4=7TLFXu(G8YS_|7_#N+P@7uhGSaeelYQBIiP$9Y zHoC^?j(RwLgDg2%!V- z?^1KZ4>|p7r*Dt}ID)O1MN{ZCowqmuDF=7a!|LfjTOhhNqA60;-+ttk;v|oz|EB&n zH3>R>bF6w`CJcT>g5+i@fz(vVM;c5rXAlp^iOJtc+nerE;YAQ4=Iiqiaw*}PZst&l zzZ?ogEjKM=uS5RWSoqMpiFM40y=ZPDZfNtfa0R?^MG-V9KLHJJJYd$7#OYlB6@D9T z=3r2#(ND4?L#S79?3t4_%ab4zHpC$}k&Xax1z{>=hds@DfUx8r#H{t-kbls8t2$;a z&+kV9j+;==%06g=#ELq)OTql1=)Ov~ENJ7P%|n~qq|??Y#zZxFTJ;C04o?0yD#X)d zmxjK6ldA459DybX!bDTdmKRaGPzyCQG6&77Q_C8GwvKYCkA`86R5*c1CN815)Jx}p zW~WDCi#IFkt22b{alt7pari~2cMX`dAes;;9y9FF{akM?`(RO4W7{y9Mg}BMel}Nq zjCT56Sj~-!`dysggaL=y6ovZ6oenTwsiyY5T>x5}wX=`3vv*sQlKo=@XBDMdI|r(; z8sfLh)pdS!!gTyR-S92)lP-GDYJX`Um{Y~8pSW?f8q#XnTWe^peVeHP;r z_A?r}(s+0;z~KJquFTC1?+gLd`^6rykU7}gR_j~ z7>+yV1lCoy`y24xP0a)kYq#)+0p>sC?3S=_q6;f!^d=1QhWOkgyXVbMABH(icNYZ~d%%+U9Lf2y5 zy(zW947p{wJ@EZ&XP7C00QX>*yW_kuwy^2O6=|4!aVRpV4@_9OY2xtF4_(X`XAdGL zlxrTC8~tU=cqq27dc#HRueP0};NIRxp-7_NYKrd*ref^o#D6X{3X0%p@V+^Fmx5YX zxk8swYzQ&y^uixhSr80{Df6&i1^EKzruRuF!Uu`v(3&UN3&_cxKY)>+Q0-&Jkup}D zpx<3&uH9{!smYfM-qRex!QeiyvX&$qtwlkfj3f9N9e-{Iji9N%7jZ4@eO zNy{Une?1$O_Z*Zh$&(&)4!z0ipYj_qnR6&e-1HulBt0~c5I!+{%+XigV0k*BmmmOa$6ZZWajTa&SP758N+Zo}#V++7F zEn!_n$!jxeZ$#%BbbyBZa^nXG6$dR7#2EAc{}7{$;~@FR(E*-+du4pj_xOH7Drr)l zW-%V5W7Y-O_-74w#CWJt4C-SZ^yd&f00;2y@?noLo`D*dfw}V^aF7wyWU(sg7>|b_ ziv&V9%x43cI$*+ItKg~rPAoFDG-;7qkP9MRo5MT0Yy8m_g8Y>9o3XBqv}NWE@cqyn z<}u(>@Fhlb?)ncH=n2BYVpp+@o!U*zviZE7zFrPAr~ohj7|*n9g^Dan^*_1Q$v;eL z1VL?des>O!fi|r6RMK--pGX5dMdzoc>oInImsY5HQmT(x)F=92Cg4-)eFQut8j-Aj zp?*JC=L3D73S*4$2PN(gqtz3*b4ys~Qy>IxI6A+FKbYxAOgHE9=JD_nmxa#n^(Gdm zF#!5r(H_q<3sx3QdA)o)o^4~<_!2k zW#`2{9D=CANWSn`6KMhZ7cu+hrhQa(H@qIl0>9D0JsT0nlzlX$9 zlg`BKiU^J$IU^tZRX(HrKg7oV9TB9|%od@r|7C#|>jAfGz*NpTh#03|7s{Yovm68Z z=LcitXHU11_D1QL-J=6lYL0(`g3-)FIK6ppe*yk8sxK{mTMR2tpLY!iyi^nA>h>SF zv=f9c!XZkpP}e)HIBe z9}nop&*tyq3kQN6kPiW06Fl}h{txVfl{fFM-&Et`CeP0l#|qT> zyX+&I%?}RHr;R_=j}($$u|c}!@5&n;K_=L*bamd|GFQn_g*1LH(>N9VVY)doT7~ql z$coi-goEFpSwC9xR#?~GU?e@XeitOs1$~jOyGW}}m^!u^~oPrS`dW`p#r^>DAihOa( zJ^hgz3K&?!H{c7d=kJ~RfX|Hi6!7>A>?Fd(Dt`+XAZPS?$_bX${etZhfNQI|Pf@F}P&h9Sp|gfM>{k&h>Tu<5DgrrxbMrg7>H zn{G-#y70%XhYNG8IxF)e+t~cJ=4UK%DjLTE%pGiHn~r{Owpwj$IM|;Oc3Mv*fSN@+ z>cY0jI^uaz+0Nv!0Ri7ro%I?rO|`F*eY0T7vu^J}HMQ#A=MjmUUsgyt4VF~M-)(E~ zsUjr4lYSWz<}W=wR9im%}{=H;*yJzAMhHk5>9zj;aD$4k@UaqA%!wCCGUsh{-HOk3X!XWtib%%RQyFM!v z(hZ~8#9}a3ypb!d&fD_kfhw`zkM>+TuqgYScxo`Q=cquIS>`IkJq8ZGx^`#K`~>e7 z!&~o_C0ma0>{%M8-4jGtR|rKXHgG7FAGSog!-QN(0R?XD?JG;!%|8i6|GK)Spb%!} zRAh;!_2(8@d{wGKQf?UHC}P){(_XmZYX6fo8FP2>IJPPnD#%sG^^chp474y#G1zVL zxIuWe!P?W;!X|wkqcv0u=a#xZgYM^7t&)aGrYCZD6tW${_Gb=fu;?WdB#5uXRm*DS zmy7*$G1{=Hnsda#-f^Atz}Juy4?IK~`Tc9msGJN;ovSk8DjBPXXzTJv74k%4Z-uOx z&=7aP`9nO*kl1*;^{we$6GFW6F}~68-#0L^Ypn^)v2WVrs%;ZO*P7Ow(ze6De)vV$ zW>4v!qCV)tvpUkgtcf$L%+dUIDyUpXq({7I7p7PS88o$t*N1zKqHH9GSmFR~s-gHZ zuheG)uUJ&adMV|X7{}|B9W=lXqakS7a)SvrX{qgBLJnBm6E{Cid^}v*{qxaBZ@1Dl z6|RFnl38JdS7cj^UvQ1^{a=CRc?~f`prjjvX#MF9_z7=ZscZ8~!j#L0PB@}=@IiY@ zyi80**C#HYRl>hcY#%6?tbD`wtPnJtlLoyOB>)bPV-E90$2)3AR~7P_Wx2p0zNmG( zTe&a3rRDi%QVk>E`|1|U_Pt>UYVdrlH4LunP4zsiEWQ!Vv}!H!D8nMHQE%+( zf~?PD<)GCG3uL#{7`95}AM2O*GtU#}Nh_dTARaOBut6ox+c?O1xW63+AIi5H-849} z`H*u!`#0XCuLRo&mUnPnf~uxb~QXg9NbXHFN*bJ_mCmUwkPe zv|2k{>NMEWouaLu2IpYyo15{wzjhdYjMTLW>5b99t9RP(R3!{AbxRJZ>9FhZE?iV{ znvmF^IZ(RlCsRn@9+6{SRWSMSo-e9uRf-0d#ZXDaig!c zFxV#;aQZrRFg}eKo)q-L8tKbI^G_O9Fm}G%HuF|XuXbP>Tru5q_hUyw{)nOMphmTN z<31(lfi1qoVd(t!V*eApb+)%UqiX%n)fN_k?&Pvo%=Q^7J>kZe$h59QGwySU%H1S$d_3TUzajWW# zF0?k?(8d(XUCnx)H7 z&%@?rbEH%E^Hv%AEhBO9DSR)r3lx*I7mU25O13#3YNjF3WGHD{-J9HV;`=~F`3pNw zX{DDJ*7c`E6lI0fip4lo8MRu6%Gle9_^LcF5B6#nb-i76uFJ2Y(|?koAc(&(N8!mp zaQlUQS+b!H&qu(zG?gvw7zAszVvTX^=f35iXeQ}M&l>CU@7(mp@i7#F{rgjP7{C6y zyFja{syA)KvFsQjzNwH!p`FlOs3-fhV6oN(RgX=R-3LOYvF>wCz2~E{ahWrvtP$y7 z_6z8=ab6s3iO-5X^YsLn+qD-CUy86MNcX-6+rLIdrvIc)UHZc1?lAH?Ww?CEFsjDC ziR_xu_)UEB?ad&9fNlVn)aI$Q$Y7PF+TYhVHr{aKPvp%!!Gb?<00y1Oml^g1D0tql z`@Kjj!cdy5Fd|bYeE}Igkye`#(KSFf6qnw#B)?<_LoDZviNxLj>Kyf>tr`a}fFVNn zP|X<=Dc%P-gqD);8!{XrUjj2{OoUejzHe*3nUl=y)gHyo6%)a^A{;CKI*Z6e;E7jW zK|dw5{$m(QLX>e8a7Y7jreuu{@I8!znq;d%Cqaq+lbIQO0_wNp>k>9;k~jocVg|^l zN%BFaBN$_AVnvw*gZnSYuV@Ucyhyel(#B?DGF<2>gkya=LUy?a-+|pTDJr9w;Ea$| zzAg};_*|;{q-DW9r(vBls)NJB_m>17SwHDm-jk$yH5FT+gzrhVYjjBxs8Pw2yyVvt z-;g+r`$jHbK*~S6Youmm1L(5ha!;+vZxjQdI`ObkF`7|&ur!~-f&UX$9GNWUPOqn{ zCyNwkC%+3cW&$XQ>2%PfBCX$_ZaG)}f$SN|#8}q8-Mfe$oaluof-DQJFJ^a$D!c_# z#od{d#nvCybL@T%)!lY{RVU6ZDFw%02>ds(;l3&@bCAp8$HQ*yq$-+AK!VA0z%3Ggi5-EQdB;#n8 zV)HsI%-0d(!Q^fX5f%TNy%|>b+QEeF0l)nJN3?Xo=@Fb{dbQ3+%V|3kyQ?1G zR1~6viQ8;xIiOn|_f|E4KIHNg;{>6Pg1JP*Pn~32V!ZT*5Ha z$*-|4HB}&Li2)E$1x-uckY(%WXKOL-Dj4=Z7oj20Oc3o(tKAX#jA;6dUz7I3wr`0VrWm1zUtBP+AHyxh}9h;822v5!)ANL8$KuDNrp zK*)L^^|H|t2^jhVH*1Dk>SC*r@pk}i`QDHmX*V-^ff;e%<)nLtjmF=nvUJyn{=rC+Xf+* zfiGYdoKHq(0I_$7r~PiYBehwk{kqC~s}f@tKu#1Pk`v#3;epzglW*UF&qklXnyi*3 zcDG8R^V7B}z|Gh{@@vIK9=NHYkEgCg6Tx9HbX>F8;F==n&C8Z>Se_u9us_|tyfe5j zN5zS;e~nRUNDTpQ#}qkq<^~9m7TeA=_2TEu#tPJ?IrK%+kD%R=e^?)eS+~Dg?i7_8 zNO!GCKwvll2w54_=3q#SYj;@K53apv*1wJ5gnB#=3#(8c&Jb{l9^~tW1}rMgX(PkY z;A~}qqsrjn+|g2pltx)?*z<_hfN#jIj!%@0DN7y5l_MlIzuf<68$l|XAT#)W)005P zuTNCkB1-yhU8gEJAETsw@eQ?Udj?Au%uZzkP&7QN1I{gr;M5bNZuR3-lWB6JTI7v0 zUmsg~rF@nr5ZI3CwV(b0x?`JFscBa6e_>(8{7uy<&7I9L1W(HXjVJ5c&}|L%6W2F3 zzC<7mQDhM5fpscsYTbrh5koJ+#3N57rL5Y~YmU$qZpBaVu=aj}vka(?CpN#Xu$+K_ z&i)UGY)E_u2B8{LPxjv$_ z*D5`e&f5m{EvyH$J^HiPG5|EBD-x1$I30v{opb+3y~etM2JzJ zv3wb6F>!Wlb8NG3|EM?oQdb>KfG%(=k?l<&WW3Obsf3Qg*ZAU!IFYuti#APtH6&Tl zuRK(*AbSHE5l9H%M@lx5<)EnG2vC)Dqi9QHvRMjFl4S)Nvua!#A-lM-7ouCwM|U+h zlX?|@GL&Hgn{`nqvMvOM9%mqpRRq&{PrUm_)IE#cS&AP}S#@BQrpqo5JS8^sr*wU> zfp*4RS;Q*>4)=BULU$bdxI1;kjDTK4(474}J=&}x?V`8X3-p|c+4szRuLfC{iw_Ug z$!4Kxv;A?s2pxM_v3r^Lh6^y%*lwpYe}wMwD=R9sDh7_8hbZfZ1+hP=g2*3{I?y;^ zA6Hmd`mhdqaqIP(1A?`OeI~Wa0x1?jvF*7gPBF@% z+2T4z_pfkzm0tNNJJKoJBv&WyDZkav;`fQ~HZcB`YcO}iH3ymDI_F`Ymm{&cgDGnl z`UP8jQEV@3a}wnTxf!tSPhTuUuh;+K*xBcAY(#qf3_d)(u-waC4+pTu6 zX$zB~a9m*0ixR;U&~Xpa>Y~$ZKeTW6ifnfjG_7%{jtT6%*WIzL>TECp0_YZ)i+pZR zo<{4sn>8I`Ak17ohKp{9f9#QZNu(15D?;5bxv%q=cp!awRN~ z-1CywG}-ar>q^?p?>q5F<;e{soO`EL-f_3s7#yGH208KX3?)r$ z-mR(J%|4(y)HX6~66Ee(XtvWfGgwlf+SXYfx^ADY-qqR6W4&4Z)E%`-xCH3t80y|z2jCZl8VS4N1_mG>b&TgiHQRR< z-J!3?J_(dJc2~%))YxEVc%wF;Kj4B4yK-r6w<8ZuKbg8~vU}fO!T@jzlOV)5ieI(n ze*5Wvar{OyeLpX8$#yzi_Kyh25&czH*|g%Awh&-BfS$=9*gQLOS?jlDIl@-u%+vahb{{aasrFVtX4O%HNj&03;j!d|blHmL z;3Vdg|Kc9t!OME)6kuPFE3&%K-s$w)cOXwCj^Zqj>V|3qnmSMTZv{ZU8`FE9hcM_L zu>FICZFMVszxcN*%Sd~j7oZg^h2^+;tGyNJR;lP2gGM}2Cb_nH8Me(POR}=kO`!F; z%}KU~FTQ7vetLhk)SIL39KvW0q-QrRTq`XdGjwP_pBhi0)W}e0_qHHM6Hr3kF~Ply zOOC004Ua3{Y-hi2Tk4yl*Mc;1`Zn0kIB@C~BGuMO4(*(XB~h#nzQ zbg${{rJ-%BO_#qsmjR^>?k>XfFayk+Ab`j|vsUaWQeU3CqzQGe#ozm&s6bh|`AbjA*X|-JC$Jcfa`OtM0 zsvCIpHK;hR{&AYW)rr`or&q}4eo*Hk|8TVp@|x}9UDlW+3w31cVC+(bWc-J7Olw*h z{IDIqX+s;8B^n-P@4-LjOmS)5p83%}xR8PW?3o0mgCH%?Z+?X3EO4#4eN*NorauTHCtFuK&vt(_Z1NzBCq>T{z|* z4$?!{K1Ix0iQYBmqu#K2_ezCS)zT}Q9c*JzU}Ja^WTlz1duLoy{w}%ZW1`S)A8?q$@(?m^I{@12Q-&T@C9>jKRy!Mnfa&@_BdVeN{PZgB-3fuL$- zT7(KV*_bo*?)R;V&RXYo%2!9Eyz8;(4t`7L+GSK=LaEJ5S81i*mmzfSjdX~y^FO`D z)>u%1KeGZQ`hwwZ-~XH1_)sA(q}$H^s<5XQFj9AnX@8Ax7RaZLWY_odQf=pfh9?H3 zW2&|3@T+eUz1@Dw@3RzW3>~W<8_isA9vH|@MX~JAwS?>9YqAX`W$o7)bUXAk7mDNG zQeU1=EjZqZy0)P=A|PW)bpWKfM~1}4v{lrb+LC$`(E8bO?FpUcB0(2E}Z47Kx$uQ_kWGa&e`i-zgfc4 zH{Mm}MCqFWDHaApJwN)JbTyqVT@YPcqd2_%2f-;$S<0sVzH4KAjp3J_{0|A#?Zob+ zq7!a%1^w@NWEfVd@CWMI*4#0GjJ@Z9UD>9+QY%5HXNx|r`>*)`clJvZZgN(HISsXE z6-sU_g$vN8pMTV4uq2=C6jA%0J9nY>p?gP@n^B{7q=qkMmE0qxd)=`5cR z@#W&|w!;t>RzLDxtHC{39~qf?V9?-Xp<*@Wr+ku$W0O`D1VZkw3U!lGw?kvLB7pu$ zE@_I9697BUtXF6Eyu|Tq5~y23hN^j*;s;C;8yiuz$DF$6 z9g((5=OOpe-T8(0E8G6Wn%9L|CMiV_v~tyL^jW=9Yh-45;Lbepsy4Br~9QLbK zUMS8E{((+4Yiegu+n&Tzl2B=(QVDLo7hb9tS~b{}_BYp^7(%0FR-1e?abGXH`0Fvf z#}K0ta}RUtK0KJG(plthysbPomn+1J?VOWSXF!F!c3((m$?Lm9C{<=Fk>#DUvf-xU zx_+&5rJp*8{k7Wy2zoM|h1%vy#0vb-KZcz7&~$<=tuI4>lj)Ffblp4l;fu+@2wn$- zKNyCky?!2ccIg|QSr8R+=w$*}#tXsySV8&xM;0+PceUhGEtBK9Dn10~P$k&(t%$VA z__!-CBlVcJshp1N$J!G^+S;YOuc0D9K2=ew#HsI>YtxlpTj%>%R^B*(&2F--E~)MO zXIM=wW2M%41{A=So#r%L>ob!|_?akrsaAgQM*P9kkM113Av^SIx#ju( z$2|uej5g?Y1-h1+a(FeTnO56A2p<|M?@c|%v#NSmo0N&nfbiK?o(iWBQ<26~r3r($ z#G1FAQYNKME!(WmWWT@Fn_MStUV5RGD=WAvFK99HS|oZLxP{vhl=OlLD*{GSyiqiy z%h11D_zef2d9t`~!Zx`cAGeyAwoGq>><6a7Av}Q>cS#2im5U+lthO~NZFyW|@Z*Kj zUXF0|_0`G?JWOj0J;Oyyf=wNv??zBn?;XpS@bfW}p^pVnSya=*8=DS4-mc?OFHj5K z`)N-Zf9H4o0ghk;o64v=`Ug-f*FX5dX*J=9o+&|S0S~rVJw1OOt_4T_K0qKf&xnjS zj^kCtZJmds2W{-YH!&7HwP@_pihG*N=v68bMopHLAp0GjKao2KMHl!^PN=g5#WbW-DF$USjkxaI*j~0?w^8!=0dqHEd7qL zT$lS}m$>*b-fDg!~{YnkNT958^f+J?vOVDsr5pzYk+33m+s3o(b zasbq=?y~}WnaMF&(**`>$h`c`*g0VxXr1zO#hE(k#-Q7a@3*)6r0+AR^UIYIwMORe=a zK>Bjkw$txnaF}=@5#ph;c4C{eYyqUuQERo7-{RJaHc;CoTWGvI9gT41rGDOGP_IGTnRI!F(oP{M_?sr|KXUzH`pWjgE~O__(QFRTQN#^yY5A!_yg zEoW^*=Oy+b4piM*WK-qQRA<1PxUQZ1YZqXOqxl8Ym!>wvd00Z6r*4PAhe0SGGC~oz zH{on>g=yAWOLs1rd%CMXRX7e0ccv};$e(onWQDC0fPRIFoBh7s9FeYn)c9H_ENnAG zr}29W9jrQgr7m1rmE-wpp0{BZp~f5wZk7nJWK(BDQ!1So`7-_AqdQDGPdY4JQ_A{OXHLa}iJ{?alD-aN4?_knW#;p%`2bNq<@htq!Aw}-E*IQ+K`DwU+n972fAhE138 z!M*aGQcRof>e@97{D6wJFNVH4R@YUWI($q!b43w+DdAt{rCx%n0JXuJw&aA`W6LDK z?k+Y%L}+;c;U>D^d}cn8>3(KY>t|?Jx+>`F5K(kk@+SCTSPul0c`I(oT4&#Uk1(Ro z2?e^wRe)KVvq27pPdv4jb?#j?MvVp+>TR=uSu)mRt#zr;QEk%!RarULHujBFkdcYn2v%4pnTu z0K%t~HKyL}jnVGCVmfe0O)Vt`XBCn|-~Z!-LaMi{b#yj$uOIyv)+Q*AW$G^&&a^cN zX>;mLGJpuKsioka4m{?2qt3<)@H(H5I3g^Kx=YMrB%$Vk<%<2S8ms2 zQ+b`|#6VXAl&|)B*P+0^*EaPZ>82Vq{x1U{^t=Jx)awoqqc!QO2f7uSOF@!Ir5e+4 z;81qlOI<=0sAn(8T1i3$chJDLHc_uF6`X01+W1C98cGO4t*sp|xw2#}8^2{!7yGe7 zapCi@8=!DwE{K*8M7!$#8vuPMnkUipBGCgSP=s?h4W5RDd2TZJ7?5l5v3&%^2q<2C z>BPSX>_`?@RbHYk5t3LrXWz-7z`J-o%X`t4JL}X}=YeY|zs8tZy!OD-B4`>``7a&+ zH8+7&u@2+zRn8KqOTUnDajebv=elZ1;eb0@`rTjF46wdEanX>MW3z$f_XDY74G(=9 zg=)?3I-=4%@1*r7n-ko28$`S$l)m{dqVG!xTOA~nG`%)8og%U|?T@!7?sZG{R%Gt? zb1RhX^do#ThmN%=eKLc2_PJI2eYI7Gp|Dc|0rtgP*LM01!J314dhy6&)F#(A+m-?tTRLRm5#9|k7 z-swU|e`#bxuU|`#RjFm64oQInzPApfQLnnRnr%n$gQ4@y!udlb_5De!>w7B-iw4fc z2$hQ_|N5us6y!=?aRS#ckRy^qVNGql%CqRJ9AT=ggDiz90IbY z9`nz362=g1n2r#83uzah@u6#rsX$nw&L(ZBMj#8VJNq6x zpex0V01ey4pDN3zAF19ovf`-vdf|Q!mOxnmY`hH!SXT>T$Vh)kHmjMpeO}#o;)t^< zM`G0oFeR%io7rAcLVNK#o6vH#UtpH>&&TA7bf4&dvuslnD$oS5K-^11z-NOD1SEwm zLj#3ro-J`D5xNc|fOUz>0MwC%5UYVz2~e)sn9-E*TRJPq(tMq{1FFGnG~B?n`^%P) z2gwy;sL7xmgAG3`A!7ttv=k%h+?Y>07q2}+Fz4R)kO0~DOuJ=f$v>Pu1?u>N%orSH zKBB3qCnb|w6VII-gu3X`SL;hYK1hr1{(hSOG9fWnfm`k5EqQMD?RxB=dL2AFdhaMM zg$<9a>eu>Y7hsZVYaDQ01obrRHK_s1Zrlvufj6P$SZc?eK9Fd3aaj$XDwV=irw|Qp zufPBZ?o^OSE#15Tq{7lT&kl#PdwL9may27cbs8I4FUEK9W5z}pNdtXTxX4P@!o&L zE9`>l9(S6*Hs((o>gfKjMfc)!s}8BgV0S6D`{50dyu!bV|-3LvlwrGzDf&efzDE{)Md%>d|sLmqY`nsy}o`L5x7L&UFD&cYqpBbKrY(n;x zk{^aWK{-njt_`9zkxpd1EV_mRmQ%39xZXC+Ry#poCd-%ZQm0bKmhjj^WlsEO^Il#O z7lA}|cE<1zC;oC>NbYcY$aHvneal&l?#BetctiT~s5y5p&C+jveCTC$6xWR#4SQO0?aQ7Ng=u%e`_tYkY+i|mwar&L5p z$S9){%B&tE8D&S2mCbwIIGl5ej`#EaQ}5I7{C@X+U-Nr?uj^(Djx_QT%u9hyRqSx_ zaeuUUTii?ky>*Z#$7oxHyLWA~jYy$s^ZaE&)ejNY8bo=?DBabt3qg`>?^c{SWu19I zre)i*)h3+)*jno;b=>zU$!&b=$5Z!TxMNeu>BBGAyKiPZ_Y~NzRq)sy++`R?> zsyaJ|!Rj-EN$;rrm7glzQGStc0ev*N9kdHQV^><1h(#U@tztjefA(3p+2@kep&jm* zM{b%8mt6go#`GG2z8$Cia#KGd`hGN9OH0VZLC{}d&Wbe@vpfGv3@_WeaN(#7hupiI z8_GA|4(9K{8;d)01RFoy8==%_26}6s)}Un9v@NW|aWrmdc(h;qnS;U5`NS>wkys%p ziq@25f12#uYo=cp{HYT zI%|q7{pw_gKb8E_u>P;(%waj;pnvZGmN@62!cfR%g_gm`qfH{t-_{Oujs3iypW~G~ zTD^C@kVT$53e7TmABgNQ1)%@mUllK!cht0eazbI0Q$9^6;(28(loQ^^^}W*W*{7OI zAMYsV817utU7`7S9KzJ~aj>NPMmJ+_ri&Kq#(|?96zRK{2ez<|!?{~~%;2L0liH#o zN3k}c>9835HR#4KxyY=p@^OU$=@O2L z#XW&$Ds@oasQ1`b998EZ?*7GcvCGT<$AT}I=13)DCb+>!#clyoAy?|CQbK6mW94;Cy?1)IW=TH zuliWs>xiO1lx+qc?SXIEX|dSrWzrow#z5V!YvM1~EtehnITWbe?#uc4{$9WJ;gM2{ zwmqzU_ZW_%PnaRC=T&SM)0 zq#)LFE*QNHkQr>a<#>F8bcKM-;}YeO{JPdBn_|e>o*up~rgP=}qCUVJ!Cj%xK&<=1 zmB&kZBu$rJfbhIAc-XSbm~Y8uAZT=fA9Zp$A(9okcTq~6lRaELGx)0y#U0l5|M+X7 zEFC2s6L(^xt@Z{6h^R;f-9p6BLKL+wo$)>$U-$SPlU781Gk*5T!2CZ|MfuE(Ea7U5 z^J??P_1_&X+vzI>4)~aDEV0eMn(x^jF)9h6aK3m-%lbBlU5Q&}bfV zMs*k^Pod%B8ZOAS5>h)ehr6wgDdzS(QgMu~J@Ca{XGE{ji#D*bG%?|Or1=lX?V_Dy z%xnuzwuMSfs8U~lpk+5yT3*z?|Jq$3;tj{Wvpev6{K1A==f1Atl3~kT&qmCY#?-sB zOi(;3_Ko0&2_`aE4m)#)+$2~?!U&*GUBip}I5K~BHUBP`rjrjpx&>QbhO z6dRp*TjTDp4t_SmIS5dXh&IStHtwsq)@}2hrbKOjs8^*`O0exXJLvGSA5s$*ROEDM z*;VJJc_r9Pw+?C$dVsw|vl*NXRNj-H@(=}ehzeEVh?rNt6P5EWC^pCoTa{#@4$@;# z?TW~f&p6%*_mz8Uj(nDidoc#cQR9Q1bn6NleH-FLehv8P7f5tlW*c@zTRnmyld>pz@ac_8OCj?yRSs9n~#EzJDP&3%@m)j00(CeCq)1oW@6sJjCE{9nlcR==Rq z-Y79`bR=vq?W&VO&9-KWGWz{>&d$=sBL{w|8vr#m+U6^iADmXA9vCz;_vW3#093ZDCcn?@p+a6Muv=Nz`2I`<&7_g5ZI z(Nqa!{|SSlW3}0(DqkK5m&wqUazI2PC~dQ+{gh4dx zO)-xd(q)RLI>@I22b%^$b6Zy;JDi4-y(dc0V-qKTK$M^dPGHfhsBmI z#ID(C%gq+71DR3HkCx9RmMg!mC4yMv1<%&K4TvKR=v?Y>QRnzCvvF2n z_1QI^)mQ$O0tiRNTdr06`IsBoEgv7}>x^_P{wFip87z-CXFoPU2Q)+^ILla&i3>3} zY&wm7&0C5wY4*E=gn^{cSG*Wi!9r7kI>iZj4HnTzeckJ~6(d}u+oV6uMcO8I0HT-U zB(UXt3K<_K0=W?0WV%T~mhfAC~+U9TAw?uG{6>5Y!c zST0{}ZRfEr<|5bN=e_NaNX69_ar<4f=;#S7ShLY0(w9>%y>~>buYZ`!(AY8g6MH34 z(>Mp~f=5iY#|@ItFI*&q=blTjPbfqF<90lR2hoaeW}$H zDKOX^K3I4bFc{m+2U2V@VCh@-3C1!bCP&-W!C$?~K9_zN=>(_TYpL7{1i{!`-!nmu zi6uKKG!x_M20nhST;12x199OEO^MoV536~(nYIIl$2)FfW`7caiPk^_t19``6Wfzk zISlktH$bGNMate%1>W~HYb;531}WiTi?0yZ8S-;{pkHgo`R$H=(R%65mW+^Kg-;{j zg^+YB_!kY^p62?b4pe^qV)N!cxFkTrLfGP^jmpOt;S;h1WrpP}gD=h)qpCO$lSa3c zAHBt$e;)@c);iY>*H&mb0%Y!4eTrv|W!T4rE&%+$!*{-E{BV_Lj#_!XoAJ z9bp^iMoZrkMXz}C#>9_W_;35io^6#}7mj;aP@`j6yUpJix;r})hMV1Wx}~7&6ZKB7 z&MFB>d{OSuyW?6&eD$Yw86U)l3sbuT3lnx!O#(u)0SSG^J#T#4>I{E{j{i6T^^KmJ zGhK9$zkpwEW&KHh9n^H)4W&3g1R%U!bG9e|Olj!`pIt1*;IR4*u7=h#&5xlkor^-H zE&8uDZ@4aFWK-&?Y?liBTpk>#dc zbCXkU@Lnd=?cNX-;e~iG_imNu5OrM31Na}cr(4eG7XU(%0#yvgTI0iJL~71wt=I9#B!<9eRP5+v~+(BY*qcz z6F%;MtZAai(w{Z}h()u#1bR_V7g#mPr%olCpbz*=|J5Um_pjLo8v$FFg&LiQ8;K8er9x4g;LilBpY&n}98sOOF zjI~fd%<*iZ>CzL@G0iU^Iw+Fou`!7DnY?PtS{MX4k9pdJML`b^^q(~)t?E5Q4Q$U< zs0@#sra6S)%O0F3Aj!v|MFPnFm2i6XkQtwT49uObFPB#=e-x{}(E%k<5Cu*2RDAHc zb}~NeU>NX)5jDxEWaN9iaeK9qr!VJ)Y^vmw&tfA!y{T> zQ*ms*He)_DF5_2~pefYA?!+zafc9NMg#I8BS3M%seft=h2V*Q45Nq+v>O<$TIG|#w zB=i*T$_odK9f7Xe>SKVXdLXzY0l0u=dsm~3V?A459o%S}04#3QSkd)#r={grH@45g zJFU;xXaLmhWbp4BXvx|pUXx^1qg*`^!U4K{-ev@$)(zl43}c8zjbPd9oagx#IzW}t z8*p%OKIH(039QOKd)3*Hm55ui8CbA#rim<@9w&X+he{Ha5F}pT9`d!kJ>Cx`$0!W2 zi@ers5r{X3c8fRQ!j1*&0HxlV@gL!0orz-2I*a#q9lfQR@mZaYHTG?fBI|wRmOo-z zzID}C03D+JSosbFF(Yub@z`obz93ZnaNM!uVTI-U5hdB{febwWZ>1b8lz>7srZzW)rCr1F1jrXI0 zKn=L5BFJS+biU|mdo!#eJ8HGuD$WbBqY+VCvI_OT==;jE1~!}sJN@AmR9VzT^h$be zdMT&o*R1*=fB13Gg?E!SFTT(7PO7qo9YdntaFaRW747y3+NkmciVSge_qDZ8$F5P$%#f-3 zE(_nDiIbn z%t6l%SJ#keGKm+1O5J``se2R~zlAAZYr#b@AK=W?VQ3RHDUEeK%|z_EOOUJ{zhjGv zx{%iE6oyD{3xjH10B9X5X!m_?O93`ydLi)m8le2{=J0t4kyh{6j}5joOtl^HcROX8 zIJ)uryBc@8@_0qHi|_1+$%5XgGE4J|!q3s6NI5 zT(@?oZT3A9B7i#%9r%eb2cF?`9^$ebCJ@RcIz0ob(Nzz|0mIhvPKb_1d3*!V93Kt0{ZRQ; zW+G|U4j>JW!;1jzJ1bj*rVfl`0)-hRQWYik+Q2?D+i1g--!9qQe3maf z6j;a4h66i|q$0!Y&W}2r(J9IT+Et71E?Z5Tn(?!pnGvUB-o2pe-2IkTN&Sl)^tvn(Z>BFzd`AQYc?Y@`-glV1$K+WdtUi$SgK0R>P3$!*wMIwk@sxo z@Y<+nW*_gSKaWmti@k0g6Vv{mjD2TaqDs5$uyl*J<8^(nCX ze~EV=wfK5&$n-8?b#E0p9M?RzpROH+saK_d25-qBEWyF}#I3E%4ep+tN zqtT3(GRdyB!8s9z-1Kx_GDEHB6CE8^ar}e?u-yXEd;iLj7eH^h4Naa;QTq^Y$2Dgt z$tfmot-b`Bd{7cv(%ymM9jOJudI~h4JHT6r+G+NLYs;O=5Y)hP8^V_dFTj#FAt}$T z2D(Ia{Z+(UcF~n-dEUqb93J zL>E2dxuaY_Bx}SXxno<=_9Hs|s;-A(%A$9D>U{Egd0FydzHv5_^*t4tzmngE%M7s5 zEb%pg@i>o`Z$Ho6zN#T-1eFUJZJVLI^c1qPTdDpr#Y3a5b!Nsn@ya}GvBo;UdWNsvF#7c|e9X%Fhx{qU5tFhkkBg`VF$e}sTlT0l-FkYe zw#7p_F|S7t%7Gvd_Q?_*W)z2FK|4^q>i0Cc4^C zWhsEY(>AH}e#Ya;7F>XE!nP2YhmynzBdyG<@bQrLrEa<~0A>9n>p#O4Yg&2>?Oz~P z4P%`S41p?Z?yqE6YcuZcIh3$PrlVRr+$^IfmC2D=DJP_1`L9!~HjlhLtG=#$vvAZh z*LGA$Z<}+#o>MPTM<${RzY`U6R{Z=^3z`6=D>EYn0^VdqU|4#W{IC? zrrD67y_oX?wr>OOlRjiScHSE?|&O6FQKsieF9>ytX{ z%#nh}2qitg17eTH46?d!$KM(lMxjo-mM!_PYXyd%BsYdan&6{@ zou1@~>p;dSwB6sSJi*r(9;Z6oKTd9z5eEtrGgTdc_!292Fi)$0vE-Q?xh+|( zB4G*)Dyv;Ln@9c<7)S;0aa~$AEQ%pGv)ys&=S$spcsx;Xt#)+@vd>|{D`{4O&-dOh zwM@7B+p^K&$~+nMnya!M#&xB1%eF_ixcA-g>_hm$aPf^@)p2l1{=xYZ^@>u;_249{qv6VEmWo%qy z?B`{;oN7u>N`;;R^8UFf6CTf@@dFAS0vDpn&X+yv%3|v}?R#nUZtHB~=^e2DrJpe$ z$iuQhnhn!nlhS6q5_ebqMav{)j~xKsid&y!?+_>B@smevxI#RAMOjj9GhzeSLP&G{ ze0frXrw1EHG!!5@7&-1itl9fChjITcfz|A$j0;<|>lUmH9=pEF?2C>0*`dnDH5{4X zd#V_n5`4$LMTQF}xH;$!zjyHZ-qm|mpvv{%%ZOF9=Wg}|$DHQAOZC|5Q?kvXnx6+u z?{B%s{^jO=;YVSaKcSRx2eByjynl5;Wt*$h;>>zCI`fX~hN1yZ$%f+LhHpmqjas3i z8GFODP5;01$!*`3n`h+svA?>1>+tH)lkS)RWFGDGI>w_M zRV}g7Y8;sSPo%)^mbDm+x*llb@$83&)yFy2D4y?z_JZ5sIO92?mUZUIvO+}I3p9E5 z>79sEm-~4+XU8zGIfvbzzZ8Kas!wiNR(#gj5N}xJ6Nr#+35cNvr2@#aRL9_6#)Pjj zi;Tc?iu)rVvk8*nO;E|OIu1P;`;C>j9-E9Ft9~!tyfV17;oOP@;l9qcPV?{Jg*Tx# zvsLFjAd-q68O|EFcav%84=4aYQxh?W?Q9zCxKan5l0x6qd~bM5Xa6Hv!)-S%^{n4U5L(v5xvq{;H)8 zaJG)Txo*3<^O{Ek+M%6WT@2hk?^oR5$J4p_ynnR!C9G}Kt_%VXj3R^lNPh6 zDDmtcazGO;299N*Zie#<#}UOq?Nc};CQqR$$<+rkn!ZcD`8@7};@|9`6TCd&Z z*tjCH@+X&yfa|iRgalcyQJ^nv^xakL{R;uDF$odj?5IU>Gs;BwTr5x_Tsh+>%r<6;DR1TDa7_^T~|z%K>UKl5F&mi+*(%!hVQd;<^!ScZky@g8->b=Ac-9I=I+@G|(q@ zrBs+A#(?PKYrb8_ZllqYv#S0fEtZOWY0`M=FdB7t*c4cD=XnNDFMKA9Lh-NEbcTy? zZ&fZ>6P$qTL0j>+_h!}7vSNq1VC+qq)|-kGMzL+hLtR?NQeL~f^Q7vlQ(q4Q=Tt}i z#JL9+K>qbdy=3VT&c@HYp5M$9EPfQu5q_N`UOh%lWr6E~E@+yv4ur`UF_CmR*S=S7 zUOU?BTy{SBDPT}{QVYfa#?WFBA9*c1BIouB2iQ|$&^VOQj;wD7rr}|AnDWZVkrLsJ z>Z4|v?`4`K#ylgb*>v~k{-*2qvUDOSa z8M`g32LaS)8b|oX`U=z)Z2&VN06`7D1zq(Ib8>s&80?%VDu+170(5Wc%oi-G{Ibo9#cQ$YORfO(O$+1MPoki z21MGm))+Y!FCdJpYK&wZ=%CtXoa}lH<}idks(C#@vmo{{ZyA9wJMO91Y34U{3FTc! z-G`2_)_aYiAvuA7FUENGC%hbmd^eU}u3or*Vf^c3b(e1J33wB-NLwypd4;yU$&S$h zX?i_Gonhn{j;Ophq}S~&aL}#KBkSs=XoX7HYQ_qU5T>gFr`VlWn8(VS=K$aKcttX12h#C&LbxA-dOtOT*HtR-9ao%YPaU8>*XK%SzSFpDzJDY+&r z5Cy86UQU+7tH<14CRPcy;%7wum#p#Otb(kOf~;Gq4Ff1~I=|4akVoua#Np?y%F)Wx zkGN>;6n5@mU^M2kgtqe=g`rB?wZY}>dG&D}2`^uaN!B(T`~f1+9zP&eeW|Mk622pP z`p(92-p_0D7PUew&7ZltmqmVA!roy(PNSN+jw3v#JHgxWarzRwaobw3w`E7@?nR*a zrRVFMO?@TBPFILWDLx+(6!Q&-p)hB@*h2K3&8%>08bVn#ylDZZUZ5$h_jH=i(999XAHw1ih(}e?Gto zcm?2pvj4sx=Jn8XN?7!j(PqGVOGG@X%&r#d2;sU2tN^p*E~hVj$2;}!8yN!pw^6*t z>PRx{mV?Z`d+N%#N1gVpQ)E`m*}-DzrosFsr|yuZP2a-=C)L)Kj`9h6C35|i=?Uo_ zxE8F^%cRmpn<61TCafPa7Jn)FAIHaIoq%Et*?T2>XgqFz7;Jh%ZxmsmTc5Kx#7i~~5eY+M+OYzp&L@dVbcP&QJzF!cKU!Uf`ZilJ4CN*)uM!e_7tAB!<>>8q?B zg_5!cN51}Rs~36;ZHxoFi<(SA4_$j16tb?IM{GdRSKjY=`5MyKaEq!_wUEGlfL0U7 zz}%$|FgK{vt-dM;V-wL8%7qpm_VV9GG+&sdz@e6suu;0>5SNXH_wzxG#iLQwRikKd zl%RRaDERt>%}`?hv9|US33`ad^V3`*zY%~qF~)KyLbO;IBglbUM2(<{r;yhA!|;cD z%HZ}MfK?5CkL>hQfGchTpD+!leR%t~d{RL~;RpXL=3zL#`mIAJDHEmFTh z3J_7-X+vmJqJvprr^S6^LH)r#gcrN<)8@1TC@T!CV)5vEI8K?vq&y3<9h;mCPO=^s zED!ABYSjqJFI6onZE!G=6kZF60KGg#6(bu}CTA7{uH;a*2} zuA~l;x=IkQgwLo|;K6=bNF_`3!{|3~B!V^ht0Xdj#V@2`rnCBTFMtu(!76G$@6T^l z2Vt#Aw`i7Lhe3^4pTL+D4$xSx1pjK*CouKQVk(Xu2pU0kzH+dV5U!^y5y$)FTBrM)Yxfx@F!Z5tc*Ls#gJ6+ zT*$UnNeD8$zHa&oWgL(-A~!Ke!2J3Gn6r>#a8dfS4r<8_gr4Xa&r#55JH)cOW-MFD z)|oZ#1d);cJK!ekKnKwS3;596h}3{O0FG=AcU^%7N=5&B6(X635Z^upgm^+{CVGFu zi%}|o+EpgtLS&2z8xG`HZ)twJlk}EtJM9Hz)guzn0G+w=RXD(0YDAyE1G}1n@HfxU z(ktr7|0XBMhG~mJ6PiB)$3kE!5p9Doj(jt!{~d?D-+#IiS~Q{~7o%SGCdjP?K!=8- zhIWnod@XBH3w~a$en*g3J>4RhAgdZ^*Y+spo;yI($;v_7A3X+cbrYJNbwES^{Bwjh z0ko89m;NeX=fKbVq9*;;U0+LU$$~d$s;s$kZd7oPL%O2(!8JRF&f&X0m!K-QD1yR( z_GX>EM`4WJrrjArE(KGGTvl`biE&CV0D_EYTSw2_0bG}-&~kMPS%pok#{lAe2g%0~ zY3Wl(L(h@jB6oNREqP;YBbyJ&#q~u@;E$#DORqGH!}{6TLs9V^tg-fftxQ!BiX>q! ze3G}*mRG(4!k1vqMgfF48nuA-W|pIZ^KroRhh>Jz7zg)R8OSeWBv;@nKpyZ#U#=O& zW)sAGLJv^e2deL!2|D+1BWg|C@e>t0yB~-+K7!ny(WbQy62NyruRgIT4A`Kpp^~09|L;jCUiZ= zC&--lCmG$(3dqUsDTeC&Q}DmHLD=K}8%D`Oj5&m_eQO)uq;+Y>I9~V;-b(}~K58C^ z5suM2s$gzM0o2mg7VR5&^O2@rE-9u=Mj{e|qE;TSJp0&RFS`(kIgxfO?iiePRjCqdOKd)g8=6>f=sN`k_J7tO?~$ zf#;(iy+c@eF~r`PgDc8ww~&~)e17hgbK6&1t9xLe4{EYlu1?H`T;L?~S`*q`2b1Z~ zfKCb;Jx)-7Fzf6)G|rFX`&4TYa+0z%zp#Hn}(#Q}A_*Y+VcP-D=DL^l?kxx7p)YRKc23rNay zPV=0VO#tJ@W^$QogIA7z@4^K`zbxw<^dp>%pE*P0d;!WomP78vs9&=Eqq16@9d6Hy zyfwp6;1J80~oJQ~~pgV)9rjUlVr zvd3+`8!A+k84L!rAkY`K^~F;J5E^X-g2tTz8TyscOnl?6$*Ug4L%(ZU$@&}A^?RvT z$c|jFHijjLpqjIr6SAt&qF;e~&`}_5>)NJMC|$KD0sFWgg8UOjLYWRwVtxmVZk=+J zngvZzWiA|t%67C-^pI%2HySwt36WG2nA*)KqL+k`Ka0*aF_4_2)gza4f3rpv^a6~n zRSycJfhw04i4vEx&{ja{#cy7Mc1OX4uhL?%d0GvqZ`uNe*4+XWDhV&bT5WtScWE@rDbuIdi=2Q6`K_mDIP@R^s zC9h^sUv3l~am`EFT#sc$s+On75;n>&+_E z?4(AkDzeA*)m1da^_>osZnpWuNEr+0ThWkbI%R_*Ku8i&?Y3gK8{Utve4!I=&%D6_ zFrRqMd&e3OU~@y^9``AjCbAi|`ns*ZaxAztYwQNb*O18)4TdiOfV#;7MiFVK1WNo! z`WS8P$xoNFW2Mm=6(d)fMKW9N)#|)Z^nC|aN%hz~AS9eXghXGXUH~l+I;rLxc^A0%j^^$jri97S~#zdwdXi++6v@WNFPOII18BCcuPRk6@#>3{blWW!&%4}H%$})ZaoDC^(|=VyW9DQe)RKTC`mw3C7>%nV(Pfb zB4Y5~hnVS7X?Gs}0?fjV32&wN*gCr;owy1Fi{1WpK~d8PjGGjK2|5=!&kyt=$i{9T zRw@*~v|Gp6)cbjPUg#kIfuN)H%X=3F%ddQitn<$cYEMMVZ5{6?F5mS8t}D_|IfI7E z#w_bc9SF$J4;k2^kP52ik(~e^m1x$OjKeS}p)TsHV@5KEbsR&tf4g%>$I14_4rh?c znB-_15Yv59P2BYCa0WMI-?@z6Q{oQ0$q?8Oj3sLg2Bu_VM``$yw?0FMKp_M&I8VAaKsf*mi$VRB`PT~IdoP>p85F}l-A98GV~ zqDzDnZ-}9`@U6h>xTWIQcEr(y<6BI8mxQw$zrvn>UIXj&Xc132zf9hdg>71S)=WmL zD$YDMAH6i*kIKAF2y_p-|C1O)=Pl24-bEU*WxHZl|1AN+@qyJeT>mgGi2nk#Wcvdx zD;^aeMgcK4PVE5)l$hRbZ`|17`~j}-I%EKa8%D<>m|Lk<_fdJ<0k?ZD(3Ox8{|&*o zYk@Y;Xyfhuyb5_r0hpz%UnVo;@hic11UzjFpdT#}`P{V)%Lg>HtkrBmHQzx!R!usM z`v$FEH5#_CX*)m%XJEjXd4*k%l)O>KPUz|TBXZg>h7_BJY~o=D8MfS!q4fTYGhhJVHNZ0L(gYUZL~`(75A zF0Eb$Rv3=v&~&UWifgQECyarP-mwDkKAH6MiI7y}o;w31R6#KCHn1sFG8Z#v{a;j|AUs7nnH^9gH>F6<1%*C zn%4vgSezJyS`MaPl2Cr57F?hYu);pswy21qgHRD`dI!XM9Y~VRMLQ01ab&}Mk<#b| z++J@4#UkSYiYvSJ3mPoaKpt6mi##L2fKMT zK(p{kNvP@+Fz(;1TJP0sUJMgLSZg${(kD zCM80@4h@Y%0QViJqGU|uKxlNvQjLuINA#*2Rz8e*{B)o7<8f~&BR%;JVFYiL@{Y2j zYF&Fyrvsq9fBcoF#M!9KYfoLRT{G8fBz*#%-2Yfb0+E4u!x$0(MO3di&dgK@;V+t2 zv@zkmtOqNwEZLtq06j?vVV8BCs>duKg1#-V$W83B7TedYKoaqXy6zJ+5B?1B_k_B( ziR#dV8!Pmwk(R?`LQ22_QIf98k$nNI)Mr z7fY6p~2h_p}O>gJCmy| z9gSOvjufbd@8LDBKSwXubi?Td@m`l6z%=ihqI43@Dcy(n;2}uZ3%KwFwJopD)_0fc%?WQr^9HzdGL1nG$t|D0| zj&{8_^1ysefhh*J0Cr#nQ_!t4-mB8I4Z05k?2HU_^n+adh?n5A56ezsCjRg$ES^~& z^C=!PxSBB)m>BP1e7w3g+!QboRlxl87k7KM4@G4IZ|-C>RO+{u9X)aznOrm^MeSv_ z;z=)u`+<(0QbDZ5t8CSyIzmp6AO*}&l(HI8BDjz8i*1W+rbF2UZ`k@n&OgAf7PY^~ zvx6Ks@Es~B*20{giBbxro_2+pqVGn9eitS#eJ8jnzGK-xiQkXhKPkY5{-13Ca&147 zXzTdRjU-5-r2t@qeJ#BR7Sx1aoSP)YAfa7|E2u}N897tp_y@^rWNXd7a)Wy3 z1yyBIbSfC^X40x^(v>q{v?*KoMFMh0tTUhqa6b{L*!*$ZU9M=M z(uUEU;vC{rE_I^fA<6c@pp%v~2qr9G5+Jxu0?B&P8&aU|2(`uCQ_7Z~toTeS39c7| zq3q2{_$^?yJtfZ(TP9pFvnVB+j=a})RXBI1E!HH@8_>6sUpY;VF2jdT?Bx$u@UN!k z4a9PTXI@0vu`xnZgFI$U+Gb+IitVQlAirCBmSdxyij6`8i2yiALP)*_9nznVO^G`V z0N2JPn~8^tdD0mY?*)>dS>5LY0mcg~L<;Alg_R+dx8IQvg7O+3f{nj?D2Qm|O5HHc zWQK3{cNd|+M)`mv@$*tt(nNoqkf>Q!?t=Y{6fU&dh3uqAf|ftpNGNWAfWgZH+x66E zo=#WGenG4~v$_c2M&jL3DX^?jbDI!h+vIPk;OC5Z{LrV zo!QBW|9^JNjURakZ>IVI&Ior?`CDe}O~B;y zSdk_9aR1npv`_wFRxbgMv>ro#?)i4Usc6V5+-22#_AVU-7&h_YpQaxA!6>?%)of?MwW_vIZ64!L#)37o_F~2g@5v$K1Zwb7XJ-4Kj>hz+=Rf1YE!jL$#65{&1eD^Few;tl5-glT8032I0>c()1&COTGgo1 z=5-FVR!@-Y)8;&aTn4U#QFLBP0Ip9MT;FGn(h>Ap{2ELyajt{Ja1Pf4XHm)oJ7K36 zcon6hxVtnCWJxlS$W8w+u}#8ifgEYQnn-CHnS5MH&ClFI1TR#j10`g+lZ*DcbR(vj zv}JmZD0lL3usui#>yCP(zlzFGot=;ULXZ4|l#sUe2|Bt1RFCF&bPTPhkrKYx*$hgE zGi4Ojn2-HJN{Hf)3X!`D;Pr+XE-T@Sh;7K+8Vp~&3|^m+P<$BDGF<_xym@{-cPT%J zj`u4OczZq{_lik7I{g82D=_?H1$bQ7O8W%tIer^;;lJ}cdMb<%giWQGqgv?er7*=L z6Gr|jw_pSIfb0Ya*fub7m%3xN&hzB4_y`aktKRMa7$jB0qdVmWrq?jzD5kfC^f+3b zVDoFTt}%k(o~Ig}_v;mr0nOj-zSkxD3&xAW#CpcNXV<~!kJP+^u&Ngtgu}a~8^pw2N{4BlKJ=+S%xyiY6XEX8o_mHcx#|HeO=F`qzO88A=0nxYpmGbI> z-4Cx%x}p|IoxACa)V%kOThtsG&Enep3_fm=Y3-Q&1DTYeZ<-02CMB`;0w~U!Ozgvo zH5dc(3poBRc+^`VZp(jiV{6i3Y~9qN2_NxSr3FiM7VUlE;Uz<=REqQ8y9Ky0Sp1OD zfV8*EhWWqUB|8cP*CVKqj!8(@Vg6f>#qWUEpJ<4#NT8FT6b&Q(%m6HGK9g(RNxA*x z!$>IDdRi;g?08V;u4zEHjuc|Y1bv)R_;t82A;#9E9WuaF+!w3f zqxWU?g=MLWG#6@FFSDu|4-X=Q9=dp!5@KnEt^1QoZ=~7EN~aw7WQoigXzg!;cC~)^ zNyC>rE&$E;7Laj0O_msscKVOk#OW-2GnJq%z`3my-9-9j({ zgXDMXq$_7zQQ)a!8Cyfv7if&d#SH&WK_C&XepU}(q*U8p$Hurk%od(*2w`f^peszMfVmR z3!Mj}vL)&JDUMT0Z$QE7#NBAJuz=z?Mfuonu#pO8x+L$XI8NEQ3szsO&6P+J7*HIi z;HY8s8;sQXZ!~2?rzP#O_2AjZ+?{NSr`6lI0^Qf=GFs zV(bR5_kYrMYtnQ~=_TcH3jaYs(Q`g=koX76y9JAVEjVuk%ldCXF(1J1&E841CWoAQ{2JsDoF>L-aK zK8Jb4EJ+IRdLzv~Vj>Z49x)4(0Hi0Fu4M-Z8xNkqj82Kdm<4wb0khu(PdgEL6RXQE zl*TOFC{SQ75TGWqwy{hU$1HY=u=vi$$@hugzLfbViA=(Onl)y52)UIg&Gi*TkDp#i zdCa090(;kMP)zL9^k^uLS%es2^VT1$AqwzBS*a+FS+e&)dJ>@Is6XSrT6w9C6U*^j zc?!M(rFJ8Xf!x!Zk1J*qokZ%UW3k_|is?`xe;uTDc{}&bAW1OPa)`v7!4J;{is9Cx z#ml)1%|r^gjWiC#6?~)-vpr6QIf54d;Wht^)=N^~Cpw7*e(GAj^km)-v%! z7wzd!natK^PjIm!u=v&o zW&XDbMhl8_tbMlzhJ6 z9BEGJR&W?X5oKEla-VHTlQa)(f=?Z}z99|*!>UveV?m_C6RI<-;80<-020A3VSUR? z*sxF`-HQ|pjtiC*zW%uS;pJ1pf)%9hokw!tdkeX{6Q-vKZ+B+<$wAnNspFhu@34?L ztjFQj%Mff%x(*gYvDgBZ7yyZ>hg;F!sRiTQ7)7V937s}MbG;sK4QGGiY|PoIcV@Rz zC(WY>N~?^D0Cd_5UQd00#yW1^d8{7K0iSR6dQ%|JKe>r^QkblaQ2PI_wSuBk5HBDomY}xst=NDXZ-Tbq=)399hisKfM`d2JhPsmOHEtUSfHjlyZ0qmZ7TifI?;_brt`REP6rF0M} zihkWcnd{A^;zWTc5y83!q7D46^0%9j-NGNCDV!fDhg}b=cVj)mjO>PCoC{e``QOmo zUNk&G`=q%odOJ5Y35|cQ7;9=R?0djta0@I)N&P%{iPO@l4>Mgv)g>p!Y+#3JJfP@HP`3LHPx&-c4uWHuU)vG>pJQ#Qv|95%yl|>7j=p z@be)j!d6f0-&Tk=@{@uPfR7Kv3{f5;PV67P1=O~;`sNa7-NzkfC;aGSj6?dNQ|}`Q z8g)hC<1aswK!wK+7Ev;&WUeq}^6Whj+brN)2pk#i;u3bsL$s+6jYZ>pPUvKSzXGyn zEa9`KL}T_CZOWCGqEKt6?n-6c0$erb14XFP%y>~O{v`5u1^WVl`w=RjdvIDJC;u>e zd^;uhO1$9j$}H&RT(CXVy_B#UC43#P5U>E^nIyPc)JJJ=Bs?9pO|6gu^#iQ&5IV)D z5=t)E6#4-1d3gM(Q-tDscIA(dy)IaL8jtyS{7IP-Lew)o-5MwYy1IzXcNbCR5Ki$F zOCb(xK=o<9GJwS!p;Ih694LS*W)kI@N4Pe*$NA_KE9~~k;g4g6$iZ3?ZJFFHQgeh` zi|C@4-GwdUm*kV*$W8iBC(eaDTVI46N2+@Sb^4L)$AqYrQu&97Qvw$6v9RP3^}Lvh zQRaVOZ!%KbGD45r&0B0h?P%d~xonAQPfRn}D9OO&1bHA%sp@=q>cv@WuZB_|5`OOq;05Kj5}w{+;=hNi#+qqfHsMT>jE!P`%vWYD^|HkoXpo4wm{OtXjGVM<>i^ zD7RaV{AEm=buW?q=AR&*NZO=0CHxV_pv8ovoI1eLomA2U%z?1BKP(`9pRO_lo3~3( zlW*1*Itfask1D#lzSztwjO6BKA&mn?I0O53C`$cvoRg#~1jMIJy0sLU5Pg*>CEf2A zi^-nOBzk+R3Dq~EzoIRjE#VQp0*|lL zC#Ez=a;Sm z(b=iLhM3!lqr67~?~VeM_j-Xd#T3Zbcy%ewH1hc%!2f*X9nnc+w^8@bRCfLp*H7pa za%#9R6yW<62ydTOGn_W74*?!0#FDf6ziWVMir(^QAQF`MbVy{Uq5$8$bs}#{D z{R8K~;j?T;YIP;3>d5{0AH5zzJmCIBkm8Mkr`Apj&MKPIlt+MCkR;<{{0n9RkDZFy z$}^`jAsw@sFE}Bz6NBYP*Aw;**F`Tu>IwcBaZSlpFY4=Ct92=XaC8})xaR?Lh$g|~ zcTHw`5=yl&1{87t(9wAufkaJ$Ds;LWQM#N;e?}iBo#Cuo50_>l*R}l2%p*&MzHktB zVoEmt6fRBy#CUyV{|XNSsy+D-ceu-*>0_My!>lR*dvAjhwYa{mgmMHnk~V-m0G(BV zLAOy|FX`xjzS6Q*%a_BG<*vIn5YL-a5;YcP+kTY&otd$%;QVb|2j+fx^k2F`P-Jgic;8bzE3q83g)Uxpc)&(<>- zdc)rTlHi!?NMqN`CXIdPGTkT`TaN0H|G1ypG;#LacU-nl2?4>J&!Kd1vNaHP;6JMu zPBzqG`Dp_d5+0m#zH=xYsO>&D`i*zgCdUs_%sM2%+r4qH_*yOd32%q*r`|9hr2~%h zV|e_Ot;1v$FI_%cYCcK_CKGsL*#pb^CLP-eK|J~0Uvt`BgVHQi(7Lu5W1RSg7gcQRr66g5NiOZxLeZqO*Av>vs@sa z!GK;c=fN+)twnhHYj+VeLBQ?+R(n252ajAskn#&}GjwA3UDSm)r^M~=LB`~6^C5Hz zb59s~cl>S4(EKC^i`Rm^ui`bCNR0}>fo(Nw{Pyfr^4(7o3}yh7|Loz#i-6w9AQnqrnCSSZ)e+gh98Sh zp}s#8^-ga7A7KTeiQ3t;@VLT)ZUm2||C7x*KcxdWe<>`!=+J-2y`SL66L=(q-TRBu z0fzyBvY(@tL1Ku*VWw#?gg9@G7-v#%5|Q4N+XVn1e4M&*%Doewp3~1G(ZjNVyJ~$U zzwHckDX0VaW%|j9Kg?}z;UDKisKMR-?w!0#_A%;^`S@A%rMNJpb{`8wq;T4dKGWo{ zDWG%K{BHUi6;Psg(+85`RtKtWWC+(^*AAmIklN8l9E1w~Z5JZu-K0|fwI2)bh}7<^ z2~0hB$o!KKtWl`*ATB$V;pndmxqf5FU&lR@@l1;K-;@TlHUhNPRC9+VpQtY;cb+B# zHa`iz{)2F=F13Ql7xIpQzltpbU@1_^$r$_4@8v>Fmz|I*BA&&RM3LJN-97L)592xP z{CI{FCxbQ!fFNJ2C1>Am=&Q1uug)WHhad(8W4V>BB46g|DA9@l#sY6=B45aV*=6Z` zl?H6QAecO>rl*0uIdtqK;3Co~;xhH0g`p8bY#$pFUXECoU;oG3^H7jEKvOE%)iiIV zfe%tZw5g}Qtn=d;5-cK`aty8Cbce-b8A|yl*GJHqzqskie_VPHMCZqUV&!!-BmYMM zl-4Q)dAsMM&+`dbz{#5(;Xc&!-$+RGkohdO4vsp`ac6pglYgM#7#XInbMz7pD*u1p z9uFX4>s3Ll0;#_xC6SDWXFOl=EGD{hQ6?+nT5U!K`op>leDQ})US~a2RkdhS1M?z* ztx0;H=~(ukbT^3JxPj%7tL{S9hr7RhJ{Ic2!Z{#!=EQf`ZpK`H%<_+iABJvqp}l`d z`@p4@w6>bNstXQSm|2vh!DQrqGv}Ss{b7kEwMMa>XV)xVRnk9|fO2ON-FQ5)PF+^a zK1?F^e9y*m@)OJT|gZIj+Y9%M2Q-JUun*uSqG9|3D$XdH6n|N3%Y znp@Oj-{Zo`#l?Z-Zk`F}`rA(6YM1olHX|bKn+rZg8nF`Qn){ z&E$6M_d^4=sBR7~j+HNod`gjS4(1wpHR=z~TdC>(;>E@BB~6L-bUjx07lypN_6MP% zy}}x`4|A7^o}A@*6yr4Q)@GGh_(5Z#yEn7Tue>0myi|$kUM1gXtp!*Y3A?VK~*q|8P5H7D^8)p3BnPM~?gbQN>H2UfG8c`pi&3TB>8m9HwY$m%4yHekxR-isdGV=Prb-K#r{`}A9P6sIY zUfqzYX5ykkb~`4FJcFLasX_0j~A_@lx>jW83P91L%nDaGTQXqCt*@UuKkg-K*M z#27$x^9a$&878_UQ@C?54Fa7U?>XY)w1SB49tsbRfjtIP{#e=pqUb04ipy_PG5%=B z;Rfqc87f}fuODtKGvC{q?$Sq*sROtYs|ZoV&UADCna#$pqDrLj;9Rh$;qCd)1!vlP zaQ{$;P-NxNrGOcrG_9kG3#N4p%SC;f0{=aAmWeCjY_)ISrG*9iiof(UXMVoK;-`-aZJn&dOh5df(V%UEWiC>I-mNz+QN) zsK5vkNLY9*6S^GhaIs{Q)Mx|Tf_1(osNmo7IIUo>oo!J?^sd4?89l!Myt2&KNe?F>|rTCye6HiPd_wEgO{`$bn|qV%z;)0k={L#t9g*FU z<^wm9*mT<-QF&LhR2c5eIX`64a@mbh&iYF-7hIiYbt1`7wJ$p4WZWa|!tOk{{^>Tl zX8w4X^vJn}ptA!k>7=ELHa1c6654cTSE>b4+0hYtduqcPSX)aSj<{1DS+bB#>wla* zd0b5E`({kEAw{weN})8qF3D13*QV?h)mV!tTC`}Hk@X5`QM8Sl%909M+B9~OiZ;?_ zX_JZ;?e%-!GjnFnOlA3f?mxM6&U@bX*`Mcq7Kb(Inuc0v7Uj$J<*(||4DV+1HZ$|U z7W>SroimkfT)SYjYCx371<$V?9rbCRrm3!uiK<;Ko;^-UX#7hh3~r_)oHo0CJ=@k; z8Et2vxiM3%z(4I>c;m6=^DW&l$6h%=)4pVXOva{6^)wjQ_q_=Q)Pa;d0weaP$>QK z?etM4aVZVy)7Q40>OzAan^WB0z%0QmLzq5h>Du&W<+`?0-Q_7K)n83K_-<=8Opd>r zsis{5^SB~mGKfa){m*BrVeXKJ>76|rrWeE+-<*8IBJIJL%G0)P+codF`X4y2ZV{H= zVVxhN6DhqP1`BRn&ibE48`)FyrrZ0p5#;p73lqwV>p)DW z`Q5!Hb8KDA?Ex%h?CYwe-7Z!nq zY7EDIr4q~EpW-ITqqr(^>dQ?Jn!@Z?!^pX90c~STTUVGS+U;z1`?RvHp=)QL+1wv? z&q~WvVBX^Rbl0$pcC!5?VP<$-WP@0n-E(GI`}MSIl48%t z+3P#T_>rc$e3;gGwCCi?wwl&;rRqmn z4JE!xdT$>cm0H(E?#XS@aM$_~*=A?v;=h(YP1>LQ5JoSXyT9?MExePrGg92>S+KKu zWk|Z2WO^`~oLdMyc%8i1#jH-3cEHSKQE|iw+4MO%sT%)omwP_}8Xv8%bEW+|>$ETl zI!Y*rh7bwc)`e}a*)xV$q^JGD08DSEYNIXflH*}p|^^wv)|_rAZ?boXAW+IX15nKx~=`>eE>xDn#% zI*6cL>V$X25?j{4rtvldo&4>~RNQ#;wqU!FdSJ+;Vfm@9NR@b)B{d}Vgx&GOYX=yo zI3IA-IBYkH)v&2Y?!Bgx0M{5oVwn@5bYN2`8 zq5AFm*lAfc&iXMwm;;Bdl9E@wvRzWB%#__i9v~#l1xgb%UoB`AtI7w)T+g=Z8q-lj zh4$lVOZEBqU9T^$?>^k~ z`n0^jZoS22@sEeNc!g4J7+p+w5D%*uy*wYH~+)SrcL&5u>mIi+Il`+{Bh>G zA>V{@gtkeo5)Bmz?4v5=JssD`dW-3DjpJNuQGJo`k>RXJEU=i#e;BT z`$T(sq8Hu8EVCo~xR(FW`Bm%&y9TL_b!k^DO6L7|BBwFd8itwapaHq*n$Zm&ZEro> z8ay>U%hLm{7MZa}NDl?c`ZWZ`zRfh~dgSS%bzszCJNL}Q`EenhPlm6&ympsLrT>qq z6JUJZE~H)@3l96}zW>-u@UWTwefUGbfDb}7y&_3rh0NCi1&J~{ipXQe$|rG^)8 z7~k9$ap&}&u#;__1UC|jTE${SODzQ4AOmzqZ6ErI3!Yt5%0gC4`m2`*su$U|ziZO_ zDfcld49#GSPWJ2!x_@`ex(DCTsK&^X+smCawGOzuq3gjhzfQi%cx@|R+S#i~&JpzJxAg4!#-_h5pSdW+dBn%SSz$F(bW&46If=4n{z z?f}+L3FF};@<4HzL^^Qy>x**+ZuuybEuOH1Gr52Mpsbh+Plg}IT?(pTX)H1-uo zL+7f|9MEH~RhNz}sManb`>zyaD6=t)08O z+ME5(!u0f+<3TfS4=yF<)ioK@`&Glh+2_IeNpRS%ib=twX^gynl@)(u#E@H(Dn!4_ zi*I9wyFxTG@fVQ6eM>sC4Xph+pD4)_g+^2!{ooMPn$QJ(ulga){Jy5xPuqieu^Fd=)qBVQ5lOvFB}6v)h~o}gt9nDRx`eT4sq}9*p9EB!UE6gVYCWGR zF(&*a&5JkD={oGe!K!dMwORfFEi_iW7CpL=#b zZ_0FfaJ${?!?HnqWn#6S^X1oK;vJQ(82|7gG#%U~xeAL&jF)%r{P8@;wkzK@5++qo z4^VHg9a9cQ-)hT{11m0SdwPJP1d%ugRDkLtBzmjgM(ze{FxdQmwKX4X(-w7A*mmzi ze#~1Kd_TWdb0|MVlVHmEgbNqmzKAjkexd=daRwN#R-&^F$oWgyqgi@06BTcm+>W~N z^0h8Chy#;9eP-S-J3KlqBMk&;)Q0ODBbT3&-{+rFw6zTEV*{yCGxZI%+IA#$)Foe6 zZ;3Nco!^y_)u}(fDSK%SjKaQX4pW;?#|BqFRquSHe!1SQa>4HB;~9qW+-u6K8ueUl209$x-~M4^OlT zlRZ?U>5@;!2m8}KVHiWUBMTrz+Xq98uNJK-Gq82_yG1Y1$Za1h=M1CCb8hW?^%l7$ z*<+^GX~n}p+(;$8^Os#3EA}1i8otue{fiYI3mA89wPv~h>oW^@oCUbiBFz@WT)CSO z7M<)=*h_8k*cnr@=a6cYvZYvO-M?+$eSG}ERpqDB%j5iKNHuNya5Th=Geo1vNB7dM zvx|PND($S$$Z05ll{4k+Lf_T%t1maz+IlW1hQao`A08H~z7lap*Sa9~M%@6Tx}ip~ zFEyjJVZ6Oz91MQ{B00hB`@<%fzpT;KWZ2_S=-JbyeAdw2*FLu5Jq$X(_P8ehhTbL% zcgI2))C*yo+N7G;4R0#T11(}pTKEc`K_w~o8~4D-l>HgbHNKSoN|`o+Ws}vA*3*%e z7r4Ye(_^;6OUtsC%HjGS_TK(D@siw57ECan*Hu#AWqz;c@wR3VVQ7AHWR`oKef6`k zwpdK39em+H8$f-+tC`7{!Q!8$W}Y%n%``y;2z_d?TjT59-JNaKwmm;>b2KYM)|oZF z)lCRr_D)+`e^XyxWvF7X-7ZzvU9HQ0s((vxjv>H?5 zee`D;8qqL!mwESn$a5{lfO7*_)nbPyc0IM|eChyXzrlj6W-+4KIBQ#1(GZVM${yPq z)Pn<)CHKNHueyC~t?#a@Kis3No1!J1SM$_tv7l=dzwJ$k<_ z>r58CMH`FR*loJAv8&+FDZw2q&6E^DB>1v9;D{DbJ15Jmp~JoJ3ia=!Jb#S3TC|Aq z-?(^i*qXrmi_UVZn0>qbdPRiAx#1GI>PC`esm(WR4W3Up zw0Hg&pQUeMZgaR+$NQ{}PSQbj?cj?j2W)$|c<}gt+73>W{06gJ{o*_uls*3!(9^l7 z;l-3{Bx)9F<@o(?m!?x?s8iU&$Pw~qU=DW^OlH44c!D#d;l461USlPVQR&LA8Akt3 z(5!qPR=1F#h)fk3(}?;(VHqQM@#=%1Y;FvXmw`z^vaL^)wa%qm{BP4xE31L}A5L)g z-7CKPCMEbmN!9CV>J?ex;Sai-*B_y?tqLcVxy|Efvf2kLYc+xIt-kmAMnC`^&&_Q_BAl zkRsPYCs;pIz$8`Byh@VD*uZIqTvBlRXO%(haepOquC4E#(_6|wFtn`LZqx1>p%I;8 z3^BI$&PwfYt%h@FFE?~vyB@21ed7(?dnK+l{w1;e`Ko5H#i5Oo zl=4@qFgjXqj(Or|&c7pPyp6KxR*w$c?%Al}=~tK5)19#YcRJ;O%MW$vK!3A!fP<;H?O>0s;B%_ccimD5(r*H z=c{ymjqBbsAI1{rfao=HzuL2)v~6`eL>yb10^-~vlcXLN-m%Vsuv~Z3RYUh{V_G0) zrPK3cV$Zqs@#5(b3$?qNa&o#m>ccUaqr^Px(G*b)Fw?VWe;ash?L(M791?r~)4vHj zUivmwXrv{Y>P;KsfB4bov)W<0qc6>`!lekqm{b4=W=E3COP%a8)yzT>zIh6-Du`Lm zw-x|&?598mY9-_JYR)tBupd}2$dL*V{IT#D=Gz!^J5#q@sx?w(f5xommyzNd@ABnb zo($5l$Of4H+_O5+V_GdR%x>`Lt0D5J_k5zKTcJy3XbuPrzccebooG7l*?PP>uBSaN z2b|xW6qm+pzMR(@r6J~SKb|^(6Bq?FKera*AJrg+Z=_2jzMb9^XurBL#D4X3r}6WE z*mGbobzn(ztW)ILaVnAFA@)h#?KL@Zo#}DwBaf7J6_*;oEcgS7xwz7y-C%kF=T+ST z*CNm@*2t7b`P0yBgGc z?55WTED-lsPx4oU#17#{)X--w_Jflp7U;t5=@zc11?to1ha}{-s;y#f* zmsBJ4ITi~R8T84{?ib+;?~M@eO9)%Rj|eK9X08!3W*JRDtPvV>J{3{62rieoAH|)G zF>sA&HK5oZ7tbD2tAtaUhJeHM52Z9~cO>76@6+tVj)Jes-$o(w@+*p~-xkJ*W|ZT8 zrh_RblrlfnT^|3gLOAmRe*yFdyAjS2TKcGcV31HFW!N*m$PPnRhFGyR%YkjF@WT87Jo-u@+HEkN9Na-p1y(DuMH!&8ai23JKQqa^(T z4$j_Arh+~!LW?^XvTvjiaey<9d#{`m;3Kb)-h*EZb5tnaxH{sUZx5jD3dwQi(xiTq z!>kZ3OE}{9(eAx!w*OHAmqE+rHzqVjE;#KAc5I6L4d2>Xcugjg<3LtV>$FSy;;Z*I zHo5MiU1qh<(xE!{itO|pzkbcTp7Sh+OpZ-4yr6;8Oipf$E0k&WV{^>6NjX zV}k9B{efJEySNBNj4k8}A-;mDv{CP(I)?9AE$COmxfCjP2FjrhYuoFb(t@=*KMsr5Q&0V85g3Ej7c9ecJbzkxo)7bAuSN>y^%-rDxz$fc zS;R;V4z%!aNvgg30fcj^|4i_o9NHLCM32^wO^I3VGmD)+pRR_B z6ob-^l9sn{W1zKu<|w3m=td+|fCGOFqok)rmxmnkS&yf)aHPpYg-MtdCf<|;HTsL> z@$3PbCCk>c<3n{$c(=p!G|L6=Ji7`!Bf*SF)5p{eD2JS(3%>beE~rnrCi5NXnrdr{ z4pURO7-%l(w3Y_BUtW!Z-fd#im$Gh$J1eYvtRO72r zYofHYx__KC9C{OZ;h_f$cQ-UY@iRDEYM-^J`rOJ1I&jp+C(|~xm4VW|{_o&n5T*U{ z!ypbt401M?w&yz_GjQopO+_=$uIB37;az(nw$S}$P8fy?B%C<*Ay9M9B~1=a*1g?xpull?zwE}oZYc}sLPIAmkMn5kI04+c%U;8@ zoIhpnO4}4?9;(&TmKA;0?aP#_3)8+{yS~=sIfOZq-X{)4kmM*(^m)SUpUI#c#FM#+ znOL*{H@kRJZj7A5W*zp;BU6#;ajT6UkH-QqwHf5cIaJaZ=J5HnMic}!;@Xin!@+FJ zXSSUy+8++{@F@YtifRb#w$H5o;%B&8yW_U@=72$9Do)&x_G;9Isn1k*98Y0H7-rnh$JxPr{ta(Yhjot@v*J${_3ma=GF;@X_IDQ5xr zfeG~wI--i5N>fwHv2#rsK^xJJ;^!R;A%k5^j(}+1f=hXPV>82qJz}6%^HsyhhC_F~ z0@iXO1IX9xS?!_S~^9SbZpE3z&b% z1T?+5ZTWl=8_+n(VYe&XKkrJpyTub+@JPD?mf6AgO5^P_W>r6uQrBNO!DWZbD@>s7 zQZ7^FnNOd|k?c_WI%mmOyeRV?dD=qJ)6p3-lvo2Kmh+EirO);ssGC@Z-Qc}a0VCfK z-9@2TT~b>`@b&F-&s%_mMxQmL1ZmisPe00p&|y;2PsFAZTRDT@jxG6qOwD}rGp*0w zv#(jdGM(Vr`J<`3B~82h+pAe}ccD-q+lkc1wNY7G;{KjJ>F?79H*byG$-Vu|k(x&w znSeI@WYjw!BZA1`nO<~i6mva~Pgn(Db(wp8=icis>pe~LggTABM2SU;4-r|tP?4dA zFR~b!Zg1?kh2~I$9e}7F{~G$^huDfx*h%hx&-kVr_X+z^G`vj-;|p>(mFG~olzyFA z!#BveauQ6A(~9e5t&S957UJ@33qgaA*WzHheYV+%a9MSoStRrzvNu>ir@|8TCB}Ri zi|y0Ffa8jP4tiV4TK%Xn#`;q^=yCSoKj6o0C670}qL}?-NP?S?!KHAPX(B4paWjcP zKtIIV0WIud$*R1j+dNq9)!`h%KVdF@-;b_-*JKs?K#&P%L9c_5BrG5^%6Z5mb)h|+ zP@(d#CpbqyJRUiMWp&CJ()pb@czZ#26kY%?>Ho^N2(2N6L1{&>dSn9q5D?gE^~RtCnHdkYoEWtlxQqEjqhymlA<==oVsM_>Wqx0_$#) za}F51>~Qj0VS@m>LE3v0mN=#H{4Eg+*=nTg{52uR)|Vda^t=$B$W|g*^MBy&8mzu| zju2srt!E2mvogk!EAWHi#NFFMM8Jq21!SE1Zr3Ct)zEG;o%nldpH#M*-1W|K8azSs zV-nj0p`|gKSRe4Bzp9n!!mymlgeGo5kXjIQ`TG%|-ZzvlA{HN(s}3PmB^K*`2(jHE z>#n#lZtQ~%{}AzQqaA8iQ#gsyr*ZCW4n|8;)CxSs0_~6^D^r!6F)jCzr|kig$5GuevYla5-# z!F$i=;>eF1bstK zM#xXq-nf!IgPh;@KI#x^J%=d?P-}FL`GdfO;#4DDGHKfh;4pXc3cgg~JvQ}+_Ow2VXF+v-Juk!Xm zhOJWG5hk2{9Z5qiGjPVGOoj)Ry<0x|>j>!NK=}WB?h|i9!7TL!YRdS};KgHp1@Soj z2ml0>bwDCf3}-%d%IMJLg+3HC9mM;(ISbEDp-SN2;H`~En-$iBP&Jyx?@%QEk2*t4 za0u8e4brPPoEdZlYQB%MP?V##o+;&55^YaQ~FT+NIS7g0QXJ1u8ZM1 zeecXNeyBzS0XUBUVELIE3{(ZTen=L+!vhpWBGbYOVbRiYMsWm8eAR3%ImGp}mFeyF?BtsB%8VUCyh4>(3Y*aDs9GfazXe!fYXj|c zy|?C-GO`TBQu-9#!du|U9604WFd7+ChVt46vsB1-Ld=A)Dzz-dkMm3-RrHR3vg8Y1 zw@fb1F3jL!m9O>axPM4b%D(B#k&!SH?K!O-{m~n(>vNW|zT#w12bc_dT5@`CUM1b6E|gEA2CzGcEg1e52PO z%PMWTJQw|a75sLHRhY_Jc`lhjv=b3dDyEn58m)t(^QC*c`*UMAxk?bJ6qYE|6C8P6 z@?zP4MY(J5V$NpnqcB|v6m-J-|G4*1r^Ql4kp^YTA+*YrAEY>(1$iG~4Iod(Q9l=S z(Ytq#XJ8uu5|k9($pa2k;#DYRP|A~P;XN|xIphjlAxlpJ@~2$X!@flKBUkjM%-nPF z{uH}|%B=Z&Pm9>%(_Ga#Xw-hu$eN4=goL$i)ZOKI8>QGj!xG}B^*dfjR@CqtAD;O4>_S^Go{j4eg9qS$j`mxWeCFpsA1s^G068W>x{UCxJ}i_bQJ zHFkBq2E?)+h`$h_BFyC!)xe0(=|n4vJrEE({+*&8fEt=q9cf_ zESQG;$FDN9sl7;p{zy~sfonT7gTb3?X^qsG^wct8#MbM!2E8#PcuJa3SkQ|lC^+y;9p9tncsZpKvC{4k~MCF;Sfh| z4Hd#b^b`9=5V~%&+9}=wFPn-YFF`j(9Dy`lDFxB4_{c0_btBu#GEND%tzdn|I#Qef z&~P#xwnYb;Lf3T*ZCyc8|~F{n`gWsW0&XYW+N?C6sNp|_M7r(Li(b{b(U zau*S{)1f`C66<5pmgP+PY_d=s2?8pudExcv$n`fdfC+%98u7F?NOcUUtN5dBI&Z z=NZ|YNN=+Ns$u)?->1y&J>rdLXpIF*-uvyNz)FRk3H8lj*ux^P4aQB%-bcDlWNkiX z2CycH*<~&@uO?=uKby&bhSWIE0jo^|jjI^)O8lQmjMc;+0DL?{{)!&ENQWJJlc!9v zcR37lCQ&!jLjf#~U&jP(Z@aj$9i^; zL&k`!&vj~%SKBke1Paf&2cdruixHZdRSXx~RnN3cc5W7n;ni31S%l{k90vBIyk9ko z4~efL6Y0qyjwprtYgWu){L?pFfI@xg;RiQkp>ednXgw_37J%aZ;eg92rXWPf-88N% z-z@{tDJP@O=gi6CFED+@M^O%JRNw29QJ`2%f9?&m_f@2MU$o|*<)R!-xU0Q1Dlv%( zsd=W(=+#-OpH38*gy2D#y%CUN>M}%YGl^TJf?e@u3+z8^5 z!$08KWOq(jX2EjfAC8AVUd{qhr(&2-*v$FLKNR|djnab0zrS*DX3jOC9|(RgL8)S| zK$~eHMF1v^5^GhmX1eOL-li_M^J}Us&`;h$DLnAvY{ntPo?z2ep@91Oy9<@h@e#+} z4qv2#TLo>MM*hGfvf1t=3E>?>lth=DbJd|G2rswggM}}Im!hcT(+d2a0fM>}1buM z;89hTHz!+(m8qCuBmx|U6=@U?^9KOJpT0#ZV&7uql11+~DcdCUH_4s|Fv9!@0u3?d zlb-XKFbqhcY478wTwWPV=@eLZT$%%)_B`C4&Xyg_Bg5_g?faidEP!4h7K2Bc;u9f~ zRWX%ocK6*Jjd34*K>=3rjr}u)f$ImDgrG}a`Y4fWqxY$A>8GLp)}AMG1YUV`WUS(t0k|!bgkxRf;43UZ3H?(y@*^QU_I@(zgrAWP zHPc*IIFduKpQt_{PT8+VVC@n5RJimk?MvZ4 zdW}S~#sPH0C!pjcHAXZ4Up!?igC1}IA=QU;ABr1HMENDuvISSR2%?Nm@~+;*x5_vg z4jh{FObB&Com+H<$3CJbQ)5?4$n)eZ&0uvrboN=}#ru%H-opZG{H2QAy+U@r zNGu){IWhjVfkv@EwDhQd$g%AEKSod%)7s6(Z4mK4)6UT-jcIC&a%IQ_$ZLC9(0z~8 z^Wj7m6Z?-dAe_|GY4t<9zBZyqwL`kK&;PuuW|x)rcy)GoRuA5r)0!%+%GYJsAEwh8 z+yp=LFEjc3-)_CSwqS6K@y-9*Cmz&256R7+{+;pmu$a`_<)h5j4|PHKrPqrMAmSOl z3i{+*FrcsW;1;2#;v-1BbYZYXzLQb1a`;_H)N(+jr{dr=K z^PLRsRxj;Uwr-!tUa!O&KKAGQ@J!BWQUt*hF@h$~|0jz^M%k#4({9!tq1_%qyFh_` zLNht(rG&Gg$B{2ZuLm9JdaoQMrQ<*Q$nqvUp#RY&i%Kj|J;1uQ;Ki z;@%L5GANYEZ%xFb6TVf=2N=)qV2goDReML#6D%yVnLeLLZa6Oq>o;!Hq3U1Y<4p(C z=L%7;Y3tuSgh(Pu~mKO1rDlsypdzcTtA z!LZylrEJ`v!kr4&?n>1aEKRr=nN|Gn@b7 z>nOeI@V+bf;5OD9afNDg8uuK%{} z0c{jOKCvnhPv`o5;W0+Ie95~9l--rkcX)K3bwk1#cTaFo_s=m94c>W``y)M>5+e|MLZSg@AoL(#e#saB%N3r%#|L^Zfedcq zgLoWRfHVTt3}Bffmb3eCl*@9xt5EQ47UGipZ_^o5;pi83;4`O^lo?O|;^Qd49S|=U zeu~R-tOF83LXB}@tzQA24*76OtoGBdd05(O!b$$C&#QqIX!UZ1F?RnabE5a)K*34* zqD~-t^WQ+^;U+^m+_9MLhXx~*jwq-^gd4&*)U=2Qj3V_e28ks8O z=xDc??Lkaa703MYS0|_JLOGe|o{h!cVq25*@rWG03Bi}}|3kzD~XS3}0KW{LKUV%JO$kgAd%xxqjNQB@Ezh9q(jnX+o z3um4}YJ>Gt+_=9)C_WhY;d|`s&hYw=pWM-bL478I^Q+!)G!k@B6J%GhqsOYF1hDqr zQv-iVe?UUBAx4?8_w#s+2Jn(sociE*!k2Jx5Q&Pm68J}xoP^yOJe$NA$`im`sUvhw zd{H5ieuk0?hVHaSYjb{4lSv>=A~Ikc3G`HcFDV5UIej7NBn}_myt1z@%&U zo*77lGEX`Yzs=;x1M}ZyepUmBJLqss`8Ef4ufIDwm9_~mmnfNpy8JXt^;>s|BOO*1 zKF@ZLdN5o(z*%nBF+#h4uGtCU;3e+k90vmPo)mSiGznJ;ZI@M6i7|<%zPA4 zx;f-FWYzSS%l)gbpMYCWnFE`4PNqZcb1dK7wmcCLYevONG zC4jtIwewl%@*{3M?Qf4Ro?!ygF7wDB%wYWdH&zlx8XRXJoM)f+LLO(-sQtYYXHe5< z0=tRgonnfvQ2%=!KY}ch=!@|3JJ3N(Wr+Ve?0X&)NFPgy;aAS_l;baTHTKLBur`hq zZM-BdtzIT_)q&rM3OKNre@t@#c@{ukTxQ4`mXSOU)Awv~z<-Tx#RO0x&O1cH#*rk!Zt*;g+*}g{9h`P#u&+ zqhLl!*qqdDjejjIU~7v(+8ws}w*Mxinh5Xo4_HRL38WQUJy~7&!nv1JL0IaINw`7* zypg^>`hZ_g^3VS#-nhq8DmXQ}zsLrhqDTk0@ zHbNmrG5hI}d{GV4U+SFjG3myvm%j9OM-B@@+muvP%;4<_D%QDc6teqJ_a_>*oV=i2-%^u1IN7 z-~5hBR^^qTWcAH1V@Vh|nm(ir`nnIizQpFb zk* z=__okC_*Q_vZ?o=(LV7mdj=Ab8T#2#LNyY^Ol6!R`nK}o4M1S+D;#1?d|xO@nM>&X z$e?c_U?5c9%{WXveRc|SB>q*LO>kl7c-K({j|C7;44YGcNj=nx2f9Rk#o8*iMT8@; z3mXCHWAFssyY(zG?^D|TBPh#Wl|sD@9GoBHcOvt?!_DvvwMEoy`gG`}mO!l?fc;b|#PvWco&}@OerC{FM zX2lTdAoCWL_$}7hNKAO@T@NJUiDImZ*$(Zf8Bt|%pdcBKp4NLtaX)|q zy#-;Qy<#su5P08VF=q^s9{Umb7)e6ISm?%N^ba={$VFds?}4MetUKHmdIml%ZSEd9 zg4~b4%0!9ZLZr|)@4_`ADyukHdpu9@_=(BX{*oIBOLEOF2P3dmyn=VV6nyXENZJBn zA=5A9#g{=a<3!SJJHh5f5{=QW`Z5-k%B-g`oEzkn0p3lVIF^%8=; zk?ldk$~uVK?+@=J@)Q)ykcc4I8+{e*^qY(cw*^Y9A11+qlq!*=;y#@M*cx|7tnX=v7aP7 z90kM|E^-Mng|~MPC{sBNfkvoPEi+Ty>d#d&uV%9cur}@gV?N}9v-nZ z-Je6UI@$rIY%$U-j0gw>UMqD%bP|V^1^o!31e!nDmvYRmP#RBpr5zx<^63T10@lY7 z?{@eZ?75S-@w$0jWsu^a%{GbUvZXQ8MdKX!X9`F9C}!6c{fiS${C#pdL1O~9KPH#9 zWjgJFT$eLCJ81S58EwUHG6e#HD}wOlrY15KSvnpA#d#NJL8|0WqS;x6qhv&bQ`rxg z;uyj?1Ep5UIy0sy;e_v{#&8hjf)4WJe%Dk`BI%GLC|5`G$d_X*Zbis7!2pfJlfZ^_ z{+gesQK7{2PBU(Gv?enR|7rvCU;_g+lm_8-m}yLL{4l(ASU<&<5sD@-+a*Dr%5^4b zAqdI#CEh$F2dR*{`a>RJUH+LMnOKKB#4t9S`y=bAoDohg_<8xy;)X0>nSXviGrBKl zN7}$ATxCD?I8#NG6@RsTI%7CRQ9_mMJDGb}4pmA)F!PQ}=X`8qC+A8*!J6LZfrQ@p z=JNbDT07aaUu+BXpI}Ya0U&8wY|(sPT~Rx^`)}aKHc5J@GFQKrxjEmTdU;FE?95^$hdY-_ReG zRxPF|q2IoF0QN2YL8MuKb^HOW<->gx<5hIixrli-^^Y;;Ur$ z4@1!Z%|3DV70^EHS6&BzH+j}H?Cf6C8CwKL#3aJ+*5p(n@?WHUG=-0#x zQC3TU{%7gf1xNy)VB(!?aaQ^5>&xJEuX!_`!2wuXzac(4!wMmc-A!%;=w`2%eaIkwT_hV02GS*Q1iJ>UO zO21_T8S4HtV1$n0oh#WpNjLwRbi`$h*Mk^3H9c}1>!BP?H1YBG=O>XPkQK0AbIuAz z0wsZ;2g#FyBDhA>A-X1`6ENVNmN-EFvHytO+-U5$&SI|6u=Y7XaXE!gXT$-~&o}Z}jDqt*{bJz2;a+Sn(r#n4IcUopOAnRG0X2d{$buitb>t zxC%ee#oI!kM8TS_L-yNx{;6=Jz;84C&%7DqC*n#Hs+0Y<34fm5udYD<5{B-7bAfsOEmNt)p8Azuw>ew>AO*FNSX;5%*ul67$E67+kLQXH71n`_e~@BYwx^uQ_a6ukY)x7c(jW!vtce>?7*D!^ZyRT$KA8^!##8Bl ze+s)sP=GXm^-_)M`e)Gh;job_mwRCrajeWSaAkz}sXz9GVCGUaP{3EBv&slBC}@)w z{i!ith}^gCai<8;aMobi{eoM;UkNV=5VPgr{aLL9c(XQ&Js}b!l4pI2UKGlUuR{D+ ztgAzS*C0KP6%KYDTKbk=D3-gz1mLgYwTIBRiuFbY>aqw6!eV`?FDw|!Sq5>SAaO@` z0R;tRkhlG@F{xmT_;10SK>~TlXr*`)u5_`T^uxW1bjE$ee=VJ(7x8z@`XQSwn&jL6 z{uO1w0+pC*aza4C@-xe(^H<~k(ck-f(p>8CSO8Gjd%(u71+9YXczQ7Tzdw|OK!9)& z4%i&1w?+VbFGYr`U}qRw`(tL<>59$B9Y3{4AkH~&rL0)E66oI&;H}9yg0LxbFqDr? zeRI+|p_>-2-XM@W1|S}O6t)01y%uP!;A4}vm;9`kfrj5A;I<9luVlPH*leSW(Zb=+ zX%(*$E$!-;7?_+L2%DpKnRxSICMCE3FJgnN9tg{SPm}59%dZ3#Don!@;-`L80R|KK zm4JayR@3-rrp+U-_#;b-8=2=^LHf*b?GQc&RBtfbg}>jQF2U}r^u58OK6#)3>+lDs zVBC^C5&XvlFn~ou9vR1PJ2p=k89pCnX@t&d*iL>0fMe)pCO(V(1E-p2OQBVl5d40< ze@P%Azik)A@Nl$oJafihyf0G)0Ibq$#Svg-NMH6huN$kv_fI{Ir12<_VMOu%tLSiq zI%t$8gFCoIY&@TIhoD@LUhiKV1Dt4!K?qD0H^0Z-HRsC+8PRHw{-hDc--D1ur)HhO z<4KPd88LqY00mb7z*6m_WS#}`%pB$h{W<<*g2ti!KVNu?r_zmmn2mq)K9z=8Xq@Ck z?l)QalqY`|m^!5l2>6lW&$GY>E#;^PZ~xcur!%CHPffZum5Vi2v+8$1pHe z7Y?xOWexW&{S~BHI4|J)=+7F$e&^7U2&k%_#U0s2TS#8P??la$nEl!V*zb=K1-4ci zQ82JGV#E^u?rP8vY)X#e1*@%?3}UrO&BK`qH%fo~eM7}S`8)P{ZB11OC$`m4)zxJ?TJAu173fVF_t z*CGx7D+I#$2P8z_HEzd}0%)q_XSgu?LjaBeHI~g$l*BamvyJqcz8EO`Fo!jGg7J)) z_$A|EQn39|zk1E^8hf+vLi*BApMD>Q+DnL5GHzug%tMsT@F4e@Ai=pj+!>81YAJTI zaBqD)F?LSAam1-ZFrg@Jn0wXI;%{W&HS=GVx@EQ19q|-eb~QPPf9xQXXd!1AX9+fh z{#?=zzFOz)7Lx=llFW-((ks>39X(?HX>D2q9{-P+S6et-hF|8%VM=9mr^yU!r4GT2 zyg_DQA@NU+{lR4pPQ3Vz0_L{~-`Rr~U1`uLJ-+iBkqSA>q^kHXGGzGZkkvB&Lu^Jn zPcG~`cJW#wr6slgMqtVprJbW#6;T_xwZNFjJHLf)fLlo{ohd+OZ}socfy}<2C1(n6 z4~skaz-`y4)*6c*(|e^JhYbO%G?&%TgQgOlY>(%J|rs|v;G)r#0WA2*f z@Se8t<_lSFQD~NP{z@2iTj*5kUbCp)#jy5HfqC=Yd#N`{TAyj;Tbnmtgkim*dtf&A zl~HM}PwzlWoZR!9ZA^mq-8I>lR2_Ba$Fs2oe&L;8TtCT$8;{)^zZj;9xi$wqUl=u^ zt7>Jzl!IFtn)T(y0LQMr{T@n$F&k}P2v1IBNpShu14)1do>uccBjbh)rN!8 z#s}Z;w7#(OeDxXgxPk!fA5Y}0;XtwQkI<}Uf4#Qcj%d&J=-AMN7N6q_FVEbc(?xGKkr~r?^myh zoeUULKJhwCW{yX5nDZ;Od)nr=S4Gqukz0KQk#aK(rjE~q@3yyPd77h9u4ayT8ys^C zH1a~7N=$C=dOf{|aKX&zCT9xBSYcXx#tU_fTcI~BnNRpUNry!lbkz65j+~_0)l-|^ z+RW+!)NCy6{&_EXc_m=T*dTcB4dDAU*XIjL*4Vn{1n+?Vq^wV!J&m5-hKpJfvu={hVm z^9uSxG5IG3FWT~Yw({roB<7XZUm-vmKSepgVszpO1kQtoKAfiHm{61 zzqQOOd6`$7TltLU<~YyPT^T`ZqC@O-Lt!|hW2R#$*&@Cm0Y`eZi`{$F);@FZ+n~oK$jG%&!O8XcoL%U4n9;(6PWQ`OCkGRK=5%CFHIKA2~KJIyIwo zYVh8-O6H0ZQyPO^Jf3$ohPq^RR%)F%P%J>a3uqVZ-@7t5{oQXoS9^ zr%-I==F9a!3PR$B=!Q-`OP9ZA`S4&#VGs0Dy1)5L4?reyvs>8Fl0)BauDn;(c~sqP zYfFB+s&v+e+Qt=KDP}!7hOTSNP35CKtCmi@rtbDpDqwk~2_yVzlgs9A#64(8x%Kn$ zPBE(aJ#gwRE5{vJ=KgtX>6NAn%RhT|ex8`T{O1%W*;c@BvD);k)npS+cqrwJc6aN1 z7(kxfZc^2A($MX{4?5ISUvA&;s~K<(~;BS)Sz-G9C)Hs8#tc;@a&Kg!j*O{Bx`wQGOdI;^Io zA^cvK<+)Dnv+tHur7CYPrNW&i+-&@2^s$Oftg9OW&et z>&U%-UpDKki~hZwhXAZrn>U`alWxg9iVC(VR9dEZ^%JnKN8pj+m8$PNi)+jalq{bt zlxhKTt8eEF*2-2I*_j4i!oI=;%}{G)mf!06SK){{5<0%CWFKzWWR|ByQ@k|kW56qYrmWL zhc5sAO6}g3otHf2&sz?6k#CDG0ll!^l+gYvls`Rm68mG3YHkH7%^Xaxxp5-$v-mB#s_L#84#p+E4^qg3^@V{H%J!JJ? zXCDtbat!+Dg18Q7`(PDude14fyH`e~RxQ7ivwDGJW9J#Cl!*V;U+5_soH+58AN}Cc z7ei$dEpFK_IkuBbU|?%Zilg?a@Y~<#+rG&<{7p5pWbf_QNgA&!_qDj#dzOTS-+LCm z|B-~_OQpbEkVzJX?qAo{%!zX`KpF+;^+g+KhYj-W7BEaaK;wnwIZsbzmg@Zy7}Z|0 zAc9HG-E^6}9CMVfdzKair zu4>SemA-ZTA?UJ=vv*}&QJew|`ep_kIjR_}C^6}!>ihSTU#g#l@6T8u|HW2T(l^F; z$B}_U=dm9&1jIQcD}U+Enkm11;>`(st7yeiDF5dFYY>W0u(N!A45!kh$-{+z)%#nRNDq*1JV-E=mL4>5x3o z4J1q4y49MIHV@h=lA7~3CqH4Pu)4s=aUJhkb*XqY*(IkP2IU%=sB|I0&@iY+hcx5G zRIv%&-;Wox*Th-4X!(w4MC0W@Jftp=*fIF@x4q|{_IS4Egil`}-&r?UMeOa-_R8YC;^C>bQx@~ASubS< zAsTWKxfxIXkvt65r*hf@_sqw+0#y-Z1oN@B?N%w4ua1 z>}XF-Y_d(c!B;=S^-n1BGkwNO`5HCGST4HmtLPI-xi#?Ntmos_=|m+0sB2)Tb4^$> z%+kGV4s&A*u3e8cP3vxsn^Agu%EP4-+q2VcC_1?>R}Xes?MKfpJ-7Yil>PinR`$Pw z8c^0tBxD*g?yQ0lW1Bcm0$GF+2Nu%``a&+S@N{oE*rPhtBPX~dH}%p&snala`qj+g zWk@MM*Y5t&9P5;N-2*%$NH)Ly6=tdK3w8Z^)$z&jmAP6S-*QslbdJt)UUB0=`&Yj$ z>sl$#%&8|yV{KCo1g?U!fxgblYXM!ayDbYDDCu>S_a8qajiz*ynk>IJwK+ea(iKdv z8yXLOoOUc@4L<1T0aJBPcQKoBKSyB%=`&8RAOILM!Fsnp(7J?4pF#da_yNasUyJsiMl-53x`@uluhpB9}vr(nxEjZ!!JmWA42V8&jL zO}eH9W1I8k!oDA=(_mC~cn-T&v(jT?!-84+7mvoes=~g@NR%xEPlEx49}=n$E`S)* zk71Vd88YKc7B7Nv)6K>89`{Omx?2izudcHW*s$?x%GQMv1Jqq!>U~9Qv2*cvn5u1Z ze4O2?oyshOx$iGe0Pay=Qu8nPH~QpP>S=B}2OkE#e8+d;{%4~t*LzI4CAHXNSVOW; zsKl*xi+!cFzO#loBzqTuY4mtR)J>kbOLTNWQ_YeEFMmb+MI|~zhmzZu)Q@xEo#Y4tPI=yUabs?${WdbNqRW}NfhyasK^*!kG8YF z#HBJ1jZ4-&%mJCYsUA+3G=7$LyT$zJM=BLbO>?U{Q8!PrKbnfS%m70fkb^+WGpC54ipHst(8EqB*cFuP;R z!He5AC~^~It`2v8v2?;(0Gl<&fA4n&<~^S9Grn-o^&4OJ*&f=AjPut|rpfU3@P zd%8+%wu8e$JHGWBeCHBp@!;|6k*zZ$ZRyf^ZcDjr<7Y${YBdmqZhwjuAaN`j@ML^UP~M^zStbd>1pj&hbsR`qk0 zP$e}E`t^35MMVsZEn=PB-ZSoHqQf(MCB=o_*{%dqIA+HkJ>d7;zJ(PM%q%wkLMcCZ` zcntHp3lnxb0uy?BD5KI1P8Ye&{bjwrhiyZ$Uuk%UVF&LLt#vR-nSe$rtrfmB$n%W% z{%kRC!9^Q40LHtT3Zk-x%zejx(E3z8|DJ(jB?6pU5N1~ zO!g@~*BSQ3NhwXeE%ar*0Xk=nZU|41pHgK5o&*OP17#1{FP2Ge-%Cm8Cn_xM#X&Iw zkrB>sGW)SZ2nF|{;wu(PifUM;KtWNVUPE*n*wvB&Tuj8?) zp~JWILT8CIhc<+Toyt3jIf%$1(3KWyH=^Fl*Q_Fi6Mp~&#f!|cDkz9)Tpy1uF?v4H z)6LBO^_)AU9vyig#R|I2Jle{w{S4+_1yytX%%Y#~Y${n511QUwoNW?2ESa4XzBo-v z8`c)8(3X6T51dnZ_aAZjYCpwWxOaC20EIT}M4X!V3GR;hW-&htF&(KU>drv&ZX-kuEZw^s!{h9;7Tv3R&|%{Y zf_bV><@_wCu*QY;E_IEal}^;O!EI(Q>X&*bHyk=wdb-o&i__&?qjitA&Mmp>CCCUq}1KP~`mH6P}&Cz%auxcgmI zp#+RD>$8h~URgS9QNrp)&ACZmuU$WX#UcQbLbX{7qg~7!iz?b)+lN$YW;q^|dSM?{ zxhTuMM?Q_-WP!5XGP?F|0Jo>-_?PF1#8;6s9aZOcYOZ{PA>D0g6$hcvZ7&sw;R6o3jp;G=5dHbpz z2*^Em`tCY~CLz`kQttA~KCn_fYiro^g@LK&afennrc@-pZVa|ukYyiMv#|8N4z;qO zsIo2m$dM&T7_vyi!I~Ga2`E35SRhxUZQ?pgv>gn7ea1LJ$+z(Aa(#2ke0{z1^94zv zlm{Sl@<76yJC(Rtz6NQbM|E!S3*UmG1~r>Y+c&nje?F}d3(m>i35`L|a}lY;x#nF@ zGLssr)mqkGSnyyYn22XXtLOCKv49$g8D4>yTUNOzMLa{zr*d-E$}qo$>Mk}72YVK- zbO~$NQr?zse|~wlOI<@}xn@So!ax_T#*hk^*WfaEG>)ru2}{-}#bgjCNYaWwGY7Np zj-dn^8U446VQ;F4VoJb^B(mRf#?g@*uME*BiZ^e4G%6pp)Xs@AxDMUH@1nS0QpH8< zuXjJ#Tu0uER_DiIPCA3G-?(dv^t!gseG!labbC4dSgJPYS!kkPbJQMTysvv0Syp^!o#Z9lTnkh_kBOVMpY>K7&T>dY3`)bwrDm@n|n zQakxB2sT&n^DKJ0>wC;Q>eD>kf8GwSl^xmev&lYFGa4B*h`wvoJ=DmJM}rMV1g?=$P4JQTMp$Y@)ioXNA`5u0jX`A(+)w78N9>*@r97qPDo#^(-8$ z^M9Sac|6qX`#*k&4ymKm8H%EiiY%!R8LiSrE3%e^6fxPCq1~Z`9J^^zbh2d0zMU5P zR`zwu8bX#aW*GCm?!j12y?=dvfAn}%v%T*7zV7S3uIKf9UN=Vp&PVUJp_w@=KM@|O zr0I~|i9Kw>a4JyiG@WxD85|uO{b4h!?P$hx*NDyAAxzc_zU3s~u z>LtQ4s>7Z7>-$8mjeWUmm+?oaRo-e+2yxkxwIh#YAE z$3tteuk-zeHvke;qmr zB|m;f*$0C1NN?g?VzR;eou$(2%U#bRG&*J9-WCozz}5~=Q%5rRSc4)5BofbGQLOz0_c1;#eO*0RxHKD_`*d>7NM$_Gw0pr^& zN6TG;$9Isg_7fi+q1Ue!BHP*W6wt!Ql}SSa5#Ap-PE2udUe~d1nLSKUxZ>^rX-bLs zwh7_^_}AQ{d6uPB;gx%H-N<7`0pZ;;x#%vXE`{bl7CGRYFjq|eB|W*j?p(eA?r2IX zJvy=MA5Y2jD1aN%(}&9xdtrk50RK>#z|x1Yy$HNFmLYTLf4|}Q$uGQ0l;`3TyOws> z(Rvd*4-SN#yL?_G7{!R+;k2-I>D=1SjURhw4#9AYzovc~e>nm~QOkRCRVFBC;C29#Vw*=zvhEf!5El^3`4E9X;(Qzlpe zcZt?6hJA^S#e`fMy51lo5FJwJ)&w|$V*eomm~I23zOJ?k66Yg;8{q$Gw8)4V+DU~$ zzDm$gwu~;{OF}t336kfSOQpfpYq>IHsKzIiNhh&=5m`wgqm|`^{sAf4r4`P3K7$u^ zR~zI7DHNp2PNMumPf2#bTYONo?0rb8tfkdGRSYp-?D$}j&PZE;!#xw-#E00j!3Rzg zvfOD{RA#hr-KHKC*CZa4Q&sMKNk7pII~+)e$V^ zv6-wHYuuDJU`_8(E};dkUG&MMztT&L-1jzGZ>kLPOyuzbe(mf1ti=*1w)$^8q%zIh zV4B7px)$clXNbM7r98%Z<|Rfs16KLXcj-{)SmqvOOW*Rn1LzP&x-bvGuKW+qM*fy7v!?KkC74rvZ9kA z2WkLZvM1i#6`{dV&Vadz6;4-11cFI zz?;^XL(|C`eG|thu)JZQlpH!&;t!ze89qR>*yu>I)%9? zve=gdN>Qfqd3>i;XGu_V<|S9z@}Pkdr%o(+Pybzsp>mqfj4#lxaymA}Q~rTk+eqwn zN!^Z^jt@KDGz5@Xh`DP507*)(&5BJ>QRbp0 zM(wuDBjV|_-j#&0{ir@qN-KAXWn=^&RR#4>P>f1B-!Sn6>!QsJWQLOs0=(mT*ZyT^ z)P3UcwBGb~-y=N;cmgPm{t`Uwi^i>A^7t*ICGOEt?%x-DZM-kVPH|Iv4BcTIk0~{g za~to^h)a-FvE?ZENR%;bCx({$SZ^jcz9iUw(r<|(j1JqM zTWM-JmOgPVaY*{Q<<$8#>_*|HNCNdyi1OBnmo1?x;eEW zTv+h)L)u(6l0THlBAwQ$l{EV!jPS#H$-Q4_Qjm+~*d8GAoqNTug|{tl z7ZVctg-1HFQTnJF>k-V8bMt@SEmxY8Z-&-4Gk-stPnlJG40>aee(+hBfCj%+=tKIOChEvETIJ!Y??<8MUxg-H*sbYu(vJIg?gC znG0tzPG5FT`JZJZ==Eu9d+g$#wgGeyZydJxB zdH%cIJ^rki$Mgro_|X!Kvgd8utHx@?_DXyHioX_N?V10TKYM@;NSNY7rOts!A^Ip3 z5js2n8-HTfOPy~4OyT<1()m=~#RjGk^2`a_?JGE_ab;GW8Rms0gvJ zXVFjCwAa2vpt*EXVD-EP@s^tt8uzJHBj)Zm z4%gT(3qPgfAd7T3;55<#Yx#`%PSw}zk6KBFBB>gX%0ejZ#v&V;_HODH`DS!UG)65&YrU?sA7<7FlRc^ ztpCK9pfi9D{_wNlq0m`(@PBI>WLO9TkO&Hy6P0W{s1o(YwvBm_rThP_ZBQS4-qb|$08`E( z-N4!~hZx5&p`Up}s63(xJet7CD>-99xmQ@|vrM|Fh55;8hNc;~l)h|}AKx=`Pf%b> z|Ef|BoezJb72tXMl6eA4uVAGv1kM(p_&idn9w(;sg|ix9s2oamd)w)s@uz_;3O@Fv zW+T%KJS88C%#3g9OyU!ENBO_=OH=rCwNt&mg<%um4(R~O82W0dNuvNbLE_wh|qV$36any#R{ysQzu5NcDS zP5L0^8vi#`a1L0g?pd(SnSAIw`(S+6O8idDn9<@hxvBHquUl^J5?ug!-T*#Sb67m) zG!YoGm*K&_VQQoRpdk$fIqxyMm zk$9YviHs|z2aiA}glwJ#qNrnIW6N2GV*1lRK8s-~YAGvoor|QtR{?W0t{sLdix z0zNW+w|7|4N3Kf{&_)vCLAnM6^fZ~ip(RD`p0YZqot#0)k)ROEh0%wJihF!T`?JU` z^8Wz8 zdbX|_uv$QruJ2IC(nkI|fcS)h_I>6CI$*Zuw9%BD7~tz9O9CBU5;)459vS^~;*Fjr z{L5if4efdADH;5#MKnP7Q#BsZ{}Dd48>;_(wmVXhIiK&lA8ptDJm9Ra{NLJ_g3zUs zz25i56s-eWb> z@d!G(mS9CLHFM^L#-sK`LbvkT?biF0|j=GNW#a$WkGhaa&D-IQk(?=W3#ksy4P>M9qVm_I~Pojk=vTCx*uSWHG z$sj{S{qvGjTc*0sp?z;}1pWQ>W}DjTbLKHSBK2*h>tLEPBHV#-lG*t0`b&u8n{n|5 z`~5G7V1f9gdReI(tIE>@N$bT`xs; zPayIzB4jZ*8%Okqd?iO8El6wK&#!{`7a1vN8Hy#$4v?6}ou{XAcfR$y`d{sgZSb=1`G;#Jx-p z{BWsU9Poe=bu@Bbmtj)MdS1opn}j=D68$1%Nut*Rk==rYj%wlb5q0`$uFIB)S&!;X zuL5?}*9&z-9uI4{KfGDp;jrCSBYaI__?de44Ib<^bUF~n2vvyae0c8x+jWO-yhlh| z`dF)HFIst;=){7)SBsD5x-kX|#WD%m7cu6{f|=n8_Eif!4D=rp^87fwlOiEf-q_Du zH44UDqvr@e*+kR(golDr8a`^5anYm08N7PePo%V|RAX1IPF(2_;@Z+qNEr}Y4b+#N zUv1CYH?^(P{C%Ch;Z*?GZ9pv}l|&bPDVy-G$Wo=Tf4RePm(3bODl3!(C&zQ>8NMS3 zx6`!U!{;DUZy4U1J>$QpR>@rkQ!#ZPLZCe=V`6_lIFl?(3Mnr<44T-7Hqudz0m?9T z)%7NX-Nea(jDhT31DV5HEW?RVr0mqW*I54CwkpugqB=UV-7z5Amx664Y z;Y$wy_rj>`FiHSP-@zE)9GCxc3vBrfW|y1j%f#1`_z+Ew{%Aq?Zpb7h z+(8nuhesbxNyu1OZ*Dd`z0l@9FUjmaX88{)quD@_sTOTsJl|J#UR{mKNcA{4`JRW5 zy%#G>3zz#y2TVF-*yRMa2?`H35Eqnhb_Hiwg^bh@YE9ZuBv#CcxN7ImJ4isaiBWi? zB;`CzNFP!Q-ofOawl#D6vEgmL~oWLN%r>=!~2d0ZEY zA=o0-1wF0pRan>=O9a{`U$%3>$Kt00i)Wi3qua4&!VL5m5Hr{4;!1bHz019*xQcWa zILt#hZ}73typ_|TsKZ+Y;+6&!)wFDlP)~KpaJBtyY?7Q~P;bTv!jR``e!vFR03#ru zVqbmx<6%l|S1AdV*uwCWl$Q-E;dJmO8}0)HznJOtH3XQ>Zg86YThB;N*D=K z=FzcTT<{zii1Dbjt7)ro$oCy`W2i6}d<5waY}*be?9U7TFua)#tmK!jQ0&&^5l9%{ z+W-U&^IlY&Lq%<_7(cx=*6+KajCq8V?=xDdUqMud+pvh49Fp&6j0I9U2>W0XRv^HF$bY_sJhm>p4;Kj?$#CFQQvl<*2RbG-$PiVXg9dA z+tAhK4`8}W)&OBP7AX`g1OVQ*{4nt?u{<`jkdDL=KzCp4W6Qh!j);X4qF*BE2`R?X zC%lW3ldTA^=xKCM4cZ}fEXAzzQ3QXrfS|uEs`VJiyG7c@Fgxe&A zgnP6DQjRB(7!UU*8Tho-<3##of6SOe?<68d4mgxM^}v?x!$d|-AXe`#O1oc1%D@0@ z9Gva?L;JKd369;uTn&Q2x@f;{CvmmW>>|{hK&PZ>!^zrh)Y1l3l7YOCz*E_HkrQ8_ zuG-TR1~qHFZTs^AhP3YytkoO~_^GEPJ8*0%6W2v_WGNcX?6H^?g#& zS>M-Lgm;DnXV=u5bcO5Y#`z;E;UnB`vcLcKx{ln5% zZdTK}#UrS*mq+*M>GUd3M+Mj)bKS%Lz&g+yrIS%+RiimA{mgyJx#_=MK`O4_LUBP# zVlD&?HP=ILEI8d}nLX0K!PhM|kmQ+ka3IDSW)C zs|GA)qujlzO*xg%0m^WK<@|KU^$vk_T9=9#l2W%^p(T7j>fh%)T*4J4(bZXt+Z82>6KP{VzVIiV1r-cPiJYu(#K+pHVWtV!oFHSM z$aft|gJ-rZBY&hyL%hcz+oeo+pAA_}XZc+~`dsPO*J1H|;{|1T;tzaWj<DleYGK}o_CT=%v|dNO51*nfeDK(oZ=FBWjaLQwtna#IX26H&s0!b zoEC0g|3s+EDNDb@5_5xYnW)E~)$%PJX&uHRGjeFkPFeM}mH4ub3pss22DOWIt+b_n zC0!z;^fz)WRG4%y870QoN$^G&I8loNS88r~?i?pK-caB;tRG_cElqhqCLzYhB|^9L z@Hl*VHqX8bg>QWF0dlBcJqn+w!)?V?_m)18v$2kFjrIN}OIWE%?I$_jF_0m)Fo+nr%Qe>V>Bv#`*1(O@>$?%bNSr6BlYxnwOMn z&XqVaxen*r?h^>IahDfF=YG;s?ZRsH!=Ae9!gqwYPN3$@-SIA>CMG^vw>vq|R)0Tow@@ zCzs*cu*N}|ggy}$vMsBnspFIwFedaZN8dE52W99ouCkqSr1M(N1){dHl#Hpi4evG^ zR?XQi=m_sGt_m03{J3}q2v%Y#B_oiHV{CeAw=G03oTx-AF!yjxBjz5Q|k;I36 z<2$koX}$`FX-&ebPqp@^{5gyge&loyhrRd3b#E|6L^y^=i5H74*%NzT>4DNvL99i? zJ)>-~Y%gJ=tarDZ(IErcWSdWNrHL<>Eg%d%#M-pKm51bNWQtuL4w*@jE&DT)umiHb zCR=RARXfMQ_6{*Z9EmhBYR1Nkp_Fjj&MZ|biJXRCHvu%gvb7WlTaZ+@Sy?nHuN zT!Bg&Qc%syp-+x$PQK^|T(+tDSenG6vZvh=Ndw@AEEd7ZqHY{N0$A2b?(L&VL@f2H zD5nrtccnp-H3WZJCSf+|5uR0QL67>#{y0y^_Pzm{$PI}*-UdA=Gtj%RC-!-H=#jk# zF%e=dzX!j1YU3(vym98alH|cDw{3%HmByTF~815sUxcd*fy&M?O|_eG z9}n=_5M>P_39gRD0+7#UgkSU*1l+Zm^gFc+GL=(y^oSfU@$<&BC;uILk=qP}P=3 zt~HfXKQf1^Tw%ObSytjvZF6-P|>&+LO)d$P5qp%;69++8XVGVkCP* z7UoSR)1Cs=MCE%}wcCjR2;J_7rx|i-Ii7Hc^8Fn*0k@0mRidH1rQ1M`m226D6@m$= z%3PJh2ZtnL%P#b+2;d!)u{0{*+wQ}j@YV4m&z_=->#w(54o}H8kq|B}A6M{9qC>`8 zX0yw)t!tMIL8@EK@&Pbo4fM77b|)Odl$-o(Pd~(ZAdjMABv^k?An`Y+_B0>DxF^u2 zj|fuMRZfI`6)xn{{+a`1eVdHc7h8C@zmSN{0RrNd!r$Ucv;J%pIom#rB4SYviKt1f zP$GajmjRL)uuUqsK)FrwY4`o+_;B!rZl};NA$4fCDyFWhslcekNvxxDcfxO2G7dvR z9ADbnXTPVbc3w4cB?eqSI&3aJoFP(9nrN_|?Ut9z%C5_tV?GX;)kQMP76bFg{l`6_ z-+HVlXze(lUE&NJPECviS+(AvqKU3NKzvl%N=1x+%*Z8*L+?nANxVZ#I5vlUXtUpk zcEmB?VrgG~W8zUc<@Q=Nj;}du@#i}Z>i3fU;+-MeUF-02Tq$SJSYk6a$6W8Z!=%IG zq1ASn&n72m1^+ncxMH;p+bJGnp|^+C9v0l7h*)0%k*fdg_F?`bwwzja%{Jm{{2txU zrCiKJ)r5xaK`2;_B<7Se{E0ZVzY6)sEumgwv+!P5{{P%?f{YE?8$kzelUOfw zJM_(shDtv;tFEXBd3q%pDtJQ23d|ls`_Jw>AlT?H9Z*|S5K@lY%Jm-6T%DsQFHj^L zL3jBo1IIn0-p~N(vAE@Ff{f8YO(nu9JOmhVe(K6;inW%Co~4NovlY|a@H<=V+lp96 zGhgmDQWvOa|AJ~?sAY&h?q%@pdhKuF<=PWrwku8Y9^nwqAZ}@^u815ADPL}=;ccFB zf^$zq^CQ$}K&kdE6(Y#EBSi)LpH&|y+6-aFGfQ3D-;R7c_|WQk7BCjYWlH%?Frkt6 zKPrtjX~mD9oWTNc#(U(GGZDpx4p9s%K-pBBEUfFtOq=A@6LBtj1B`&sc1s5g-wCjjcA(X>T z00%nJPn-A+ZX7K+CVBCjLO?l9m}a7RbHpcYcbk`N&lhgIeTw3~63lh)kzT2<18?(^ z9`EA5o{bgr9M;8Zgv2tV#O>>aIA&5S_dHb9-udJ$pMNdlNJ=Al=6_AnJBj98bDqLgU)chkc1hAd~k(LFh#}>=5 z^0$Dm29hjx3`LA;6tDj={7>DRx9L}vNhVXutZUYi}`{qaTe zMgf5dxk{Gl1cOjS@PsJoxc~cmb;f%fb6m@ILjg2eB9u^jzMV`dJ+n9ogjv^>>zjo# zlrH)>m6Z?F0n*e9V)s|5_S)R>ULI9;pvOf6rQ>4jAtev-LE})F!rcJ_0PqE*H**Xr z7@yo}R7`v1aL+h9GR8+WFJS+}NTrU=jAk+0{)W~>D1Ze`!lFiuG~AY9l-xB<@@sd9EAx!lWC!X4$*Kp>1Cu%`gl7>$ZA*Y z_I31dX_rTodjYG`z%IEL2=HGQ1@fqV9S>|@*`4W$%;&BJAO*|_713bk112=Q;%+I_JmM->Q0 z=Z(rq4&G48q6rhnjXxPs`wWgW(#$v4VhP(eXFQfq}| zxCT{u8)1CBm1+fv6O*I)0m;-_*}ny710F^#uGU+PKdDc?7;rB0NidT#Dj^F{!gmRK zyAn4Clq(D=bPSV`&`54T?Qo&bu)-(JhC+*FsBe+x#;$Qy9tB@h-)+K;C8WTvb;WOz zWCzlK-@DC2yzNksfUwcmS{MpN85^18)2l#rDYrbTcJQ7~QO6-Ci2Z<#-1R;w8gRu?6bO;MngBnrHAdzi_$5b zt&s#0+;FpBkcW!=fh&7a?#eCldDnEFJDu_~`I&|r*k2qCI;p8=bEY(Shk2|KBoQG6 z=ewEm)W-@eEKGx4&bJgE_@)f^!98v&uUK2>iTU6c{!1ia{?OD9FZF@Yd3~_NK9GmM zX7idaT>4=<_WA7QieUJB&85?DAbQ<#+c^Vq%f^H zIRh0hYxJG^f@O^I>TDxyX7)Y{D;LkTDQJuFZSKzj?R$ONdCN*6X?<4eJhJBOECVM6 zVdSNu3W+b&3HnfwItw*@n;n2KDdeL%F92SBLVvh;uVEz zEI2zjw#BIM<#zPe`Q@gkR2G)JrWZK3{m<0TH@HB`xiQYIfVp8ddb=HB{sUy38^QkG zZP&>=Z1|AP>qiGGdWP2;?x-|ikwY6*FpkZcq@q?0=D?NtHW{1R`WK|oup~A_Ty|}5 zHPrq;RR=*d1Sg$>YLN|ZKV7W$B%bI(?u$jewh+t|YXOJrqh`w-+7_1X?}CQ=S(744eH(VdDEc#r#k^rZIWByG^&+WNcj1k+BP zKcn#HwsPr0dp1TjM<)rAl*we*NDP_2OWO^Pnq|dGyoabo1rpidu?0R*s# zJQ<0tLG6K1JC7PSk$9`#EODzm4gOr9`gA1lVQUlZ@kpbD=|r_K5>QBXpXn6PK210< zVnzn3S8!!{C@UHNRcz%qhycj=SBD!{TTX5a`tthSAFm`b@qpld(o;`EG1n(sy9+Xt zY9VSH3be}Q+q{rEDSiK6NKopzzIMD+@A}6B{>thQnH|P^$Z@VrLtWoG%R}3$PN~Pd z_o1pKAIiWn6LT(SCHk!vDj6H03M@^bguRyyty$Y>FKK%YSY04W5 z5ql%NR~OTdN`U?mH$|jNY^+$bMb--`a$LN71K2fC=PG9N+8Z$AtrybH{{|z^{X}_r z%+Q%_gvlqIsqs(u#|~WXp6P?NtRE1hWJfJ3yfFImR3xg$q#K>+Mb?9=lc#N$1=^i( zE$gCAG!$K!43afDn-`q&eE^zw?8w>gep{4Z38cBvnH_l7jKv}+gj`DE1_qS_)-L8T zu{|#jg>jK3USzhf(|rv16j`=1<9yhlh5``Ydo4IWORDj!t@@hH?4~dMnf1TbJDX+5I9FsVXfa zgl5(RA!V+%L__4{0DCUDF#+th!Z6fq?Hpv}6xxRgN(gGWsPf!LwTbqyMa7AStNdGu^>B?$+88BX?27FoJMLk67L++(b{iYc>7_Te? zY!?9PL_5nhTM|KGdlr>P4~P@AKQrplvnFng3_#EL|d*u*3Z9)~^LRv_xsF=jga`!g8!D zr4bv@yE$N3!FPE@&fhr{dc6m@Awl zLmoM)YiupqR>TZ!-9PeH%q85jHK5xTT0;LaHDj;(Z_)ba7yeGiS<-5Vf&9sX5Pja0 z88uFanLp(4o1FT_m(}|NP0;X1*Ez@*iUIVkEQ6J$k@t(TZQW?P%rtWw4{NQop9B zT6o{J1Pb)_Uc1xnJJTJCIk5lC)Q_TlLo-w-j`>_yTf68d3Cm0ZGjG2!ttRgPJV2_n zRMg6@IZ@;M&B=W84|EZ5F#!+^E=wl;Is*V)SZ)K~x5IU5PW z1H{YKEZBcqscx_4S_|0Xk0zjjm+v85UbeJg{{l^I(q^Ig*MZH3N^=L=$F=knNG$J| zC40>fET*h!u^RNRpX;gS9=EeEfJ9RlJ|7%%at>$#&-&{(zWB)^i6*Y)z+4fPd!e|H z^{*WV*WAH_U=Fph&Rg!KA#OkJaz5`}QxZ+2>FS=C@4np!S5*9(0qvLn1Gz93ZQgmy z^;(*xc6OQ=6FD2M%)S6;%r^(~ z)S<=(GcDSVeGuh?#*hX=If@io;XuV_Z(E)R8guYO#+);@(CbI=-GB3F%z=Xd-MZ_h z%v_d@gj^_FmjG;Zuo-xpZ-A3o4V<(dXhsv{ynMq(=q|@L-Jq=9P7DoYO+cB=Hr3qU ze`K?K+<>99J&ER?`{Mj~hies7aF_LX8cbXWvv2jV0d+>zmRHdyp&z-+T3m^Y4dlj> zufvgwMQfG&#DjLSNRIg2PH_j!_5~flXv~*cPcPYYh;-Y1jy+U_yWL|Bm&5D9&zrSn zc8sG2FsK>+vr3<{@SXZXIlPqSf&o=m)gdQeqW?s8yWPedxl&;-N4GT#gArO(i})tKzhrY{hHB56dFGF0=WDVfUN3Z^H^;}CF*6DvfsnVfuWmCH5i9B zV!iePgER(Jy7=D8LLYL>KB01(@ByVRhn8XJ6;?i=!f5eHygPBcoxCB3pF%4bD7bLF zQuoAs4?2Bj=U#-npro};^qXeA!Ny3pDCR!k*e?F7?b=*{c3vyWEdZGz<6Z;QZaeyZ zF6#c*wY}Utyb(3gN(L%`q|BzD^lLj8Z|^2Qw_z&7H%vvvT&TC)a)E$ZyF&1x(Htbi zQ2&3mYkOsv*|wHW$a3QDOQW^PPfo6RySH6y)y_Mc&*L}Y0zb6#6^nGW9sWbKYcJoj z!j13RA1)7hCldKW>&j9Ylg1J)Pv>{id7=1l&1Sog%>&s3b62E`HHf*odFB|Uh;6ZX-oz{`VhZBwjNCwsR(kLXV4lB=C zDQneg(!u5v_^S2%j!|d4^V@nJIX<^ov}lpu)l=nFJ4mI+SgRCw9Hr4aySr_89=frK z2(fS;QeowJsPbV|MYQPjzoKrjHi&3&-i#GJ!g$S?w}LwJ+MEx4cX6lZyse4RX>?;* z)O3*LxCqv!=TS#ThqjuU$J*VFRW2>@MygJfq33HAtlwWM55Ky8{rb+XE|ab2l3rI- zXz(joM`XMal9Vj-_4C`hdw28D@Ni60Qq_ns3+Kcp6+I2XDo)bgX0YY_>J_B@dM6B+ z!#hT=5+$jHos6s6%I^|o)I-VtI1(^B8W@;{16$I`sjR$bifIPy9wYEy`^OS;oW z8JUkKPo8WVx5nX`H{RN&`#^uI`?fzfZP~JIC@>E{6g=@_XMx>N_(bF%QNQ=xpgukx z#oP1Z#Iu935A5%xKh2Jk?YW=+Y%*$Jk577pa`E*&j%o8poXxhKpE;)QTxi5x_Kwpd z{%)|uRD${Hyb)){RS=0pXzgiq{`~o!m#a2~$XSNR80570`ucurO2gUGCtaMzM`Lz7 zk5$NXx=@BXFsoO0?_BCUG4?SbA>j(Szd8BZm%aSkF5Ytr$ZxX>S=Nwz)x0^~Ch^~! zzc-Ezwl(*$%gD(5{qW)Eom;LsI$pZ;qNBY%u(tONx1jPPJIT^GR*r|N?JFG)xcZoE z|LSMgE*MGm^KKWVsD&peXb(&(gk|qP|vxyNaE(Crm@CY8h$Y z+7tgij8psVx5cVz`}dqyulkd8hNWQpfqA3-EXVQnPo~K~h6|q59!K9MxlX+)m&o;% zpZO{>`l-D!maX#q+<6+66r-16Tk}d&ieKi!E$DIhu36-@aWO{GdF*wg#8;sHcDQmS`dIMxLdVf>W>7T4@w1GEA69@DptTyO}rm9ce6TVM=??1+4(t51P11mP@$Gx3GriOdBb5iy`!&3IG z^HFdAlb^LfcIbYl(IOUy0$jVO*tlO#yST%V$)VkOUAUnT9TsGJT~UkjqM}-YQ|4LL z_c->Esrf*@ViPa&4w7#lt1&!>+rSc2*te0;2* z<%paN5CP)z?(^#u|GjmqynNl>y?dK7siS?p77C9kPEDy+{iqLrTH`7{yN)9NWZS-T zDM9ODhuZcxb$7aa&Sh!a?hst3PEfuIv;DV^&-nl;gPm)dEL$?WYaWG!m{TZ}vl<$P z-h$ZVNOeN`u~ky~nXfhD^tG+5;xnFqY0j`a_)O_o4{$LPqztm_YU2!GIW6g%mO6V6 z3=5PGuWQM7Qs@h}(svQ`wlce6vaaN=gABG)ZE2v~u*uqJvA*$iEy3Q4SU)+8# zC69f+#T)sxO;zCgd;f#3p@oZ@2F3n(e=~KtVC(yrh0P|kkCu7Dp=WhsO2HHeFf!xaDmiherBQKkoWyXO(MCSEAvFu3pIyE1>Jj(%wI!p zPSefI|9RZXVd|&y!NdR37RGteaZF{{XVxNS&V*btd(vC$ts04!j@4GESIQsj$Z{gv zhAhqVT0Pu#jCH$X{)Z3s$yXb%&?jkmn+LZi8KhZ%*wIxvF*!lOIngJxGOFF~*su9q z9;#rpJsr1e;-53GuJkvh?Q|Hr=;`H^1Seqcr{|vRY;36^OXYpSyUhNf$768%vor=`B@3lB9cIB~GC5_~*)nlb09|2h4#)phqeZwDppI+@BcIi9n4#cZ&gvc$Hl+KKHPqV+?sI@qjTqiJ*Rv0wN2#1 zd{OS+zMXbVu$_yayMSOF_Bl9o{w6*JTQCJqLq(_2=;x7_aG<~DmYiivD*r0eM_nWM;LfyVpNG53@hzKj zx3V;0z7IZGsp&Atjcv=bTa@pt#3!>l&&KfkFRIKDr2ui=gV E2P0(XWdHyG literal 0 HcmV?d00001 diff --git a/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala b/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala index 0c42fc1..9f5f0a0 100644 --- a/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala +++ b/src/main/scala/za/co/absa/standardization/schema/MetadataKeys.scala @@ -42,10 +42,6 @@ object MetadataKeys { val StrictParsing = "strict_parsing" // For nonstandard data inputs like the Mainframe's century pattern val IsNonStandard = "is_non_standard" - // For allowing separate infinity patterns - val PlusInfinityPattern = "plus_infinity_pattern" - val MinusInfinityPattern = "minus_infinity_pattern" - } object MetadataValues { diff --git a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala index eaf5497..36113a0 100644 --- a/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala +++ b/src/main/scala/za/co/absa/standardization/stages/InfinitySupport.scala @@ -16,101 +16,30 @@ package za.co.absa.standardization.stages -import org.apache.spark.sql.functions.{lit, when} import org.apache.spark.sql.Column -import org.apache.spark.sql.types.{DataType, DateType, TimestampType} -import za.co.absa.standardization.types.parsers.DateTimeParser -import za.co.absa.standardization.time.{DateTimePattern, InfinityConfig} - -import java.sql.{Date, Timestamp} -import java.text.SimpleDateFormat -import java.util.Locale -import scala.util.Try - - +import org.apache.spark.sql.functions.{lit, when} +import org.apache.spark.sql.types.DataType trait InfinitySupport { protected def infMinusSymbol: Option[String] - protected def infMinusValue: Option[String] - protected def infPlusSymbol: Option[String] - protected def infPlusValue: Option[String] - protected def infMinusPattern: Option[String] - protected def infPlusPattern: Option[String] + protected def canParseInfValue(value: String): Boolean protected val origType: DataType - protected val targetType: DataType def replaceInfinitySymbols(column: Column): Column = { - targetType match { - case DateType => - val defaultDatePattern = "yyyy-MM-dd" - val minusDate = infMinusValue.flatMap { value => - infMinusSymbol.map { symbol => - when( - column === lit(symbol).cast(origType), - lit(parseInfinityValue(value, infMinusPattern.getOrElse(defaultDatePattern)).getTime) - .cast(TimestampType) - .cast(DateType) - ) - } - }.getOrElse(column) - - infPlusValue.flatMap { value => - infPlusSymbol.map { symbol => - when( - minusDate === lit(symbol).cast(origType), - lit(parseInfinityValue(value, infPlusPattern.getOrElse(defaultDatePattern)).getTime) - .cast(TimestampType) - .cast(DateType) - ).otherwise(minusDate) - } - }.getOrElse(minusDate) - - case TimestampType => - val defaultTimestampPattern = "yyyy-MM-dd HH:mm:ss" - val minusTimestamp = infMinusValue.flatMap { value => - infMinusSymbol.map { symbol => - when( - column === lit(symbol).cast(origType), - lit(parseInfinityValue(value, infMinusPattern.getOrElse(defaultTimestampPattern)).getTime) - .cast(TimestampType) - ) - } - }.getOrElse(column) - - infPlusValue.flatMap { value => - infPlusSymbol.map { symbol => - when( - minusTimestamp === lit(symbol).cast(origType), - lit(parseInfinityValue(value, infPlusPattern.getOrElse(defaultTimestampPattern)).getTime) - .cast(TimestampType) - ).otherwise(minusTimestamp) - } - }.getOrElse(minusTimestamp) - - case _ => - val columnWithNegativeInf: Column = infMinusSymbol.flatMap { minusSymbol => - infMinusValue.map { minusValue => - when(column === lit(minusSymbol).cast(origType), lit(minusValue).cast(origType)).otherwise(column) - } - }.getOrElse(column) - - infPlusSymbol.flatMap { plusSymbol => - infPlusValue.map { plusValue => - when(columnWithNegativeInf === lit(plusSymbol).cast(origType), lit(plusValue).cast(origType)) - .otherwise(columnWithNegativeInf) - } - }.getOrElse(columnWithNegativeInf) + val columnWithNegativeInf: Column = infMinusSymbol.flatMap { minusSymbol => + infMinusValue.map { minusValue => + when(column === lit(minusSymbol), lit(minusValue)).otherwise(column) } - } + }.getOrElse(column) - private def parseInfinityValue(value: String, pattern: String): Date = { - val dateFormat = new SimpleDateFormat(pattern, Locale.US) - dateFormat.setLenient(false) - new Date(dateFormat.parse(value).getTime) + infPlusSymbol.flatMap { plusSymbol => + infPlusValue.map { plusValue => + when(columnWithNegativeInf === lit(plusSymbol), lit(plusValue)) + .otherwise(columnWithNegativeInf) + } + }.getOrElse(columnWithNegativeInf) } } - - diff --git a/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala b/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala index 1015632..c288f16 100644 --- a/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala +++ b/src/main/scala/za/co/absa/standardization/stages/TypeParser.scala @@ -34,7 +34,7 @@ import za.co.absa.standardization.schema.{MetadataKeys, MetadataValues, StdSchem import za.co.absa.standardization.time.DateTimePattern import za.co.absa.standardization.typeClasses.{DoubleLike, LongLike} import za.co.absa.standardization.types.TypedStructField._ -import za.co.absa.standardization.types.parsers.DateTimeParser +import za.co.absa.standardization.types.parsers.{DateTimeParser => DateTimeParserImpl} import za.co.absa.standardization.types.{ParseOutput, TypeDefaults, TypedStructField} import za.co.absa.standardization.udf.{UDFBuilder, UDFNames} @@ -323,10 +323,8 @@ object TypeParser { override protected val infMinusValue: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityValue) override protected val infPlusSymbol: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinitySymbol) override protected val infPlusValue: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityValue) - override protected val infMinusPattern : Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityPattern) - override protected val infPlusPattern : Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityPattern) - override protected val targetType: DataType = field.dataType - private val columnWithInfinityReplaced = replaceInfinitySymbols(column) + override protected def canParseInfValue(value: String): Boolean = false + private val columnWithInfinityReplaced = replaceInfinitySymbols(column).cast(origType) override protected def standardizeAfterCheck(stdConfig: StandardizationConfig)(implicit logger: Logger): ParseOutput = { if (field.needsUdfParsing) { @@ -513,10 +511,21 @@ object TypeParser { override protected val infMinusValue: Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityValue) override protected val infPlusSymbol: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinitySymbol) override protected val infPlusValue: Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityValue) - override protected val infMinusPattern : Option[String] = metadata.getOptString(MetadataKeys.MinusInfinityPattern) - override protected val infPlusPattern : Option[String] = metadata.getOptString(MetadataKeys.PlusInfinityPattern) - override protected val targetType: DataType = field.dataType - private val columnWithInfinityReplaced: Column = replaceInfinitySymbols(column) + + + private val IsoDatePattern = "yyyy-MM-dd" + private val IsoTimestampPattern = "yyyy-MM-dd HH:mm:ss" + + private lazy val dateTimeParser: DateTimeParserImpl = field.parser.get + + override protected def canParseInfValue(value: String): Boolean = { + Try{ + field.dataType match{ + case DateType => dateTimeParser.parseDate(value) + case TimestampType => dateTimeParser.parseTimestamp(value) + } + }.isSuccess + } protected val replaceCenturyUDF: UserDefinedFunction = udf((inputDate: String, centuryPattern: String) => { val centuryIndex = centuryPattern.indexOf(DateTimePattern.patternCenturyChar) @@ -529,6 +538,45 @@ object TypeParser { pendedInput.substring(0, centuryIndex) + modifiedChar + pendedInput.substring(centuryIndex + 1) }) + private val columnWithInfinityReplaced: Column = { + val replaced = replaceInfinitySymbols(column) + + val originalCastFunc: Column => Column = if (pattern.isEpoch) { + col => (col.cast(decimalType) / pattern.epochFactor).cast(TimestampType) + } else { + col => castStringColumn(col) + } + + val isoPattern = field.dataType match { + case DateType => IsoDatePattern + case TimestampType => IsoTimestampPattern + } + val isoCastFunc: Column => Column = col => field.dataType match{ + case DateType => to_date(col,isoPattern) + case TimestampType => to_timestamp(col, isoPattern) + } + + infMinusSymbol.flatMap { minusSymbol => + infMinusValue.map { minusValue => + if (canParseInfValue(minusValue)){ + originalCastFunc(replaced) + } else { + when(replaced === lit(minusValue), isoCastFunc(lit(minusValue))).otherwise(originalCastFunc(replaced)) + } + } + }.getOrElse{ + infPlusSymbol.flatMap{ plusSymbol => + infPlusValue.map { plusValue=> + if (canParseInfValue(plusValue)){ + originalCastFunc(replaced) + } else{ + when(replaced === lit(plusValue), isoCastFunc(lit(plusValue))).otherwise(originalCastFunc(replaced)) + } + } + }.getOrElse(originalCastFunc(replaced)) + } + } + override protected def assemblePrimitiveCastLogic: Column = { if (pattern.isEpoch) { castEpoch() diff --git a/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala b/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala index 4899580..068f071 100644 --- a/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala +++ b/src/main/scala/za/co/absa/standardization/time/DateTimePattern.scala @@ -26,9 +26,7 @@ import za.co.absa.standardization.types.{Section, TypePattern} * @param pattern actual pattern to format the type conversion * @param isDefault marks if the pattern is actually an assigned value or taken for global defaults */ - -abstract sealed class DateTimePattern(pattern: String, isDefault: Boolean = false, - val infinityConfig:Option[ InfinityConfig] = None) +abstract sealed class DateTimePattern(pattern: String, isDefault: Boolean = false) extends TypePattern(pattern, isDefault){ val isEpoch: Boolean @@ -83,9 +81,8 @@ object DateTimePattern { // scalastyle:on magic.number private final case class EpochDTPattern(override val pattern: String, - override val isDefault: Boolean = false, - override val infinityConfig: Option[InfinityConfig] = None) - extends DateTimePattern(pattern, isDefault, infinityConfig) { + override val isDefault: Boolean = false) + extends DateTimePattern(pattern, isDefault) { override val isEpoch: Boolean = true override val isCentury: Boolean = false @@ -118,15 +115,12 @@ object DateTimePattern { case _ => Seq.empty } override val patternWithoutSecondFractions: String = EpochKeyword - - } private abstract class StandardDTPatternBase(override val pattern: String, assignedDefaultTimeZone: Option[String], - override val isDefault: Boolean = false, - override val infinityConfig: Option[InfinityConfig] = None) - extends DateTimePattern(pattern, isDefault,infinityConfig) { + override val isDefault: Boolean = false) + extends DateTimePattern(pattern, isDefault) { override val isEpoch: Boolean = false override val epochFactor: Long = 0 @@ -157,9 +151,8 @@ object DateTimePattern { private final case class StandardDTPattern(override val pattern: String, assignedDefaultTimeZone: Option[String] = None, - override val isDefault: Boolean = false, - override val infinityConfig: Option[InfinityConfig] = None) - extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault, infinityConfig) { + override val isDefault: Boolean = false) + extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault) { override val isCentury: Boolean = false override val originalPattern: Option[String] = None @@ -168,9 +161,8 @@ object DateTimePattern { private final case class CenturyDTPattern(override val pattern: String, override val originalPattern: Option[String], assignedDefaultTimeZone: Option[String] = None, - override val isDefault: Boolean = false, - override val infinityConfig: Option[InfinityConfig] = None) - extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault, infinityConfig) { + override val isDefault: Boolean = false) + extends StandardDTPatternBase(pattern, assignedDefaultTimeZone, isDefault) { override val isCentury: Boolean = true } @@ -178,29 +170,26 @@ object DateTimePattern { private def create(pattern: String, assignedDefaultTimeZone: Option[String], isCenturyPattern: Boolean, - isDefault: Boolean, - infinityConfig: Option[InfinityConfig]): DateTimePattern = { + isDefault: Boolean): DateTimePattern = { if (isEpoch(pattern)) { - EpochDTPattern(pattern, isDefault,infinityConfig) + EpochDTPattern(pattern, isDefault) } else if (isCenturyPattern && isCentury(pattern)) { val patternWithoutCentury = pattern.replaceAll(patternCenturyChar, "yy") - CenturyDTPattern(patternWithoutCentury, Some(pattern), assignedDefaultTimeZone, isDefault, infinityConfig) + CenturyDTPattern(patternWithoutCentury, Some(pattern), assignedDefaultTimeZone, isDefault) } else { - StandardDTPattern(pattern, assignedDefaultTimeZone, isDefault, infinityConfig) + StandardDTPattern(pattern, assignedDefaultTimeZone, isDefault) } } def apply(pattern: String, assignedDefaultTimeZone: Option[String] = None, - isCenturyPattern: Boolean = false, - infinityConfig: Option[InfinityConfig] = None): DateTimePattern = { - create(pattern, assignedDefaultTimeZone, isCenturyPattern, isDefault = false , infinityConfig) + isCenturyPattern: Boolean = false): DateTimePattern = { + create(pattern, assignedDefaultTimeZone, isCenturyPattern, isDefault = false) } def asDefault(pattern: String, - assignedDefaultTimeZone: Option[String] = None, - infinityConfig: Option[InfinityConfig] = None): DateTimePattern = { - create(pattern, assignedDefaultTimeZone, isCenturyPattern = false, isDefault = true, infinityConfig) + assignedDefaultTimeZone: Option[String] = None): DateTimePattern = { + create(pattern, assignedDefaultTimeZone, isCenturyPattern = false, isDefault = true) } def isEpoch(pattern: String): Boolean = { diff --git a/src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala b/src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala deleted file mode 100644 index e3351f4..0000000 --- a/src/main/scala/za/co/absa/standardization/time/InfinityConfig.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright 2021 ABSA Group Limited - * - * Licensed 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 za.co.absa.standardization.time - -/** - * Standalone class for infinityConfig to be used to define configuration of infinity values - */ - -case class InfinityConfig ( - plusInfinityPattern: Option[String], - minusInfinityPattern: Option[String], - plusInfinityValue: Option[String], - minusInfinityValue: Option[String], - plusInfinitySymbol: Option[String], - minusInfinitySymbol: Option[String] -) diff --git a/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala b/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala index e1647dc..dee3d58 100644 --- a/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala +++ b/src/main/scala/za/co/absa/standardization/types/TypedStructField.scala @@ -21,7 +21,7 @@ import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetada import za.co.absa.standardization.ValidationIssue import za.co.absa.standardization.numeric.{DecimalSymbols, NumericPattern, Radix} import za.co.absa.standardization.schema.{MetadataKeys, MetadataValues} -import za.co.absa.standardization.time.{DateTimePattern, InfinityConfig} +import za.co.absa.standardization.time.DateTimePattern import za.co.absa.standardization.typeClasses.{DoubleLike, LongLike} import za.co.absa.standardization.types.parsers._ import za.co.absa.standardization.validation.field._ @@ -393,28 +393,6 @@ object TypedStructField { (implicit defaults: TypeDefaults) extends TypedStructFieldTagged[T](structField) { - val infinityConfig: Option[InfinityConfig] = { - val plusPattern = structField.metadata.getOptString(MetadataKeys.PlusInfinityPattern) - val minusPattern = structField.metadata.getOptString(MetadataKeys.MinusInfinityPattern) - val plusValue = structField.metadata.getOptString(MetadataKeys.PlusInfinityValue) - val minusValue = structField.metadata.getOptString(MetadataKeys.MinusInfinityValue) - val plusSymbol = structField.metadata.getOptString(MetadataKeys.PlusInfinitySymbol) - val minusSymbol = structField.metadata.getOptString(MetadataKeys.MinusInfinitySymbol) - - if (Seq(plusPattern,minusPattern,plusValue,minusValue,plusSymbol,minusSymbol).exists(_.isDefined)){ - Some(InfinityConfig ( - plusInfinityPattern = plusPattern, - minusInfinityPattern = minusPattern, - plusInfinityValue = plusValue, - minusInfinityValue = minusValue, - plusInfinitySymbol = plusSymbol, - minusInfinitySymbol= minusSymbol - )) - } else { - None - } - } - override def pattern: Try[Option[DateTimePattern]] = { parser.map(x => Some(x.pattern)) } @@ -422,7 +400,7 @@ object TypedStructField { lazy val parser: Try[DateTimeParser] = { val patternToUse = readDateTimePattern Try{ - DateTimeParser(patternToUse, infinityConfig) + DateTimeParser(patternToUse) } } @@ -438,9 +416,9 @@ object TypedStructField { structField.metadata.getOptString(MetadataKeys.Pattern).map { pattern => val timeZoneOpt = structField.metadata.getOptString(MetadataKeys.DefaultTimeZone) val isCenturyPattern = structField.metadata.getOptStringAsBoolean(MetadataKeys.IsNonStandard).getOrElse(false) - DateTimePattern(pattern, timeZoneOpt, isCenturyPattern, infinityConfig) + DateTimePattern(pattern, timeZoneOpt, isCenturyPattern) }.getOrElse( - DateTimePattern.asDefault(defaults.getStringPattern(structField.dataType), None, infinityConfig) + DateTimePattern.asDefault(defaults.getStringPattern(structField.dataType), None) ) } } diff --git a/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala b/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala index 76500a7..37a86f3 100644 --- a/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala +++ b/src/main/scala/za/co/absa/standardization/types/parsers/DateTimeParser.scala @@ -19,17 +19,17 @@ package za.co.absa.standardization.types.parsers import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale -import za.co.absa.standardization.time.{DateTimePattern, InfinityConfig} + +import za.co.absa.standardization.time.DateTimePattern import za.co.absa.standardization.types.Section import za.co.absa.standardization.types.parsers.DateTimeParser.{MillisecondsInSecond, NanosecondsInMicrosecond, NanosecondsInMillisecond, SecondsPerDay} -import scala.util.Try + /** * Enables to parse string to date and timestamp based on the provided format * Unlike SimpleDateFormat it also supports keywords to format epoch related values * @param pattern the formatting string, in case it's an epoch format the values wil need to be convertible to Long */ -case class DateTimeParser(pattern: DateTimePattern, - infinityConfig: Option[InfinityConfig] = None) { +case class DateTimeParser(pattern: DateTimePattern) { private val formatter: Option[SimpleDateFormat] = if (pattern.isEpoch) { None } else { @@ -39,34 +39,15 @@ case class DateTimeParser(pattern: DateTimePattern, Some(sdf) } - private val defaultDatePattern = "yyyy-MM-dd" - private val defaultTimestampPattern = "yyyy-MM-dd HH:mm:ss" - def parseDate(dateValue: String): Date = { - infinityConfig match { - case Some(config) if config.plusInfinitySymbol.contains(dateValue) => - parseInfinityValue(config.plusInfinityValue,config.plusInfinityPattern.getOrElse(defaultDatePattern),Long.MaxValue) - case Some(config) if config.minusInfinitySymbol.contains(dateValue) => - parseInfinityValue(config.minusInfinityValue,config.minusInfinityPattern.getOrElse(defaultDatePattern), Long.MinValue) - case _ => - val seconds = extractSeconds(dateValue) - new Date((seconds - (seconds % SecondsPerDay)) * MillisecondsInSecond) - } + val seconds = extractSeconds(dateValue) + new Date((seconds - (seconds % SecondsPerDay)) * MillisecondsInSecond) } def parseTimestamp(timestampValue: String): Timestamp = { - infinityConfig match { - case Some(config) if config.plusInfinitySymbol.contains(timestampValue) => - val date = parseInfinityValue(config.plusInfinityValue, config.minusInfinityPattern.getOrElse(defaultTimestampPattern), Long.MaxValue) - new Timestamp(date.getTime) - case Some(config) if config.minusInfinitySymbol.contains(timestampValue) => - val date = parseInfinityValue(config.minusInfinityValue,config.minusInfinityPattern.getOrElse(defaultTimestampPattern),Long.MinValue) - new Timestamp(date.getTime) - case _ => - val seconds = extractSeconds(timestampValue) - val nanoseconds = extractNanoseconds(timestampValue) - makePreciseTimestamp(seconds, nanoseconds) - } + val seconds = extractSeconds(timestampValue) + val nanoseconds = extractNanoseconds(timestampValue) + makePreciseTimestamp(seconds, nanoseconds) } def format(time: java.util.Date): String = { @@ -131,14 +112,6 @@ case class DateTimeParser(pattern: DateTimePattern, pattern.nanosecondsPosition.foreach(result += _.extractFrom(value).toInt) result } - - private def parseInfinityValue(value: Option[String], pattern: String, default: Long): Date = { - value.map { v => - val dateFormatter = new SimpleDateFormat(pattern, Locale.US) - dateFormatter.setLenient(false) - new Date(dateFormatter.parse(v).getTime) - }.getOrElse(new Date(default)) - } } object DateTimeParser { diff --git a/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala b/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala deleted file mode 100644 index 919ab63..0000000 --- a/src/test/scala/za/co/absa/standardization/InfinitySupportIsoTest.scala +++ /dev/null @@ -1,228 +0,0 @@ -/* - * Copyright 2021 ABSA Group Limited - * - * Licensed 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 za.co.absa.standardization - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.funsuite.AnyFunSuite -import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} -import org.apache.spark.sql.types.{DataType, DateType, Metadata, StringType, StructField, StructType, TimestampType} -import org.apache.spark.sql.functions.{col, lit, to_date, to_timestamp, when} -import java.sql -import java.sql.{Date, Timestamp} -import java.text.{SimpleDateFormat,ParseException} -import java.util.TimeZone -import scala.util.Try - - -class InfinitySupportIsoTest extends AnyFunSuite with BeforeAndAfterAll { - var sparkSession: SparkSession = _ - - override def beforeAll(): Unit = { - sparkSession = SparkSession.builder() - .appName("InfinityISOTest") - .master("local[*]") - .config("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "CORRECTED") - .config("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "CORRECTED") - .getOrCreate() - } - - override def afterAll(): Unit ={ - if (sparkSession != null) { - sparkSession.stop() - } - } - - private def createTestDataFrame(data: Seq[String]): DataFrame = { - sparkSession.createDataFrame( - sparkSession.sparkContext.parallelize(data.map(Row(_))), - StructType(Seq(StructField("value", StringType, nullable = false))) - ) - } - - private val configString = - """ - standardization.infinity { - minus.symbol = "-inf" - minus.value = "1970-01-01 00:00:00.000000" - plus.symbol = "inf" - plus.value ="9999-12-31 23:59:59.999999" - } - """ - private def replaceInfinitySymbols(column: Column, dataType: DataType, pattern: Option[String], timezone:String, minusSymbol: String, minusValue: String, plusSymbol:String, plusValue:String): Column ={ - def validateValue(value: String, patternOpt: Option[String], dataType: DataType) : Unit = { - val isoPattern = dataType match { - case TimestampType => "yyyy-MM-dd'T'HH:mm:ss.SSSSSS" - case DateType => "yyyy-MM-dd" - case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType") - } - - val formatsToTry = patternOpt.toSeq ++ Seq(isoPattern) - var lastException: Option[ParseException] = None - - for (fmt <- formatsToTry) { - try { - val sdf = new SimpleDateFormat(fmt) - sdf.setTimeZone(TimeZone.getTimeZone(timezone)) - sdf.setLenient(false) - sdf.parse(value) - return - } catch { - case e: ParseException => lastException = Some(e) - } - } - - val errorMsg = s"Invalid infinity value: '$value' for type: ${dataType.toString.toLowerCase} with pattern ${patternOpt.getOrElse("none")} and ISO fallback ($isoPattern)" - throw new IllegalArgumentException(errorMsg,lastException.orNull) - } - - validateValue(minusValue,pattern,dataType) - validateValue(plusValue, pattern, dataType) - - - dataType match { - case TimestampType => - when(col(column.toString) === minusSymbol, lit(minusValue)) - .when(col(column.toString) === plusSymbol, lit(plusValue)) - .otherwise( - pattern.map(p => to_timestamp(col(column.toString),p)) - .getOrElse(to_timestamp(col(column.toString))) - ) - case DateType => - when(col(column.toString) === minusSymbol, lit(minusValue)) - .when(col(column.toString) === plusSymbol, lit(plusValue)) - .otherwise( - pattern.map( p => to_date(col(column.toString), p)) - .getOrElse(to_date(col(column.toString))) - ) - case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType") - } - } - - - test("Replace infinity symbols for timestamp with valid pattern"){ - val df = createTestDataFrame(Seq("-inf","inf", "2025-07-05 12:34:56", null)) - val result = df.withColumn("result", replaceInfinitySymbols(col("value"), TimestampType,Some("yyyy-MM-dd HH:mm:ss"), "UTC","-inf", "1970-01-01 00:00:00","inf","9999-12-31 23:59:59")) - .select("result") - .collect() - .map(_.getAs[TimestampType](0)) - - val expected = Seq( - Timestamp.valueOf("1970-01-01 00:00:00"), - Timestamp.valueOf("9999-12-31 23:59:59"), - Timestamp.valueOf("2025-07-05 12:34:56"), - null - ) - - assert(result sameElements expected) - } - - test("Convert invalid timestamp pattern to ISO"){ - val df = createTestDataFrame(Seq("-inf","inf")) - val result = df.withColumn("result", replaceInfinitySymbols( - col("value"), - TimestampType, - Some("yyyy-MM-dd HH:mm:ss"), - "UTC", - "-inf", - "1970-01-01 00:00:00", - "inf", - "9999-12-31 23:59:59")) - .select("result") - .collect() - .map(_.getAs[TimestampType] (0)) - - - val expected = Seq( - Timestamp.valueOf("1970-01-01 00:00:00"), - Timestamp.valueOf("9999-12-31 23:59:59") - ) - - assert (result sameElements expected) - } - - - test("Replace infinity symbol for date with valid pattern"){ - val df = createTestDataFrame(Seq("-inf", "inf", "20245-07-05",null)) - val result = df.withColumn("result", replaceInfinitySymbols( - col("value"), - DateType, - Some("yyyy-MM-dd"), - "UTC", - "-inf", - "1970-01-01", - "inf", - "9999-12-31" - )) - .select("result") - .collect() - .map(_.getAs[Date](0)) - - val expected = Seq( - Date.valueOf("1970-01-01"), - Date.valueOf("9999-12-31"), - Date.valueOf("2025-07-05"), - null - ) - - assert (result sameElements expected) - } - - - test("Throw error for unparseable infinity value"){ - val exception = intercept[IllegalArgumentException] { - replaceInfinitySymbols( - col("value"), - TimestampType, - Some("yyyy-MM-dd HH:mm:ss"), - "UTC", - "-inf", - "invalid_date", - "inf", - "9999-12-31 23:59:59" - ) - } - - assert(exception.getMessage.contains("Invalid infinity value: 'invalid_date' for type: timestamp")) - assert(exception.getMessage.contains("pattern yyyy-MM-dd:mm:ss")) - assert(exception.getMessage.contains("ISO fallback (yyyy-MM-dd'T'HH:mm:ss.SSSSSS")) - } - - test("Handle missing pattern with ISO fallback"){ - val df = createTestDataFrame(Seq("-inf","inf")) - val result = df.withColumn("result", replaceInfinitySymbols( - col("value"), - DateType, - None, - "UTC", - "-inf", - "1970-01-01", - "inf", - "9999-12-31" - )) - .select("result") - .collect() - .map(_.getAs[Date](0)) - - - val expected = Seq( - Date.valueOf("1970-01-01"), - Date.valueOf("9999-12-31") - ) - - assert (result sameElements expected) - } -}