From 288989ea18e6d5b39ead539ee65ffbd4a8a8d2f8 Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 30 Dec 2025 18:35:34 +0800 Subject: [PATCH 1/5] Support reading and writing of Variant Types - Add adapter pattern for Spark3 and 4 - Cleanup invariant issue in SparkSqlWriter - Add cross engine test - Add backward compatibility test for Spark3.x - Add cross engine read for Flink --- .../hudi/util/HoodieSchemaConverter.java | 22 ++ .../row/HoodieRowParquetWriteSupport.java | 25 +++ ...rkFileFormatInternalRowReaderContext.scala | 1 - .../apache/spark/sql/HoodieUnsafeUtils.scala | 4 +- .../sql/avro/HoodieAvroSchemaConverters.scala | 33 --- .../avro/HoodieSparkSchemaConverters.scala | 33 ++- .../HoodieParquetFileFormatHelper.scala | 63 +++--- .../apache/spark/sql/hudi/SparkAdapter.scala | 76 ++++++- .../hudi/common/schema/HoodieSchema.java | 22 +- .../convert/InternalSchemaConverter.java | 44 ++++ .../metadata/HoodieTableMetadataUtil.java | 6 +- .../hudi/common/schema/TestHoodieSchema.java | 16 +- .../variant_backward_compat/README.md | 41 ++++ .../variant_backward_compat/variant_cow.zip | Bin 0 -> 90526 bytes .../variant_mor_avro.zip | Bin 0 -> 87511 bytes .../variant_mor_spark.zip | Bin 0 -> 88480 bytes ...ITTestVariantCrossEngineCompatibility.java | 142 +++++++++++++ .../AvroSchemaConverterWithTimestampNTZ.java | 4 + .../apache/hudi/HoodieSparkSqlWriter.scala | 12 +- .../command/CreateHoodieTableCommand.scala | 2 +- .../hudi/dml/schema/TestVariantDataType.scala | 174 ++++++++++++++++ .../spark/sql/adapter/BaseSpark3Adapter.scala | 39 +++- .../spark/sql/adapter/BaseSpark4Adapter.scala | 88 +++++++- .../spark/sql/adapter/Spark4_0Adapter.scala | 1 - .../spark/sql/avro/AvroDeserializer.scala | 33 ++- .../spark/sql/avro/AvroSerializer.scala | 27 +++ ...stHoodieRowParquetWriteSupportVariant.java | 195 ++++++++++++++++++ 27 files changed, 1003 insertions(+), 100 deletions(-) delete mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSchemaConverters.scala create mode 100644 hudi-common/src/test/resources/variant_backward_compat/README.md create mode 100644 hudi-common/src/test/resources/variant_backward_compat/variant_cow.zip create mode 100644 hudi-common/src/test/resources/variant_backward_compat/variant_mor_avro.zip create mode 100644 hudi-common/src/test/resources/variant_backward_compat/variant_mor_spark.zip create mode 100644 hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestVariantCrossEngineCompatibility.java create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/schema/TestVariantDataType.scala create mode 100644 hudi-spark-datasource/hudi-spark4.0.x/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowParquetWriteSupportVariant.java diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/HoodieSchemaConverter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/HoodieSchemaConverter.java index 5e211c4851a0d..828f6752ce7ff 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/HoodieSchemaConverter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/HoodieSchemaConverter.java @@ -324,6 +324,8 @@ public static DataType convertToDataType(HoodieSchema hoodieSchema) { return convertRecord(hoodieSchema); case UNION: return convertUnion(hoodieSchema); + case VARIANT: + return convertVariant(hoodieSchema); default: throw new IllegalArgumentException("Unsupported HoodieSchemaType: " + type); } @@ -445,6 +447,26 @@ private static DataType convertUnion(HoodieSchema schema) { return nullable ? rawDataType.nullable() : rawDataType; } + /** + * Converts a Variant schema to Flink's ROW type. + * Variant is represented as ROW<`value` BYTES, `metadata` BYTES> in Flink. + * // TODO: We are only supporting unshredded for now, support shredded in the future + * + * @param schema HoodieSchema to convert (must be a VARIANT type) + * @return DataType representing the Variant as a ROW with binary fields + */ + private static DataType convertVariant(HoodieSchema schema) { + if (schema.getType() != HoodieSchemaType.VARIANT) { + throw new IllegalStateException("Expected HoodieSchema.Variant but got: " + schema.getClass()); + } + + // Variant is stored as a struct with two binary fields: value and metadata + return DataTypes.ROW( + DataTypes.FIELD("value", DataTypes.BYTES().notNull()), + DataTypes.FIELD("metadata", DataTypes.BYTES().notNull()) + ).notNull(); + } + /** * Returns true if all the types are RECORD type with same number of fields. */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 9d0da7f534afa..e0273b4f52b0b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -73,6 +73,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.function.BiConsumer; import scala.Enumeration; import scala.Function1; @@ -281,6 +282,23 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { } else if (dataType == DataTypes.BinaryType) { return (row, ordinal) -> recordConsumer.addBinary( Binary.fromReusedByteArray(row.getBinary(ordinal))); + } else if (SparkAdapterSupport$.MODULE$.sparkAdapter().isVariantType(dataType)) { + // Maps VariantType to a group containing 'metadata' and 'value' fields. + // This ensures Spark 4.0 compatibility and supports both Shredded and Unshredded schemas. + // Note: We intentionally omit 'typed_value' for shredded variants as this writer only accesses raw binary blobs. + final byte[][] variantBytes = new byte[2][]; // [0] = value, [1] = metadata + BiConsumer variantWriter = SparkAdapterSupport$.MODULE$.sparkAdapter().createVariantValueWriter( + dataType, + valueBytes -> variantBytes[0] = valueBytes, + metadataBytes -> variantBytes[1] = metadataBytes + ); + return (row, ordinal) -> { + variantWriter.accept(row, ordinal); + consumeGroup(() -> { + consumeField("value", 0, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(variantBytes[0]))); + consumeField("metadata", 1, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(variantBytes[1]))); + }); + }; } else if (dataType instanceof DecimalType) { return (row, ordinal) -> { int precision = ((DecimalType) dataType).precision(); @@ -510,6 +528,13 @@ private Type convertField(HoodieSchema fieldSchema, StructField structField, Typ .as(LogicalTypeAnnotation.stringType()).named(structField.name()); } else if (dataType == DataTypes.BinaryType) { return Types.primitive(BINARY, repetition).named(structField.name()); + } else if (SparkAdapterSupport$.MODULE$.sparkAdapter().isVariantType(dataType)) { + return SparkAdapterSupport$.MODULE$.sparkAdapter().convertVariantFieldToParquetType( + dataType, + structField.name(), + resolvedSchema, + repetition + ); } else if (dataType instanceof DecimalType) { int precision = ((DecimalType) dataType).precision(); int scale = ((DecimalType) dataType).scale(); diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index e4d97a2df6a29..142786567f2a5 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -21,7 +21,6 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{filterIsSafeForBootstrap, filterIsSafeForPrimaryKey, getAppliedRequiredSchema} -import org.apache.hudi.avro.{AvroSchemaUtils, HoodieAvroUtils} import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala index 4055206cfc5f5..04f18c1c2a881 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala @@ -18,11 +18,11 @@ package org.apache.spark.sql -import org.apache.hudi.{HoodieUnsafeRDD} +import org.apache.hudi.HoodieUnsafeRDD import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.catalyst.plans.physical.{Partitioning} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.types.StructType /** diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSchemaConverters.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSchemaConverters.scala deleted file mode 100644 index 9b068afac83d2..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSchemaConverters.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.avro - -import org.apache.avro.Schema -import org.apache.spark.sql.types.DataType - -/** - * Allows to convert Avro schema into Spark's Catalyst one - */ -trait HoodieAvroSchemaConverters { - - def toSqlType(avroSchema: Schema): (DataType, Boolean) - - def toAvroType(catalystType: DataType, nullable: Boolean, recordName: String, nameSpace: String = ""): Schema - -} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala index 11451f3b7ce08..161b4770796ef 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala @@ -18,28 +18,31 @@ package org.apache.spark.sql.avro -import org.apache.hudi.common.schema.HoodieSchema.TimePrecision +import org.apache.hudi.SparkAdapterSupport import org.apache.hudi.common.schema.{HoodieJsonProperties, HoodieSchema, HoodieSchemaField, HoodieSchemaType} +import org.apache.hudi.common.schema.HoodieSchema.TimePrecision + import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.types.Decimal.minBytesForPrecision import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.Decimal.minBytesForPrecision import scala.collection.JavaConverters._ /** - * This object contains methods that are used to convert HoodieSchema to Spark SQL schemas and vice versa. + * Object containing methods to convert HoodieSchema to Spark SQL schemas and vice versa. * * This provides direct conversion between HoodieSchema and Spark DataType * without going through Avro Schema intermediary. * + * Version-specific types (like VariantType in Spark >4.x) are handled via SparkAdapterSupport. + * * NOTE: the package of this class is intentionally kept as "org.apache.spark.sql.avro" which is similar to the existing * Spark Avro connector's SchemaConverters.scala * (https://github.com/apache/spark/blob/master/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala). * The reason for this is so that Spark 3.3 is able to access private spark sql type classes like TimestampNTZType. */ - @DeveloperApi -object HoodieSparkSchemaConverters { +object HoodieSparkSchemaConverters extends SparkAdapterSupport { /** * Internal wrapper for SQL data type and nullability. @@ -118,7 +121,12 @@ object HoodieSparkSchemaConverters { HoodieSchema.createRecord(recordName, nameSpace, null, fields.asJava) } - case other => throw new IncompatibleSchemaException(s"Unexpected Spark DataType: $other") + // VARIANT type (Spark >4.x only), which will be handled via SparkAdapter + case other if sparkAdapter.isVariantType(other) => + HoodieSchema.createVariant(recordName, nameSpace, null) + + case other => + throw new IncompatibleSchemaException(s"Unexpected Spark DataType: $other") } // Wrap with null union if nullable (and not already a union) @@ -129,6 +137,9 @@ object HoodieSparkSchemaConverters { } } + /** + * Helper method to convert HoodieSchema to Catalyst DataType. + */ private def toSqlTypeHelper(hoodieSchema: HoodieSchema, existingRecordNames: Set[String]): SchemaType = { hoodieSchema.getType match { // Primitive types @@ -241,7 +252,15 @@ object HoodieSparkSchemaConverters { } } - case other => throw new IncompatibleSchemaException(s"Unsupported HoodieSchemaType: $other") + // VARIANT type (Spark >4.x only), which will be handled via SparkAdapter + case HoodieSchemaType.VARIANT => + sparkAdapter.getVariantDataType match { + case Some(variantType) => SchemaType(variantType, nullable = false) + case None => throw new IncompatibleSchemaException("VARIANT type is only supported in Spark 4.0+") + } + + case other => + throw new IncompatibleSchemaException(s"Unsupported HoodieSchemaType: $other") } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala index abf805a430f33..6bb9dffcc4e03 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala @@ -20,13 +20,14 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hadoop.conf.Configuration +import org.apache.hudi.HoodieSparkUtils import org.apache.parquet.hadoop.metadata.FileMetaData import org.apache.spark.sql.HoodieSchemaUtils import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, Cast, CreateNamedStruct, CreateStruct, Expression, GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, NamedLambdaVariable, UnsafeProjection} import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampNTZType} -object HoodieParquetFileFormatHelper { +trait HoodieParquetFileFormatHelperTrait { def buildImplicitSchemaChangeInfo(hadoopConf: Configuration, parquetFileMetaData: FileMetaData, @@ -46,7 +47,7 @@ object HoodieParquetFileFormatHelper { val requiredType = f.dataType if (fileStructMap.contains(f.name) && !isDataTypeEqual(requiredType, fileStructMap(f.name))) { val readerType = addMissingFields(requiredType, fileStructMap(f.name)) - implicitTypeChangeInfo.put(new Integer(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, readerType)) + implicitTypeChangeInfo.put(Integer.valueOf(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, readerType)) StructField(f.name, readerType, f.nullable) } else { f @@ -55,35 +56,43 @@ object HoodieParquetFileFormatHelper { (implicitTypeChangeInfo, StructType(sparkRequestStructFields)) } - def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = (requiredType, fileType) match { - case (requiredType, fileType) if requiredType == fileType => true + def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = { + // Check if adapter can handle this comparison (e.g., VariantType in Spark 4.0+) + val adapterResult = HoodieSparkUtils.sparkAdapter.isDataTypeEqualForParquet(requiredType, fileType) + if (adapterResult.isDefined) { + return adapterResult.get + } - // prevent illegal cast - case (TimestampNTZType, LongType) => true + (requiredType, fileType) match { + case (requiredType, fileType) if requiredType == fileType => true - case (ArrayType(rt, _), ArrayType(ft, _)) => - // Do not care about nullability as schema evolution require fields to be nullable - isDataTypeEqual(rt, ft) + // prevent illegal cast + case (TimestampNTZType, LongType) => true - case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => - // Likewise, do not care about nullability as schema evolution require fields to be nullable - isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) + case (ArrayType(rt, _), ArrayType(ft, _)) => + // Do not care about nullability as schema evolution require fields to be nullable + isDataTypeEqual(rt, ft) - case (StructType(requiredFields), StructType(fileFields)) => - // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations - val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) + case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => + // Likewise, do not care about nullability as schema evolution require fields to be nullable + isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) - // Need to match by name instead of StructField as name will stay the same whilst type may change - val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) - val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + case (StructType(requiredFields), StructType(fileFields)) => + // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations + val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) - // Sorting ensures that the same field names are being compared for type differences - requiredFilteredFields.zip(fileFilteredFields).forall { - case (requiredField, fileFilteredField) => - isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) - } + // Need to match by name instead of StructField as name will stay the same whilst type may change + val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + + // Sorting ensures that the same field names are being compared for type differences + requiredFilteredFields.zip(fileFilteredFields).forall { + case (requiredField, fileFilteredField) => + isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) + } - case _ => false + case _ => false + } } def addMissingFields(requiredType: DataType, fileType: DataType): DataType = (requiredType, fileType) match { @@ -195,3 +204,9 @@ object HoodieParquetFileFormatHelper { } } } + +/** + * Default object implementing HoodieParquetFileFormatHelperTrait. + * Can be used directly in Spark 3.x and 4.x environments. + */ +object HoodieParquetFileFormatHelper extends HoodieParquetFileFormatHelperTrait diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 1afcc78df058e..1cc2e9da657db 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -21,19 +21,21 @@ package org.apache.spark.sql.hudi import org.apache.hudi.{HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping} import org.apache.hudi.client.model.HoodieInternalRow import org.apache.hudi.common.model.FileSlice +import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.cdc.HoodieCDCFileSplit import org.apache.hudi.storage.StorageConfiguration + import org.apache.hadoop.conf.Configuration -import org.apache.hudi.common.schema.HoodieSchema -import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.{MessageType, Type} +import org.apache.parquet.schema.Type.Repetition import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate, SpecializedGetters} import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.trees.Origin @@ -46,11 +48,12 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.{DataType, Metadata, StructType} -import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.types.UTF8String import java.util.{Locale, TimeZone} +import java.util.function.{BiConsumer, Consumer} /** * Interface adapting discrepancies and incompatibilities between different Spark versions @@ -374,4 +377,69 @@ trait SparkAdapter extends Serializable { * @return A streaming [[DataFrame]] */ def createStreamingDataFrame(sqlContext: SQLContext, relation: HadoopFsRelation, requiredSchema: StructType): DataFrame + + /** + * Gets the VariantType DataType if supported by this Spark version. + * Spark 3.x returns None (VariantType not supported). + * Spark 4.x returns Some(VariantType). + * + * @return Option[DataType] - Some(VariantType) for Spark 4.x, None for Spark 3.x + */ + def getVariantDataType: Option[DataType] + + /** + * Checks if two data types are equal for Parquet file format purposes. + * This handles version-specific types like VariantType (Spark 4.0+). + * + * Returns Some(true) if types are equal, Some(false) if not equal, or None if + * this adapter doesn't handle this specific type comparison (fallback to default logic). + * + * @param requiredType The required/expected data type + * @param fileType The data type from the file + * @return Option[Boolean] - Some(result) if handled by adapter, None otherwise + */ + def isDataTypeEqualForParquet(requiredType: DataType, fileType: DataType): Option[Boolean] + + /** + * Checks if the given DataType is a VariantType. + * Spark 3.x returns false, Spark 4.x checks for VariantType. + * + * @param dataType The data type to check + * @return true if it's a VariantType, false otherwise + */ + def isVariantType(dataType: DataType): Boolean + + /** + * Creates a ValueWriter function for VariantType if the data type is VariantType. + * Returns null for Spark 3.x or if the data type is not VariantType. + * + * The returned function accepts (SpecializedGetters, ordinal) and writes variant data. + * + * @param dataType The data type to create a writer for + * @param writeValue Function to write the variant value binary + * @param writeMetadata Function to write the variant metadata binary + * @return BiConsumer function or null + */ + def createVariantValueWriter( + dataType: DataType, + writeValue: Consumer[Array[Byte]], + writeMetadata: Consumer[Array[Byte]] + ): BiConsumer[SpecializedGetters, Integer] + + /** + * Converts a VariantType field to Parquet Type. + * Returns null for Spark 3.x or if the data type is not VariantType. + * + * @param dataType The data type to convert + * @param fieldName The field name + * @param fieldSchema The HoodieSchema for the field (to determine shredded vs unshredded) + * @param repetition The Parquet repetition type + * @return Parquet Type or null + */ + def convertVariantFieldToParquetType( + dataType: DataType, + fieldName: String, + fieldSchema: HoodieSchema, + repetition: Repetition + ): Type } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java index f4c6189a50041..95d8af739c073 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java @@ -542,7 +542,8 @@ public static HoodieSchema.Variant createVariant(String name, String namespace, null ); - List fields = Arrays.asList(metadataField, valueField); + // IMPORTANT: Field order must match VariantVal(value, metadata) constructor + List fields = Arrays.asList(valueField, metadataField); Schema recordSchema = Schema.createRecord(variantName, doc, namespace, false); List avroFields = fields.stream() @@ -581,15 +582,8 @@ public static HoodieSchema.Variant createVariantShredded(String name, String nam List fields = new ArrayList<>(); - // Create metadata field (required bytes) - fields.add(HoodieSchemaField.of( - Variant.VARIANT_METADATA_FIELD, - HoodieSchema.create(HoodieSchemaType.BYTES), - "Variant metadata component", - null - )); - - // Create value field (nullable bytes for shredded) + // IMPORTANT: Field order must match VariantVal(value, metadata) constructor + // Create value field first (nullable bytes for shredded) fields.add(HoodieSchemaField.of( Variant.VARIANT_VALUE_FIELD, HoodieSchema.createNullable(HoodieSchemaType.BYTES), @@ -597,6 +591,14 @@ public static HoodieSchema.Variant createVariantShredded(String name, String nam NULL_VALUE )); + // Create metadata field second (required bytes) + fields.add(HoodieSchemaField.of( + Variant.VARIANT_METADATA_FIELD, + HoodieSchema.create(HoodieSchemaType.BYTES), + "Variant metadata component", + null + )); + // Add typed_value field if provided if (typedValueSchema != null) { fields.add(HoodieSchemaField.of( diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java index 9783ecac1b5a5..f6db0eca699b3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java @@ -346,6 +346,18 @@ private static Type visitPrimitiveToBuildInternalType(HoodieSchema schema) { return Types.DateType.get(); case NULL: return null; + case VARIANT: + // Variant is represented as a record with value and metadata binary fields + // Convert it to the internal schema representation as a RecordType + // Since Variant is treated as a primitive here but needs to be a record, + // we return a RecordType with the appropriate structure + List variantFields = new ArrayList<>(2); + // Assign field IDs: these are used for schema evolution tracking + // Use negative IDs: indicate these are system-generated for Variant type + // TODO (voon): Check if we can remove the magic numbers? + variantFields.add(Types.Field.get(-1, false, "value", Types.BinaryType.get(), "Variant value component")); + variantFields.add(Types.Field.get(-2, false, "metadata", Types.BinaryType.get(), "Variant metadata component")); + return Types.RecordType.get(variantFields); default: throw new UnsupportedOperationException("Unsupported primitive type: " + schema.getType()); } @@ -441,6 +453,38 @@ private static HoodieSchema visitInternalSchemaToBuildHoodieSchema(Type type, Ma */ private static HoodieSchema visitInternalRecordToBuildHoodieRecord(Types.RecordType recordType, List fieldSchemas, String recordNameFallback) { List fields = recordType.fields(); + + // Check if this RecordType is actually a Variant type + // Unshredded Variant types are marked by having exactly 2 fields with negative IDs and specific names + if (fields.size() == 2) { + Types.Field field0 = fields.get(0); + Types.Field field1 = fields.get(1); + + // Check if both fields have negative IDs (system-generated for Variant) + boolean hasNegativeIds = field0.fieldId() < 0 && field1.fieldId() < 0; + + // Check if fields are named "value" and "metadata" (order may vary) + boolean hasVariantFields = (field0.name().equals("value") && field1.name().equals("metadata")) + || (field0.name().equals("metadata") && field1.name().equals("value")); + + if (hasNegativeIds && hasVariantFields) { + // Variant type: Determine if it is shredded or unshredded based on value field's optionality + // TODO (voon): This is incomplete for now, we are only handling unshredded, fields size of == 2 should always mean this is unshredded + String recordName = Option.ofNullable(recordType.name()).orElse(recordNameFallback); + Types.Field valueField = field0.name().equals("value") ? field0 : field1; + + if (valueField.isOptional()) { + // Optional value field indicates shredded variant + // Note: We don't have the typed_value schema here, so pass null for typedValueSchema + return HoodieSchema.createVariantShredded(recordName, null, null, null); + } else { + // Required value field indicates unshredded variant + return HoodieSchema.createVariant(recordName, null, null); + } + } + } + + // Not a Variant, create regular record List schemaFields = new ArrayList<>(fields.size()); for (int i = 0; i < fields.size(); i++) { Types.Field f = fields.get(i); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index ebf323243150d..90c09cc94668b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -2059,8 +2059,12 @@ private static boolean isColumnTypeSupportedV1(HoodieSchema schema, Option getInflightMetadataPartitions(HoodieTableConfig tableConfig) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchema.java b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchema.java index 3a374b706ff87..69191265fad4c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchema.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchema.java @@ -1075,8 +1075,8 @@ public void testCreateUnshreddedVariant() { // Verify fields List fields = variantSchema.getFields(); assertEquals(2, fields.size()); - assertEquals("metadata", fields.get(0).name()); - assertEquals("value", fields.get(1).name()); + assertEquals("value", fields.get(0).name()); + assertEquals("metadata", fields.get(1).name()); // Verify field types assertEquals(HoodieSchemaType.BYTES, fields.get(0).schema().getType()); @@ -1118,12 +1118,12 @@ public void testCreateShreddedVariant() { // Verify fields List fields = variantSchema.getFields(); assertEquals(3, fields.size()); - assertEquals("metadata", fields.get(0).name()); - assertEquals("value", fields.get(1).name()); + assertEquals("value", fields.get(0).name()); + assertEquals("metadata", fields.get(1).name()); assertEquals("typed_value", fields.get(2).name()); // Value field should be nullable for shredded - assertTrue(fields.get(1).schema().isNullable()); + assertTrue(fields.get(0).schema().isNullable()); // Verify typed_value schema HoodieSchema retrievedTypedValueSchema = variantSchema.getTypedValueField().get(); @@ -1141,11 +1141,11 @@ public void testCreateShreddedVariantWithoutTypedValue() { // Verify fields should have metadata and nullable value, but no typed_value List fields = variantSchema.getFields(); assertEquals(2, fields.size()); - assertEquals("metadata", fields.get(0).name()); - assertEquals("value", fields.get(1).name()); + assertEquals("value", fields.get(0).name()); + assertEquals("metadata", fields.get(1).name()); // Value field should be nullable even without typed_value - assertTrue(fields.get(1).schema().isNullable()); + assertTrue(fields.get(0).schema().isNullable()); } @Test diff --git a/hudi-common/src/test/resources/variant_backward_compat/README.md b/hudi-common/src/test/resources/variant_backward_compat/README.md new file mode 100644 index 0000000000000..94d45541b0bb1 --- /dev/null +++ b/hudi-common/src/test/resources/variant_backward_compat/README.md @@ -0,0 +1,41 @@ + + + +# Generation of test files + +These files are generated by using the test: +`org/apache/spark/sql/hudi/dml/schema/TestVariantDataType.scala:31` + +Spark4.0 is required for this. + +The shell commands used to generate this are: + +```shell +cd /path/to/test/files/ +zip -r zip_file.zip htestvariantdatatype_1 +``` + +The test runs on these four arguments: +1. COW + HoodieRecordType.AVRO +2. COW + HoodieRecordType.SPARK +3. MOR + HoodieRecordType.AVRO +4. MOR + HoodieRecordType.SPARK + +COW tables generated are the same for both AVRO/SPARK. But for MOR, the log files metadata are +different. Hence, we only need to generate test files for either 1/2, 3 and 4, hence, 3 test +resource files. diff --git a/hudi-common/src/test/resources/variant_backward_compat/variant_cow.zip b/hudi-common/src/test/resources/variant_backward_compat/variant_cow.zip new file mode 100644 index 0000000000000000000000000000000000000000..e1efe0f447f6c77f223a587c35a9401b4875f0be GIT binary patch literal 90526 zcmb@O1y~%(_V#gicZcBa5(w_WA-KD{ySoGr7TkloyK8WFf(9qp2id)w-MyRp-@W(i zho0$~X-?I2)v5ZO_f(IJ1TY8`z{^GvMXUb(;MWHj02}~E9a|F}OM5LnDG7jerX2^TD z@%(omlVtM2W=NftZ4>Q!1$TkpY!{>jjW>gNJKb$%_ttau+9hMxVraRAYN32M@J&4@ zwLh=N*W^?^SREqD8ULC$)-zEbtqhjUl*%k zE^7HtyXH%u`USCtrDlVAPTMma&3RoursqW+?Wd(UnabITw`e%?i&*cl!MT@~POdkf zcAolSTQwN;g5aVd)mf%q>sy?5V~$W6#;1^WF3=;twteb??@J8)N^heQS0IGI-0V7$ zh#U`5)meLbG6w(j*e&PZ-6yD`fjqtXm~m%D)p8OUaHqT8;iTn}WhwbImEnAxgUwho zg>SXg=(1w6Yt>M6dt2wpbG@yMf4gmA$LaW(Yd2Qbg1z&&HY)K=t5xrOvyJl!o$V;y zL+WdY~7u_PdEFeUm;Uk5!)PY z&u|^Mz67GvdaMtAHad1MvRVJ!WZ7nV?F0`8&+B zK0J8Md085484B(8NHsaFIGrWgG(Co_XObmdKZ1uWymi_-FzP#yuS#W9fE@*1q1{_}nQH5SeJ(YX2l`b|N~45U zldbeM3lmUm}E$=6>TB|L) zzAcTjXka(aZr|Che;_E~;!5YNFJ*+};Y()@qQ$xU-F6xUJbUPn?hJ=_}rt9kogTb=1JheEDAr8K% znffk#v$^JwBtaff(;ZmU-CxKp?YOUCa{Lg2i@WiuTDsdWcOdLF&Lx2h0StC*T?R@S zys!xFwP-@u3Z}#gs>Df1{6-}2BCnwh2Cmm<(fifF_2hx0S0ZL-I*{Q^a@(iMB;b=# zJrW&yQWKPnjZ@vC`Cc?f(^vAH!y+CtWM6nKorfN-Nbw_b>qsW=@>p_w6{2BwL{|O2 z1#H(aYeL14gF_1Y9*PRT!Q&)VcUFn6cqQ8KMDukk`|E`s%RU=2Y~g6O#@GT@wnn|9 zUaK=qQ>deh+z!hq!C^#Hcqh-m-u>++#d@o=3&uv!XofLaLe!nXKF{{_Ir8A@h2ew8z)+f702f9VFZ%g$VTB^7&2M^o~D)2pMN3q0Cb0&ntGGT5 zpW)zL1KHjf7}|d_AntHX!vnp$E?np1P}vv<2~Mc6jn*|0=21!X-0u#Ikr?gIHb&`n zaXi$Hf*E|A%6{W=P;KoJeSbTwPo$B$ZFgWd(etswec5>FJ^lj1 z$jS7@POqdMt7w{yv0T|cHqIfP)QGNUL2NjfpvaKElcYHdE zc3i(sstRf%HORjn0|2ZsO};w<`SsBwZdyAzqH!Q8qVKpDiUtYvDXU zX`iV)SA)#n=A~+DPgBoEzMtoxw}(y(v15#{T>@ew+qC7GLvO`nhhtoJDB>ae=95z`y(O*F*XEv6R%Td;HJ4jp5XTwD8(-mJR)>%-qYzoy90`PEimV*D=_*d1xh9DxAn4eCw5U(EJh<9EMx7w@r9 zr{ZVBHzUt&m>2x7M*W@1Uk@easD3n!{LI4UYl7z{H!}NS3IL#W{Ci11{<&TM^DkXd zd$55(G~fUX+|Bh|?~Ej;;^7-=^C!s6iP0!bPQIbi9t(!nwQy1-qTp% zWUal~T(35_!er&11>5b;b*H9=dSJJ)s+7iz(IS9UDh#^8+-@jV^4ev$bl;MKnaIimx|XJ3brt~x*&TFsNglbxxQ+2n>xCzOp;F^NQ7Ouwc#_uG%1YnF;1@c*ct*ji zf5*We)~Nqvjo>HNXiY5j4V<;~4GismvF!hBHQ=u;|KMKfc1YaZ=S!;e+)9*x!}9k+ zXie>`EWg7n-80NC>-`&B2ShVKchVySTt9|JH^gOE*-s0p3-XIfyUuNi+oDSBd$AQx z*M`BNN1i-$ivhWR^d3z!oV7?)_*OQG8v_ZUSYXPi3+lUbs4*syBZ8^Y(k16nkGolU zhn1}nUiX+r?M!~-LzFGboT3nFU&bw*N?l2vLC`wRQ~S4H>NTkVVL*%V#*IQ*+cdrM z&Xemb8s(erLowPGkoi9_X*t?<>cyD31OG7qK;;Y*QhDyd-#`HXNPlu6erY#aYg;R8 z16z9&1H1op4dNBl-Y(LkchwWkaU8As)xnMX?pfO6{t=@Anq1U7x;^lWv2T7=Z!j+D%D=5EvXeYK(nEl^NlTv~ zo~MafA-I-KHbsMOnH|ZqCw{)ofmhr zGenIgvbmoFU3M`$m72kzdu+66JpCD68*^u@Kh`8nC!U#E_9L~vIdxTDq77hPjSlC) zEtJ1%v}6bhvxVnm*{mS&jDHeIknaTgs=@kG>lq7)c)sUYVoAYkTj<`hq#a4s#6aNQ zH%lGKq!Omq&v*gr%LGTn6723g5Aw{~$d4U~xd`NTCJ%k-8T`HCIe#EI^xSeRaFk(%|B*TR-2>%eIPjC^UB9TIY8(q}{GC`QeO@w2xqE$s|bfLMUXDG!Qtkz9QW z3sCmPG*8v$?mt^yHq{_QT#F1ritZl+-

4oLDMV;-fvS1=y>r>z1*r^Xu5ht+Q@7u#nhN@+9IQ-f}DV7KjO}J1F47;MEII zYquv&asICgw$hiFrUaQf^Vk6hbA)mus@e;7<|BIiYM2a(Ajz2Qx&5MBb##_W;e_N; zEb+P|ZInee1#d(Q^=a2S(HWT9`4NJ~?aA78nVEa^WT4RUx)4q;cT+FXeOM_|Jhq_X+?`CJ33by;wOr!d z$zUDXE@&9-d(wl*md9D@6R=dkj;Se;?BinxeX{y5#~gO}^3JnAz*a6fr4)okSRCDc?T9-0STg9CRaiMyU%tA z*>`AQdj+bfA6p)t%YWi?l|KbBlwQg91%#;DL_w{x%EF|vju{8(#&Zdm9|#FYvGFW- z5%nQ>ZRU#0fsQ5rV`}C&@!-&#D?^~511MuuXbW#QC1<-D{QD>x^!r0^IA8?J1=~a! z5tSpDxQO<)VXX&_*{0p{K6wz5 z$uWXdn9-c7QmMEB@*Z2<(jT}X7>SFs^r591(%R!*o5GR^C`5Z8s|NtGFa7*4#^4`Lbu2{hEBG8P&>#Q+ zi2S31U!3Ys5+n-)BkeC$`K^r4?1YGq;3%-$XX?G38mY#UIS*W5K{dm) zqkwTy(QLOx3x|#M2V=c2;B$!Gu&+Tyj+bt5+yO0vfotO&mnYly7X;$wPOev;eeGmk zB&`be2-81&$D=q3=cd9V_gn&gUonj^j8Sdu3oZNE9ZN2&FkQi>wr4Q=Tsz_MCC{fE zNP`yc;Op0e*TC4j@p?fE!4O@v!rfPwU-mzbXM>09MHVjRoM7w}boktLG7?!AH31^F zBNBphPhbK;Mq4wPg7vP+*+YPjI^XF(X7eFqIxxIpapMHdU^=`KY{Qzz^pB28EpjDx z#jLJujA%qaC{reNkH*EOKPeyM5YVaSG=p9Ti%eW zC1NT86g0ekl*|D)%_FD*_36#HlsKJ(xF~qZd9^|gIm@Ytf}Ok}T~-Tnw|k7c7`5Xq z&Yo=DL3U2waKr*2{s6P8T4kJx31=NMHPVgc5~d*aqH2-7kUSgJH>8f-!;@0CBHY8{ zp#E*CLlVE?5fE@%CBCZmZ6J1|z*ow5-z>~4*NR~mKDA5r(a1pt`mdC(tFM51n8=Oua54B(a#bjh_TW%=S*l?K?^cdIyumv#_X(3^b|v?nKpSbAsYL(A@0 z0eoYdzul8_qWPXeavJBIc_re6%?A#!k9fM7#~?^{Z9~gg`i>b-{BNJQzg=DKUglr# ztOhMyOGynGFLX%fYxCs-aUhFFQ!O}B)X5(v*fuRDn55ul%TX)a241=7F?7|&6d!I4 z`%che1XSvz2`+(E2v9R?~R_+ zc&RWmulnKoJm%G58QW=7`G~!}v>OFOc?tJ4!Sveq6A4RSrcrC&;+F?Gv7Mlx3pCl# z(G-gCm;;&!k0;$8nsp!*kyRC4#4l-tzjWde%PBqt=~5^k|N4$X(fF|tgZ$CA#Bc=`l$155?{l_ggeK-zNGxYH zom${%6zjWSyOB=;xB=G;PmbBnj}8ynn)-TgQ!aoS-;=k_ zzk$1-YmAw%3EGO4j1em!%{M^D`T7uuFeKd6ZDa1}kD=0@xq9HH9ZKk5i@J7#n_~|& zq{QC^SAnnttj5}zK$kdb35Ld#Ic&&kbFf8IgIUC&{Td$=kv(o1z-}!%8W`{75C{%| zjmaELd_Bd<8hF(B&iPS5L=UsZJRGiM0L5L{qHg4_D>+R7{k^BWSeX!Ai?C{~3^dU2 zR;>5+Br&pK+GouC0_@V!LG^kcx=Q*U3=ca=%s5w^p0&MMR9q-nZp1SRZpnW>OzaLtHsh5Rj#-mA*hT&4Gg)HfbN1E~Cn&@zMs!LI*}Xg$t@eiKhcv~Yg^Fcc^o9~kE}K_O zOdUSOq~pxinBb$qr#GfrEnyMp_s;>0q;;H4n1U}+nUe$t`&a*d8r@8Ww}=pIeMo0+ zYLe|pTagLTI$2snGH)X5CObK}b%%NSgv%4BlQ}pLzz8vXxMHyy+ZO)8Br> z6HIQbNFffD;w0h>of78j7C`Cb@@eNjv}JQG-0hvIOI-Cj@p}C9b$7K~IVvY(jD&(z zEJ%pBm_G{u6V^6*f`%%Q;!CNzq1(_j^N0+zYiqP@E}hvqta(xLk-G3y*hoprNO4MD z@nTUKdT7AM&M4nRK;2mPe5!IY7ZVNZnAaY|U%ei4au$Qw^17`Mp#pS;hl)m$G5|p> zX{sNbEcWOw8XlP&VLd$YM+*|9CD-%F?5V`sYHOYa)S5hUH6t&eoXF&^>|)1!p2z&Wz@|4 z-ADS4ZtbOl_`vaoS__~uk@45(`?C?C(`OD<__TODu30yUDlX`nyC|N%U%B;p44L`3 z3E8A6Yiw&)N)(3T#a&JwBOl;@$O*~M;GnW;W$*s%`qlr~_1W23{xpAcF#mD>X84)w zw^=Rlp^zyBDV1_MenD`7E0ZE1D-ekHr;I@Y1r!T{$O`iCPOTc_$XL!nOyjDSh%c3Y z@{5$shfBMgy@20KgZ0<;#6bQ8a&z2z-RgFI?B>*0dTlld4M8r;ogHR=Q{zHYMTI^< znoQ&;Bfk=j`>E&dHDO+WV~B;?Ap`@x&zrVcyAA{ah$Hg$C#RUnODPFR*u`$aBzSW* zh}3j-&L%t+dmqNIcin|?PSd&tz1}YF`hshcj_$)g0{pD?%}fA?#FmD0Fz-82Pu9Q= zC$0wWgCfZoP1P`Plcp$~!&Rk&u7jA|-~q`W@?zD)9xt%4D?=96^vDuHn(G)Cb!Ifu zY(jmF9h7!DzaG(heie#kSf868=kb?SJq#I|g3Wapg#gC!bt+3>mdSfRj)E50p!v=K z&Yv4zEFWvfb0NeQ*y{F|IsDLWjbZ@Rr!8Og8n{ou-j|#ty&sOO*G2W7Hf!2xzFktB zO3~K_I{bMK_b2~c61CSZmfVtLFO7KVeAZKu(6z%CwP*p~!(vc2i{>nvGcg#8`&MGA zA(t>@)259n7$dJjW0gmb!f`#td1r$&RmwC6b-n>LY8epLsD{Z(gN zvB%|WR?X8cSXHKWv6neKw0?y~XRRBRs4mz?IF<;(GLm%b8J)c;`f+IpyZGA@vV6^* zas%v~yG6NQTz@8+fd_$xpB~6aiF%wgo8A^c>#+KRL`a(ZkKN(-^ zT5I*oQ~Kk+XIsnd#h3Dny9N>ilKoF}gN2y~(VsH`=9^8>Dtp|UqL){gnWfAMT|Fhn ziY7-?D}AI`icK0y^3YQ`Sn6_3RW+BD&Tn-qKINWPtp!9Sk*pp}hPlcjWrSaB+xd8% zzan1dX=WgPJz2MfQw#l$hFAE_4W`W64F4kUY~Uk*MpU}cY)9%VdC=OSYba=M)iwH< z5M33f*=9C#Yr4Zg-4e+_$!P<#&XpuP1xPR5Oi!g@@u@8#QWbqo`IHkPlnjS@wIKT~ zrKfr$B)_&a;e?s4HwXLz!ChfR+^WWXiaMbo9rdzQu!Q!wLjdYgnI6wcRcz?~?Nmcv zb#B)S^{=#_&Ip^wDDELP8~fdd3!Nm&zJJ;OB5L?+<>qH1=5G?hjxiiSM- zKfm3pmE+CQ!zPb_gh%!c;9=TbVq$U{#ieysejA=hANvubUf^PF@iDE?(u27;Pf%G@ zcSj0N^Wsq$g8!?aGoIpZfw#A>nAMseHY}p849Ej__Xa&lH3qVqFToV>9SSDo@R#l7 z_Y>fS1s{Fudbq?;Yf%dLq9j4oQ@Hi9*pat<%DF_^`KOty45N`O9fHVCcNWnPGC)-e z3f^MQ40Is@ky7CAJvtjI)BA4Z6q?q?2HCJ#Asq_zzke68wt2tg|JmAU3rS?)_>dX| zzQy@rGF6{3t70fCzn`9}cJsi#tE3}drd{{+>rny-3q`W%F zv;uPT3^*Yb_dFHjM5eqPm#^no^@#&srn)wi+1sro$ZJljM?q1{Z>HjLF}V~N2R?3T zisJg>G~XZ(xb?0W8Jxoy#|IL7<;s-!=)OGwOr=*cn!*ExPZ}EyjN*w;#pmjzwO?=2 zB5oi#WlzRaFIVnigG8A>InPQQ$!cN~DqDQX^(z?$fw!#X8;YiaRj=p_EA}||GH+b# z1Fzl&8q1@xfecwVsU_IEi5alqAJ@o-=nm3@zI#xX^cLJ0t^2gz-&Pyro8C6n$le$&tMz<#B z4wzMI4Da-^-V#u5c3=Q z(Z?`PQQl16$>g#-$asrPW&}Jk0BE&Ij%(3IPGxmxHo9;&`sj~XcOEe7eUE;Hfru%v z@*n9)7>3A#=smpXK(d^PI-`s16!`c$>e(XO-v&X<7yEi1>(yi+sEz~+youiT0TdF} zt+IKb*9qsNEGk$_+{n&>;(}5r5!nGMlS-$D{Qw0;IGF~l4QZNU!XyeQu$%?iIe*pj zE#C)?5{kS_{;b+xMQqTmkvDZ77ABL^(ljaUUT2yn6$Dr`i4mK^T_HXB)0dM-^$mj6 zw6uUVv82+~d$MFn<3n3xWK6UkO(D+ZPsY}yD4$^5xtC(wc_4&#p0^O`!E%z;5bs^{-+2z|y@!_6ISRpHT3y%FW4}&boFUgf^r@Zhc%_h1Mh$xa z;)&xFf{<~89be|YpCGhK;B0!=*QEI++SbWiU1QSvb^o9w#JVMPwPgKdJ1SLk{X*SS z-{^}V=jguHQ@u?~D0{bz2BIu$YMY8+KT82LzdqOL`Gks0P*{IXW6pkAZjqynaa3(B zU5;TkR>6gZ3cJJGf{=~?<3pzg-7Jq}NV}H0oG@F}YGjkwp*|uOCdGly8CGS)N5l^~ z@8yef-n_Vs8}CbJd9HkZd6J>U>t&|pejUymsPkSED>wElC9d9kvekRG^VheMO%YWB zR%5=dxRk6od_9C)nYMc%;PKfWtF;zmMO-zmJ(QR`={3q#=8F}P)P02;@|YC;$-lALH# z7AyajQd&t&spp`Hy=llcq4b=4;Av^;@rs3)Pecq2l-{-Q*l1EpgoLCr@Eai3E3qm- zG9dz$>~>Y@JcG3?xOZ$&1FrT2?SPA;-OC1d_GF+O;N?)s)01K{hrzwB2CbT%R_rwe z6W+vKOUwkT$JePrm1UvjQ3fqk#k%qeJ+aKQM}xa31}#tTcoTcb*0fWk@nmI(;Br8) zR>Tf+^PK?avtzN6eM5w=EvZ^4>k&Lq%9`YoEpE6|*to8^#Wu%GKDZCI`uob`BnVFt z+?nN^gSYBv4%V5E))BH?Et~<9hxzD#A|!Ti%o8$98v|Jk5$o6-Qt_YKI9spu z6C|a+9x@0B#B{I-uiHLs^WQblMZrrm#FpGvu^t4DeUogQuLoYFV)}e2LDUy?`O*a{;jXk{`a{+ zmcRW)Uds5NlY?*m+v5KBY~kzwy6FEcefYC1<-fP$|CCRp{kvc9xA5b8D&qfK{qkp^ zQuxQ^(En*;pw%}px7YaZXE9IYjCzH0U z)X{JsxpYGaX|!6S(T3eo^$e~LHRT!3V>$#v3~;XfYkjG1TFb(Fi5?+7(OFuj9Q5b+ z(!tSd724qi@(KvPz7xhcZ?y9u=4pIePjo60{d0^%k|AQggy8xzhd-QcKO`{#6$TUe zW`sq>lABY?RHP?E)nB;qvse=azsUgy=R=-mj+-SQUHSqj)t@_CcB$_#_g=A!73Z7Nl#cr9qo$Dgvze66cfp|hxt%QnQ%-pLWo_tkLOGlShK z7$2JMLCV?pXbLtyufVNGf+_%2wyYy55U0oi?#h`tXet3LK5yA$H9u-1D}r440^fm} zeyqqonJMuS9yu;tV@1i*(gzHOC=@_^E4$GnG(ak3h#-~_kXZ=KJfZAKl8al?iQrPA zLtTC*FpXyd>->zh_bwlKvpK&cVj&Pi3iGIBZoiUIV-c$oBt4>uUlaihyQa*3U+I~^ z>MI@w%Mw!%HE{`)cl}4e->E^uyaY%7&%seuXIPWRrq`i+R~^1F6TopEjl(a&QGIau zOK>ze33Aw15%BP6Muvc8ldm@S>FYCq5rjD|u)h)sALghH6qwcSI14a_euH>ZRMU=4 zs`CY)b>4&{^(8if-EoaHVGU5&V?H*9A|ZOO?S+i-5*wK%4LyB3Ke{|Qq2u-NytzG; zccd7t?BmM5xw;M29Z9geDRS8OYs1jiWNh`Na5 zw^roM2`FJYY(2}>4BGmP-xtRH zLNhPID8p6fe#-N5MO+2|0F8g_%ztp7zfek<*jRt>-rvWVpU9=lSUgsn{h!HdIdkC3 z>U9Y~M1lG%tDEZPmnk1PKLA)A-k61jRbj*so6V2}C1Hv?!T?j0!5urPdr3{^NXF1U z5j+v_MH4Bmq!Lod<%oKyaA;p!H^ttxuBKkIaAUeJIeLos$=Sc>(gl1t4+~0YjZ!(V zkYH36E&ps8z}$YK1-)+LNZ{*z;3O#@WtzgRi0FVQ*q;g)Fh)Fs*@Jh2(dl!V1n(^{ z>i)K7#q&H`F1%n`-40PR_&V0iv8+%Dzy$Or54HC-fgunmvu}{zmp+(&eO`HreI9NA zpf``#rngMO@kp=@W8YTX5#J|pH~Z`gS4G)Nat*w-Qro@i+H`2zt{{x?SauEDi#qzy z=~~UP5gSqJ#M(;~4y@N!8K>F`u;e-|uv4XILq`n3R!eC|!+Z&Q_V|zj5}AWy3kvnF zB*T&sEc`>?#?El}V^JuOPx%@_NvNgAHYp>YD8I1x3Ds>2J|!7dY-eZa;az8o2I>=^ zD5eM}X%$gC)A}9omz6R~D6(9;NAMYV<#0`{C3S{Ef}6p5?rGS9S6x}U+aD(T9r#eY z=oeHz-JN$Ud8Es z@6@l8`|B{_sY9Rp%6?}?TG@Hu5qGzx5!zbU`SH2(vgV1wwbb?Y&}?~|`)8jo)B7HK z#}}?`3;QjeN7dp4qei1>_fJP&wdJffLQWtfM7?>F6M99kHR`B=3W2A3L9QO%k88B25GTy@!ro;y`2E>82 z?d%FC2Kp3gSUJCSSvwLlh{7w#iyZ)dgTmXXCxfMMx@+%zOQ7lqL@#-LUV?yco3Ct* zd6g9X$=8@inEucd%QQBII1UlIG_i_2!#iJN%Eq;+BCY!TBMg+1#OAAHY^n>))967r zUxe;RiJ<^pAB>9|KIl<|YsWY~5`hfedMRL93A~4KJtt?VvsQm+v{;!{wu$&|btJ0_ zesB^BBh6VGZPW1mo7sqVoWf9u>Yi}ztPOFyI?j893%2TyRa%Yl42KNs2qYH6%$PP# z`kUD#uvxNOb_Z7t9^@O`S@tv9Z^`f_)0C5<(63ipBSx>0f_Eo{l?KRe0QAYkFcvpU zK)0lQZQ|Yi`OYC`^(&!c>cB&v)MJLmuPP1Dqo5fLUx6^4s=v9*vP6s3qjh!#tcae0 za)QY%ddEp^oQg1pGN^jndIqY|?k)hb=cTvJpImU!MNj*HROlSlVG4Mdd3$x{yAoW=-j}77%1Dd5@40(IH=YdCPwkflRc zJq0N(7>g*JRqhW+>>c9A=S0zI{z1tG1u`Tqh~*>he0W;1##wnnTpc5EP9 ziNL944=|YHH+q2(%s0UrUDX31!hm}u9kY65HAjww7v6O!F2Zb2`^$0jxRE!uN^Fwwf;cnF77Gs0nz{hQ06pZ``T4ew2EHfd{1`eZg^N_h_m+hT zjg$P7*z~FII0Cvizto0&GYD2jiKVda*!Zx4GgE1fKF3Vbl?xe_A|;T%1KNhU??G<` zJAX|KCj07$GA5|W8Uq<`4KZc3)HyE(116F$qTL)DDs9$C)T75RyZ8|<+x6Buo@jKL zis)O&g|o40M(EL=EQHaZf`Cp(CSI0SyK$ZfnAA}SQ$0sk;cl`A{w4Cx+X@u+5Na{T zT+Irz4~Ru%@3ahLKp7iUhh&}<9G28WT@g(s1WeQvSM*C7Dk^QX_sSb9I8$SjuvzC> zuOZQ28<_`Fafvn;SoC&SdP_3aMT~pvbn>-ham^*3vEyZd^WX;UYX;?&5z>qfW}5wWc3A(`Q}%s%8Ubz? zq~{3hRxf0mAl!~Efk8+$e)Q(%r~ufN%CWfNNKCZW5>{)Hdr>c6K_%fNTT(14XZj?6 zBh!$orDxrpsJ3Mw5PgC^S2PkhLru|3jWrNT0? zSH}X_#+5!t^Ad%b0xS8f2IHL0hTDarbh1zMMof_oUa_<;ND{L($X8^!>+x7>)Wkkq zrrv_IfY&da`Ula__$Iir2mvKo`X@>UK-LdrhZG?|Y~v*OxPzTdzI zs}=DN363cXiPQ}GAi1r$En}(niCc0Y^A>ubHAqKkKf@BcKn5NPyLL6x=CE>FPr!uE zHm@VK*A{EBN1OKMQ>s_76nPnymdb|_x8&CpP(?AEJu8Z2=&WpV-7yq1;goP^^5~

<{2F zP|3{1#dVOX)kM{fjqN$2vk9ZBKlFuRfUGe!wk6VspdWmg5I@?@+nvvLkZlY73>eTC z$5vdCO)Q2;NW3y0)DVq}IOMYIiz`bMI6i9{{$vFT?slXK-_y4*t+^*eGYckSJ5c*I z3#HLuHxFin+@n`Q2-XjMfA_NO=8jHI>eHGB#-TbjkS2C(`8Atx3LvAuqRjjT#>IyC1Rqv zc}o>5nhp-eU3}W~8R*(fHXeOOX{a0GoB9gHWV5V7tLW=vJh3$yaYE(H=$>x_oOfZi z^t^~%9mL)$cQX(h9;d=d11aS`56GRQtLqtyFWx~xdT3wl@ZKjg3-j|?;rq~R$^h&7 z1r!%Qm>|}vpN;_X*qPL)n6pZVg54e?g3uC4+72PqtL8u?b$t!K;G!X$7Fcg}5q!q! zOAJ{Rgwh&oLz>=BPE8)}lEL9Lt6(35*V#I0G;^g1%(LEfvov%G-_l8#A|zPA!=x6W zQu+?IqXoN3SeO7exJ#SL@mPv^?&MIaj-BjCrU%{I30G~FBcJDLW5p=p&r(lojv=WB4yv9zcslW|1AA_w(gJLx`r zBTh^1zELrbQRs84FGKJ7A%mX5DNQa2P+T2-UG43E-i;$>bB(jt7&$4L+< z(-npLoZYH3Jr2oj5yEqRI|J}-6NmmH{=C+3%=O^W_>Nb^Mg86t(aw;xBu%|r#>Pis zGV*woop0bVla?g*__|Z{nTm;+FJOq{;eo3GBEdM;b$Ss2RfL%4&CgsHcSH?B@iQ|b zI)U9o@z)v1VJUCyIGf!OXt)3|^4wgMg5evcsOn+dBt=!;T(JLysu%x)s=5A+s%!s5 z)pb9m>I$nDs>a9m;gCZ`G!Db;;O~jwv2?g0+$%R0GHSD@(wo!kBotjpGGW{}?(Y>5 z!{X2G-<>(KyOZg5;omRmNSAz_#7W11Pj}qLgPnFlf-sgFQ4%Y43fM0hOJCdf4!S=L zWRvn1!eℜ77ty2SQ^4`t-<$7V40evPyO@)NSCk_^%z22 zvC3X$e1POWMp7zB1OeWepM!C)*eJ^oiikixcF9pz7J5uB%`r zJHK#4+E4-9ag{u8RM{XJD5{YKT-LVuI0&HhEzc_H6~{sUEWwBhaZyim2%zo>fg?@)E}FH~Le zFRIr08&nKSadk83&TIdGCPu$fm{93#`ZOV1-HJ4QRpE{6Lp10?HP zVrKz%`TKaYIDVpA^?>a&kD6^gW9o7R!PpuxV+5dXLamdyYz^vjjGl=Jd(J;qygrH6 z3TnCnzjKC)bb`AA#%{i8H*-H;81iO57Jz zdIxiuwG=;`LQL3J&*IK-s}SKMV<3=YMNdj)&z&1!Zbb)4l`)s!csmOJK$4 z-d9GOyp}28Xd|+pQcL`<$BT2u-FH?!Zd@X~R8AvKdMSu?TpxI`DpBrDP_u-^V5Y$Yqz@JkW-Zlf zyClWFIY*E6fbS}q**1Rk?m{`69QxRt?pZ3F>&x9VQq&GZ^XavDh7_A6wHVV94 zXxJb;Kmv{DCkn^|mUC$=pel(&`?ARm@6zxkjL3uwo81&vWpWtn98|8v7aVsLohFM@ zq;h@HsKGIimhd}>VMd!Ruwh&-KdrO#HnMYYar1b2*}6L0+B(zUS&lz_|G}GIIQIWD zR)1f%@}IH#Cs~!h#_E4g`F~3B(Ekyu|48{ST>Y~DbScdXZQ2~LrlIZ#g6SrE5h=f8=3VQ|tO2HP8Zfjld z+_rAlQrSu~rYd4JaKlw3y`1lsSl>dL;a8(V=Rhr{)n+n0KJpJnSLtE)ExGr=z!qjb zmPs3X6UZ$)QMlt*9FxPk%pye^^b-N#*rEkAdYrCvWGBfO)UV5_4_$|1I~o*B0PKU@ zjH8)=1UK+8NUiN82~z+w-Wn94c=%%T5zuOuC-536P)tONl;G2%y}|D3;aRGCbr%?E zGW-0~C0Y%1bWUlG{ejF8l6L?TTATsEsS{VhRIH9O^H|U^y-HpN#5Kf_^_K;&sC5rE zR!G(?%{6v}v9R84W#_{}exM}%jD=#W)i#$0ufj35^^kk{-~bN=vhs$YB<$E6)fKay zl#sanq~>r1J*6--=hMZTNFq$`k}xmocyZ*on0Il5x#1rHVew{ALE|P-pAs&6QdY1B z?$q+0f8Gf*5qiqo^)-&E(kMlQI=BLpeRe5_TXnw_wnP_W`0nit1d@M;czqm zFvAhPh1zEZ^n5+CVV0;NNh}`1yeMq#Vg(F7^Kom3nzj!;H0bWwnaY5u2Ln5m{<_g8 ztOV9^+OcW@?aPcp?O=Ui`%bEo#=)H)HE&+z=@4>i=~`9xb2Pnurj?fFk|Ao9l3<_r zh}j>d(GMrcRgK$~4-yh3Jj(wiPj7vy8mJo9vOS;L8@dp7oxqBx`>#?0oXC45{)?x36rgvHhhy1i7^dqrNC=Hy|>#1b{mAPj@1eE(WtJxhJS^%kkOxoN;c~M zRW->h@**w}q{2~jx)`m>SPRA=7m7Kjrtqe_8uqQ~?AHv4%bHG=QTQ z=z3N}JcDr${pXT1*JU)(ZE)T)i*M z&&B{5wrHpcdYHKtpQM)a@KgvC*L{3iB-&*H&AR+duLZ`-O@z*CZEGrj(TP7ZV}8}% zx_{A$#gJ@_t^!KIHkv=$OY4-A;lrTSB}FCHt;U>>UPA?=PYP=ekU0bEldVK(uwz3u z#ARyTmaiETt_Mn9wV%VSueD8Y3D-O3ns0m!4|nm`2Gb-r)=DxYu#fTazf{Zw}eR^bMXN8TU^LvUzm?(>8Xel^v3=rW=Z2&UXL z%Gs|NX@pqo)k}(WpA0fH6c7gL@a9|(WQeeK7&;eb$vnVT4(ZLR-*Oo$uD)%yVVxWd zyxl?Ggn{zh?Z<*?u#sx)s%?+5q*3&;1|)`zNTUb^aa={j+Hp1sB{~?`piZe*?1&V? z7y+$FVBr{Ka6rqsD7WmTQzS7^ui*0q*(%@hD3^*YivqERPR%8gq_U!*UYAH%d*A^9 zvw6&Abw?6(m+Elp17!o)$7&xi$oD!lt+}?V4G4Z6W;*z${!L|h*OM~vaJY9s;pF1$ z=L^j!dZB!X)6%REYvA&-))Xu&=SRY+6t4?u3U!~kvL|Dr42usSmqazSTlc#GA4`% zLU;_An!l<6RSzqqV+~>&Vz`z+Wy2+qdD48P>C5c&=b@lQ7@KuX#ECJ*{s0RP%G&IF zAl*SFf&KPpjaY-vq*f#_t}(c(9yH4YW5EMIg8O_{1?t1ANRni^@;R1KNX57kE4u3i(89J%IY`jm4s!71Yb8y z?BFbc_2U}M{J~(5!GLD=I$E?YLH*|u%RP$qA}=B6`*+(2f!l4>Z0ZzFf-q|RN+Oq* zF5U)Lxg_3e;xs}ZU!mZfqp4@6G?PLa`BZIhzstBcRcehC;fzDRPqykWMf8loIqdQY zC!O>Zj-+cEjM#`;v`S&l+)glLrZXod6)Y zM1@8__S&k6wm2pGeyqTfELiQ)Y4Oclk<{{2vTP@wa`Uvb&rTbN)jq7)_316|p&GY5 z^5nrv1&o!`Qx5MoGf?b%D~{h*l19ziYjCZn=P)<3tcMy9*@(bK_ml1*4x12A6f`R^ z1K$DV`hnJPSq*~8(Nig)FSbt?KTK!TG&&~MB=6@*(B{nGcuZUd(<3rWR;^U5lJ3g8 zILya$!IePB%Nc+5(dQLpE3ST94y<4d5dgbr#cqT3X;@ca8*fW8jqQAUxXvier8lr; zBTPchPX1PU!F^?DEG}z}fib(@DE`rBq$SowH;g4a#qXZk8)I5y5?sx9`P>4~x85I7 z>^7S0vTD=ZPg^*%U;8VQBHcW&n{5%mku3{se8a2aFpvGgZ{&(^u`gk(6wqqnieJY% zK8wUm`wpM;nBEvMgJVDqXSCDUkAAeoMsc~WoHBjb*xNZAih3w7m!>P{)l^T{RR54Z z+y;F1yZbKwyhQGAuax{vD*os9L%7!@z-2yDBI^IXmfd(r&2Gkb0mSAZP> z?;Oy&Kl!9^=5h8dMNhE!2n+gpyz{FD!^9L*&}b#^ay6YpU|lj9K4NF~JkMco$t$M# z*w>xF1PEJR&#&P$i zE5je``{kOpuNS7>V!^qrL2eo;SLC{qiBCXMHd6e zkHiS%t~GG-Aj9x31_e!NYaIRfq3A}0e~^8WA{n(%9by$<=Gs$HmYzc6>|gThao`X6 z75n${Ywv%OU*Xwa?EUfsGV8Xmg~ z!1$kH!XB7+;8o`|fVnqVx>QVG6D6Cd7jdYVFDQ+3YtjXg5RnxoLn7=NEPLEFU)HU< z-yF=meSP5ZwA}XiHIese?_?DJ@nX>Bp^)mH9b%Pg7`Yr5NULsS_lWj>{+P3E4EWIA zFr&?D%HzCt@!-h8>;KXAj?tC<@sH8V1vthuiHx}V?X=DJt=ve$aR?!rmumF=gcTYK}mixw%w5waQd z$LDp9`n8=PfSJ{J+D8~b%=5ncIP+E?Kbp@r(m;14)Nd@5^Tq2EO}ufQ=l(+{Z7qV zE0VL~TR)2RO)hJk+3V&PdtLrwubBVNUiU3jN6H2N zPwcgIsL?6u%u*lYV2dtHCVE)M&{UM>D&uhJuG zf(>}Y7+?Q^Sw~0WN5eyw#x;}Vd*-W8SUR&-DGk-%n}9^JzI^F}ct zbcW)#)#Y{L1KB}Ofy=5EJrTZm5_@7e>h)?y?1&q3z{!HR!Wh{bfGLqA`odue$dSCa zMT$=-*CW`9X*FPk8(`3{dc?)tYvpIuFeqj{av-LAb?&NlF$U*w@S}FrOx@;r?N@f-JrP3!j=lk%)0pOM+i)0iZoZugU(I%lS$#L&N|LYlrAb+wXJoUx1P4<91s`IfynX3 z!HUP!hCL7ly96`3kqp=+syL!TyZeUU&BfV_j>BNsF-V}9YZ0YuL z{mnIuxGQrJcAgk9%*8F;bu%ypy$3>oact?5Mp(UxDd-T#3ivmDmfygtqYS&2Yg5 z$#$vzUvM}>!Qk$}#ZJ`O79?On#0ta=>!CtN512~&b*sna-R-4%d>e(~PLGk{zGS?3 z=}RTwxIK~j(wG(%G#E<7kI@;@PvQlYy>?-2<;+h#Ot;0~gP)~oK&ObL6J*NN&#}yc z%O#iCF;xO&s+X7+_*HS)Q}XqK*X9(~)PAsITG>`w<6yj0*jCaNlN^T1Hck5UhD?KE z?~cJC(OYIX)M67PPS_B$7^E^F)DFS@E^4R$m5O`2rtV{^5epF1u}KWv^HbZQUFFKl z-5T5ix3%jMANlMK?wO3ImVaef*?vO%8d%jhMI^IzDFwa6=Qb+e`vT-dMdsS7>8pqS z1jCs+9+33)uqqTYy)KWr;`H^f$|^48>tOZt?Y0gJIBzUeVp~T_?buxk`ecg6s`YvE zk~p9h6RJle!A5w?|0zaSm#Nnw3}+{1j@3r|XXK}QGW>$+%=%Thq3R*;P_ zPFNDq#FWTltF7|JS_4;X*%JXM+mO?5SgD@%IV5dr%a=b{mDim|S`{LVXx8u$$%w?- z$hbUl9oc@U{}67q>Ahtj4LYPJdBqFWEb?o!$ra+IuwL;E8mb#tiiIue97zMS!y31+ zMh|M;>9_r`Vasc&_EvOYyE1txq$m|TMe$Nk(^=V&L=-R9U@qqvk=Nx2Di?m)k-qk7 zq7^{MVDKiq+$;5UrD22c1POIM+AbqY*sK?~f~uuUnJ-{+s^{WQrjw0UdTvqszd^6B z9S0oz?R?!ltu3B@z7CGQzK%Sva(_p!|IyRKGBKR``M+b3|0kp2pM3WJBCBsym;KYR zEk=th$H^Vbj~cTI6xggQmncOlQ$z?>Qw`1!d-s<-9R9MS&Bs7J{qb^jT@+omq3Bow2A_?VuaZ+q4IA>JeD84;N7q4~f~ zv!{b0AhRKc!rKHu2`MmQhY&$bBjY-Wj$irEUL_?WlT#xQ%Cc8TCHhxtboga3h~pwD zh@QTl3?urAfl$xV%KWWz$W?$#0x?89?EzQ>Wtn|omHHIm&4JineOo^)a*HC{f_FVy zw1g^2e)00!GPj2Sq72ti`7(E6HbVP`>q?Z~QiJhPO^{T+yb`ByNVu8V5xV!CztGwy z$cGG7hh-Cqs#{af9fVeYRcr-lTevpjbIFS0;9%ypYOdF8T5L~yL_0=M8p{<*iP|4U@*^sd#i!43{A2+s!iBihDZ)^;On`Tv z!mUV&+|o{}qw5l-`DF=w|8xDj!>lr!*o8oJarAGsb7$2wTWeTNpy^>?4UF&|W!#7|8xXmZG z1&ds`aU(Pk89A;aG}N;IzB-&6bbqEp28(SB!m(MuS)o&o{c%!|QiUgX?`zAg~y|k+HEk>>N05DKl@6%YMv#rX7v4;{Mh%k1>+A z*O^;(pgejmYOSy?@sf)^V2uu2#c&EcBtbmdMB@3mvpD+We8F3c@}%mqL)C5xOP&2e ztV*vYYwGG>6>>A2f&4!;^#6qh#$UzPe~d2vX@33-bG%V~{A@Am}aE zcO*hW6jzI&QjtXp3qwWG#qU0eGTiW1JTX}1-XPd)Gzi@RAmN}nJu#&CeG?pzgz{!{ zExIe&oJZN*PCl3Wl|Ju1ULZb{kq$G$VEWlx&z1;z`ZuItLZQCBM?DjF)PzET;(?6$ za*BcwP+|oRAOUGa#?25fU15WIND29*rbHs;C}G3W*{ieCf8N@@vZk z20cxwcIJ*C7keTIz~b@Ffn^ouW%PoQ7-oRe{bjrLEB%nn^YY>H+>z)~5U3^H)nnHu zs`KCnXDcGNW$Z$$0`b3rBbC2J2EaorLso2Ej*$MI-(1y%$YyBwz025F&a11)89+pMrJ6ig{} z1$jSjxFY8fYwbZ{B3CLRYH=9Ehy3cnez(BU#0s2a17WXLil$^8<>vVnvnVb42s@^M zzRiSMYgN!hD5fmNP3_uNDYM)ZN)uqVSB>z+-^Hq^b=y>D*&%GLyW7r+ zOoNwSMiw~@?*y0BfP&U(XPACT0WVix@7iuZ;d5_aUx2T{__td~>k8%nZl{Hy`pK7xM&V|x>=llT-^JR58eJp3!_6j5FD+8r!^rf zeVT%_^8rtPA)|5VS+!HKV%ciAzWweWI_m$+>EC}Ep?{9s{(J`gx&9Av+y7tb-#<6J z{`q$PuhKu~$1bv^uZ`6nbpIRaALGBIf1GUpbc26!o&PHR(~|z`s32zFaE>q@g~DZk z%$N_Ql7Mf&X;s#4H4^fJ^$ut>j3h5u{meMF{nJp7?=hgmt2>)j)#n`@|Mge{?<-m&_1l@jknw%_^(e+eO5>=s;452LIdKK~e)Fj> ziRtJ{wU?S5(340x`vWFMaOL^r+5#xQC zYC51cDCkD+wsyaHv*xltPfDFGKHOWcT!q;3EsVLuB|he8byj4459`#G-A!$YgtIjL zuHY&|mdv#w<26EDPx{RU*it7Of^A`L93h|GWNcrZq=IcfQs>(!52L|Lzwd5P*L@-y z(3^q)k&(cyAEF2D|3{|#6!X1;o>Y-l(s(8!kB7lyE8_nlRTMG0Vm)F)P+%?*6eg?M zTj)IbIq~vM_+HG>dpHT~TGgVJfwnNEAl`H_bB@YIf>jvf0``2CDYYp%R277N%tqns z!@E%`g;_W8o;BELjeoVd%!qEPL?#s*Pd%i8=qviuR``nk2(0VueH!tTEGj^^!H!)7 z22I;qY4R}k+i5X2$Wr&04DGw65NmtuL|?Mdk+Jd}x2@wt8U1VuprR5x(WW{EHJ)Q0 z>FG0|pp&t`QJ4&rn*Rj#Vu?hj#RLNzHd96G-1| zd;K_A^!s>jmiK$J`n>9X8}{?#cpGQse|nU!f!FKp4$^M{Jf4rO(RP$FeG$AlkwR9W z0OoL%!O_n@NeVF@nhb{iMiy1w2)O^yAdWd`%Y-;NJLJFu8NdO0> z4%TE_8}m$SatKoh;SfSjZl(-|UmckR-eB2q)`ale#kT&Gj)XGD0jM>|`E5P&fywFxzq%Kt>sPT~D1Xfn1nIhxK z;P6qcA*7Qay4mV$ZN;00t#Jlj$ncvVUNPX;Ty`UeO~FH@{*~I9J^?tB;SJGceh=P| zL{R^b2hfseA*^rq?=Wh4s$p$Ktrjh9bJv<*Cm`+yINp zb-8KGUlpd&#>C#cqDgdny@`36;9+U^Oi`zeq59TbU@pG;B_|y0Ru;=F)F9A!>(XDT z-9qVCYIni)mD+)mE?engGa91z+o1Oe+3%JR* z2-{zy;GtEX4lTkP7;}UG-$R`1<%?jU9C{< z&uMj` z4zDqDp}tos&OiAPuid<3iviNBD6+~DHFm(gMzI#mm~C-%xdnpaCjEV*F+MV;*V^@f zL}KJY82eE>AVYtUrNzKS(jvf8w~pEx>&@Qc*KDl?#cjv0y$DXgJGR=d)hGU32Wk;I z9U%tyOg+Rb8QP4+@t80%JzvcZ>O#sbGZErRwAsZDtZIF}4U=H$$4Cj^G9fzauakdp zdC?J+o3NzZU``QVK%-5)j9bd0IHa`|(nIq$A7@9Pw+EGWm>g(EGrcCO9R$lljeSyb zsOl&ev_c=Jt79^xS1W~#EK02_$}^&A+122{lRt~GD|#~rl~%q&J6t+%Fzety#SZs% zoxpW~RpU#nJi;#p4IKM?r_166s_#6Q-B+@Xz!L~%AHJJdIQQQ}yZ*1xZtM4S_?<`W znI+3!ygW^xX&xe0i8(?j_z}t@zYmV-kH$sY1>=aHtSkD$v_{@m?-{{S<2hv zE)(CY%#}3i@cLZ6S&YOir-JcXWf5UDjLs2uvhP9xU`Xj4b|1FK z4J$Pga>{cb!!0SmhWm}`n_dSho zyZSc{f}LtwpYz(jw0v34%DjF*@?#;L7)|#t1?`H43(jlbvAU*OsP!xcby@{R%bRc& z9?`V%!($ngjwQG+^n!hZ16Jd(F5Z#e@o;CdHq)>j<}d+%8IBZ~=KVi_p zl}e{6{P}3wSF>Z0M+I%Gl5Tjeaqug&(|&+8p_A4QbJ3Su!JghknPuhIaqe^7jAK z@1Qd``C8xp&BW08e_UVox2Md1-P7_vF1IkUvv#qu)pv3>bpHG1_Wx#A!GEwF(;iI( z;IAWT!g>EM9v%PbJ-*hyS=0XYcC=sq7~|K0a{sxzVr^$e$EeRp%gDe$%gD<4-xV@0 zv>;WG)zQHm^5{U_6e^8S8X^sm5k*XpA&C@RHG>H;MiY#Z_0IBE*b_3OP z1WR{A^Hl&5DI}m(XogUHM?z?O6_D z)(FIE8t%jrJ>7%l=uL#FVoVVU4?>IwhY;()9Mv^w5@|@h+XWf&plJHXF~E}G zV;m}xVBn2}_PX))9ZI`XV8abDHX-a00tt-txA+g>e0vfC!HK@kMM&B&APV0nj*DjA z2YQ7@U?2n#1>p%a*+r#8?oE*&-x(ha7D~D&RU{N09yuFCk_1`c0*NTd7f6n^7PYh> z0Z4|BNQM+e%G*Tc2ht?!ir(MLp#3rud@$i15w|Beoxo_ANAwevnr;U%7E1cJs=6+eXlK}uE}(vxo>3~~ftPy|7qi@lFLUz@r4X$A>7o`H&p z0OdFU+7LvUU%Se-)M0h`w)P|rdbY5sda76P#A{@EH@I@`RH5uLsf*UYmp*Ro* z)7g|E@DFY=y`AI00tdeVjFLrDx?% zg1Vv|K!J|-Chyvf+9w)PyTeOj0>PhG`|AQ3<3x#0DV!O||L7Y5wv>X5ER3^Y7LYkW zK(^r{oCBc85>Qf<(N!Hd#afwA`crpaZG)D11^p^jKX>jS(KrFW2(M>f$?^>^PzNqp zvTyf=P$E;tpuOdcVUdGT`7GT$KAGXN zXHg_GoP|P$bHT;U)+K|DpM4`Y(OTGEliVo+7uUL>?zj4&<%38tCq(5e1ysZs z0SK5|DFp)v!9l=)b0p-(r1@YGnj!o0TZ8D4BGx(1rS=Oxi$?B_`+`+m>l^rbuFg_Yvb=Svi*vcKQZzV~1ra&;)Dl$d$oCB5lLE&&q;9I$$voCvX-Co2X zULvTaGk1MlJWnt^4+^`Vh}P4wS?h8{4B)~sFC`+%Wxy=841gD4;zHe6LTQ?753)g^ zupY^vq?ZGr@wG^Q_JC;H%X9iLD8u?&ijSye8}O8{sPpN(q?>a!#5v^)%K!$GJ)WX3 zpjd&EaV};3-f^p=P(Kge=2Z~r+#tW0a*yCdZLn6W+nkj(xe9ylT2)9nEOu)>THbw1 znVbt<-KFU9Z}a1ss?(Lxr!ZV+z9TJHri?i zTmsGvW=Mp7!B7||hU~7nG{e7E{cTJG5^(FYT^UPUEcj9ZWpluugEVE-GcC%U+!vZ2 zQnHAvt8u8qZ#c1U(bAaVQbb%`#G3*uhc0J~oE}KtDHQ1?8WbLr~o|t^}+W+4SLV zR*Ie+$?zl5$f1MwXmBv04#4``L8hD=08z6U%RNxEI61t_yBF1rDA19Loo=7e>C)8s~WxW#qB#T{`+blw!kd2fsS?;lM zo6=zvdng4YZHz=f_2+_>fRfqzHDgQ{gsElV3MTtA5Tyk*lg=4vzH{|R%H`ti{L)-D z&1NNzSPX$8M3hV%1QikJWbSyzNEoi_76~`6ax!jo-kG8FErcXYyth*M^VkxYjsYIW zTIB_nCD|S^Z5YV@kf;Q;gNbImb`x#F#=Df-jEYe#cdpsp;npulYaVXN+TnG`9s zkPxOaz*)ZgnVN-L{Tgd>rbTq0`!W9*_L}1flgrZ#wFMdX)w$fuRr>ch$h9Fy12DJ^ zdaxP_`?2-kxS{tE2EHfx3>y0Cer?s+TP(Upn-=rYrWlh3*|AVRncNtgYcu#Y1qB`Q z0t4r6OiuvlCrfjNL1J}lCT7^SSI&+6R%IU6(y0C!PRT@+UE4Eq_Zsq>lN<=K(F*neG)^05~AOnPM5Iw zvrB`pk9lFZZS*^u2!~@OUbc+a>sN=>@w|-c&1JfGqU`86wP4j(yz>15*G~0|Ki>{c zWE2;+fWhSbC?qk;-2EF63j5)m$U=W47w*$)7qN{Hg&^lJ%VNZRsLc1$n%5LCyao&+ z%D%rl!Kzjfhkf@nP;8~xoNiXHRPW!@)Rgx9IBggrjs-T6)RqBXV7qtYGP==tj?HUr z!fc-M?WX@WpjN~Wm7fUic*UoH?gbYE3D#;ZkEEuN2*gKD|oo!G5r~zU9t= zbSN8#8lD{d>s!wCBBJ+ULw!eJ?c%LYx5QvY;_#y?6IIoiu3P=eY2+S2&AgQnm>B!! z0)^7Udq|2N!il59CaJfc#7PcgVs}N|ceEO!&JGmory9Bm#pBvstjVU$z|bmaDbXZ2 ztC&rlm`>Kz(#oP-h7Y0kh_U1g;ya+=t2rHl?F@w!B}@Z(eE~;k|xTu`UhAdi%M@?yjTG1e~D; zN!#ItZ|m+|S~8mooyX=yq|_b%gg$uuT6E-L7$;vfw?%10pz`qYThh>D>xK2lquOnO zQ8U)U4`bA4-3XSa#m4j0*J;YJ`bftp$n5eb*Rry-(WFq~uw}8-sS54(4~zGHGqvKI zpr=^8rPvkkGgF&_JK_R|%ao|I@#*J_SZQ2;vYaxPw!*uomDAUFyBILj&5t1~#os-c z9(KIrOc|T~9PoEt+)Ynw>k}Pfn@)0C)z(Z@F`dyNuc+{HACp6NtiM$}MF2QiL|I;2(}sI0U)mcN2d?Yjb6%kDDWj`zka!I9^A zF7QaWGF;5pez2RWz7{C(`F=J@&%x>u6L_v}(;bE}one>X72wz2 z9^QHrJ0nuxtkSbhJtK(gPY)}^+vK$QiGICRDG%9lS4_X7wdnIKbvoq#E>Ga4mIn#G zP+d7VS7pUE zi=&j*bGO#b?tB_QSh}Hna7FZUN58B3riLcJ*1SDVbTJn?4VD#_&on(qH2|m6B*&u_ z;nS(+{hHB;B!9K3l`04Dhx*uBGN%R$ov=Q4hlVddjR2(#LY8gA@%N zd`F>=@a$x1ShX*Ys97YPznGU#eyYI*z_fn@QAeAamObwzxt!kenyQ4G>n>6-GGB84 zmU?rIanMs=BPX{R`z8v{z(avgQ1avijU}^T!y$7u&jo zwrqyOYGvk@I@ePwPFTEE0SAm@!rc_rBo}?x-c!v~NT|(DOU5!=Wlu-k@8mMjJgi)^ zXEx7c(6*0JS9rhW(l6HB+&_j+kLpv-8}#>i7kIx7! zMAwI)wwmqUuUjc-w*#WZcEoClqdv0FztK=fp?j4_+Y~FcM3gLXSd$YO>W4pi#1++_ z9AA6!2uiFxz8ujke`smwYx9Wl5>^>MT!5b4I^nItF?Cs98mk6GM|yGXZHb;oVsQmN zaWb1o&n<9lY|g)=i3$2QzK_Y+;a{8{B(g5hlj3zP9C^;wg7-G-Cs|p-?jo2ks=t_D z&Wpc!mzSGkFEy~EUs1;azvf}5K)7MdeK}V-CT^2Y> zw=D16ldMz$F6z?=PIk1*9DAJwB-d1Fd62j?2y*G=zE)dy#4dZv-zKxf=^=a0pC8PC zFDG6r4kONb_%di}?6&NS&BO9Kf1Vu{h`xk*k#5|!Uap)ItsZ2>!Z-0E{s>Iw&9Jwd zpP@YaQOyEP-X^;rL$E^&4e^L6A8knb9qR71weLPJ*7Hb?pd%@#!U{GsH>-Hx?r;to1x(4P@d+hF!W_wh^+Zo8yCuJA9W=IiTPx~1BrZ>W&|rG}b*0%BIDjYz=p zaMgTb#RiklIZBo&{l;%zZ)ds=CO%OOeWxLl%_qdZ?_)+_vGT{bKIsKj^;ulX2ID^!^jF$g;gN4IuAEepy;;Qtva_5lpYsn5@_MiQirL<_ zlBT+k@VETmc$8&7cP})5^lHLKQq9GZh}yiJwI|xaE!jMI#~ZMRwodaIrW>+TT~CJG zSGdg*6CN0rY?-R7sTH%)c61lTH&UCH*LsfJE~JHIBO~h zHv}DlDt4y~s~cOt&MzKz-97ApkD7EpF!>c^A}MEW*6v>1;nT5E`%+V0xCp*@S)r3t z%^S<6)?{vIKg8-;4!_DOEvTl>4$I7f84F~;T=?;R|7LCBmo>S6!?s{E>dh`KLlNqL z(Q&J+a*WAaV0Is!J8S@lacOkWR8LM>*v1%j3QR;7BEazc zRnh$qmPpEofNX^X008*K5_SGR-tj-N#J>u=|HcwIm}wcA8UMR>!QW8gqnRQ!Egiwv zH+#kwXh6g=3e|o~5rO{v#$1@u;Aj$N2U8#=z(G(zRA?1MEx}4`x+h9C&4spt9W^b* zC~6n<6e!Co)oylXNr-l}cfZg1es1ziX5yC9VnkGEhI~-V)%a z<2wAy6HXl}tpIy*2S|}%dhvQ#@jX(QgSeuJ18WTf zu_C61PvC|lryRsU5g!ESC=7ymD3L}x*re|SG^C|4X}vsN__k|Mmi`6lcKs$B{KzrE z9_8W%Eg$D88nk#5G_1&Uk&0NAApYVAbcQp8>IK4}(zIX=RK_flgpRDi>BoifvT)Qr z;}ROhA`-Om+QV(+8Up^3*QSh;`TAB7^!=EVn$PdO)QxWqace>ah1^8K6e2qO>@dS7 z5YdYj@)U>=MI{9!%=ig?Bq*T-7$j)2d+&Zd&j&Lxs)Soz(2GDqxio z5NA9H0V8xGBP~X?S<;J6hlvo_TUg14VJ8G0L_TUu_ydw*fLg2wDk^~(byf;`pjSE<8mPBP6U>@)=TqA*`WvK03#LFhQFTyCt^t-a=N9$sg^njVPUHkq z-ufVx4whqrM5Z<)*rS9bpio)j`LdMRIrZLWA>=dS;o1|VvlG6S*QY3Ws`1O);xAj5jP7;Az8)36!D7EwV^jK!un3S4Nu4*z?>wQ_sp zUd6>G=lg))i~~k+V`>L?Nx!bOQ{{ycePAFlhpM8EW2k$-*^E<|9`l99(Ht!mITuZb zG_TOGtctHvO|&kQhg(NmXp6B?_<#^@(|XvTesM%wZNzk_^D284kns=l_HA6mJV%Y+ zXbC@q@fBA%zawKTDgjc3*MD(Ds4tE#Vats zHio(T2_2%kC?xDt=83uHujS7b`;5ZZi610w8`=rHQ869e27+r zQrCz|OAa$CuqABP9(^o8*0iwCN*x+W7G5Q$E#tQmycrr52$ix>KA%x22s`&Zbl9t) zzu~5L@i^QZCkC6VO!C;=Hmosxl1y)z3hN0&J8K3NG2rhUapZ3tahT|CfTSx$N`n{4OL=TgyOx;M;O%p79+he}hb8*Hhq$i))@ zxO_EYMHsnjF;%SqVtY>8er(ONFe`8X*&>wMFlNpzi35#G6f(8}HF(L}t$IT!$y?Xe zNXS1D4^Rvh@=ZF6^D@+yq|PIljrMqmyn8kFgVPcB3I{2DBN z&~c>lqb-@5x_E^ukTU{URbVzPGg_gu8nvuTo!zRanbX;sLhZ$d#G7nfYBX?|PrtP~ zq@{XuM7DI>OK|Ha2t>%upTGerOKSB%>ww<^IWr(a2%UR!A+_s(Z8!tmc~(rTDHd7I zEKrd}o^1%y#H=|5Q5!Xn?- zwS_~)-?hIBDzmbJ;6Ags8n{^m*$=Z=DIRjJ$AUyLivlvB1*`ssR;F05 zHC|(pg(>$Tq(qo!uv+5(&E^kH0|t(r>V7nWwkN0yh;k{Qd=tiCPPaysiiNS!hRI4ck`MFM^rAg{Q6L7*M!9XHu#e=jH1R9_SQ(3^uD5MANqS;ByagAFu1}Nm&s~_wv`*Yt; zudCJPtX;P@vhMj=vCGgJ0uFj$fLinc^=5Y#nryH`VIoXE=S7ECdbfHa<<6a^VBLHHwq5cc5w7=eo;G|7uy$TK(sw0uNWo|>uocaZ{dQ(%%EX0gHaidv zOw)dwJG!);`@-)voF&%R`yQ!?Z%mK0eIAURrE&I!vCk=v;>NR#JGO=MO9Ct%ScR(>ywO(jqCGUnc321qUk3kX(ml?W9R5M zQ1G+==w@f-V`b)JXQ%Y_@m5kYRkA&F_FXZ?$g)V~qa9+Vvkf#hg+UB6L=ub^2!tJF zlzdtna><_*DTtV+W%~RXK3x8g)qWf;GneJ4Z2}+gs*;zZ?+^TR3eIPQZACl3u-kX{fL`8^m;VZl z)9xY{&Lxq}v{KVrsiUIcM3iQ7>{$MosE(ClaGP7Hh0b>a_}Bed0(Js2znb%BIAu}utLZ)pKW)>TaB9DbsITNooWtn%hWl4 z1OzZ6HKB#kXq(t{nyn?v)7Er8i)g(0*n#K7O}d#4g3`VGud<0?{RMM{z)S3 z3nx8Y>PUsVi$DMHV`AZSEp8b)UNRXlc3QT98CP8MalLk>Z0XEX=b`2FXSj(Qui|)! zmzdqptQ6;_oRRVQaxW9kl8G}cE{2Kw*Uq6R4`MlOS_cf4#oxzi4+)b6Z%)pWxKXd) zkECV&=&nxM_Ii~1r@Zol`w~>LR0>v%Mdf|;o@t*Svb<=x#HMG)#WvV^PbDAp58hjb z6)bUXMkVc15+V;?_;a9Flm;8=vsNW*+wR~>sv_;(o*EUTy=QwoR-F~U# zWtew*JO7@Yv%^Iv+s|HAbMaDL?WcDt8=J{Haa}H#wfc0w<@$!z$nT)XB2TC1x6tFA zNpjH>5*+wrJDRN$@3zk#$HHB&L;BHupyRY62!hVoJN>0U$Zuk3QSUQ)m2SPQwri^S zFz1zK7H3lVo^E5K->dG~jxgd^2h*=8v${Z-*NRO!9%mO`9SrY%=qW>YNFkRloNxYI zX72htOyF`*2Uj>Y=3;AA63GeV{Tz;lb-wyM=2v43onenLWWuK6RGf+#Ti44|x|ioa z4;v-d8iWgEb0#-9<>MS{{F-?^!L!|vKR+zU*nT&M7;#A*g>0dtwEFN;cCa%tOumN8 z4Pp=%@Nex?QCZ98WFtFxMz1F$-cKGSPLDtx-A`eBwlwFZWO35}T9zw8!>&AUxFu=L zPWTlCVc58S0D5%$-7Ct?HgUkVmVxKTW6$01vATBQW^gX$5aB7id+lC*kv9gf9R2H1 z=es-ciBOYbnw!b=nRb8bJx-gA3|vDC4+_T}P@~wQ-#JvT+*-#x zGdX)lhk44gX>vDZPb^t)nP>T&5qZmZ3~_3OFgKfza%IKOVfnjTENVPn%C1aQpt@=> zeu3*l7+Yo%{LcfO+cqQHoF6G;k5`=1fv-|msM6K$oozQ9Go?n6Rvv3}`OFn!Pgo$u zHV@ccnpf@8ggYZp3cmO{Vi@u(*>Pdn6=#pHW0%4y$N0X;+zh2E3qis2J|mRG_}tPP zp1S+$UL(+L%g!8Iv2SC!^@9KfzHHq0B?tVqSk0Cqimo5az~S?LQte{C^TSlS&88)v zm9v2E5rdbYE^1R#$HaTDjRkrjU(3;ycA0XW?S}a>TE>U*r%?vQ^o8bcNad%Bp6^}x zSekok&6n~@SMLhEzUyE5nv9*FmV`CaR6T$FrHTud^IDF>`6JqJlV3crxrWWwP-!&O zerfT%U9(vGsXBM-X(VU+%pg)PQ*$vEQLe&)m(;kbxNqq;&Zj#$S4)$!niO?Km`nFO zyO2#geaE54E3Lel*52dx$wFqbP|4dD+q%hZtSQM!zW2Phye-8B%7`P_hQ{31b(}m;Y4Z{BohK&+jp^P%79M}$LfKkzQBnCd1F6IB=H#6eyV>NU zsh54->wTm)hZ=br3w}K!dlQjkMjFLu>SjIs2b*tye7zq%g1Sc~m#UlCT&sNo=k(|y z`NtLdPRCsE!pdGIXbwJU0=~Vv58L6~=mg<58GNP^x1Zh6rgJpEP|HnU_D^hZyVb(U zgj*t>Nj{JG5`OFXPM54lw$XkqlyD4#1UR=pHzYnfy5P z78wc2*!FjuQp0>a9@V@qJudzD_f87#7xYrlK1?p9Mr<9Vp4tn}_2HBp|Z zJ%Mh&(z!zRF10eI^z5#+oRGUZ5;m3bSQ)l>sjt?j?eac2($ucrs_DADmoZtB&p8~- zMaS2=sCxrS`28y#lD^Zs@x6qPuBIJt-S=&PZ;_u19O48ec71xCK)idMSvP9EEy(%8 zMeHB(kxwg&re&-A-lY8(g^wZ=AJdgEe@Mby$1wFFXZ&JA@=ty-?-vO|{oqwip|78hUI9u4+>i?TU z_}_sp-U&u1e+C$U9^M*=rvMQl(OyY%vq4Vt9vBVf}n8-jloasG|!F*35#GIIRQ5d1G{Y!_PKDk$oY;D8|pz}Q4%wFuC=feirP^2VTm z^29)#SF%FBz5tt;B_j<4&T1gAP>B#lf?7+w3B)78B0-~_gN2I?=H?S&83AH%XQXYK zoTMLrzUJU;agMKMwO(~;`(AZjQZl@WQiom~;U5jIStMrzj-x+2_nZ-L01)B^b z5$oto7^T%Rd?|vi{SJ7M6Cx6+;EA=dyV^n^PLxPGI8&j$j}XDQgun8vIkdiVBMx{( z03p07P!c$VesrQ8^I-77Tw{<(L1Pf0Q4V2W`&0lxsBO86kf;f|O!)_G;mO9Km3CX%Ld}7$QlAlW6<^5yK~O!;xbSa-@h?igS_L zR1ry(;U0FT-GbWv0w80!_wyWkWfcWDoD*+kw2Kc9dzOi0zMn?VO!`PD?lbkqG8zT_CJ2@#i8^eAG(hwPh5;aiC?c0wXVnIPa6`k~ z10(^yRRmoCC9G!qkDxWix5`3PP~VZJ!T|^o^~N_CaL@@Dmh!ta0tF~oaY>J#6+mPJ z3?Tvqc}_NhCW8xVi5fPy75&lvHaYCgfB;e?P0vYi-5%OB-DWMS{$|yPt z0ER<)s7Qz|xAt|{)dGI#IGBx1H}n}UXrEHM!zO`IL4?C~&OnLa2aBxc$3#ji-VxG4 z=tOEM{|Y5A%sbAZ9q0u>jt<^G&rO8@G3uZZ_kg)(QIGEF87)RagfGt8B$Cpgt{A;W z7AFF!5ccU)fI%A;L;(ef8noX47pg;L1|bHduMJbkE<*`N&<9Yqw*t)MNq1tvLz2D# z64eLd+57gdh9DO-k6eoY(SWyPD#-^W5UPfYzX07k8c+<8VxUMXZhtJ%!#PE&FM`ni zPyrB}NL}$H3UYuWF@|?TB3Pg((H}z)`5z5I6zJUDo7dOrctNvV3?RvO5cIl5*P$W&gxGF#h*f*`x$A3+f1^xoA% zp~AV~;b!ZQ!N$wJk&|dGbZ<${5}k`{%`od*ebzE+Bp7&fnOte4MD#xkg2Vd|hcQS(sAWsq$h)i_xs@UH!pvAw`D;YqDg#qUWL3C1QNFc-w2oZ=3 z0mi=zf-FR2Y1Zfh@RhVfqTf_PC#r9CIGXl&9=CTT5|>^J2QTf<#p}6OcJWPJJQc}x zgqnY>tz^L|Ix@c7|C=DF)kMv`C-eTs9Zi{j*_yaOqCdE$l(GSdgx>$F$J%)uGRU0* zaTV9iFFFdkH%$rvz2j2~DMS;$?cB>5t8kDMHkOVHHj?Lk<{DxSF|`@GDF6{dAS=~^ zG(VMcC3=q4U)(@*Q|iJZ5|aW4kR4*72}6$Rv_A|KWDZT&H?p(j(V7orgv@nKy|CCT z!h=Grgb&wQT=$FY9emLX!(< z!oQL>kAWfTB$8Ry1uk~2$?&Jsz;@0a3?rYJaBnK`oxjJXklY~cH2@g_v_vqG&QJ;; z9*()S)w4`t;{QY2JGNKasBN}MDiv2!v2EL|*tTukwkx)6+qP}nc6RD{RY%M1I>+qjF|R#{#SY+6+%@u&%g9Q ziMW!>GHv=ewnsgE39RD$R#D=lRD@3GN*}dL3}jfz|D*?A{C9d_{jx@0i=(?^V`IS> zllc%jF<4v=UM#+ED45TfvAwuHpC;uE+#s%7Bu##VTU~_5{@uCn@fGpnu!a;?ss}4NX03qH>m7q4z_nThzDo}8y zKEMuO&qOCk5;!7lFl!E3yptA3^K_hO*skcciA(wX7y&$+a8hpv8Jkf>6T8|K)&%!2 zJ#eSVD~`GDV8-O+DqCqr+;L+%{ce*H9T&bX{tNBVT6coA<#ZJruUbN(#&o?EhQ%Z_ zCpXtG_k|=FHW(%(Ffl1EjNNX-q+z~cU~sNQbVre0H7`kS-)E{&iDiYT>e+6U_4Cz8b68OTrE-Y5#BwPWG*E#XTqh6gml&KYhtE zyHI|y)V`~HnY)8y=`q}IKD_U{Up1Fn$)?Kjy^2G%O?%LGyLSv4h0|R)hL_m5X2d1* zt0#L-<_vhlJdOmNaZF($Y<(Ck4V0U5BZFa%hf5^o^d^?qcdVfY^@`OX>|tAjME$9$2_8K z7O(x?{e2_o4p*PaxrR_kRfNL{jXM1>v(1g{jEoPpA1Q(!T>J8X1qL+kbLs01sXrnS zFw$-Hhl-t4$rZcinM*Khko8+D#urr8rL*FUV)2q!JUKB_%u?^mR2SrAWHBl*V2~pV z$&-q>1Vx6gtW!fyCMe;(G8KFxo^-YEaxC-|GBuWdH=L`T?|d`$D`h>-ePQCA-?69| z?zos}b6%sa#ofP|-x*u`*PfV;X}@W7}YIU}D&xOGcaa z#qXWGPuEMnsbO6xG%56&Rcgp(IQ9H=aZLH#`=})#Y=JLq)dl?)sGPuLm29Jhp=i794IKrw^1O$pq>F}`SncAvLDE=PVB$dk(sYdF_VPS3j$|SdKn;<-*(B)n z6|h^W$S2QdL|iAfRq%vkTB#;+qqC|rKIPAQGagEIPB1~wa)0KppRa;ysz@z}ZnGbE zP*G`Ge7#Zak8dqf?Sqa(FSWC1A+l$0$RLkXqP_KTa=|r}r{7$GzJ;aC@z5hqwq<@R zLrY<{?0CDEtJ`bzle6o8FtS^;V>13Kay%L+7H0_+2A55|9F&M>oi!u#3!+>F)?eiK zsG%@iJsUgv1Ahs)!f1sRj9<7BFZDiQx{blk&O3eEsFtY7u+)NYtUqGDh#NJcKM`tK zZ=}I`*M8yWYv%*L(UlIZepZk2(UKqJ_NqP{S&sOoTSHa0XkT4CD~s+v*KD~GV!uI| zX~k7Kw|dQB+AnPtzlum+{gx&n!7n){>A=$BE|f*@#V6pUxTeA0r7Iwgh;J&rmXAo?UHgTOV|hQ zEtA9oZ`&hi`PvQk2Ucqr>l4W@#4B07V1ZXqhGg^mVx`;oZj z+F8$w=4$rfMgFtM$s~NjUI`o;b5Fv9lbP~f4Iamp0)l)Rm()(F>tiWIkhrCH;5Zas z=@74%&uZy&_CZh(TgoS3B5K^{1E(sw16}9!YyBuy5+`{~+r_>bC#&r{tA@{2vFcSp zvmCFN7K`0{GXrJqtxBmK^wHcM+2hngGtrBJ8pi{-ekf!ThO=Y|!YM$CNp zNa&KB_-Ln-%;BbLigFc0oP|A|^C!?gw+ejVUJ0-!;-=S*^33IAEmdu}6&7v#=khvi znQ69~zvN0u53M@-*T2sf3o)J`@E0M2??25_=%@N9UEda}0x7U5b#5pg9+K&yf7MSa zDZV(z1!6!ito&9sa4P(~J*aC*UHkav_@35;e?L5xAH-aJ=-K#g7NPeVmm#iLPGvvV zksup=oL>gn&qdm1N#mmA`PuWUFwmbgJL;9f>}`4V;A1Mj607X!G~>DS_30+4wWqui zTLjB$F6nLjfoEJt@-|`3LQ(2G-Bpo33}H*GpQ|z5Q_Gf7$&7&73Y!~ONhnvhAAeD& zSPt^wC7K3TM zk~;O6F|Js5%i>(|li~{->_lWtdkicyWvk3wLXO@_n2G8qGhe~orF}A(PIhaHXDW1l zTazF*`9|FXF&2$i+np|v-iL~LwyDw7*r(lR6?T76Mrw*-W{eBd%bv{mZA;h28ey+| zojse>2nEz^R>U9QtdtIrS7UsLpI8%w$6727;CDUJVsaOtKImL)Pcavo2L>z0)n_l& z8+)+UemG6Gbl@tO)u?_9)Su!wYCk3OdW;{p3$RBB2w{h_Y9h$8vbvHE9?Mo)p1H!T z7d~8DRj6l}1e&Rfq37H76S{glUr(2)+gvVL)xZyARxxPV@|H@MN;chzs`u28`v9#+ z_gbAy?=p5;hvu64)l3Kx5%_l{iaWJW(JfP0eiKnv*d|4h#)+d(W90 zf}{4UJ?=jIM)bN87zT5fEdOzHpt}FUu~{VI=^P^29b=ShUZ@>akM&`&*jHV4DW_{? zeM8Lnu2F{2coVVa-$Z`I=Dt5$x;c(vqSip8l}l1`CT*^pmZ27U8&S8ksP~g6V0n5}IDJ*qJ^vDQvO+ymf~RUrI%& zC?(ukXMS}*30;oH{Ak}FfvP{ylXUwF3Dn5&tv6&txLf{ieB#P z1Z%pL-i`AfS$!XJzS+e3$4-e|Mu5YwhMC?%*p-2f0ym}i!_auig?>S$kl7V_gGAHN zb}Gar3m4XxbJ&i{3-Nk-q1gG#(?k?fF)QTX@0}ikGtI-pW zmt#a)Vq)g!e)AD$be8Aq_kD>dON@IC{$SI)NOLzuH7yN~>nJzU7a^oJ7i&8FUHg^5 zC;18N`m_T~WQ*#<*oXD7kAIi{sUKEW69P68Ev`+F9FhzU2El4W?z<8gieFWXHo z&EuR^FYbtT_s3H%6qiSrP{4B)6@}h!nJ+HZ|2G#yt^SLP4SaF2CR~CsKanM|Yf4<- zW(065XoKD@H(v^fOjm_Ut}i}PufQT6l`zC3U`YHj!-ymmG+H9_A%1;m>rkjl7noE2f=Jb{I?DX zb9lE9y9tgm0bEx2DmyJA{UN95Aj(<$Ffl(ZHyPIBnwKLK+hGWxT|gDC6eV6UGLNA! zgq|ULEK6|u&j9jcB>)C2{}|~#0&fnrtS4Zgn|>agv#EaAa{ai=pJYHzWF(;Yt;m1M z+1J0yS%e8h425+uAxvXzUXGWnGg&SnK z7AMMlOb4nv?OJJlZ2i)yL)pYHOS8ZYH96}J(BHjruU<^tySBwmfP}27kynO_;wkgY z)6G8&LgyFEpKvy-YIKE&W+vY~@23$kz@z{VQ(RV(YI3SK+xIB|pxI#!_CC-(6-yGO z@JDSJ>AHb`hO_0=%Xoau(zm}2qAzo>IJbLGm;3-OL;dpI8Z?Y$4QJn|nZhXJ>72+2 z6N}49SapN}Zm`H!^94ZZX!D6$`DpVPJjEGTOkR2k&IBnCJJon8sVyac!(( z6j;DiFbVOmTCc5$obw6^v6so2H{7=HPHTtJ&Vwt!hQNaP{L%!6fWj?e!HL$`Noq?Y zBEW;CDjGe9U!bR<%81821Bp$BL(nDNvhyF%JHZAh2x7J z2&RKi0{0K$ji-+w_8jQQr@p^QIQ%P|RrnLmviu2WXNvB*Osc=a+4eu-tkYLGdu0A^ zT+HL2T#WZWb1|!ba8!dUL}h{qUr)85qLr zz8#$Vp108eXGxCL!O57z{jazf*B2MVV#0pYhi5-0m!0M8<3Pxj%Y<&pg+^JDyE6%< z*WcpEc>@X;JRIVYGaQ_@L-Y@H;b_?eh0F`B>59SXS^GTxocP?i4 zHy6{az!2`NcNd%ysc5O2~7j&Q%O&)FdJfv&f9EV+@M(m@LQb z!v8EE$SPVv_YHNoWv6MaQ)tRfttWX;I~MZoKh6O_5erB0`!e1rw`0!5wI>73i?Ik3 zCE4@JpS+?NT`nm>gd-$|?Bfl0fTBTWroD%kR|9~U7^Dp4fb*a^TJ}YF3S{rk5zPZ- z4?XkBhVkp|m5zb!Vs~b@8Ldy#cX4`({}^wIusJ|~8*i(aDPsmSd|$>}mWblt#+#&e z3sb9~{9@$f#lu3}a^Me_9bg|>Wqn(IX;|E7@Cax*roCxA>S?SIL8xRr33fj+O<`MS zgfX_es1VpKpxdr)!Te1gJ#ggTF5%RMRJN*J z3Ia8J{wAqFrosaE#rBnv5gX0pnV3t{^@dXiNGKv-gk1Q{sL~6dFI@@8^$#)EMLsoZ zVd;LUsX&kY{Pq~uMGlb`+HgJ){i}M+tA)fQB{@lgmX@)erkbI;Dv8bFgyEq1;hUQp zwi>$-J%6W#gr+KPSm`*w;+ zfkq^o0|I0uLfaaMFo=Y>z5Q+z@O+Wve1Qfb_)rLlu$vf+$Or=CVi@Ks*6mlpSvi?%t4(jf*3(M?vhYJQK$#dl95c^`M2^*Kq0~1~)cGdwU0=-0uOfQyW?!5gejE*9L5|PI#T1 zHaXh4?FUpcS%XbZOit&JRV^3T9j~Cc<(g?Mm3iM+f>uALZXLL9^Cu|h-9TZ;?kzT+ zVkKVh>x?Tb7E;%0HiSe&dw=hqblEro!_O0ukVcSN(aVf?80R%-@;x_$E*y7wqpoL!Zq?EgL+IFi|H zi%c4(7A1|^7=TuLe}FyQhzZ_Bnsl;sVmoFxzW{@#K2ZI*WX!f?ib^c7;!?@sG}72q zabm?!`$m=#F1BIbLbZPtCVQlMw6t20TQ*k*th%!;*2(p72)_{BCX{K~NrlGJTEuD= zZ)far_6D2w1oA%J7vsIJbx`KGQg~PZcbBTS(u}QaGwg-7@ZI=(xaEeML$M9KnZC2k z@zkb9O0Ui++y1N8cIVp7E$#84=iX|w9CnAqntRA68s)1)<`1{7@R1w@$q8?ai(z1x z*I#Ig#2}GOG-xzac1!x5W&G!e?l%;40Oz*?MFDn!+b3O&lTsmuv28r`=HG8uvko>TT7&H->XYG6qaL0gXi0ZUk~tR zj?&qp(!CMJqsQ#$8d6w#<{5=XQKq$1q}@KwcCZ8SN11Th)w{{wKBEE>sle}y+ql3z z8>%vi5Y$dL_-dW?>rv(Wk2;duJk+pSBU8K#LO^o6&TTIXPHH6=|A2{T_aWW$cBSDA z!2LaBr)F?s7G$AR>HN)&pKBAU&r?AEQgDGytOyojytPY;O(=EbW?0QiF}oG z3zcm}m7dBoWjh`YhEIWz_}_boLwuII7a*-DkUed(yXW*}KT>jE>n)4Z9PejJU^P%F zGPKc^wR17PX3fbBLT%%!4^u;+&0ZrpWkwn@oQZlxBE0CYYlHg&&Z-N6!gk}ZYagKc z?ZY}MDqG@FX!DqE!{93Q3gK>u4Yx{N~T<*c>d&q9a3M%kC#n9-+ z)Znebm5=qSXR@z`QY|st1TOCg72!f9eZ_}7ow5Dz?OL~kV54n^cY&xj=5;qm+nT;^ zhr#n0`;NupTvJHJ%*73MJhT0Q9rSnnGl{{xrSJEP*7E4<)Y;SI&&HA1Mn|aS$Jjxu zi*d$MoA#Y_eFFWeobNlos~-#fQCcRYHm$O9kT7jone^QHJ&SgFhJUwlxy8B9SZ*+5 zV|8r#jMSz|d_)#-7fd`_6uG_x?djN^n+S4TC;Swe7Kqm zp}wy`E&)aAJh0x#BbkRa`P`xjKQWzI4Vs!j}4Y91Pi;z|bopw#C|Gp=UOVcwcsl3ImV+h(Py!)F**AvpionFO7lpu4B*ILdZ6QfI>jO}JnL zaR@etA7S)q@^K4S-AxsgE+ll*_toW3=$BJgn0t0)eP@ZMSELj@Y1kks+P_|JilXiL5%I3 zuo+Ye8c$K`=Fl`|@ED~vAD~G#ZQ)v@J2$&cs}@r!H3dcKs_8{)DJMQP(0*ENmO9Xu zoRa!-?4LAAte+-oY+Df~kyLfHcr$CXl-)Wbjn`?ifh#n0{55nl*RRAi{KUu6uDaQz z-ST|#!fwDSNo;AsEWMLquDfW05mja}hc&|xK#yb3_q+S9q}eK^*|KR0F2((d;+W~W zdS+SW7|r)~;rK+wLB`!`-glA@lbKk{>G_#?_att0pUFsRf`nqpp~1^e)xw~RY<1X! zccspv#Lh6hRPHNQ1wo~CDtV}UwcNcDJ(~#}erA4$25lyDgVbvB^7xckt|3=R?kQ^# z&)lQ!4KSM%G>cTpVtX-AOb@PSCYO2hSfPpIrokbqCsno3I@VG`CXXD7V zSr%YU;9Amgyyui^*%1yNjLrW2V%K5J!)&*56PQ=0<=bWTZrl0*CHrS@e0KLQHY`VB zd_3B9l=7BwUZZI$@yw)4E5xTXyYb5Hh67n=LImD9vjY)f`VlR4YAom#w|$Jy#ayUyil?OLWT zzQN}1a>KjaH}_J?%ol}bb>ZU zmI~*V)_A$+-nK0Kj&VZRA8Hv^8lKZP75ij`X?lsB)ZF@ZQRgf0_9t|QX3>@j3lSzsU#9|8Tn^rd&LOa z3lE=vSlGFBZL;`Ea!cixsvZF;P=x|g>HEdPgf5RT+jE`nA~&Yl|Lx)WlL3JW(XiwV z^k0tHpLu5X6D?e!{|0+NK&Q+=Z`|G3sC_iEF5+G`ta3Kt! z$ijXuD-_k>m*UjO1KJ3Kph(c+(&OUo6b5VC^p7X?(H?I+Md7w{SRD-v>-c?>YTV3qb2C!CHzC<_T> zB$6%#*1)eO?`Xejzz|OU+W>18v~3>UxM2S0pRgf3vJ=tR7y|-b{?-WFw$=p>(q;1g zQ^KNw(o6%p+599vH^OYH8#qK-eT0kM;J71E+JVAK1c`z@I0DF~#2`k%JYVV^~m19R00;#?o3nuf`iATWmA@rm~TGQ?syY78gx zI{SvNvJ6F%X=J#jLlL>JzPH6rzHb>Yj3MGS zg%5G+t$(N38h{yW=_81wii6DzlS!GqnFOxPyNTr;fr!|V1VhICDY?2v1GS4s(<$MR z4dx?l%N8nkn=OnI)brCH#)iOdmv_gN!LAerla>oG*Byh`2a6X=+!x%pmj)KF-WM`R zn=42aEO9+C@4K~sG~$L&m=1?FA`t9@PAxcx0{J_xl9?~S~~M`Gj)94yU( zOtifK(zIZFF>r_jIi!Je`c;35gLSJ@1yF$>5IDnc=Y1nE;&JjBgxw8w{yOt0zV(UV zLs!QCO| zyfGdL<-F-)oT0*lNgD=veltctZ+2rAv1kD&PBsv6i@P>Ld6sD&y=q?5e=?2uHDlwD zN<5(kM4Tse2`Ajd@}waFa3BLLOQ-3ZT37;|An<$x6Q^`@48|U+ZIf>J`RdKdGlO1}`%5VcDA>^H0?mOHS_*PpFN zBqfHj`L)4k*c$hhf40o&JCFwY6M0t%wn}&%`L73u_(C8j7SFQh|2m(Eco=jG>aE-D zp5G2WLWx{uFP1wxG7WMF8<*5fBt<)Wws+uFhLV`8@7MKn zyUpWaEiD#{oXwUmjFt!i{i$B!ySJoYUQ^!IG+*JbghFgISP>S0+lMYD8aTir_?dG4 zIBr05!C|y6{Hi)j!bgT-D1=CFbYN38F2SJ_cxXGDJp7h7eK#rWC3moB$+Lz^VaZ1% zTWPNN7yD^UhXX%Of%HHW36Kf=nXi>e`A8^+d?yS>B|+n)-s~3-8l{@2LSqPLNaF-) z^hYY!2u%MITxHNY*?K*Ey9KYYaevsEaL5A?!heAr9>E5CWqC{Rqywq4W0FSSr3mu`K?>vO>Nw z6E!xtFHwvl>q`{72%VL1V1{nC=*>F}?(9s3=YLK6{{8m+81(Lx$(GL22b2annGXm- zXSU<+-rnzSb<~I;c`qbn*h`IlFAg4R2(f4s@HmFxQrWhjWp_!jrfEV&M95Bf$*YO6 zDd}m`6e^~q7I$c9(G=7*u>sOaz7{DzWAsqxcpBsk4;k7Nt6QOTmoHF0lK{_2SGH;fi$0mmrpSOH ziOWS;z64fCzMClV(U&MjRq3;OiD3jN`0%wO*9-x(@B83@l0;w1J-H8__{F4q&%OgY2)CTytk54gbI98IrR*W^#C1A~#`UzdZRf zB=;i`Vr9ZsJMd?d7J%mOt+;v%>aPei*Xy`Pnwg%qmhKM`OE%Zl(J|0572w7WVOf4% zW_NZ)p+m9tfeLhga2p1oJ|D^){K-a+XtMH{e|p)3yj)Cp(Y)7ct!$hPs`i$(G02bK zjCbn(CN)`EfNeT-z|fVSTq=dqKq9G8s>%-Gz=)Cph?q>WTX0FdGD0tl$`wA?7$&Pg zya+$3sqo3Rd-23}5PZUX^L<27hCMB}?nf*fdB-i4%7dCruCh_z$B?J=ZNz46;G&_{ zP0P@3YQ#Z8EzH3>nQ-cM$00?H;l~T2$uevQd3xZXF)PZEDRKegdH-g7yj}Qp=(5*Y zLsEMb;;~g)RtMkcs0YLAR#}?dR}Ze8QF*wq1p~ z-(=!;bs9QS+5057_a(meo{^an((}mk3eW5uCsqsi@Y1O5&f8F;<>7v|FeBb!1oWjf z9IP!%o;KTbX9ZEWbtK1$`(RQzY+ypBU900P%0mtNm38}jf@y-%CMQ@p0@#PFg-RVQ zky=JWLFz1X$Hs=|*ROw=rzOWKcE3eKr-wB{+e6S3=;%!4kI{g=&{9bOF414ymBKQb ztMb_n0+b9#@&Z@!>igtY%nggh7OQMv%Lfj0Ia`2B72stVRL8gw>SS1 zP98a+Sco_fDSTk(@afP{c+fxES^5@ns0=!))S@`T%`__mC-!huOp`l9c{kKK8+L(Av_&&2GD#!Sqk9W*UP z1onxxWyBttGC`W0tL)kHREWZxs^n=&%Y}Z#kKxNkm|_u!?!ix?d-hEwdMb=gGs*I7B9mQwwo;2krX7&ePTvdbWMla?4~ZVo;8$Z%{E>aeoQl{BubL{PUAr*un+NX(%y?8JWriNdrA#x;&7{+VL;mMD+&cY7f*;DQv@NJ( zt-A^8BN537pW>_iADSmIU=sJlRv`ZFD%?*`%|0unPWy_`9kw;kJ{ql z_q^Yyryw)>x@Zo^Fj@M;ZeK&APM3!HVqXX%k81&--O6}lMP)^X8&#}F_hnYI+mw5;g#jtHK3nz% zSAiNA?+hh6e9u=2UgFWNIb~YVNz~$FmW*l>@zGP~!&SgR6gi^hvY^R{J$I+vtmc>U zdof3Z!$DYAzR38nL+l0-V^Yp8rDDnPX^cyUQrAVz$x&i**(FuIH0Hpw6q%}?xAoNG z(uDviq)JJxr;~(AA7|zr`+WxHHp{V4F=owV#uLPf*w$7n?W^oXOPM#a%6ipRqC`J*nljSmt-AYBwvoh1=ONC-@_i?eAe?j_4-(Op{`( z7}9L(;oPjmPBFu)Xf{blW-LPECkX-0_3j|6CD!Q=Nvw#9OhyIN*9@;N&+`aPQx(=# zsMU!tKwg9?}rojXbmDy7@uqH^oVhTyo_cMU%#lU=R(itMI$ z_!0xbs6pR5=Bi8tdKQM`i~HxvDsS%iBCHh5bvx6OGU!CDWMVW`uV)MwTblM=?aZUJ z+M+S9dtDK*Zz--0`@s9zxRVOTO6A4e3=Hk$Hk}hAo_&Ju2OA{F%f{)ntI{(gD$wm! z-_Cv?L}zB4shVDBw#l^ef3EwbmFc;tlH&V1Vq-RvY9TJm(BV*X^h*4i(|7FCUcRSK zXxW&U_R9p7kc?vcsClvp)r6)sqr*6PMVnM^vDIiES*mFl19kICu0&&=aoB1K2MPmu z$XF;_5f8@xddihfHC+JIIbmt1+DCpkpU#eZ2xcOAyD+E-T$ z>mt(ajM{$Kl8q{0RoM_iv~043ETvt$5_r*ByiNWN%J5!)@=%nvUhU;4I=a=k+8!r6 z5h(}V{&Yx?yIH41*4N3BF6-#E^P^Cm1MNPh)ZN)4Tx%f>;KxG;82n>;qOT>Cts$uk zLB5nj>#hED``rPvTASk_4D(A*An8+7PLS@hjpj*kfh??TNihzoSEch@h||W8Y{f!T z%ZiZtbTSB6Gn>_x0mJTy#8oA$3YW537Q&v7#h${7oz}^eJM)r8L~b`{;@y47Ug^() zESu8NM+$N$mu9n(+%_n48%xPZlZQn&lqq`Dw#pk z37mL#5#kMvu+a7wwe$8Fa>Y}rA8UcF4vi1pfhHQovF3_{7EyT z-)2@O6US(i^*9hZFDn;jtoR?0Uu=oOMIan)D{x+uEf%hwW=RHO>=#t*K`uo%Vx56m zbU6Si{JTq=ZNodJnO_7qkI+4Gt?nZJ?cvP}jqVX#4(a3HEVciWD{_Cx-2eR*xj(&W z?g_5S|6qw4|NkP||8SAwzp%uA>rKDBy1#nUzrDKuX>SUyE^+)yEfyMR4ofK%k+}v5m~`zJ64t5h3#^648o0L+C2IeO_Ag;uaXJ}eeOfUG-$9|;$b2E_{m-Bd^A zZEBJZmc>ON%R5$f@vEb)_SOu_14DpZgifdsrjGFM)&3#D}82oz5^~7AgA~og+-HS%xOB3 zex1Jf?jxpX#)M6%Z!wuNFPP05oy zVR_Lagg`NZ4abv6L!)e;DY#TczIEgM1~6FPh%8=W)skUb(AJXR!VUdCb^UbvZZw%g z9}O^O&?Bg43M~ygBw1PcJ|-pe8{*F5bB(voq76D~&&+SV`F9CRMz9Gj%#>QqI(b zhiER8`GwS>kY4M;IRY=#z=mxvlttZM)Stq~zh>rQ%X`wqFhhq1lB7n5K=Q3{_7Vvz zF?E5`<}(%j3xM3nG4^|WKZh@z_?wD;BDn|VQEW3KFcla8;j4^D0of=5T&V-=^NFMs z*EfR~$pQSpp~0N5=6mDG*blJXebe3ENH0v8n-)4UWpKOXm9y$vNH7dZq9! zcz8emqgAJ@$0`vjfR~m<4zmz!K0L70?D9881n=iv_`-;O-qqIr-_br&k_ z)a6|H)3QQ=F%#7k7(<=}Bx#bQe#idhf)?_qjfU)*uiA8T9#+?5!lyvT@$B1!13W8c zW7oGN|1n(HP-_*f7w0Bk*YSP&IPsgV+#@e^_Pua;!5MOz#BVc%CL6SOq1A%45&G5T zwJ`|?(N&-BW?)xEUZu3gB@2OB_*%J`OByQ_YqaC4aV`@N_}x3=G63NMID$Bq0PbPU z^5pO=wm9r9Lc<0#&?4j?n|cx(N@0L-da<$^gvm_+c|PI{+IA0$+1}s2i&}(w&-o12 zafl4b=XB(Tk4wL-Ma3|=5F}iXq-$HrzOA%@Sa#T@@w2Swg2dgC@yY|7FyN5VxRmn6 zwnG`3_$;e1h(OA2DO$JCfQt_rI>j8VJS?xTq{yyX<5;B;$e@LpM}m>z0n_PB>5TZa zrR))^@hwLa#^#$CNH0T*C}I7mWqQ}s1#9Z!u&$auW}Z?W6Uf7Y#9si!q_S>Q;ES4uu)=GKmI^O6=4RmFNhdtyh284skk~AL+vM%6#Vg} z)!9*<7eH%$X?2E&7Jq4Vzh!@=era_$-md~%h(9D%Ry}*7yQ|NrHTGV`J-oUE@|rsANdl!-%vuE% zbtVo6*Ws1Jcu+_QugMgh)bUL2E0N6h-0dIpq)4X$rPNpu;w)}v)9i=1K_CRqgqut< zL=vAvM`Xw(jg{;&1=c#1uKtWRc~xutIBm@21t$?bzd=TqziJYANUfy7v8J?>W)t)Q<+8&=N!Gpk5vSJhY7zNB+>eBk!b$8omOKnXeFa``Z4^(y1!M3IJ*hd!ZSgHHEO z@QS=a83<>|jh`F6$%1*AhpRu3TAei6k=bQrHpX8I8`}y~jXE$ZY^_NS1&Nr%cB9>p z-!@=M zxjH>jIDb{l`}(0aP$FT>*JbhZ4_iClTid4ZK@pKbC&tdQ&gkakc+B>lU{7fC36=5> zSj?V2%)A!XuO>W@51n#tm6)X$+^8LqRhKhNFLLs5n)#f=XC5a)UJ4{&yL0GkivGR@ z8fS%h?QFAo%C_~~t`{>}hclDkZ9fU2MQ7d>8;km$q4KWNL-YwU=jG0$tCHv&SS0kl z)w2PnXu^i8-Be*p^Ydbp^V4~=P&+p;+32bw2=?|3S*z}}Fu}%TPph&}o*E|OV>U~{ z%~c+6R{?@-Uk>46Qlr6Q>Uf)tl_~9M(NCjX(T-t*OpQ$K9rjF%lbQ&Bdo2}lq#7P^ zud*-WI3-?9tByIlBwSqL&0y>T`ZSt(X5%$6qIRx10zKgPu)WP8L7o;&(C=&Jy< znu(2p>SW`x(9A~`d35Yk0w=OgB^ z6b5Gfq2zWi6LFi;WR^1+w;S!{we!<~4U|-M)QiR)S=pjnl(UcFL41aH7ol0)v#!(? z)5-MHvHFY_3rtp$U-puPg&REvhe%~83Z4vVHrkQ#V&m(cf@(A5uIui}1R*$1@bff>We$iJ>58%pGd!tLg;@@hJU?BJ|c z#pNo9*)9@9DYWE%N)9K;xAct0a?_GuNCpFD|F|#D(qvhVM_Wm?=VYguJTf?5le&B#Q?& zI`=WK@;p^)i+(J~^gOqJ_c=716&tHCknuRP+8bjcJ9d1Zx$$v)xpo`RboEN>EbA_K za9PW|prVh@kf?B#-w;?dT?AP@<=nSqBdBE9LUhl1`DE1&d+VvmO4)!pr|%=CQ)lsL zZE0JiXZ^7GiZz4UsX>Tv<@TU;<<2cxR`)h6e@e|#E!d*c6Kmpf(wxsGvrxt~!Pyw7D5M(JKA z8N3}LuctlPnxZKjDJ_(|7}=!QZQakR`#@2{^9q_#ASXq5g4vnet$7s%%}k=Ma8x@ z9)cN4Sj~~q0<2nBM)A#OAL->0U-WQO-Bi^?ITe1JU6(i7Q$lhh6x8(CNYVTZ>9-Z8 z?VH~&N^Mj12c&CxGEL2nNO$b}1s0#IDS4FP4O#4^Ryp{lKV`UMC7COyGvpyp0sbHy zTHDH7QDlEC&g^;V90godMP2`o8ZxY8BgB9TPiZCei~S8Xvx(jr_nG6DHo)iE8S-AS z;&!f>OX)tWyL z((wbGL#pgdOMd3_PGgmIa}hcIVUW?U*Cm(wDbYyf-pzyTXlZ4PD-onM9F8m_qg9eG zx3yXV@f6WABEmhneOr3Tu<>HY_0$g^Spg|(=Gt}!L*}teQ2J#jP_d!6U^iZPJgT|^hWvoc zI*k}LFLGGk8lCxV;VF@XfzHEH!$4E^8f|`d=~;fXObxIT*lp-lxAI#9r>i=OoyEJP z&U!mTCorbMwx~cv2OBy9Lk!mXF8W8CC%6q!KFE4So2O?@mNc!CM(;f5jo*nUxr!`( zN+s9Al|uZm50~QF;cB7y3ietAg=S0lh*R}sm>Yc|aVKq9`TxV(TLsnGsM)#@+}$k@ z+}+*X-QC?ixLa_CpuyeU3GVI$cXv3G|6i-u>h9h9^r<=*R57b4m~izm#`}zid1YIh zZOf;-)j(~;yw3A{mpI9m#mk{vb5zG@`VV2xJQo9xp-$Z0-D2h9m-B;-;Z_fkTt0fc5EYe8 zunWARrAW)*V6lW8gXdriUDly2UTcnz>ag#BLZ2YuOGL&qbY5-;Zrj81)}9u1Z6qbT zo|JDW=G&SaM2o7l#(Z`o3i@UlG7hz`a@?p$=QBBv>?u#Y$M($fJ>}A$4W{>$s^V$& zI>#6wuW?yAF(AE?zrvCOK2PiwGCs9WRKAZ31@3NWbYS-n%ivPQB zPW#u-_|(t-x3G_mosIn;k_*PaAzIeIF9+%Wr^`VIWfYBvSiAud=Ndm7*h(-$zg(h7 zXkerQS`~`kI9OxiG^kJgEC!ee7@COMfQm6%c@7vA`oab|<<3esZ5~nbd_(TZgDEN0 zPcZGfkGILs!$IOt`|RI`gNI!1s~pEpsRksV-!=W^v@R{ieVavU0K$j}dzuzdqHZX{ z#kj~G~`-c&UyGf7}q4VUSWQImg zCI^fS|Mo>h;rvnu_#&W!O$;NfBog|uSO@1%EThe*HnRnDB;N+#8rysNXBXAQ_HE2e=v(!g3ZzJ&fy9 z3W2)5wAv5ZYR2@h!M=mS`ttMEInj7uh$uBk5D_#m=(@$h-@$Uc{PrhJ zi~@bc55TV6S7LfNQb_n76Vw7w45*&eIE^sn^^|%r@gPOCY`Q@QsA}k7M41fj4jUD9 z3bdyDw=r^D!LGutfJJ_b+<;W;DVPTSTMcn#>Jc@ah-?HX{w>A%ba*ymFsTB)#<*|b zsrsk|#t zptx^o^5p}tJ>=amr-aJp)DT3dlpz+)^FXp#<9G~&jL{olrJv$iC;)1U<@}q0Y$zkD z0Piaw2sR9NW3xJG4>v_BQ5UK|WjQu~uHQeCzJL%w($~t0D$NH&D7iZg43Ve7WEz3| z^9xdBtmh1WwKAosMiZMMrR#)RVhqcYT7q1FA+IyHZRf`$HKz|$w>F72xl$zgGgur} zd|&GN1S-5CneeXUI~tG-#svlQNm1AFFI$A-U$zLEacTa)tg}auckowVJd$B3zh0W20;eyC_ac7i zkyBTpTb@YG4^M&4wWDJFIrux~WBu06wX1)${~xjlL!siaXh{lvD=L5|0XeWebix&D z+F#nbqEw(|Ax=Z=181nkdSKZEk^&p7#^d!-`tlD@Tzpqen~7ee}BDgl2&U^EEOS}9zmYyIfqWb3Fb ztAn)CPwA{Msew-3FIGrQ7$J8=f+Ap0OT(gP{RreDP=9fM$RFJy@~Grs{G?Fcc;Ses z8GjuMLTO9pLK~L?WDyo{f5{@S7dFUPTtka}SUzPD9spTH!+;4u77_C)i$I-)`IJRW zeaa%x+G-j8DT_$|ltn0?(Uk1TDzXEI0@pDBj2Xyz#_mdM9Sp#drQl70M(5IY2tY&}SwU z15dBzib*RYVx~rAeB)K8)_b*L23sqjmiiUq1=2CdDOduvz$;m*X{|h=8ML7HlXJbL ze?Btz-u%~MMJa%s?TtZ;1PS@zJ5%?Vt2Rev9kD%Evwn2{`qTN7oYmCY`;Uw-;4|Y> zHiB4jj1h#W9)BcP;X5foak5NC_s5zNcOnNJGi=+jR$5mOw_8OV;Mh4kGl!_*JrDp+ zg?sVy(Gijz`dpMFWD<`c`m&IMKM^w(w{nd&LdLPHa4TmLn(5Fh?+Lc=$X`P|&Xugo zFhQ;=fv(0!RbmCt@v^Ta6!o_&VxA4)ib(u)MO6GZR|Ka2Cpn8Een z_2(~gHdv1ad^lQ$(mBa&$1o0@aoTTu`6T!Y{-3hyPgevXz!gEL)bi{~$trZ@#3woZ z=ZBW24rk9sfjf&koVM3NRLYKXEVleinYcW|uUWr>?mu_%GWU$Bg{#rdp zy~Om#IlP(=+*6r4h6SCl2 zL(rGY&l0@dZdU1rav#QU$@fb7KV*uHNLS8ib7SA_j~rjx1i;8}XP=iQDHx{GF6zF% zj`*t$-8z5S-0;13Q^bfNHH)Wle0hVIKKrotWc=wBHM@M=uLttjdX`{K)Nn4tqf1Zv z{9``e_Ff`{Rdf-(GIc!b;oh}(L@KsOhpLK}+mqMq%{m8()}rh3<&ERaGUTDyfuwHJ zCgaPddY4a`Hi=l91ir1UZeNcYq<8%ZG;fipy@0)U?{UBc1q1E_ZkE8x9oDVk@G(86 zLSHVNn9C9uXRMoZ0|W|-rUUykmJ6xnZCRT7K1R?cX2}^VO6N)ovS{GG_aY2@=ZbHa zm7_|^oC2TdBMdH&Tr>VK?!xi~n~!dj%uV~3s7dFkx0B*7$2HgX@C*XBHtQ?ozTX^4 z`yyGcZ8VvtE=;?sn`SV(wlU%F@>#ByYuifm2fEU=8xJLv_J6E|3=A|5!+Bd`RELIyQy$>_R(1F!VCpk1;#VC%4NGh#7PyMFsu$Y#2lDg7Ga5=m?OV znDrk60gqPMOy7Q-j`Rxby^$2CZj2KGgExu2ZInTrG`+oG_>Hol5=?2CMi==%nF2VI8*>oRZ;ECrAWooptoK5!)3J zyF7MB42430bq9oIqZZ)lGk17t=k)0r_G90NvbsKw6`a1PC_tT1Bx=yN%j#@-s zub20fm^L^vMD|3mOtUtd4W&L3C}w8zOcvM5S#vZDER`s|T|Gw8`Rb}`3N5_lvL~_{ zP@LCReRC+L;Pe=0OUeX*(mc;^5quYl%Dnb@9A=^)Smo|DTn_g)rg^C@RMmniyf#K3 z2K4)ab{gs$ad2G^wAT_^^EU5#-|NZ_8q(OTU&2l|G=ANWk2H8>3cS`Qc2XGu{)Y^O z`<>cgaJQCIK&SJ9OI{PUm$gdJE@qag@pF+j7Irp!|43(1X#a+aa&{d~2K1G6&50y%>D zKnr+g0vemrK5mMbZg$Q{d0I2LsZ1OV?O!Os^Kv6bz4LO{OS4x72YlV?ZSC%vj=4nY z?Yh&~2)7Xosj}e5JlKW_Yu(s7l&V@(o;vCQDOj+Yek1%-i_*lEtz}5lytX2%!*0Zq z46kb$DDP8Vz2Ud$!#=l%+eh8ww)NsOQggd$-__WcR-ba(=Go)Bq`7!ow(qMo+$~v5 z3WkhM-`!+&k>^$)VDI(~GoU9Q@3G%ZANi#4R+fApJ%+0i_aB=V zft_009wyFx^;8WPa9}uj@>WCB!cQvA@9VSDX9W!$6x5|2_W>rsKcjbY(2inn|E7nS&Xd>v2z}BZtP0SlIv(bY06pMRqPiTDv!@; z?UUsq>xx6iX)giIT9MR!x)hDw8a z@AB)qZAz5RYN$o8E38C|jE8^3xFWZ&_&g`)UX9kK z@`@o!T|>A|Xej7-YiaA*@3s$L46I5`MMCw)y*8|9=*+V4=#Y<-OkIUHUrV|y?hSQr zg2m9)cl#DFco;^*JKsjAcU5|;t54t3sERN z(+li0t#_BXh)~_ocz6*#oqC5}CUrQQw1y}36yr{`au=B#>UIfmoIA`~m2HGr-`kgG zl0tD8mUgWopFOIOk|l-W!nT^a_&BfJH5G>C9pE+RJNuSqcQl^)IB{3V!!sMw>lDK* zOw;k@GSk?JW;j1wmch@ax^12|4xU7)Uif%Q?`o|Rt8aQFs}w6n=i52tXyE1SU3~O> zcHRh6uy~$T3HGx1tRe``s-L2K+48&<<-6XiCZ1!yAGoo$jUOiLy1I--v%8wClppfK zN1oc!=D0pw54|Rij02Ct&faZr-iS&Q;2pH-7hxFUXlLtTwcEN`s`SL@sq#dgE;3lC zu#y$NOdQWCWNL9Oa=^7l6kDLd9i3ot*srhtnmtk}hCD&MsvSDgbsM71;`sG2f7>U; zrPIadTWa=F9%TQ*a#N8hWt-KN?09j>oWnQm)JBeR6xx?WML`h<>snSO2_y zzobUa>^wjS-Qml*B*EmIV{T}Na8#|R>-t)xP&0_}&|bCklyT;uVbb7j*dM<6KFgxS z_vc+YV|GX^Ms{-3aRM(R(`e7p^{frv>?U5Ov7Iw#N+|s|45D264|~f4hgcEXwPWQv z_{UcWPPV~}+st=#=+#r(YSLW$rBRAyTZk1MS9Ltc(XSPKr8@A%j809*o)0Zu0w?_| zB^E}Tt}oStB20Jvxf22h@AXU1<#vv38sG6GXgoQNYjxl`9MxD4eoHCvfynIAJaLW7 zc$(;x*QYGgPSDYII(1jLSMA-W)?;P2v3lsF-hFi!TrH)wAIhhC;5jqSaMM4&jkKFw z0zW-BOtUW@-Hz&*B#sxkKQD&i^O#uB%1vRY9ib~eR9F$Y-9Ed^r=}WQOZ5*89{4jf@zN*Z0`;Mk84@R*l~5dLoZ-50xfN#U?|LXXxjk`ESh|i`l#e- zMorOC^MAb<{3!nEsH4wv*Yz)6_46u@v_=vt6p(8En~ zOUfuikaQH)l;~TcO}PR>!m~Q0ANp{ux1p#{c8j{nI75&#o!%=P`N~3b@k!Kq&wBC-O9oD89wxA@Mgo6CgHR!7uhc zCX-fkK?o3jfwH>#KvR%=5sW*lsCmi>y$PB(h20GiK5hQC!Pm19e}-4Te5`?4dx>%V ziiLPD=R)#O6wW8tIjDq0Ky0JwYk>h`B{liLs!K}G1zbyLwlMMUJ;`E4h~7c4mV*WE zH^5QSl(6LHt`Pg8;i&z;IlXO zRJcdc_-gfwP4qNn4AsaWpoV$EKSXnb2C#)Ox8LLgLK0WDqzQ!R|;#()gV9URyDr1%4 z)3vndw$^N)Gkwe_PQA3QCY=`b+O#OuGkZEP6r*{+pTbq#0=p`2`7eNM^-JI=`iB3Y zYnV|j4%(fybS?0MV>bKc5t+qPk3h3y(7{;NfyrV@(A+6q!G&&ArV?5WqdtY3?fahT zE>yeL_Y$9KIvG*9{sr06C_ZtJWAQ>LHLY{{XUaRL?60q>kiaMTlEyb*Um&IzgY8RH zs|KP~pq(}Yp9SUYHk663TDVpFXez)bJy#Rrhtp%9^?o})BzYst?Jh-flO#9OJt2+= zHL{4YU+F+=AM)HYd3t*J6`9KR|gY@8u4* zvAPOn7_5d&8)7L4|IiO+4mc1xR9F(sCS{Plh5IfHs>}J3p2nZc8qDkh35w!rgQQ803d47APpzV{Y5S97e0S)@U{cSPHDL&nZ)mNh^F(g@?f5 zimoFt1S*?rFr{S?7%uPVva3$3LnSExAMh;s)2NPO^2T{mt%k_absi0#KlUWdx{uy6 zX-w0yF1Z^0oAZzOE8Joqj_YZSmpA<>;XhzZEKx=&BnkS{kQpble0&BRPe|>c`7^Oc zCOggY=!MRj34dUn*IlnI(=jx7D6lk=%0A8#_*BK5g*I@Na}+slRMaJeHH|xZvdLsl z6D)C}I%2H4AWpk-L&3Ogya&7duk*^1$K*+{qJ z=(AG%^_LBjdQCjp`D-;^Vog<8G`hnYDjFa)h#5Z>ZPF#gGz4KiRpgzXLF=l z-wWjN7qRJD*GOD5I+~k2;l=~h0@1}#w{lpbtQ5xAWgZE6uIw(Q`g2aRl}ZKJQr-2R zoPJ|GBlnrZft1O%B?N@acOpz(Ff&AO)f*|LrbU=+A|9B9P@d; zm*JS0-?ow>IpA{SB9Hb5)YxIUCQ)>He>VnB_)gWtmx?8O?xHrS1BJ5$1=d%5Yo^@v* z(@krABeXuN1LX;~=|pVZn9*>dPnk6HlZ;>#&|O*dANA7QE8-)GGpVyb9F)N?(#CE& zzcfqN8n3n;{^=;3>NM)u61gG*b<^xIg6Cnge?H;MGjlcP61vOrcyea;Q;Ea_D|jD1 z`WFI9lN}qotd}grZpTE6&6?h^-tFmw{5=7Tenc2H`IDaNeCYZOocvGlK~3}cS+Tmc z5>cLjkirYjRMtcEA7*MSx{Z}P`z>4T zq= z+dj>Yu79Ex8j~@0nx8^Fa081!0atDkzDS|tCawt&TFA(< zHSA*0jdx_GWnb&G`;shN%|j2-_5^$thBfDw#apNtwe?7$kL2jgP?|22zSXYtJm9y~ z4T-cU1**}&R|H#PSUtZaC40G9<$LcsUwbY>()L5JE(L1tfY$B={JUjd>4Xg9 zf>f`ztOSEik9FtC#_A}!ZcHX1^kE5Q5k;FfI8!QXqy<@aK5~_k8umG9jpTmp_nY1^ z?P@eGm4jc%7#e20-bjnOYRwF#(%`_zIrf1Z)z%CjQBd0+C8z&0%5J^T&=7EirIWm@ z%RH;sGN}@1O|x6a&28OHke)VBIQH7|WZh`&fFmDo_tW9`$FsP`#}i3n5+8ht(Se*1 zicmK{_E()aBuF@Q;26yyeS&7MIIfLFm~)){BZ!A@-9my;J?Ff-_RRiUM+d>8Eurpd zIlPjPkd6qhn;ecHz|`v1`aJ<)FzEmIB=Uo5>HXgZTs|L&m7XMP@d4H{dBBJ4|L%eK zKbNonl?|c)NBIgUW@-K|UTOc&#jDaiq|24EsUHy8S5x8MuV2mm;B%2P`NyIP5MjB2 z{>x*#0K}X$;2r7%K6w6@@ACgHR{Njc6s64VlZ6yfD$6fPyl zl<1ii=&S1)>1!Anlpx_8ROuP%8FwWoCdbFAp?;6aG_dePXxqCS^zEgC(^?4_aFOzMq1QS7SX|L(8_Afb>wS=ghir&jU7CG zog$)(gl+Y*5`(108}E#P3a3X1u2-ll^V@snp;hGV1mLKg3|4Nzub?A}*jL$9Xl`Vs z5eL!emtAyJ5j?!~ri^8Jool>%opT&8t{D}%yUXFEQ zs}EdB{KX0-0koB0#v4i0YGU{TL6>5$I7k@-n?xHCNsuU@sAU_IR8U_c;kg;sU*3{a za_WYVP#SO^Pvj38(Q>WdBFa?eeHbMnj+ulAXu7)x=)nL@_mDAh!50kR9%NvEL9O#| zgBopn0%Tu5z@Qf3X6-oMsA5#8xE6fPRAJA4m;4N^3sUG0O_V1?#|$#fMX1p7E8$sj z&N~d@OG@j4ax2Kumwj7$!{_Xm-LDnWQ~! zKJhV}1Sw#?hkm4rpnOC5XTGGYu%NNFHeT z^=b%#2xmow{+qk1+nybYC+ z`3H6mIcG9SRx5^0H79VOs3Vee*V)Sa%~C#)B^Q2G02b_=o;65Cm+(h z2o$uJ6|jvXEjkbXlgPd;z>&mwNH9;C8X)vggeVc+Ztqc;uW_F9zbEX^N05xZZV9Jv zM%jLsCQIr97=nBQo~eY#&aSo zk$6AuAc+R()B3i>^;ui@i2Xsyk>-&e(vW#c!ObXvwWpqvj4kUnT$W{>u~7l~Gz7RT zaF=XuW<_yM;2TSQ9CBfxWBa0{FiGleQ{_*fh3Z7rQcmfgKueN-K`Sp(br`>9US#ox z*Rk;pAwNgWdM6Q!4<{`@iSzNSsN0;%mwQwnY(rQq_E_>Rp!{hL#;L?*hg)ryp9cPZ z=MOzZ13MS_6!bbH)j{daP19D^goz>gc)59&gQ92UBIIvLrAk>+(shv7u_3DKr3+b6 zUCT9w_WU;=Wq4Fwr08e8qVwZEv2X-@!~vGe{@qd*Wt#p+K+`>$5<$I4l=|{P*r)s& zF;tg8t7A|kT4p>nhO)CWg7hG@b^g6Q)J0Si%=0Z@ELTIQCV!+cwa_B&I>Dn7I~e2B zS7i*BG=t%HgBrt0Cf3W1D#D#i}Xv$O%&(@IVO_B7M~YEQ$(g4qG@ZighP^{GN9{LEDKXW`n&6{fS1{4O#KPyx~B@L|LVF0 z$>`I^&r7Eo&g$A4SdtpI`8zQMa#7LKaFLXSA&n*S`yxPO?BY|*;Y?VCiezU;a*r|M zwSmDtahBiU+WNrkOTTV$%&B+3Nydt7jfu#jxF@Nr8n$tyW;{(Pjw>rh@uP`3K{&`2 z#yUb5hRjfm=@jntk|GZoEB;fS<_(ai!S{m!504E zxKx6T6~w8H05l7#-9Kp-zGwx^*f*_z&@6wCM1RsODs@Pbf6*-6{x5}FXkfA~tFHRP z`W!bow3;Ei)I-Y}ni|XMvi0>fY4kk{F&?)4P7Z%KI6b`50tqa(>Y5Z7y4@fmc81NU zT(DLKx8i^2WV+Bl9+FjxDc4TC1}!5k+Sx$00^RVP?Kk^uZg=RNJ)WhXB-K3=cEQyx zrWd++R<}$T>lkj zqt_Oc;1+p2oREiMXZ1ZIQ)j>4Y*)52<#)RJOUTT6bUuFe8_`r=&95IFYClmHO_Dcn zkJ=r!s6ctaX-IO`h&lMEc?p|;J(PlJUZueIfLCC_=Ka>BueLGF%`fC#Yu>0_v#`}S zMBwN4mWm8-70MvpNJEK$b6`p3CAxCfdy{U-e|rxM*L-5^dItwP%5Z#WO}>IBrn?uu z=WX$|79I%cUYGK_<{gyfJ>&TB4K|V0mfnbzDV#0ct&&Z1XC^xsc)Q|Q;JBtr z*qfvpp_Aru*>>rv(|{MlA%IxzG=_ZuERJS2DTt!^c0UY+N0L&fU%{+YrQs!&hq5DL(S&lH|t+jc=s|> z({PioIN<3~d|pr6LAwSVWe*R6OtJINS&i<*vh694n#99xb;h>CX&WflAF$+;Hc{H$ z6~iC*%4bJ>PEy$#*_Sv+eS|6HY)=8fp?a)i-djk?gRz?(R25NY2E9}`S=_E=P+a|X zmO)SC-q&|D#SMF^R*S{g8*9UlJd>uoYid0ARaEM`$}cKseUEAz3k)6%r+9^JiVxnd zzDf19#%~oz_ic2AnjEUo+3n?3IYxLKg^UBgb8e0%wbuFW&Nl3DHp!1OM07*;1%wrQ z29TU^%UR*Z${rXaaxCf?zrOuh|AqoGQ|iebw)XBUzg4Gj8dO^mSvM~CWT~T(mB@MO zdS#~9p<7w!E!}uFLC7T^*cRKgZxQV*scVwi*zJ`pQm~vnuxYEwD)j8VGPSEl*6#rA z*a!cebG|h-Ww&X-GIw0GPkPvNATWfa(^{%CA!dLvVJKDxPx{5iKxJVN13wCG+~#n1 z-<^$wcV_r<`y@8dYvG>9;~izUK=Q1&v(TeUnd`Q(<^D3vcG`a(Uh4^%hs+(BE9U7T z&5{y9<06`b6m48C=@`F0a;Z7@8p|ufEUjmYL@7p6uy0&Wm^5)*7SZcqpsJp>RsDmL z>v*KhVcjwoPqWR}?|Ymob`#Eb@!z<`Lq((ab5S#t_(+M`#SkVZp1q1oNML|iWJ#_`T8=bFobQk;GHf6}*%sk5;bA7A?U z&UBypooh_}h&P(B!x4M^ne!Yt(Py1djzpHVeG1&6Xyw|oz8^jPEVwystEafWe!Kj^vpS7CpgYHtXp}VA|Unr0>8Z6HF!{Z7dJ$RVe`_t zoZS2-TID~HLpH~HY&*3o|ED1k6_A(ss!X^cSDrC3qvNjOIP}`77}_Y@x>og0?w|9u zIvf_{-reo#hFvsD*2R%^C?1zi5G1$Ox~YC}Ei+lQJuSm+bFk0_j?m%YKP{$JjrXFM zonqIWMV&8L4zn#eoozch@tA#k8I9x>yJ=-tJ+&AbDZ)w#)P4OE_s3{s20JFiJnsRF2J4`ED>pz!jS^3=+D**cGSu z^q8^(1LsMqr+Fda_6~=nb9su!<6G~+*E`l|DvwY}7$0!WK3|5uf46NmK?`5GarC%{ zq-#KL=^R`w&6d7Y2c3eae#I$8ITf+IZnNbE^TtSNy!VPz!df)Ih2DZ;mY#^5B1yLS zbU1MLcFjo@s_*}DoeK^z`s2sddGD%NUq`4GOeigO?V+vTP!Y}#+3OpS$|dw9+fH6* z$-x*x3X->jj%>BSzHBYkIjCYT_hOaD39OX1{aTUK=#R^wELRl~Hzr!WB(xmu8?pDv zqd;D)R_DGjW+JZn+}En-KjaJTyPW|WG}l!abHo>2}Ao zc`~|2sL2VOef+o(_(=m4wTbcXjaX zH0Ao&<`v&P+`?~4b;nlny#*|`xj(Yl+^(;&v8OM`(GS!3%%s|zptB>pZN=_Vbg|46 zsSFlofuXB)WO}O`8a{OIYce-hjT#4kOgVX7UuTEj?ES>@#A^1j@T5E-@Caf{#iCbzX zXm5)B*7WUczMn3}eHZI!NAV`!xp8h&PqT5 zXgKgYvDG*{zhbkAEf#$)v!vm}G&>GuZ=q|bj2f>@r-|wAru934#>V(9|0IE8cHQmq zE^P?i3gQf`e<$hV8HM@H97XD|mb&r;~VuGn!>5 zwws|7qMP^7UeXO_?b{WP(BoM!iSZyA!xgq^|fscj5Q3swV?U_-{2k_h)?fJiDrDU4}dz^#&s?NOrx3jJmQf zk2D{kX>Zo9cR5VwbRCSl;ZrldlIzDLoJnLG*--A;XYA@ZI#;Y5#EMNXtJ`U^R6Ct^ z-Yl0YE{9uNt0Y>eAL?T_`HGauB`N1WZNY~cAkOAlXB+3zr^W?Vg+|q^CYqB!NTUpK zMZTr=5trl7HTbFdF_9+svcu{ITUfb7S5C9EiY^{v>*QNG|Leu>hs7@)^rid{;D2Rh zpOefa*rA4VfS}195HuQ-+hJz>dxx3vua5hF-eCsh zO!o+RXkch-h$2{da;OZT1z`aytR#>oddlczzxhSu)8f()r}D^~xkW0d&c)H($OA+M z_9#??iq%m|w{%6csU}-hMLyh2;}IMER6Z_z-kLWtI*s?&oF=c^4zitS19<}0+ygak z?RNtCIJahzygBI!?w>TQ83iJ$hiXm2fKs5ARjpx@<-2(7wZn)uAfaFkknb10rNj+@ zMFK|i0P`udg6u>shDNhk|WGhE9W896dEzzdAXQ>WuqYW~o~4iDHf zjOR%vlBl-~C-Cn#d^9r{KE%WdB=$ykRa6(vBaSfI!l8Jkrp}jVK$hTpLA6bXF7?mP zv>PZ|=L3lekm69S9)pNYWHt0GTn&T}*_$yy>_0Jws2BCAB!lo%qXa0)SO7{gHsPqa zuxejgf}sJ(sQEIwgl~wFGGb7kyu<-eVXCeM{0lU?LWRDPrLPldo;ET9j-48etBNVpzr;rR9i4u|F(VPelRI1Q0(ZLI* z$Z+3V^awN1hxLh=99?=~xAA2%Simd{mVOZu=Xg#~;4@~D{3~VxY^MNXrY=CtM3@+a zx&Y)c!S4X`1-h62tM$Tw1Dvt)r;n_OoE+hs)?YrdgHIn>1M~q{OzA>lK72hLA-<2i zWs+SUX6UMZCwasvKn%yUK%0j{3lZ5sn;%#7f-K0G*&uZ!m<40AM2)(Pu}pc}QlKJC zI#^Hp4JEPDFODFk-Eshwy-Z6jC=allf{oU_^D%PnCItEm6un76G2{{knybv=OTIRx zJd}edU^xZaSO_ZOXh}g^z8DCbU;n1TH54>}dlUz4_>%P$IKs6ci1htKUR{HRc`l_q zfDfKhTQFbNE*_jkoDCcQA{u^n5JD~gTb={E?O_4w z&-@m=1tQiAcb$B5`@z)Xpwcj7$e`}8RiI!Ypvt(2!?X_4InqAT8affaMfvH3DY^!IZCvgouR5OVEmbia^Hk-HQ%k zQO32O%PEbpDI@Y|j17A&_TD!h=bHmRRN)>ZI@Y%zCr#a|2lytAZ}TdfGi|9@YPs<8 z<_y7i06Ei_&zvdKQmwK7A30OSXU+ti-a%3U$eB`lrvKKF1(74oVn2M9_LJ&PlLW%3 z_aSK+ERWS`1msI503Df-RIxyU*;i~DA{{@EhH5w@LLe7LK+a^ipwC44$yBJgP&E`& z{EMk5DUm~^CXE|Bgq}O)X{dQZEKgrD)ccWs{tYD$GwYdo7S9F1ROHDCWfB6Iiah{R z!3kh07`bAy%LoBX#SDO{DA#^-0P32#!0c$Zpu-4fvfv4QaJZi_)8*xyHS+i66ONvvK+#GeLq4 zCdNSaU3fUqK4(+>LbA9e)#ut&@aGyeor{mMCp=Qs=6o`b&K^RtwQc%Zf% z=>JhSXB;k57eBZ!Uu)OK(%-yuVF?>iT0;11xVRCdP6{W8t5qX%W-`r}SUr;#+ONiV zSp0K)dGAYYIMf<@9MgP0>Xb!jE!xLJP0?lD08n8KR8r-Lv^MMvKPNUlhgX7px4)qB z=l1f~0h+KvER}=rw3r3{L-PTUWQj%JH0anTnEHIEd>3s4s=p!+*kZi3GpW zM#JO4|52i&n?@Z`g-s)sK?$_f6~7Kb9ZSDMScJCs58l$xpRvoW(f0erjZR`j>v-H# z)>{j`3fHsN9jXndB;oKy1%JGvpQnMhlp+zEvY6g2X#^(29E&z(27pwc09I2zkqY~- z_<-rJX z!6r~SiEPY&W(T{qUUZuMAjej?ZV?{&js*>|C~=XIA@tg6CdWe>j=!;B6m zTUkvu)YR<8Y_BXYuP!e!P0D;(%LK{NX(!bRAUNqU+l1Hy+9Z=leKM80#auzyJ{WSL z$<$vuLZAF*nq=#Jo%l;i<){b+gZggvq`7`;rtp=9(BvEg+Do_{`%gc#N!FoViPzHx zvCbOT>;A@wMfSc>X_?o1#z|bdl(Ok^P8+(u`Td(^WrqEARD9ZNCjyV}LJo_i+0*RW z%pOj{EK;`bLKK{k*o0fDM>>=5#S2H%IJgEjS2 zv*&NH*@XDMNvwhiV-hq1Ke!CWL$L%F!%G()5S`@x)ZrV)10OCV>C@Nj0qo2T^Slke z=kUd1(H*rjIG_FAB=A@RX2+{$Wm>gz9MkRhQm3oI@zC0PJA#R(X5EE@vL6^xQ6C!H zrr~!LvaIf0)~ZP&d#k&Ja@_7#6-9p#!WKFV8Gc*HpV2jpqMF6jNS!C?6Hw)@h0r8rEsadG)9{WwA@b#WnlGiv+<5McQ3~C~ z0?w-xgBmGwVocj1v%zWDiWnC<38#!_wfXP5^yta;Ojh?ue7;hw`s)?{PB{?#(9~F3 zYOh0idLCmSYwzhm@^R^ic1M*9>l8JWfla(4*ULk0l%tW>a=aRX@|>zMy*;KfXIq`q zQZ9S#qQ#Kv7w~V|dTlBn?Jm_r!DwXGlhpyQg4{Q`Po3>qx=#`?jyoPMC$4sszDS(K zRtE>g>j6*ia&{xOT^;1T=jmCOO}Lvb9akY8tt|PHRVwRCUC$XduXuG{v6N9Ru;JJr z&KO$qa}2KYHPh^@`OdRBGsBe|p*~9S5Kl7n*OD3`r@KA_4x1!2E{)7mC`Xy3-*nBi zzLjsms4CU)nd(+RiJ)iL$mnU3Lvm$h8*Qbn-92U>&~n-`?6dB|TjOrtsZ>qy2oV{! zi+UvsiB)lt9lIC8vN+WH)X|B2eJt2_H5$%7Kv zJ917J!fEo@^5#wv%(qye_Tg?8s2&^)@AY=F(Y1UMQbFUR6}H8y^TEC`eRN8+nUR0t zSDiak&E zo=Lacb@FwxFuEV3i`xR519te8Y^m+@8&<+K?S^xHWJqSux85l~YH5Twms~_0$00fz z))%v9Cy56hz0*Ey7EgY6nc~2q?wg9>UCGIma9}jae3UQOX9nU(4~0~CDDz@njW@GW z=Ov4KU7p`ALO(chUA$J0I6bhA_IAFuA7dEonTS5ckfp%w7C)Y0|8P*WJHZ=fTXPza z+e=>8k=gRPRGen^>fw3czhl4XCpMd%sy|yKYt6z(sf*a1aJ)ytxFy%?sI-&Z+nyrP z#qx>g_VP_p?@d01lf$mJ(1&`sw_g=Q=-c>%ktr9&Tv1C+$?@>#=|Uv7ErVe5+R9>3 zOUp}OKlK}qSJl%&>inBY$?sK+z3hfM<`^36?@pcLB4u97HG5N!7GCG31mYchraBhK zMKLv@FoKcCU+}CGa0B;`Az}%9-yced`Q}gUm=%3Tp~YJ#eBq8 zd#Z`XgHw`{w#Bk>Pn)l5U&{R`&_Zd#Ki&O)G2pp5{03#*Wah#3LVb$#mTS(t?Q7F` zv|SAlh+Plq9+_au8+CbB^d$65-n}HLqS~!@%n{nv2WN46E1xwk*DeZSII}1CU*=Ut zhiOVSzjFRj&^^vZ5kgC`p&EpMs}~=Q7H_F#>!Tf=8A8XY<6HE9IWNkjYeG9^tFNk6 zB0ed4sO4)h+=>f>_8LmqB=uOv&(v##r_GdK+H^QsXOn~M;GWoP^UC9Rn4t0E=(x#| z17D(rjIHZO=apvWMQb`5+{)rQaefMC3yIOIEw}a5biQj1Y;q1PZF9yFI<31udT;c+ zK-e^a8f9W#m#va)*}v3^nw2O%UnQ%tGRn+e(2kRff*HOC>|Fgo@#y%tk{S9=onB40 znE3)5K6;gL{4)%iN2}CcJKJ%D%TJ}g+0p*WuV7^a=Ht-B({GW(^R9L~hU#sj1I?WZ z)aYx5w_Hn;N%?XYS@|(3ZS>_S@l)S*@%C>FIbWWoA)mMbp*l;4=~VvFk{NY09$A<4@jHhwku*=%JZ& zIT_^wlZvj|KCWJ-S^w|bGkJ{PGS*USi$mK}L_TVUT<_`|RC$3L^ByKz8cv;(T|Csd z>?!x@j$F~{oWfvynsdmQ6GYapH@d>V3s;%yB7oK8TkW64>%{`rIX0|w_S@wG19|Qd;w$79RPpq$`h!1ybGIJOW#GIB5h@@ALa5e(w1?uR z;X9T*++GS4(hj2t+t%HeRW&Z7p@Ql>R7{K$3h7-{f^Nd}(k)C>>&?1Y#vJX6kR6Xz zjNxVPh2BJp#p+tF^YQpj$EVmcKj;jWU8nrk=+c|=a{pyHfxl_v>vkQ1Os0p;H0pc$ zZI*_PjF#KfdtRFQ2k}N!mAB0P3CEr(ZR{Cir=$LDjmiW8CC8#sXPbk$>_AS9aj?<% zz&&23rnf=Q;Bk)XeVwrfkyf)^Uh6LFnVzS>jq?k;P5O+RcM;hv77mhDTAiow1OD&I zq67Gh8W;Q!eP6PCUz|gZ>^0!=NDtksy)~3grg?+ zY|T64slJ#KoaOt}$KFr#u895Hi}#d}K^^~u!;gQvWBQDjR!X?5)cb@61@{Vi@)2^WomP_s;o!&-b^S-#Pc(b1gB(7-O`#?&7l)>x~)7 zmSyg|#l-|^YmSr{-mWakuszb>|3S~eyKFPR;bz@^3(?=ky5ys4s)F~nm*t%gI-Fm@ z&ah?b)*hq34#r-qlCpkK8y(hge#WDa>EL6|7;)CaH(GCxd$Ur1O` zcl8JvP~#ichM1*lo;5-mTiek?wru;l1p?W&2Ow;y*~u)xnjJFSYyKLJpljS;X@7U? z!8}8rSlc1&fB|na?G1zfp-!;uibzkXiwhPA9k_Sjee0uQc+b-}Oot9Jb0sLA{QJa3 zQL_=o`KOc2%%MlJm=2rl_{l_+c27{Rc_{l3Z&EI{^UFD1oV{};eq#D~_UlLi+X=JJ z4VM{I+uib%+OyHKbFcc;5W-KW*KGhAiFT_sq2vt6vDj4~!#AHr&q%yzxLlRa0J z%=r9gtNO2!_~_UWXL_Br)AX31s0n*TWVsxMNYeHKUy-#-H z74Fb6w+GVUv+Ub_MteDB-gHG6`qgyzHYE=6v9>tY{smWdu)s{e&c<}w;_5M1JMoor z|IXw++})Jv(uRw?nrf75E%)4|<$h4w1QKr?1o* z%RQsOirxOi`SSxg&c4Z4dCx5-g0k%$^zNzXp2ugJI`s^{7vP+ps&EU^#0GaK$+k{5 zAmbEhgQnYYfvj~?CO1Txvm9HW&mxukq6FAv*urib#%Fq&*dbBPZw}Qzg)=KX_cUto z*~u4rwB{18jpAzoMe*ym3(ca1IKxc7CS7#`~wBhRfySF7r^&G<3dV*juR==p1? zy1&_L6=l#gr>a_X7JaNXOUc{zhVUL2NY6mAMOd1e>{x1z*QwhkT2t?8PZisJDcIhAQ6@RWtrB*p^P^9} zNNsR+h#E5Fow_pPM2$GSqnuUvqe@4@ZWrNXl&SsT6b_XM=gsdPj^QfP4k+UvY7v%d zfNAvM*x5Na*zg>jY;oFbG3b9p3i^=Qhn?Wv_q?i#=>n|BZY7?_f7Z~ot_^8{w-szK2gjH93$Rt@};~R@K;Xaf;scD;&q1ioVfcwOh6RvW@NU)6Y)NMV!tG zHumq6DvvWum_f0#ieA;yk$p7s53>vWv5o`AXIp*HJ%s9bI`R(ucIYMAT@R{XJBpRj zG1oh}&nUR7vFDEVsRI)|4psecN9T@?srbN`^WB0na_RGB!x3+VwkTF7f9#Jh_UzKV zQ23ZzD|9c?GC0cpB>vr5Zmi2CjU7i|vp4RY!ZrmDCcDpAMa}5;h;_SaGhtC<*8aZq zM|kZI#dWFQ&r^69Wzc^s+Sf%vc-l;O{-)IYyP1flG!~s{DXkFIHwXvAnM3>b29Cyf znEET*@kmzDyCqdk)?+1P{ug6&YV_Tt`*m|{b9~9}Ze8t!=#-E;--*wsxlu|7PCUub z{ESaRh);KEwnyt18uXX9%$V88%sZZv+4-u@wc##eR6|?l=QgVVs`U4@A=BS2=~(Jx1lfTBI>2Fq1;kk-xyD` zTbk~VV@#S>u8bJnRnJ{1AzgAq2MUuCf3MP_i-<}|v0W$~DbbjitcV@ys8ufx%o-P|!{}t1D@{bwpTISp3rE$rz&-&}hDyED zk@J$A?#-V*e#EBzn`WxiTC$5qM~6LMCsFKU{Orh(1Nb%k+WbEXP9K)XZuTtu7p zvy*lS|M8EkzwTcAs(j|v06QcnD%X;<-WPB@C#L-zig4cQ(S#nIgr0sT&EnMy+qS@A z%NvWwu3aaT&fWS4`DNE?wbb#%D{e_vw*CUNf)L14{R|yGU@J0u{~)j_sX?k#rlK_i|!{3BSE1X{3}%bFJ2*T5shQ!1VLD(UzNwhyP%=-^o&D zeO(BD!Q7Z(@hD*{sBVus1Z;DTn!ae+Vi7%3cY5*ktqFhW?OPYWEp2Bm089{m;egZu zNK}p%Bj`usH+*YQ3Gp**K4KyU>}3Nw%wbE6 zBmHFIB>5(}rf^9$IRRo8CF(mMkYSvb&az5wOs#RZmCMl!VU0ATe)|Hg^l(;JZG}YQXR$rusI=9xJ5=L#Z2DNtg-F$yCgk@ z#LRH(`qAe@w@*2hN;RrqDakDG9DjZiIbTw{ceYzE%E>VJb!=?p>&)zbLqabb;qiT+ zGr~sxP-Q9`{<-OL)z+xn!jo(4;GataH}171VwdMG+PGsaAo>3=bG@GV7Swj zW}9BeF@|dxz-oxz<4OG+b$}=Ql77zqKW|;0_xe+BNA8_JL?4``d2rrm*zaU?Fq8g{ zkP2K%(^dRIA7BmkF9v9v7W>u-Shk;SPvEq^Z9FdZ$gi{r2fI^~k@Lr8(+(*XhNmSQ z-vRx^E^vNoTHL%z;=M*P{+~2H+Wx|#YHPaw4}I#P>Y*>bmOp5$c=+sXAe4SOPhWmK zTMV7DAFHa*_DcE+E8FzMulTDMJz4JRG8DB=6@-3Gep_5O_$wn$dNhxw*JoPvUSK=D zTadQ6!V~{9yK?KUHaK<8SU#jxv3^;)_`}J(lb>e*^am- z%?`(bIycv}7umC;L3>5eT2IZ+^J_oW{P?{1I$T7@ueR2+!n2|;sTnLnK}3A1)!%8oGCX|8|G>pk%Kofy&zmD*G5m=&9vZ2clOb>PvQ znRw%qJ|h&~$kVnapghi`qEI@}=Pa_f&+W6nznW5r-M&(Eb&aQI(<9kAajcEAu-^{m z+yMW3Xk}hSs|c3jYdBYS_?Ohglg@7}6!1CM5DLx_L#oqtoPMfVw7gfiUu!XcbnA5I zG5=XduYUDTh~ zuA=0WI5ylgT&oh5sF9ZK{DFhRkF{mb<7%m~@OwjWt5Dd4IlfhUf63kZt_cWDVeA{* zFYo?zPI%uaCoC!|Q7bJjaam(ujIPSy#U#%iN> zMV>pl6POn5xyzHG$W;Fs-?L*fRR-5A6oPFfU|xR+$0_aZDJ?hQHEDPh^|m4*LgVK! z6WgAYK^~vF0k@s{sdjCaCp1(CWm07Srqe%Hor_60WM*XYioX9@gltd3k@IJrs_cH( zX_YN^k}8kj(U{9>DClI`*Xw2Qo?XYT8*JZ3?hSXG^NPHR z3V-2p?zB8hUtMQ{dLYi{Q*8?~G7}|_7|i$B%h}H@np{$yEhlti&Lv4g`#@$36NjQN zEtkC`q~q5#Okv$=n!bE&bGK$zhR4Iikjp>Y7C4*;(qKDV_16WrW^Y<-Pv(sb-UG^V zSL$?}rF@vy)wBu5Oa&w$XL!`u~P1NkA z&C1Ndktqwhl*pBtqp&P(`LClIfME{GpPZwgE)H$Gj*VNsEa0%?Zt{eS^y9Yq(#Ic_ z*DOf6G{S^?tux4UYYIOhU485Pv*IBj`L>4(ymd`x`zTT)vA$M1?gG|xb}gtyz+wwzvZ69 zzi+d^8wI-`J9C$e3`pO}$c#iR+!j`QZG^F0a(T7lN@;Wd;_Ah*8f()G??2<33qIP*ZgvM$0 zSQx29!l>8|2w`w;sJU7<7hv%Wad2+2LqZO6i1FEbqzzBI|7uSy=R(gz5rAag%TTwC>sN^mRB}0wP4Gisc4S<&o z3J|5-p(JIQTIHOd*uw!5uK>}8cR>mv@Pz$!oPKTmi!uGAWltfhd>U4yeqV9qog{rJw{x6{NRe+w=n&l&`?Y^t{L!jcr*DD^Xhel(pFqVk#tB4>-5)IQj z6(N%ZT^iC~u)Iyu#}AT zA~7{4Cy&bfm>P>i%WM|fN{bUD=L#4KAnQI*VQ~kolmkgI;B`5I5U3aK!rVegV+4cU3%d*tG0b zMg^t^CJF=jD7F3A1BQr-q%|9ZB%pKgU&b>~(4gp%%Z&k&?QIC5U$wfK8rA(FWW;U; zd87p)`=K^KO#i<{26jk_q^;csfMdnmO+#Dhkl?HX42_~AE_ZD(6In-UcR(QaK#~D& z+*J)W%i1bvU}q4Q6bFGNIs-+BYsU)kh-eOqN{CP+LtvV-o*@GBUEND?H;C7K1F8CG zGi#K&t`!D>Sr`tbVkGf;jwTZg3k32va7zuW2+vK)0|sZz_=%zkuUr%Yrb2^a6fvPO zLnEO=x^bO2gthy{GDDvNE%0I ze7q?F*I6%hy`kgfFz%_8nD`gK72PUpOYq$klJ%{P7}OWJA})O@L9#ZA@qAgG9u?(3 zSpyJvu7db3>iZ!A3*Az52*Cx1kf6pP2o@*ZAf&Jj@LAVY7FUz;A3`LvI6>079>Dss z8|#^n3Q~4i^Y=f7@P}QN(6tQh|4nm|Kyc9Gs`@c_AeSXGLqNxb6?i2`I;Lobq}FHU zFyq0cs6rTjOYhnMB5|b_s&6HMU#&}iUwwZ!Rf524J%b1!i~3|Q+67FAV)C*a^}(-U zA;)!Rf+}2~1#32bDa*#eo?zJEwXn%iJ>I%y{S29;7BE{Yt$D@D*wkS<0$TZ+goza1 zrD>?cw0>OQpbaEAR3K>OYI0iUsDuF07RZRmyohWaYk(SxUbGy{!3!Oc<3jGEjvC6i z$in0DvPS@{4TiP|0UZ2*9J$5{dDeur=HqhI$pSyrMoxkAQLrqk=gn;Vh}*`{2v!50 jJxvZRLF7B3Q8AaAlI3hC&c!+lISa`J?v@qF5M1d0F}FNs literal 0 HcmV?d00001 diff --git a/hudi-common/src/test/resources/variant_backward_compat/variant_mor_avro.zip b/hudi-common/src/test/resources/variant_backward_compat/variant_mor_avro.zip new file mode 100644 index 0000000000000000000000000000000000000000..59244af7dd22612e0873e208a652b22c2e1668e0 GIT binary patch literal 87511 zcmcG01yG(z)-@j7-95OwySoH;5AN>n?h*(V+&#FvySqyuK=2=Cwstl%v-7R}R6T(z zD&6<(?$h_2bMJ$kBrpghz}uH;G>zsjKm6?<1ORLRCp|keJu3$tOKUqFJtsSBT4f~& z0O0SYCT2IUkE06|03gUUAOHXe^6P(+dmZZSJ2*f~2#sdliDif2>#sOL007kgY$zIb zJr-srJwtkGW_@->Y9=E(CTe|SJtk^;0|RzCdObE~HUL}c(9%G3_a=_z|*>-&cIN`zP_|t7Q zwYD}iGx|SI%AXHO@>fIBm{}Pbx#$=g8QcG#!~c2^n_my2Oa}x`{_C;)@Ol_&|JC7t znTh73y|vXZr_1{FbiF;lJMn=r^iW-N@PW6_VKGhdxitf z<`o^+c1C$UVw-id1x!zncPR3TL#g~2cCf4UrSyhC8#pc;WRDuPDFLBDOK>Ml!dTk1 zy$h~V8!Vd@TOPyE+n108G8lE7?79qM%{_pBgXP<5?A{J=sR021Jb(fKQ2f|Fmg{9)sVAe*KPD zR>m8D4=Y~V=CDrUoaa#;BhyfV1UtlN90)vl#g23UvvyNr%2wqGXcv*#)Hso}UK22L zeKhx6CENye>}*fVCcizC#;O3p4|TfaxSZ$RFrBx^2PqBclDwQML}3~tgH-0$1Sq;P zm}%R-dHk8UJPG1#v#WTA);`Qi(w-iSort!38mO1~o_To6%pcr3)nD%isL)%C(-_UP zFk#WPgm5y{sv5!O31FA;*>>>}{RjHzr2)>qGHv2{s)745B+q#t-9hA)!lN>uy_7Z7 z6|Z@68ZajhM@!E?IN@FByrA_HtuNc*6nCVB>IX->XnsaYKtkdCmUFYGOh^0SPC~rn z{>1KM@Qyt#wcR{I64bk(eyM;++wN=yoQgc=ck1{vonTz8mS8da_>CZhszs13YM(oC z@wdTPzw`_?U|0q?8)Wc5>W5{3Bv_Dj%Q^;EmeOA(eetCLz3r#Ykh& z%Ra6c^Bcp9tb%B%s zoD9NIS15t{@jYRKToP2rIH|4O2I7%qK~T{|{Nt%>SxCm#*S<#l8U}kOgx7+mJUMYruYdT{y3nMeb|8y>bznsh7z|_c6@3)Ql z%MJfM(INb+p&iUDjV#Qp{_lMDTf%vYCnm3cy<$gT006}PrLljaIP~mv>@3s40B3 z6TQGJNt2HWHm6;NX$pcudkOEP^zA8LtwKIcb8$RCJ`yegbO<=NNZ=#W=G{ce=@v zBsd7I+Z+ha50A_2;i7OiEi_YGy<^u*U!=D7?G;j&KAhkI$pl^h0N09rKNQ$aeNB*S zuFkEV261{qr^ZMn7;J>R`oj4NJk$spaBqg->s(D2YkB3O6TI1w>TA$et+eK@5xA$mRcLg|vEDqs+2qGWQQJFoC0 zf&&Dun~6icdRp47xrQkZ;*sqPl_1ITRJnO8e;L&?dL!qQHL7(6R<8^o^#tKPhM@^5 zn~gJo`VT3JjVKgtom0*igLqJ0j%+O`RTj%CNpK}EX8`_n7-=g!KmIgAc$_HUlUp5n zKu%<5S09+*XU7m^hm~B^v0W6RYwUM5YIC}Cdx7Y7G6)Oxbse1Ky1PJa>z3@9*W96u zprL6fVOC$LN5>kBhyy5x=AhPckSuCZ60n{PvgfG>mxc#veMIb<8_)c z?g&wncH!2K8m>AXNvli3FvTcHCkiPpwL(9~RDPmrg%ieN%b{`t1UKqa}aS9KU=M{~ut+ z&dAo$$lk%o@UKAgGc!`8+GsET&tdzEGWn;lwRwf?KZmW+XPRO!J{2t^7&1B^uq0@+ zaHd@Tj#{XIVo5tdxF{TUSWM%9nP;Jv!+PFHN0p~EFM&)DG8}CJ3?E_rhYWWMoloXg zV9q8><|&X_x25zP7gvw+$&6^%X*3gCU2`=Y!0%nW$D5=t;y7Vr=gB(3Ai}lBK`GI@ zddc99TqEwPtm8ezPHeD>FxiO3EM+UMag+as*Yn z<2B|;N!!!0P#GdE_SNts&Ry5bE5w<0fva$q_`P`x8Y=l%q%bs1Aeer~H3=XE@S4QB zs=IE-3GDDkU$=6dQXa(L7lW@A`^O;To+h?VWGg7J`?<;1o=flLEa*d4y;mq)Nh9vn zu_N$Mp)o_Ou4T^$t|S%D2Pv2#fLkzxs7bz@e21JR(I!o$cM0&_nC-jGv?8nsj!!T@ zW+v6*VL21GD&`i57p8MG{BV5~h1TTMX}FirMaRu?q4vjMI{Ppwr_dUV9o<*VegdNQ zAVs<8fV8f-Ien>02j%6;#(`30u#A1fY_i|R9E{W%x}R4DAv|sr^36jq`%pHAi#h#d74y9W*;b{ztCm0V3qA=mO*_m8HeAAqn_cGGTFbRAp4w8)=49t`Qdt=CVsT4+u^csvp2SKHON( zei)p#(wX~>GGEc%_cGj#_r{uT4u4z@xPyhpl?{9%2*#te4$TS{TN28lRpv_*0%*8B z&xa<%l^iSL;fDbw{a1FccRmR**RG7ON4!Im@N7g9@Wr4m6vCe56v zPsU4GY@5w5YHUcl;TMvEp-U4YXv%@4OOKiMNKXc8KdlGu^6k#;$u+w!j5I-yc7cju zDlDw%P}al=0-`o4ddV<`88w#_!Ul+bI?r-5+*q&6?E_v@7x8lDl0ZmG;jzjZaaUfa zRdAbdQ29d5L>^W%*t`O;WM`(UuwVHr80dDIo{Pq?#T>n2roGZ0%8{5s&PVud0!`de zsF+g{n4)ygGH+Vn6DpZY>79oa0uRycZuZ|hxNj| z_>FZ>GOLoKMF*s0-(xQAVlq=ZHk*Oedj+t5&2JAc2E(Ad+}Xf*0qON4B*>WUC44rh z>hYWby0x+hVsN6M53Q7>@T#`&Q7jhjRZz$Wy49gfp2Mncu|>lJ&FQ?a1S0Yc!S$Eb zH|jpfxd8H>Si15ayThAENHF)KrW=#ihPcZE48o)2t%IOmRW5U@Gp#*7JwIHJTgzs7 zyUud9-(3wqxb3*Y*fW3acZx8heOcV)9y(WPZ#gSH(7*!#$p8B(>g~M$>}An2(=+}f0{)(={={jhQ)#tYrh|8yfhCV=vd>;*G39)^XN${bG=Y&$=aj?6Xd)twSP33$atB@+XH z212X@jE8O1_U0`-j;XZAv7^xm2C)PbALIR5W z*|3A4nm-h&kLktpXc#Oky9t+nuzdNaGRAjI;|6mj$1CF;a)%prE8uFI-_)BR-lfJV zAQ*B^H{(qLe1rI0dBJxBVZUb5i$^ymr~|!f_+7yyz@fwSQ(_omKG<_elxg1yNvX51 z3w}IpWUtbQZ5)7>V3E63*150(;4BLSMZ8O#U9*%zOzIF2|bBvSQ0Y?QXh;7E8Wy^1_gBKh+=do%x0Y>|%KR`s~t(9$T ztf&w+MEK7U3iDY%=#wGr1RXKqmq} zC~e}K_v>B&^k|yC)StQ~9zxFl7)8NymvX>X&IG2V1rU6=WM0`s9K^xAylHkpl$I7h?|qvYQ@TkmdIY{ zgnv&hWHp{3m8Q(n4j0;rNEq9ds3(M4#tGuhrZvr$RG;z z%Htb68kH#Q?d4TB)>LdhHzDKx*jgguFd}0|WuHoyrPoA`@0&_|qa9S;~=hCBDhrNFzn@#^c+5AK1`k8E6{W;b9 zErj124u5IG{at^Wn%O&8+qwRW8ux#|ssBJ|fBu+njP~vM6JO=GB7jYmxRL4Womu~% z-I;G8{CsCJGO*MC;m&;X{@2ggoF*xwi#gPgF`%3kmhZiu7F0T;7dNe1?iJh$@=S#>3eAGUn_4*+ghV)6s*_ zhe&{>amxsxN3?6?3kJHs`8NUd%Kqi|S9t7c17G|hIBE+NFW9sa{%s3_oKRULg$<~I`N6RBlkpsku<>mt}5ytH(+?I>r4Idczu3GeqYbtWPEXI`<4^#YvL?q%A*cAgg3b};iBPb5XE76`f!_dzXYhy#@ z^nm@IH~mdcQ*qDD(m&V`vy6Iiu$C!P0~6KuapD}Bu)v5$x%uGCA zri2gnT8Goh;EGh`=0{MbD;ydhdqS?JrTH_1ZBSm#Y)7OF!xh3rU9p2zNH7^Ka=T%4 zQyK-3uHGuY$2AnBO5&z%6hqacc?zYN)ABl-sCsm$=&Hr@_m0Lga6;({ejTYOP!7Eq z5tbdv%ha@_p3$GB_)GT&^y8yKqP7_f@Q(<=HdFcZah;pfpJvWSy*2Q%ZoD2RwOj9= z8^#iJ(!=5u5^@nw_Uv_yYCT$Nb9ZB6r|Fh+51I z8Q60Tcdix%s=kD~<(sGnC6VvcTI6JZsiKJs{jt`D%)2E)?wo9@*p=R&7Zdks;N-M# zz~ta|F#Le;$LXC?=xOL|7qBimHjTZ!llf`wnIl#fw0jQraXbN1B8IFgjy((IrVcdI zK)70Hvx^!;62^SOUJH1yC{Fo20!|(G!iCxlvaCsL{%PfOy%NGvNOf^k@=|a!U2jk2 zdShZiLNg7EFB9Q`&)o@9Kce4VJu`T62yV+vtE|uqcGiJIRj93&A@}x(C?SCko>CM= zq(q5&iBd_Te{lE>@6WP8WSmm>zkvLk`}H&A8JOtk{s8%Za{JV&$XZX+A)gb%;uFAP zv)QKDS>@)LH`Kig=nG4!HB(-W>{e2)2wirw-`Z%@TRjMj2Jism0ofWzgj)CGWM$Lk zw4A4YN-rPH34QW9f3o0x$?{y`TA#?`ANYc>fV0!pP-dQa>yd}%7-TVo&0XfcF?7zF zz=7WSxjg(1VQT{*nBRKCpIVyC91DMqbKt-M$AYY%#t|MNh&mz!>7cjfI7bu@y&qe# zLS&s`L_QH1F2G1=L+Z9!2tYJI^Fr7JJTE*TU7nE+jC+y*yK(z8{w^9w+Hj@|1cz;! z93(@pg8F`TV2l97U_xo8WKyy{YYu0|k7mpk~q&Mpa;i}x!Mur_eoP!}ETqBvh4;2U{V!70C&t>FN;Y6ABlF zRZRFM)44-fej-E-*$FqO>fh&!)+ht$Wj3;!Aq5dDm|Tj>l#-lo%GZnSaNZun*XO)Z z$frzeR+~%)r!n%jUR6BRuRtb@p0C7$d!Y&mFFv(r_UG z9$s=4$wc*BP4dLy{HK=mUm^c%0r&T0<}b~3OCtw8Lp_IoFzRvts)F)+qy8_6(%;9X z`4Rlx?Zdjk{wY&3jeCnTn1i~S?A`6++_(qpjj!8%mt32Z* z$1a}cNJj?LNdY=%$G{9s%RG>#EjITV7!Y>Bjmb)8ct4lChy91#KwgrJpT0ZO-21Ct zrRSa;kw`FXjJ+V(JGT}s?meT}Ro>lQQrc?e4~B} z%*(NerY=Nf(YRxP)xspq44}Z*XDc7vlo#>wE= zsIChHZsx`Iy?;Pm$v+Q?w4oHW%_g1msX>$aqhM?vreUT!Ww4`0n~pb?|#yz!%z!Qmg78f+jYnJB)nj4SGgU9 zr#m_9RZleSw|$HkEqj+7Y%PxO?cz>dpKjLoqVgT?pEGm1f9o*(H$}(4Y)tsWv7`CV ztB@T3-bZ?y=)W#Yiu_k|{`YlC!T)al|E_TPmqpS)?(Bb6$z=SuKIq@2O5T003(L*<}Alt@EE3P2Nmkm=8*Fya0}!K(3i0p$kTfEPN!zBCZ{u zA!j2!o_$hnkpwb>8O>H`EXFhJ$1|IFobA?H z+;+GFW07~+&;umu3!MrNz~Npq8fM>U5nSE^kvb+!t4Z6-a*)@Q1i+Jb0Z?;7f`&& z2I4Q0n$qLF(#!$)%$1{~_s#ZJQ$Ihw0Ggif!+6V@fL@Oi<9@e2!2D(NDVgK%8RA%_D^qfD# zEa8y7U@<`<=c@4tTTF&6naOul#e}8}bn)l0z=h9W!JKt=G z?c}PeQ0@x3i>#D8ZN2;C=W%Y?ip9!XO?zy0zwWR$SW3|Gij|gPr{?`0Rjtm(PtjGC z)-s5;lhmESP}*6Vy)x8$32|#~;~bG6S{K_an2sYwo_3zk9zlOgF~2uv{=7BvZxrSa z*Tdf-``0M?qf_%Iwa4FN+grP4GdX)m+5ZsPe{hd}>SB0vYyKpp*Di)v#|An71G4g+ z4@Ec)h2$zAkYj&bK(Sb)AHEqHLKw{rDCtNyW^`Fhh-bs9(o>L^z)SA2mw5hIgzwY^ zgP=`yCAa+;xAWfp%4W6mZI?S}(XX)nbdLKEus(Q$wf`%u8DC+I_6FodIO=Aw3>B1|)J9npbFp{MmF`Yxp6K&Dt`nC?y( zFVY*TC)uiN-%u@5t`I@)H~`GjWN@uO|0955uD2nAMAOn&`{se3#(ik+3wOw5!enPy zW;x|+Pa=WrMxiY$I7F9j;anp1YQX^~2y49#?{@{pE-Z38N8ts#n6ku<4KKJ_sm7@e zYRa$8%MCC6G_0?vc6viK!!M|Y^E)0$S)l=0<&y63!ggOcZ#M?YL2#}LhQWGS5{0Zj z$d0u4>KVv1aAvTDK`qq@$%04rwN{7vs)C3LyU~)Fbfval!=~6gV=q-hZY+RiI|~{U zBIoxctxGrFvbrjsGg`9V9^sYO{`~c6h+@>eXE~; zO^U_7(cz4*N%47rq+s#vgc?LPMXSB{m!x<-!T57hoEPCloXJO?^3_v6j`W=KnZ1*T zPOHDsLYf6RzYgAj9KWIPn8}t@;PmrydhGzh&DGS0Y(grMM_6gzk_}7{&m(1uqLSt} zgY(l+A@562-=WO1??1}dzko0*4Yx*+Y+M3pJ-Ic|{GJf`Yf@}|O^V6|SZ_)3%69$@ zJ|%DPX+F?+gU|BqD}0X0?Pf_W=m+{FbZr`d)k!tYHiA&J*G|)44A8LJp7=BXZCEW+ zCBmc6w<^(4jEQmCEdsL(}wp*az?aIh4WPJOH_nxbrba1Om+IA0O zM#8nD0*#B8>sj%_p{PtmB6EZ`PhVFoyLit#D+pYTi=&+(rz3tD^2M^->S@#2^wVRJ znl;mz@jd7Lf9NK7)8HgW_Oe6}003aL008>`8MCDO{nyVvA{`q$%O8BiH<|t?<HThf zp0&J3u6OERukWj**D{UD)VKag62s1kYC9#JHYJCEd1ieU0tQ~F_<<(yoM;XwpWs}* zs|s2T#7A)Kvs?pQo7)%}sS<`&Bl7%Z>sF8!#z!VT4#2jL;Y>h1n%OC&Huh4v34p1O z49a0VoiOn>RR+5ujWH=24(?kk9 zeGX_dyL?AY-Fr~5IFBtN1H}wV+N)!iVnJ5iW{^;;9Br);ZCLAR?1^9^sV*m{gFs{w z<37SbHdW|YOusA5Fu8o5x_@qd2MY9wAHOvG%mT#?ql1KisN=luWDPB?C@in?JAV`* z23KjgH&vnpQhcmh{7`>)y0AticD30wy*cjAW>3A2%NCj~~X! zBQ@VCF*@XyvAfp}NMlKMGeqp)&4IrXPl^2DqVlK*$PxNeJWGvPdbC0V(h=t9VJS>* zf`Vu)-4Z2qK8pz($GY~6UTQQCtZWrPl;gp@>Hz(ia~6D?c-=USz>ZZ0;SR8VutR5c zDbtWH&nkY*zQ?*yGAfyRb+#*1gG0u()~}_*RH~&Rz6l7qpJmWaCdt%IJ5-L7k|jMW z;!^wa)BJl2wn3=3PSIivyO8S`0k^)EH=S!cwLQ(h=e;}RZF@Mz^S(V?e|b99d0xC9 z|FmAK&f9}nqEUvo+}^_dtNKo8MW;i*l_k^ozup; zH~Zs$==lbyGluoa4>)c13l)SOU(ch0s`KaX1q7W&UN&l9TV~i$hXD9sLvtmWaB5Qm|80B$s@U#3UVhGHV7Ln!ufSy+Q#OIx>@c)U(^;Bv25yRK0S`pJ3}o3z&j1!C85RQ& z;K-b=m<;UHh!mR-W33)QNw5|w2xs|LX%t}$DU<4Zyk0C{*v5UJinG84;O7x5_QDu@ zvZI03;RU%?uA;<(5QLr6IW)7l?gIJ;)z~d$2-=j0_8_SXup#MMq!t?%L{ogW&RxZZ zDZyrt)J?|~?B)jV{EkSYSH8v8w}=SW09`O`GEU)zj-uxD6=vf6S?BF{BFN` zm~R$2rHSV?VpAkDtU{HRVx|!D;Vy)Vx^q+prknKydAXXj`KuVz2vp$piE>qrMIVHc z4bqGp#Lb`t%ICh$-3P2Hu;PH^;MbqI^5veB+{>zV7_6AGN*n<+keyt40B@^lxuZD) zbzFlOZaZajxy&EeGIs!UJXajU=%b-=$=QXX^DV3{0KM?So)iWyUZoX5!<0K^I>wY> zj}uzFL`V|tYL$Sj8yLVo64Erk1?%%mo5wbzdnM zio+kG(d{q*A_90o+&O1Z5OM6(dF|5>7rK>dGI!ndy{dWqNfmlAH8Q?v?Qpx~5J$j+ zxlE&=p8>`3RRT7HqRn=I^Gy!-sVUKDql->*1RxZoHoH^FuTak_%!^RC!F{Zue9tdX zj+gdj9b8k8vY-=W*j@rkwWL~0e|jTW@)(ynJ&17B+>4h%GAF`@h0G?4EzXdyU1gquR7|S2z9+|M z2w9i3c~_IWy{9l2E~kMhv0QvfYS0~-cm*u!y&5^Yo6sZBFxb#k(Rk5hD5$x}ztuav zhrI$eKx#h^4ueh#4N^Mk@7{T24_-s0{h;2DeT<+Rv;G(c4KQrkyQ}@AJ`?|Ed<%ZfgS&x@XAv-$;%3uPCD6y^iTPg?W`HY8SH0DjMXE(7OE3ly1 zRJ~3F);`{azT`z4d6KZUq8JPH-4$EBxRz~QB@t`aU#lIRKCFj|PJk_0Y`EPhP#nbF zrpjQ{!sckXeyl!^_Ewm{=x^p7Oybg>c03JmkDIz!3ZP61?ZIK@BTP;ar7QH!%f3{Y zgg(JafJL1-`$~=_)>$g2WdW>Maii!xceJE6XIzj@(-fm_|6O>Vs7t6vY}~EiD>*}U zch*SdjIEMTN<63cUECA=YUvbDE38}&0Wyn4`(}?>2SYeaqI(RZ~sPwL+*_A6TuWXQC|3Ji6mh$~OmRpH}Q0lpekFb~10Qz#y;&>JiPU8)r;9H0y z#JhrNGviS;Fn$=mg#NwbO09-)Cxb;mwZpSaN_$zQG&lK_$GJ+}mNTp)3Tg`+rT1b> zj{C~vdGpHYV<4H6*t%qN-ari#?BUIpayzTTG9}kFx#>0Q-Ur^Uj&80`n_HLPx3<0; zK9j`!17kWAOLv?WaDEf9WNN-&~g8clrE}Li(u={c}kFq~`NmNWWjD|Aq1M z2a))*@$*kq;%_niewqHmn0~)=KQGhIF}`lcj;<1FnM;O52 zts`Mt(dG9?;@Z&OF0L2knG}oDodLby3!PwEU2!8}6%nS&wkVyiga?ra#Vp7xv0D5cd{Q7?1?;|tjw~*`ZT?*6O zM)Yob_Zqx|gqug%;7rQf{wu8pq-q3dqG@)a%YCcOF5_z6y)fxa=NnRiysA2#S@BFU zAWOzW^6-U%T2_dTNK1+pF+n0pkHtRp*jw4r^e&FyYXp} zi8BurHynVt7=(=%th-iPCPij(z1?tZQHI9#gl8d&L{d_N8iHish|dHtXROM3<)r78+U=yBZ)AK?WDn4 zGF0r9xZk8;8BrS``xlw2og)=uE@ceFDwB>SeEXnkD)v1$D0>AxXtbbrWEPD`%^V1( z$gsSPHebejgdz3$i=n((`RXG*Tr=0g<#9;@Le!3?h8sgp8Z-tH@%LJvoDU;v+V2(6 zhXP5l&vNvk%mjiNn2GT~nQ{ng{TvX$OaV_CA%_Hy$qXvP^omz^M(e37hiJXRyz0?# zw-g_daJ%z2At&Ry)IcViBNI{FG?4?p6Z^1{uVG3yyb6bUBoM}cH{pPjDxG4ZJtofs zEc>oLCb&Y8wF6lhBdIiLv%g`R`4+>Df0n2dsmC$rn7^tATK)Y7D1A@&JZ^{~vV6cY zwvFrML3j+eE;6zwgiBJ`1X=aLWy+O{Br``xVTRG*qmYOhl?729H4s$kM|kvwLJ(}D zMi#Cw!Fae&L32z}<|^QnxJa=4b8}KP5OAchcqo?O(s3c24YbtfVl{%N8J$iAcuZHX zvO(o`y-INZ9q0&@8Bv8HqXz&75}X0rn25j`aR}whdjM+Um6%0yzBL5RN7}R%@OeTd zEiUTA6j5;Fjr_2iKr@v%1Db_aKUFcTM9yG^*>W7zTF);gP)504=S1QvbnD>Jr-FjR zjcUD$seYACID+n0coH0Aqv}h{qmZ3`{Wpjd^)$@I2X;~|Q8eVI{DaqN^CD6Uy*A6j z7sx$xDesu>wLZTjfO7Sb*p(EVZQv$)s$1xO{bCMhTL|!&+L*5TWr z!^GKh2I-Q$14-{=qSJeG@v`0JrB1d!KjLWWyW}{>UvAGWS^2_Q97+p7I3v8EVDaJWt89o0_$%2k7V;_^oXRz%UuA>Gt8B3UMK<)6irN_A z_yMqbF68t?;rEp3H5-EE0yxBN9Fr^dIko_B$CMdVPCt(TvM)?;8isgN?5$9 zhNqg6SJl8$GNFn|v0QvaYA_s`*zLR9e%SYs&ZG*N<&MQU80r%s*5(i_BJCPW@&*eZ zOKOXV=>VPHMkg3&qjg%FA06jjq>chrNEQI--K9MP2t@;mv*fi`h#u$2(ZHzmO9I=c z+X3gTfu{}J@o0lJKfEub=VD!I8y@Uz=ke`H*T-(3?b*9u_OR@Z1KM*y9_;XuLce~J z+qx<#d|a@vqf}!GHfe8WCS-q5jr1)n4jSNr{zmB#F{RL4Gpugq?ff7HZxswgm!w~T zZw=5k-Jv99vqU7?mFmtn}Lb@h4P^zN)kFYLe&%2Sv`!wY*x7qKz;kLR(Ro_wo`WL@CpJigd2|reHU@ z!p^88XA=o9c^ehtMI=U-o_8wv5&sxt>)f zb6&05N)6)=rzL(M=qjD#wQlYR43b-@d4XzSS?F zV6XUqv!K5=d>$XW!OT$D=$!Z*II23xRMnd%Kg#co$p?Led+1Nwd45+95ZxGnAny8! zbhO5xiJuOM&am<^vv~d_iUXv zFMo^T*S@j;A9VN+HOHT+=RavDc%z=bb*y8K1C)Jw^`UX-|6>;TQ*HLoTG&~BBFMjV zU;LAT=ql#$s}DVix(5$!H$c812n(YX2T)Dr>f~hn^?f@kWd|0P*Qnk$w7Q|>5&hsR z4UEI-=7YWS5%$67<00{mk!&Lvpl`q*BpD!3UL_U|LoNzV1T{wGBnrJQKXPLpzsGC9 z;w>;BUmC)o|MYw~YoN#*P};PfFoG~;S)T@Y{q131pfLjnVtxM}tDs1dIr39+3C9UQj;D!(5xbo6z(8Y@BL9D+P=cRW2 zYb{=um4_jO15wz&2iP}IJjcc`KspK-3sazVUER+#X(%I0NhCkcm!g%Tw`QTQ(G}tU z2y5h;VE$s_@2lUNp)g!0i_%TEq+02I)m2;1MAhYzWF3efbxft)TzXjCNZW}${k&Nq zeR{QFI0EF^`$QPOBzAH5gt@=~$%H?r+W2)zXU8@ibRBy_G8n96(6}bRFy?#?-v&oN zP9vz}lu)<>JBoSgyf#G+E%3oOk}2ZO97jF{&#Km@fV!`XV~vNqY-l9aI;0IAHCEXRR77LHFm zCl8%?&%Yjj^=iG{J6&c`Ds;LPG%0-8TOC_lW{ork%$qc|=zQ_M+h=)9N6M+Hn@xER z27tLLqF433LVd3KAuWmUGIYF9HeIFBS%hWprctmhxp7_nt;Ku(AOv8SgQllRslP0F zfxSp%oAJteYdCsu*^2XFTNBouKG@l)xfMW!7SsFb-4W{55P6`ie!f>7af@9bbP?W? z_!DpYYB{-=cvRy6kCv&Db09GOyPheMu&7i?Qq5==zV9Ki0Srx_J*OR!p^kCf#bnPF z3R%*xYsm3sa6d3f!3se~(0LI`Sp=qzsd1Bi!KK_E{y424dI}0G{}Bk3*_y&@N>8OKBVaZL#SR4lsso7nM_kZtgX5KcPAz41t+QmzQ|OJJ92eQ-;pvUOKu-We z`O1!?%@UH72>?CaO-0!gymGjw8N^LWOvS4ijR4(}29}n$Gho;QbiB(n^nSyv-y!47 z%JBVZDcMuTY}}8~{XRs`d#>zDUwn|N?J2vQ6e2Kgo(Vt3p4C-0_`!}`bgO9}$#_v( zn?>#{zfxL)tU%(KD=CLwdbECxya_{Ltxj%PBOMAWM28)h~@xE97vz zAWxvwhbPHaM$qn3Y#`!TjM5@27>hs1!~{*z-By}Ct?q077itFIDelAUZTCJo zo914%f^4PM#(t=V-~mV#$L_mfGY4#n*VXLBBbdE?A^|Y zIE3hg=2Dr^&OQ#@NrB4J;iJk+?8+1f!02%!pC2zuhg;RQKHq819S`g< zVEu@FzQc({pq;`N|GNKbIRcJhb*^!XQUv;5pG45QxX1!no z5yX#yXpiGK!=)G|^lFyQ78Eq}g~M)W5@QeoCzxqMBy`a1co3cF)S3F_zzN_rg@CyC zv1+sXQxBIsw~Drgl-@E0PKCGQ#?EY8!)mK^+=%jWjtyq>)Gncb5dg@!78w~scgYZ0 z4Ul>PMTJ%srPVF@5=RnH;=#HsHFl68!?RjA*T(=AWWl4s}}jNHH`-%6-!7;_HCqTu44|3o>vTpwzib z)#C<&zUc;wmgKhJ8F7 zK8<^)rrq`-!hwSH2lHwS*84|n_LO1Y0Fo&$p(t6-Xz~TcD#}!5Q7JE5a~J)~B23_% zupqHD-7u8{7YX~rF)H(>yVWxy-zE&m4%N6*zBQj8QP^5^own)N8&OmAmZD$7UK~pA zpevgCa?r3eNSgaH8{b)G4}#U3(B}EExPDql;6L6-ODrT@Rejoy7-gG#60hC~rPi92 zSx8SvQ~7971r2)k1rV|6$BI8Y3^8atT3CU>T6tZit`nB@ARnYv;E_IN%Ey{dJyg8_|1+>U4 z7QFoXV!IKeVe(oBQo*bdhs7=S{n_`sWp2t?avEto^=w(r^Y4ktzKCogyQ^U6EI{Xn z2q>0fC}Zyw!e;Q#l5$^={NP?i#Sf82&w_WBSLNp1!vf(366Cl#S27qpTkvNFRZSi~ zluvC^%fypRCKW02P%W_@P)uFspmHa%Y`LoDJX9VnK;*Yi0)~b-gWE&^GnuPr4*Kf3 z^pVNS&eO})&h~oqack%CamVFmHTzc&{5J#ftII%s`u~tR{>dBTOl&{7;(vc*{7+uQ zT*Z{O-tL<)JD_j7R8{TlR}g(y)y3(&&vMDoqEo%P(@h2ux#;bTTr{sEbD577O5 zgDmwEdexk^B!h}r#jI5zQ^)x_#HOtinV-+`d1?XfER$<7;Spg-j57}8C(%;IFzz+_ z0N_n}Jb>VgyR46=PZtRa5npCB?Gd!Yum+i1CX^_BYJnanAPw-~8GV5;JG<$x4ZsYm zKNTf>N5BQ}#eZ?Hd&J0<2n|{>@w9FOmypC=>$J~Z6y+$$+$n6G3w*lpShjMt5Hrwe zxPgCa%2ZU1o)guquDTvuQVp=xJyCH|7)JBah<5AosHZqn2HEWUU{oGh(8kZ>;z8hR zKpv>+D}Z`zavT9jv>)2Y$y(K3{F!eo6a{!20}Y%b@X?{o7ghE` zCOaItUb=`m)RX70be2>w;dEKM8bl$-O_GT;=FJ>ul63vG-j!C0@3^>dYvgn;vq$pi z5rOV6fGjYAG2yYOG__90!y9+~tYHrVCA()0Yt#f72ApZ}ZBX=nBaLWz>t^&hxRAF` zP*Ze!70Dr8ex_xCjOO&HbKh?6T$PQ_JOrTwE-+w@Ejn>)^P0dsnyg%_W^O4`%`}VI zRV|i2OmJz_1``vK7NtVK?>nw~K79RFGQPIT<_*Qe$@XCXxXb(UjJDs8$C}Bi)7Ta5 zf8pQ7^{tQK!g1JR?WpThS^3q&C|8e@3F=0-jPt%{!@T?aQ$b71vS)kEvhUH%q340s z_1dY`>50d7tpxtq{utQfOW}(}S34*|MkI{MJsuF-UOc0w?-kf9W7JU#`=KJ=uN@!N ziYdcAd}!GqA!>NS=KaIjKMGu~Y+E41p2LX&hn1&)Oq_iTZEt-h8E>@VROY$A3EK$~ zwv}Ra4Ir0wjQ2UPndB>^#Aduu(K^i_P2mJ>AtZ(ZZ%zEY_e-NXZTA=)Y1^(1JNCr3ZQHh!iEU0au`{ua z$;6)6ww+9DI}r84k#bwukGz8LEq4YC#%f*lCI6uI%9u-bPlD)L z*vS-f&(dY6RJgDs5^F1TBB`5#q;$xyI`%6xJ!$K}u7w3&xS|R^b&n7NR_dzEpC<>kS-95&~!)P&5SBmf-*ru%;NxlFgOJ%D$v&I6+Q!R1c_EIKdVZc zL^>W|%tAaKI) zVAN%HDAI6fZm=$6M{S}*yk>GRNk#8$Dhhw$wa_#-U*76#ep1p zFe9!$O#T35LN0~5vQrMeD<9yP6lQNX^1?sU~6&9IF;30PE|| zAFqlWu)|CkZ5@7^lGedI{>t;TfFRT*)EvgN!fWFxfZm{kD4;?_Sfoj>j|2&{)(nde zv=N?&ZqI{ioZr#EH^9J}i@K10KHLIOK#gRlst_95OJ7({W#qt=3nxU)tl&9Uey8Ai zl^ELtUUy}cwgO(DJe{=xO{i*-6<&n_>XabF;TR2RZ(MVg0fqy*aXrB^O4eFUGV*8f ztJ3xde)FZwl!X=ZzEuqgQEZjv88V$?fjU`m^at}L_PaG2?Dtz1xmLkV_O?kokUm=e z=uh;0`{+-Qa1WVCjLR2bLXXvFiDR^H3^jxnuoq9+p^iSwGG)=}u#g$C<6&f}O5>?f z*x1=5w|&RKz7yfe+7u!e>6FR|`g z>Rq+Q@dI7|ifzYY3gx!c2;*H1=}o-^ikWh@>t~7TGjIU20yA4tJZ$3^Ii_L9(Y>9k&SvzD zD(JE_k&36f4cFa?RAd`|;tr&a9iG<;c`7OeWTA8aXi)3~J!kBM#VLsiH}@ z^q!BmA+Ro5NB1@+^w#~`3;AlJz~bOXgW?iYZI>>L)2V`CQsHU0EZ6G{>_}Ah(z=0& zL;eB#gN=y*HVQ{iYAawZIyHZ=Q5e9+hrif35+b=_AgvByBkaGk(N^e#jk>zb-Adn1 zccN$5Y$a;`uo0%RnLH;2Q@fyrJgW)eZ)_|GfNz$v9r@vMIX0paraV#%f2E-Rk8Z&K2PD$}#}xFxDJuMvvh>s*iJAWwDf^$C>Sy^+QqT{^{wISx*05y& zlwCqc=cDzYLHxk1;51mQ>2SuHtM@fcC{Pjm?%=;`_RCQFBk(Jkm_NuX5#P<6F~K2O zk?3mtl9TbVV00=T(W|g z8@$km-umK7>Xm&`nFja=VD%{#J|+x}ZM6OHEMK_JI(v}!7Bd3Cv+h5w&V8>_72!%g zJgXD#7H{pEQ=$%J3I0%sC5lg22=d4p^v(EK3~nf2K!xfElotrZ=kET?ql$O|@~7rO zk8>}i#AM#4-~;hm1ZUZm-wMy;u+LjZh4ao!vEtwD&PkqBX!5G?ZKIn^j8C$ns-cKC z7Ap_)3n_pav2Rn|ZPZrF5PPC3Ms#51_yWBis>uCO{&*Jqu6_u-+5KvM>P?z3ho%bO9CQ{0pnsFiuoNi^?1khA@iWDa5`7;>bIetv6mF5~K zw`_Krw;S{y-u9ibCBOt!hK_>j270jLQmcQLKF z6Uqgn=|97tkv}TAcFS}82=W88Qz@$q%q3A^+3S`5ba{fHMp40|W!?fY+O!Y*etMj& znsV$JWTb&!jS&82hzYCAV0P$KAo~vGaXv-;K`oT43)cCDWBMndJuePTv{;3jf7f_cib9y4ykhm%Rjmm)mUL zx52wtbAji3g_nW*4SNnfuVW~NtH67h>%IsG|4N?yYkA1GQ)lcAJ98q}m0F;Sdu~sA z?zMA_tj4SbK%UBDzoMA6ymp_Jb)M0l)xOYa#R^LpC2kD+_P!;sa%nFgH1JE^7$92K zv7!M7@CogQ!EFvYu@{<3<3{;GJMbdAv7-f3?hY_L1{~jK`zQcue*#D=-Obw03Q0!Al)|~Hy zU2EY38CPz$_XlQSAO+z=|6n%d1G9br%tHSSv*^Eg?qa`xV3z&FTtyI&9Q_TmFQTc7 ze=w`~2eb1Zn1v0=o%;*3O@Co_SAwYaZ55Gv-|+eezw1|S}~I>2q(zYB+8<6#UnyWuijYZ1fT5+S-u(@@FLot}?RUD3eiSPjFua_dtc9 zj0Ow#gqbAZJiLB@n}@cxIugd`NHDDy0;iD2=xx?`@`8WX7v_bPDn3X%3^%5aZj{Uc z-A-hwJmF+!8*qL(5jO}?l>%Ef60cHpENE82_knuC(VAIeSR28-OLc&XXE#faY+|YT ztrQzF#dy=?^1RBA>iAuW+q%j;0ikfhdTcP_`C>}^fCsvd#ge4L1jP%8Ie|0s%<&h< z7mDttZ~@^T%|X`8OTj}tK>Z(>HSk=@jTq#@n*9s2?Scy*nC0yRV3sDBCs3jTzSzBQMX0`f^k$`JGuyjEip#hk+`a5QEJ}|5Hf!THdW^DyOFgx;rS(iVU z)jn6ekl=b99NXewcbx-ZmV@$i#|9Lowk1;J1GC@$VD?obOI-zkS!aC>o<7?8CX#*- zb09-EFVL5FMb7W&S5Lm5i{m~J?_22dfIcXsmnJ1HG zs%z0kq{1M*5nNlvonwci)6bNdj8d(6qUb$3XIigqhun1spTL#Vm><6al&aUc2r|e~$ z8Wf2vpNpqy(i)wbvWAwV2do(jO_5)d&f2_EC(LEIFf%y4W~NGFw^s5QNJhxlCe|&b zYRjRSw%tY&Z>aqW!e_fa!sC2hvMU1&c;hipUg3mhw8?+efkAU>pWsT07Hyo>HVhLP z2!5jUHG7JQ3M9}Y$eyUw;PF9k%BERS(qhh$+`wXv-wgdI;d6xGAYpaUt-Z+88++kO z1~JbelPa8(agAaTRwF$*L8X=;SFy%A&MuD5kWr!&HnpU*DGSetWaCh0hlhJr<4}M7 zgIPPLY{#ux^FMW9FQ=wP)<7apo;(b~aD0UwyH=2h@S#zzt}JT?y?4y(FUPQ#YIB z081B+QX5G0`h9tSd4(+K>`X;ORUqr`$gc*nPnm$O)M6+LWwgoI05Iz?4#2Ek`J3aw zAIyHf>P!A<^?}*(56p)Ag;}rj?mw7?6`N)HJ7!J(j@kMp1tZ;2Yq#n2Fx%x_oqu8$ z4+U?HohxIcEIenm#^7^mfKV+)JRe4H`J6x&x?&FfUzi;-4EsA~!}drUuDdg@4Jn>K zFq`rRv&!Dc0L(f9FdNBWWA}fQf&Rhl|11M_|HwdnTznj7I=y_?1-JwRxW0RH9Qyv( z#m)bA82$g}WHY^mDPSDEk*T5c|8TvDAqFSm(G@m$7uq=xd#XX*xo|ncoo5>Q-K6+d!M%-9~q<)%9{Gv?s zi*7-&d~~1%4Z68p2K{xzX=03Zt?=~f{Z-)k=+&!ma*S*8xovXqng8ON*FM>&o3^0< zf=ztEUm$&R7R{?=YLXQW?+j1(6c0D^RQD_^3s2`v_b4T@1`Gm;K?u_hj;g0O-zx-9 z1TJD|!v}LfB5oK8_}UNzUfDZB#sFU`Kq@bd@qu126$FdGw5Jv@Tp;K-5)t99F|Xh@ z^t2Zd?)Rw69L_|XboOslD^eeC3Go+ zkSWT-bsz;y_AtYs5pT`a`u51F36VMMUY}tmKAU&f8Wz69;TXfbk|_m4vdMG@`G-T3 zsdH{31&JvgB2rBeeJg+gRYv2*r$CGY5vFri9w84P+uXC!q!e*Li<>uI!=UX#moTg_ zLY6ov4Q1$~9JhQd^3#BMlp<0K7I#D^7KIAh;N(E=4T7SzQ`M&yAxzAS-k$T41lmy~ z4G1Ob@UkQ3P^w5v;=dW_1B(oIgaY@F6k~)Yhl=hq0m}!&fbTZquSfjZ!XO2^36(?L zmV($T+C>dUY`d|0Y+KeNUBVf0!&2Zt`k4FzqX$~x4

B#=r_ZO+lnoU7T{Pyz3PT z36fI!%cQ9rT9fqA9JadOFD%tB@$?5LFmYjfxu`qtC96VsSJyC!2{OMdSF2Lu5V^X_ zCdGF_2l28e(Oe@tf#Hh% z;Pk)J`bHpiyrISTN*Y8NB>{mOY9g0LCGU3y#>%fH3EGVuhFGw+9#V41m950RY>+L- zk31 z9>pC><{<^8nDom%ms3H%>>&RFo(5e?Wslhx4)zF&H0DMsvY4M0l3a$8P*?cOD+QvE z%+m$rwQ5YetVGUi%Vtj^6sN9msH#m((pl@opRD5TSozj7@CwHfews;$UOTMLNzn;Q zSo4(2*)&|XNKbE46_%mY5XTcEf&{i9qhb;(ItmiD_VtZ?!A_APO-(H4wIG=x!!bQ5 zbT71$0`Ve^53@+2Ed4}98!et1iy;dFHP@$lD}+X-3JtQw3rrkv_JdOq#!nXc!GoQc z?9c{75TTr|pHx}ZU)IH)p10_h_kLqrvbFFzfBO6~6L2HCs$XFIcrJH6HPVWPt(FHZ zH__+|BS;@6zLa#XO@om}Lx0-KqZ?)d>zNb9kd%t5r;Jte4j6fGp8ryDasuXd-d9=A z)xdsUS+EIJ+@>EpG{dT9-p4g~hkz-HGBEg7+@DH$xh(4ZE1`#h{O~8uS0KO8L^W{K;@3`B*pxul9`mI%aUWHdh>9{96LtJsa208#uH*?dWO2X$AeFI6OJe2tmgzlMLDaYX zRxv}t8^iXPO$RX)6-DEmWWFsN3!yyiH?gE01VYZLW~c&|5ARjF7bXJ#<@Pf*yU4JU zfc*4soVJ!Ox6nqugH}6)U5k?!DdtihMB*+ru`1LtD#3ckT(E7bB4Nl5WtbSZ1Q;3;$dbyTgMlz`)YUd!j*Ub@$xIz% z@o_L|hC)xd+WX=PFg+syu8qmlC)VWOh{1!o`8Q(nBwx`q)eTyyQZpW=^!ini;UxAX zoOm1+N@9#)N@PlTL^TVj!uv;#T@)*eLevGHO>#ah+osYwKrQ92phL$ex)8-$V}9y&1FP%My(l zXV>-RteXusoS%qwVS8P0R1TeBZN%LrRx7S>!*coWEB!a@qX|1lY?irn4@waHYM-uv zJ7P$vtU9$yMP}JUmDg^gqgn2jnjEyMqdS&b%vlzAQkS2rG$}wLdEKiF!;3Ug#m1BT;HmY1pjR1pe-2##?e2gn{WDgI zP3i9Ydas!rEDY2pXFw(J=FYT^u3=tFTj(R~#sju<#?b{q+fuGi?EK;^xRD6#tdQD@ zZY`}eOr3z1c|XemT}#}Pqji0~rB;-s5;af#4?$z!sf$~Dx`h)3N85()PCFi2*rmNG ze8b<)E?~)z;ijjmc;{?#;maJi$ESeC#s)dNs4aulCT;N67DJz(6c=O1mzR8gE}(1N z=r`LAhj@3dIDHpr-=BzyXz8%~TxkEgS=IPDCh@sXJ>{&^$zp9>!~GK%j=Ytfn~Te! z9o`M+OX1Vh0&tqZyUQZUg?E2+VCz_lH44G84=g;qexk~lZnB|ev+t99J8&aT3nW@4 z!69Q>YhDN+!-Frq=N;3Tk~J>_(ZLb){?G6NpH9npPM?C_ifIFeO!c0YeEs&B%VR_$MtAGj;jwZkF_GWBzE$F-h8~ClV4}$dIQH( z&Qavff`=??%2$=UB)p&2o8#AukpP`Wo|+7Nww?}$qs(-AuNF#UXXDl+>Tdf)(`HZ# z`3u?&y{Xh&tNp}LB0seD{nkS=nQuBaUiOQ}{+3Z8i=KBR&s@pKXw`vEa$!vSR@9A` z^{nvDxI|W~{X6?9h3!uG15X-R8C%igDcWu}Kh4N)1(sGk@1)p;oXy5XH23mNl{b%d zMi{=T&0IIZc74yc-mu5TLB%!3eT5oZ9kGDpj zOL+ps_SK>RQuDJlNFkCy4jY8UVgbJu3Y;%mS5>8h`<6%)2`%HNXS(jQ_8t0-)%?Mu ztg~?`;PEF+R2u}iIquzYj*oIN+x!RR`?O~#5NMWa+NO@y(ips`r%lAnW1C#obdsOy zTWpH>-P>C_#QCLb8O^W1NGcV0(z&y6?RpCnm)d&B1!k6XVI> zvsixtlXbY=C?M44gN_A@f4FwY!QmUs^>#vXwj*=z2-6Phc9W56^Cj?Eg!Nih@%2R1 zwdIunyC~bk9ghZ%hkxJ(vkjHVV|DlV$Il~_N&*Ex42iGjjtgfIF`Biae%TSMXy9np zpACp07sOrNpAP$wi!Xj}s5j}`LA@Wf?)}W;@II~m@}%Upvrv0CW?sZ!bKc%L8pqN8 zcz^RNjE_2DzzTL>V(yJzj+{NEdF_@U^DWTK)#&2Q4X5qgWU|V=u6AzP?(L@1iRiSb z{3M>kZme#ly?hKI80`*$*~tpCbH$QFzf!&8zSzVV_0^NkzR zEKSGY_HZWOS;Ul#;lm)QC%u^7FJ9gZ$5FL2bW~xIG)+CY3uUxP8}H4(xkU$Lah-n9 zI+nU0RwWcw^1WWqrw#Ja?MJPq9pQb0Qjlj%f4x=I&TH?@ytu!ZU>B7I{M5E5hQ#By zolZHedak@2pgBx9D9y^CV;!jca*Lw-0xR(5&obiJ;du%2T8}R>`WfR!iTFACQ%@ zpuN{iL8p7`*%k4gVhfqHJ>Jv@X&Aw#i4xhQcb|n;CcwEeKNe2WTmokX_}5v;eq{yX9qM0hO$z-q7Ig`!XIML4l8*T*6MLinT&`vh*|s+#AFh){_kWa<+C|^YkXD z(X;Gs9ZO!@6OQhbhCS_ziOYTt8^YR~N4gp!8_g`;y6gBFftQ)4N0fk%Rff6c5XAMK zS~r@%dyc1gw*J{;&4P3s^r`Yz+>zHxg!Wh4`)9k+ufy7qY1Mk#qByI}{5*9AmSEQv zCMGx0f-B`!*G~*)cCeJcI<^~$t`Jt#)>yW>@99HclZ{rc`xDRO{52NMHjXD+tK?P~ zW-O*dOTS*%+bb(L|CSBCjCf1sif~-J?dq__u4e$-ZxAPO?HKDko$@^?C|#9^z&sMu zj*(kWrHhZM$a;}?_R)?ldRq<#{RGPwU*00VK|RG$?5o;lF8(TZQ8n5=#?Gtz-CFrs zBKmMg%rNXEfki6}fk5D{IAdk4#^c@p_qp45d#a}7s)MA{_2L+U*>zF=IX(6mjMwXk zsZwNs`h^QTBi+@zS9e#5>RpInpy4{WSfqeY_}y=PB9*b{DZDO_XWs>N#S-bwv^!$Y;0`YX6=)| zm6d(lecv5}CZ6RP*bL*w(rQ0peb%sCwpg}~oWU(RDncq|TC0BaCC|fDS#LF);i@5U zM`2>>!$FZUpn}sAv#@fGo|xw1mN+=X&B}B7my4bES;b~m%-3VzKTA0Oc|wL}hD~S~ zkcCO{{a?t!nEpKr`*VipuPlrSu;7`Cj+vA5zdIpwrUk8vqHzz23RVEqL^Fh`Q$-LW z2PPzl9!HpQTEk5WiY3R)#A1kKfu>WfriF%B()6PO$0&|kt27`02I`#hTnq`#p?mH4 z-i#dDa6$cj&BI0{(C5eoJKlU&6&1~X_lTUAnhvqQpp~=HddNx@w48Qc_CPGVuF_IJP5b_ERbuaq$!RI~-bK(X&{IE#e%moSEB#upF z*#-S}jmSs@BK8$G(+~@l0*xqAW*Bi`;K3^qibRng5QE|LA%?o|a&r<1gU3Z89(}}M zV6^*4a5C|v-9K1-IGOeRywOAIq7AQO7bkn!L>&ZRN1+nG1u0FwL36l+ zn|8&GaTq9`XTwT#%MO#sgmAnP2}J{9u#q5t7$_`tuPzd(h#ynkDY+sbGF%Y()L&`k z3B9O-mXAmiq7Y!aM=4VvWeS_fKCH2ugzySV#Eqv>N&vcD;%&t7>3YA6F@usuKw zCK2=v5vs`4jx*#pqUKa?hh(H6B^O?Q$s;1hK#l_#k|6HzYEWzt{W$G)4k$UHfvTSf z<&;R@Hz`IT2TpQwLaZhSD0GM7V24L#2CEZ3k&;Sd)Tii&XChP!ZN@L8!*Or|UkAV25G4aKA|b{` z;-QX`or1lf5Mw9~XCPu=2 z4pX%0heJa2!&nrd98?4mr3`>faIgl>q)o2{!zB$_1;6S5`#KM+3{xjduetDG1>m1!sbJm7Rf*8Y1*_t>R_N|-1umdXoR7O&g>0C=xLKY7F zL{A)E?4B$rpUaJ|fX|)YtIv#RxMN5?XQF^!EZK0H>{ zz+NoZCiPxm=`|V(XRo&{CN`1$C7@@ZUe172=24PR9v0|3tiKJVJaauO8;A?c4`8qX zGhLoO$lcF&l-fbwefWfy6jxN+@36sS^TO8NVdz*b{^>ESfiI}H#nAeCgE3H-KQT3* zb{b&Cr0vt+`^~CJRQBZyOlIrRs3_om3$BZpwI&`4lDEv6xKf4$4aEEu=aKV03|jN^ z4}if)DVtp>5UQAo`Zeg8(OY-cJ-;7nffKFGQ=)r#Xz|S@eB4zpg$DZ+%-Ka)U6dG1 z*dSIT03;BB8Xn3dB~?&ee#a;W5hX4O(pI#^B=2;Udoqzh$}i@@i2n@PvYsdaJPrz_ z=9?6de|l;ue{pUQwGd(ExY$D#FzP31;8-aTeF-AsfC*RNo33SISxVu8p;PL~U}NqH zr+2z&R*QB8tyB0xFZR|zx;IPpB6OFoK2`0KsI?MAGL%_J62VBrgnZJG-2B!WmWkbk z&fe7gqwhRI@>j_hbS}eq6pTo?&|~8DnP7;J{Rb|CYZ_ex51X?6{;>Q$WYw$X3c7bB z+@IpN;{zlxvLM{Q^IMRnR6*{a1yl%N02iZ71A;IEDE2&)Pn$@ZT8;6Z=_U{`5?!A_ z>MMXqc5lP{+&O2^b@{K_i(+i}AEYQ;mo-R>JVQB){iMCQ52NKV{P@JOLNMhZCP-!& zx-3+q;aCgYaM?8EEmC`Pp1c_qY90n2KgS_dfU|lWpF=C1ESU zAmqCY^;96rknr&JFB3o|AcbY=YUIAEqM_7 zgIJi51O{(Fb|_CwuZ=J9lLfPTx+_FdXRL{%WU9*{0SRaq_`Y9Ylx(N}fDt?Kt!7YY z$B-p(CF-cMO(TR84FUFWnsn$^@vg-=`T6iooIKHdEO&56S)$IOXgFhaEO$8)riA|s zBhIN%CUmtQga3x<8c2K)JSHz(1_ipSE>D9zA{0F%Y3ntQTA0aotmdC6tk|@yuKDvB zuFN{BxS`UizzB|((-Eu7!yJtzH_yeP{L@AHrxcjwkuL@zP)+(!dU2bHjkX*ia8f^f z&zO6OnZrr~J z+>XqLU~Pd86yMM&%l~36=S&f~JW=U<+~a>0U- zpKa8tih&lr#BUqbT?~Tmrs(e^RNa7Yya;9IcO8u;n-F@ercH8i&RLn0e1Wcqr)I8r zpY8qnR%-8|*492Hyky>6@wCLT-jwE$0)Dz&e>>2p?LU5zDst1XA(|XzNv8CjE+q;#*$@lT;G=_xY(}MS#MCI^-eQUph4L4oWa_5nY%AV{v9{Fqg zeY?+t(2?kC$!JNtMQs%>Xd5rDmu#1W!ln7jfFH~YI%v%St*Uc+PSMQh%S%PVFh(hE z1m80_p62)5j;AB=GwGam+BZ5>8VG3o%uuL5NxSwc5$gMxKI=J7 zK+4_zw8=-bCHsCe&TjS`7{Gunpy0ULY_=Mo5q_xz>4HdkfE_a-pX> zpm~-Z&|VO{JXrWdIP#LYq;|~r|w!cyn2l%_vk!?8Do|^!}=tgH!Z== zUMsmgyD8VIw(P0Icx4Xa-SCX}Mxi1sR9%di??$FG0=k$U zX30_E5;|A3TeDCVjTp2$m_<%pkH-xzGAA2g6tti2P+N;gJ?vWFJudhCF5~;dbFkqT z>{{r*xMK6Y{>Tv4+O1WKsRZ**Y3f(3o{@3tn;{}XId^~McMXys`P84JmBe*yotZH| zzK*NRHZ(4ouAY`hrsMfsidgHOGUK)9HW5%AVJ#Dm!IQt6k=59Q=5bP8y{?gQU>$L+ zM`@4IuRHTTT=!&kHWG+V_Pdkd39-Nyt5ASq&fSF8N+&n7hJS_fnB88s!gE_F#XhNY zJE8+Snu_n!>J2rwi1xTU6F&0Aeco@YsH?o!L>$%YfGiEPu}7JsiJLCy1)Bix(FU}8 zYfLv@d&(L*pL)>Ijl;rYD*txxrS7hsUnem=rjplvCw8T`=x;2r@p|aDe91#h3r|mC zhZ92P7lj*aT#0fwH3(}cwFh4>tT9F*R=nH2Cc6e|V-j!$B0bP|Gp; zPFKR<6X&?oTQu>i=^NX9B9KT#c~Ymsr8#w~_HFM~*v|W9{F*{zh7q0VeGu4t_A2y* za1h19q={v-hlSk7?fmL8t2R= zFVpk*<1q?Y_B@^XQGxNaI6gT1o33E%z9##e}A6D8khP#ICWOYY-8UhWSG_~cZa1{wbSh4ozx!Rs%hxS zhT|yQF8WTa^&)U&E;60dxB2Oh%}>9G)*UD4)>g)F-(>r%`(SnAyJo1xe)khDV44_y zQa9BWO^>_KBZr$ULu4yW6;PO1*`N73@7^+~q^Ek8x|cM0B8P2`S~1<$U@Yozj)M-{ zaoWla#-GB>Wjj>dZJo?aZEG_>@?1n9X`|V!_`7>K&RFbuF1o;86}R5~Zro(&Vw7Y2 zg6^=L(4uaooOZl&jGgn5pgI&b^IbZ17@^@(V7=p|;F{Fv0xjh+TyF_Xhl#!!~; zFOSEHn@Bbqw>2~-pNG9MKdNQmi;VI+J_iIw%~LZKq^pDJ?9ohGA35%htK;_Hkn_r` z=X(~BjE;EwxIp(y1UgT)`VWoUFzaHoE)Pvu-d_?5JG67)i)LdF5RN@UX^G3$S4XbFZJwNVE%fv;|cblE2Y|Tv{cbzfs z@Xj6ubrB^ugkP=M^fv#6ryc`qNk&$eTE*q~7LcskKgVs6=ES4K z^((u2jtphQI-f5B^C2`2*dnAuTuXmnor-c^{JNESvDWWc{xkxHwh!l4rW#Q}O|LSz zjqTrGmHoUYPP1wLS_b*@n!jPv%%*= zt;iA~6E>ET9}Qm>wxcics$s|A6#CH?b+ESczEKRGeVN~Du5|enlVRK9G$Ms?vgf)m zc*u+YJi=`aRcEwcHE!)|2Cm;IeYW7a*)ttXQI*BRDIn0&pl9i})X&7fBJvti<0gvT zs^gXQZcrk%9BXwQuZe zE5(h-a)d5h1IYSfPkFh9;|0hAjxBoIvzl9}yxzI{N~(}IL}KCN_l*aSx( zbP77_`&CbcA=-219tSyk0L0o0zda5y!lGWu&%3b=9*!}Wp|k$q7M*p2Qf$|iY#miP za#m!GDGX*uquJ#+ol(BTL^7A1LHkBr&f{HUlv%(Ll6@4y1$~FUO zKJ(bX;$FryI|;l4YlsA=H?E|5h{cy@eiQk9n#DtW@?S0n-iwIbG7N0*_5V4B=HpO^ zHa2xM0T2+7BH*d_KN^Sr-xSV1Lbi|I*}p!R=>J|g1Ju#}1Z@AsD0G!OD3{-=W_~~v zgl3{WgoGA;5O6ujnSx`H`G|15|F$E$=0s^v6Y!A@08gp^jgS4;-V=QXKo!u^+0x!l z|L;A)f1Gso22|qyZLb>N7!!;iBOFjydnMF^e?DfnFhy1-+JOPAnIziqI}%`o#n09~ zN0)YNO&6+dB96#1KMkhSlT%B2LTS4)5b#y(bA zOhI!&eRFWX<~8)by88aFWc4wbI|L;gX65iIiW<{UpcLpOGaI;M#m+28ols&sC}>zi z-z2f_m~TRu%-j&cL~#T`S*XczlwsSL58~ncP@{;dkzmsw z$)^iAj$dA;{XpTG0ET)O?p|J)4a2@n9X5hTayT?j{}4p^B_R1k$eAWm$`OHlb`9X^d ze>(l)fE2)_XD8^+d08hI0w@6H?I{z<^0X?U`k<7RT%t%3gSD!8KTtx95)*}k1eh{& zUh(BD24MxneFUGy4t2<(ilLZ=o4=r%LM$fwgCxP~VduP|FC#|@HE9c{3;HsV?Y{(k z@wBTDe9Z%7V64>3C5V7IdNDKuF#}x|Ae+4x$ zBJKhmxxlFV8%8X*Gu z;CJej)PDW%32PUAP>RsB=YIKcTN0@3rS2ekI^jh@oFqVB>A`Dq!1|rHF`X2PfUpG( z4CgyUz7kG;5dmjz1NHRF#UslJd&hyHDGM%qijM-kX2q)p@yLHXjl$m%jRy!mIW=31 zSAxk+*~j0FY8RELoHCSQQkTQyQ9}9%FLG*hwVVl2RII*ep@~TC^s46B77EDr*bRsa z1`{atLlBI|i2j`E7 zxtsK(E7mU@6l94knk{iQNQb)LRCTW#GmdJ>f`?o6pTHvM~DiDS!&Z7oR}o-uL59i0kdlG*xi*5XYOAGz}i0r zK!-laTAs!23v(Bb>5`^QyN?1OUM_au?iOh%6G(WXaGG#E&C0gy>kCgLRl1uKMTwN& zaeIMlGYTnvposs{VKNE?v=Z^D?rJ*|Fg>t2HL%Xq@1$L;+I6=jcxUXyezH$7uH2uF zdEG6FL+c>-ZNHt-B7rDU7;A-5vSZtb?|l>iOYI8oSo{B80EDBMzlHYi2b;mw<-J{3 zfy_YQFAcN3Ln}FAF%d@1Q^{6u$&1EZRJpMXWii?0&3yrh6h9aeR52TzcA5_g2^MbI zfQ(3r@C_{l6t;Sit#;R?E34BK&%ms+Axg<@<|ed)X7mfV=bQUhFml%Na>N=-O(}K= zT+1Oz{jNy}MoAHes$?wb%GlJIO!~X83Eu@)Xoo^PlTDej?2yNt805l%6>;g^o8xT}RMPlr@ao!W|O&>8Y%-jwCrXqmDy<3M8qCzv30$NH&wXQT5KAo3~ zJ(R0$Mm9d4W3u6B9s2vyw~UFhG&$r4v3?+4s_ruB>NFtt^qIH;2Lwh*19HzVz`0kNp^?o)mSB}IqtrGW;7%w6 z>1Aos;V8vd7CsWI)>9H}!O9|gsQW*a-3?Kwd-s%|s>BUIz^{5isvx1^`@lQEeC!X= z6-i~g5O=<&>MZ-5EwTy`igm>XG+d}A#RLAzBB}gbls@mMTe{XRvnRJ%ME2AS{i6cd zp)n)-_X=PR@}CMImP03MTW(j0-HIC~GhzCpJ;o~!yelt{r$!3W=Ah|i^Se4%mz#EY z?z%-Yy2!ISB5oU?R(3}R+u?E#U%AmWCxJg-*5B#9PI22X7*ei(Q`Y;dVp2Acz=Zcu z0}?0Sv=S}ik$A&OL`m~#tWjLo3ldkqiJ6@pX?{I)*KFVMar6t!)ZBP0Y}1M{3X{V7 zY9EMZgSiuJW-F(C_274q*6qcB%)Z5dMBIE%GIlbSUz}X1pS}Y&a*H;3tJg40a*Vz| z%VnDvnV1)u!H?->x7%guZ~T}Wb2(@jIKZS|c$Gb18NV8q*S5**SbF$o>rWoBLSlX3 zz32Prfak?$KDPNeppvD_T-vFk|5%G5vNy}a6~Wo7IDmpKnL`@m(t+MQ4DoyXYJ3XCo~PZOx(M+@>Ho%jR{UbnDvNa_og!lM)1qpJgNq zYtXWD*keX%xR&fnqdy&uk~Z9JrZ{!`dsYo~v&UD{zh_L{Uv}7epn>m&@V)y{oA5A@ zxi?eta`Z3r-@vsgJ4-0l2)GH#mTp{HRN@yUS{w`K@Ai z2d8_r(=w%izPS79Zv6a`1yR-#XJ1n})UL7gth&dQRFFS5aew~nbs2ivLba^%F3{0`1QJY+sDIV<{j`PB})m)+1aQQBGENtqt*79nsY-7XUiHcwXC z3B2wv_Q$=k*x+B0^9k%ECn{DCFe@apm{GVt*P@O$kq$X(r6_~Secu_WL*zNj+|t)p zcgA|~cIn&z*ETtI?#5BKpT1hW4O>;=xXALVeiYto&Zy3CXYxqS8=UaaMNfA*Od*-> z&C8O1?7%`teviogL0h=o`+Ghde_kZ_vMa;#Gs^e%^&cr@qtj0&U(SVHLw@xtZ_M)@ z&}O*v&_tW}Hcy#;miuko_`|6@tJ&xI=UTJ&SD$;X_Kev=IIWkb%^IpIAFX?z>xvPV zmLqjbieI~#@3T7wcMrRPw{HQx_@CG8HEzF`p^V{cf1x8gkbSwRA-f$wT0ntx#tF|J zbNt+VFMG~{N035cOyA=1?N{=9%h0e%oL-wsHbdzX{R#Rub<6ztw#od+PnqDxwrzX3 z*r(4s+(!}}IIedt8-r%#chel5SdVL&Vvt@=tSS`Xkm?%c(M+1@b^V>C`ln#oOfB~; zZLOrbnSp@)V@%TZpBo3z^fPtP0sC?$u+Zvxw z#~!g?eaAJE-iV8N?@Npqm_gXExU7bSlJ|~(7F8@<#8mDcPf(|Ik9^_c+!bY55tM6h ze4{dEMJ~`z_ARr;nlX>_5$)!P&wS%1Lw|LHSVre*rTt!KF}S*BA}X&Y+}EyC%N7$| zl)(4>PTI$SMN$cDG*`8}VfesqJ_ z^jbr0cUHe%bMMFISWO=*DuU8_RXN|yqyH;0irZA{^Rar?eq(CNMiU*1!@#*WUY@>VwbKA~yK%8`h7V9)W|Z z3IkiHl1*7Gt6%5K%VHKum9lS}Z1^gc!kz;HzVozfs~Xw36zfIEE>sd#iW-aXXI-UU zO^yYjOK(yz!M)xHcLT-WytMl6?%lexxi3i5X=624=blolvUx(f zw6@l61?kU>NDopJEhf(e{|;M2Lb7i&78~?^2<9wfHeZe}+}!?7TbuIZvB2|j#>>w) z)v3!=*_`J~eHBgPOgXMurkc+H_onmV)frvcU@#_i2ZMiRAbUP?NzN-J$}9{y=}fsgAo&F1!?ees0q7Y+>^hO8z~k^dgrdzvjR zAc!w=lLs1%r`ftEZS$1M^%>I2%nf#Npeoyj)boT{yd!-`{+)+qvLky9ZPgI4M!te% z@h~DBpMa@?O_a5fZK{xp#{*z+hcaMkEt+%GX zKc3ea#3Ov^2Nl2Ck3-qyFVk_Zlad_Mz8&feq(bu)7;IlYDH>_1cX6#!cqzJq*<4w7 z5+Gha(zI$@ya}&Ar+IR~sS$R!cO2O__=S9JK6je>g$~|3%ldQ6$+SYwR~56QAabW-(Kg!jAB|V~42ycplV9 ztEYAS_O!RU&UGeA(xszKdb7lYdy}$5DP)z>ew&mL?+sqf2 zNm;87_B++_hU9FC2iGUjg!foo zzxalBNb$UgLniA~Xhp2$QMiLbN$uZLrU$R`-vq$FZNRtepq0qpWV-i1$GiXHVgJ_& z%3`gWN|Du8T&gYhZ#dTLvFaY5}z*Gtt!;l zh#WI3bKaHt@$p%Q@{+{FWwp7Q-;%~=VDTw*UsG76{$#v`$cliB_hJ%|@glPY_&m}B zyHj-ykkUt!^yKGIfAYW}p!SW3uHB-i97NIg5Q2hw{Dc@&O_)X((5DZD4ZJ`4~>TEFB`B`Rt@(xP>kUA zG}}f|r5c$1`cXt`^tEoAK84i#Ti@?oJ|W7hAI6j^^zsZdKIpVAVETk;oI)@T!}=xt zK1$%((WoI}z&RM4EW;leuhA#!1j6RH;WNW^XzDE@gdU9nfWIh*q8swZU-bQx@g{!w zi$WntP+hiG^g+-9tFfBo5XXKk$*F9RfJJN0@vOv9@7=dIQd%N}#n_1v%rXLmEyFSoHL^YVes1JO? zKTyh4kcRU%1iIzH!cZxy93Nuy6*XZCbpv7`c4L7diPRO1VK9Gi14l+<%<@+Y5eb@CU8I&YVE*H|mEckT^)Qw%!)M zyyLkKgR$wuU|da{5)bLg*6&1$JJkdjj8?h;gAoGbxjw=p>|7H{+}=u80s&w!4t^Mn zBOeB%6Tn~`FEYyapJhY39*bLa`Y3s2zX&M!cA$q{>utY(yXPX}Rsw_N6I4jM#D>A1 z;r5NC8%Pw)zyRV-23nTO_-1aX4|9yh4+=G2zTymp`C%~1hItMG48}-dw5#{P-fQ!m zl1`yKxNILR)!46Ip>Fd5)>XSEU@od~)UU3qfkXvtA0V%KB%wu`($&FIcobStAm%=1 znU<6@CX|*%y&{aT=qjgdhII~~Q=iQs_ebwQ;_+_`Gr4$&yrZj)Q+i7;8x}62wBX%B z8FOS#ja7Y1gSx=Q#4w1)luDW5FtIRxD7b40IlU160bt)#6jZi^xn?P%p8Rk7U$J9p zmnl9#Ug8JHiGQ^<$@p27o-ppODH4a->iJH(()9tbTukyc2pMiRoop zA3Qb9F!B6ikkqsNmdo~qn>Fun;>fh~c#*cgo6fZL9*lR!GR1ICJ;u_C`SEk55ObvX zaAbnPOECo$BmFS#Wek(+lTCf8S&y_2gHhp}reO?VFjC^cd>D-VNcaPX9${@wDZ!!L zsbQd)9&Zwi1?sx3rzQ?qdjq84A0%&-yT!EFXA9`2v2Q`u2WVdWNQH=s6*yhK6XFu8 zyjXLDN172dka!EicSR)vdGsluTL8x_inh}3;)n@A^1?ny-pJ9p0xwme!UxGi9rzzo zUX*{Qyew%T=l@7~SD+Vo;gzJ9C^8$MO1cCbYwbfD=mgqC1;eeB$7BZiPSs}Ee>}Xe zMO;5ar!Q;!41pH+%u2zGr1Lyj2WI}-xf9ocIUabK&S~A8-FipK> z4(EuJgzbhittUl;`^#WF?YMt0osiLC;5tu zcP1&TifQ^y@iAgAUaQ9D`nzCohFvd(9r&>we+|h9m!B5HD0{9ib*Yic>>7^UUA8qm zA;z0E3xxR3S)^}!->X48B+~=9O4Y*^rjEQw)H0}Gd}{JT7(%dtHW~3|dcmToL!|F0 zT$NS`tq0!S%SYbL8ThR%v~tS0Q8E3PW5OUDCO3%CstI%hrIBMeFH4fh5mHBfIxE$F zmdJYE4kL?Rn>{N){VfgC^z-{xA(u%m7_s}jPn?!;0L<4+2oi8IYg^OVK88B_q!r?|zt%u=mdj$x1gT=VJx z6vm^Mf1$jB|GDP1Sd|5!ys)>1F(6q{*5nD>OoMK{Vw+D9ld1UopzZbK-gv0&-yEIO z?hM7XM&`AiJIC&Z==fr+?A%Mur30h*zle6vO!F%}O?52Yl+$sQZjduY+f8nMi?lz^%cI;JREq{2yl>;5#O{9>oPx%&F} zjkh7mEuOYcINH?*t@GI$8N$gj61Tr4xMs)kuF6W>7CmO%K7`t~<6_?*It#UMK z?TdU{&KK2#MDf!<*UPckt(32qtIt`NU}i_P^Ozlkovw@>2s=l3O}Xff&)a!+JWo{r zC<~_=&sMQ#+fbZSygUC3@zao7j zR#`Vc3!;W6n=c!~3(r+GKUh*{xqgAoRzSsYKkJ{_9w4@&^L#5)$*$dX%$-Oix3KP@ zEJH&Klf=3bRa09Ez9-ZttfJym+VH~o{%{r3!_)_oH5z%(QR^dFV!N^ZIgR~2YYNBO z>F^%kTTCwyquI?et%HDa+P#>1a^}cgJz8wf0?tWKsRv1dmgD5w*YZq%GTh33?o@`27!@g#RAU*_ZSz>iEf zYL%zAa1A=Jy!ZGHKH?$u4VqN`{ERgY= z0q|&37v}N>?^aRw>xf*X4Rs=?P-CB7&d)FXm}JSUxADF#UOnzVA_mT!irZ4U%?xuU z{zfYrfy^=5hgNKYQg-X1Q|Vy2!D@Tcy6ZhJ4Zc2=oGM)BxZSdNo!qR%+zi_o(Oz0y z>z`db^A#Ss#blEuTPlDsp*{T`%eA<`_N_OZXT11@)_zx3?P7>5*~0y1tuuI)Av&d? zb?%f!+fauvm3CbX^ldUeb(a_2o#d|w0 zq??S+%I-IX+gh0~p6orac(gC{*R9V4Chw2x#%v<3G@f{cPxh*c6C`i(y>>e1el-Zz z%!53RT6H9E!OW5V=vcn#!j5nN5PyGgfwBZcXu5uSLookg9lbKJM}gr$@v?w(On za2n1fCYb3lk>QFop79ol8#=KEc$}v@B9ZW%9S!r#U59O){VQaL=p<@`)bBOaCoA~r z;opt(QMw=}LfJ}MnTpSa&w{^;;)xuW&OhqXWVE|ce7|yi{dKC7Gbq^ioN-(EQ{&~w zR3}lW#+s4H24%E6K;g0c>CBi^o{Ee*PqQqybx$8n%(HoWb&o}c$!%R#CXsE%?2<&_ zVR~H?t9ZWJiGvY?y{yhBde)}nT#%+Yk(_u#nEzcqlK1`v zy7H?;=(71JC)(t@HFF#tX;^C^{PtIc7sQ=#P>hTI981go4D$jxwyLgrKY^Jl1}XU^Lp7Z zQYRnOx#K(NNLv&Xtot-{(aZRL1-#F^Wr5Q0$#Ckq`ea))$7AZJAZ?lq6R(d~Hz4M? z01gO+bDMlE8D2RUem<>4Rv+NYy5cp)Ox&0w(AV48E#^8{jmF3Av^GNCmetzgT`7m( zT{gUUKk}~8+O0l2awPUQEIea)xpKIFUkl_0A**_S%=K;cM|gp3SJ`4qyBEE@qe&U7 zieK_Cw3cYw-b-0zA?7tlHx$1{HyEyzHy&KCyE+Bt#1R}6d0n(tm2N+d@B73z=mhGq z5T5OFv@{k+&Uy=M|ET^ddTB?iF~wnuedE>`eG9YR`{g?;f;Z9K9s=T3rnYHer>}bZ zMQ@fRI46D{!~qxnT1Nau<7sS(5Chw&0B$ucDA8v5?D(Tqrn)WzH54>!03=e z?YRTdmq}yq<(;OoH3i1T^H0{?tacG?jz>;?Tva=;7l&j^>ZcMcDhz`C2&VFhxQFvQ zn_7!^Qmw7CL?DMQ+Kr`k?#u=QUK?WK*{Pu~H}a3E_8ji4mw9nTX1Xn^O!kkf>qpKP zu4I*WYI^&}Nj2XCCSxt$IMsD(^bXNYhlN3X|ai_(_A*Ook zxAEjww&&`o%<9IWc_)phQ~SohH33WxnK?e(BrA&RhnL}wlHf*BdGO-fUoq9|(4|@9 zh-nz<)M%c;va8~BY_)hj{$N!=4Cx+tRMd$;Csx`*)qsd$<&n_eVVI1>PPXT%s zmfISp%|l+Mqr)WAHd_GVPB89mYd=gIyjO=A^UNSm`?DZ+T}?(29(6F^DV^I&Epc5E z4Xc6UdL~e?rU>;p6Z1WZp0i@~(A1|aEcqc(c667V16eixE(~vdwO6}fQMSJESx>s- z`#QwxX#PQHrxn7GK$tmAM{A;#4KEIo;?a}5i)&5VsnccZif_^h1UW5|xl-klG3(Tr z={@>FoT0KwmeuOmGMB8mk;12%fy!2VRoO-=sy=1aVT0pxmrF$`XR9>??8t&6#VQqP zS3HK5rOqjpxnX%SXG9!Cw&TTDa}&t)R!yx6A{8Tjg`#o$68rmRyeVp$OKK*^d|C94T2n@J6zQ&4%3_3^6{)kabLa|Kw4CV zPOQ}5CdsK)dxls=tqN$%^G|HWBVsnw@kIv?EgTNYqP$`}N@ku`z6B$moP4+c3GtHX zZNKp9KOlcNd*H>6TeZ`}?f zGm$Ia8DG%@hdvf8sdq~`1}4wYAtGvqfm3AY*C*V6et6FgnvLvzn0^QOx8m3aP#i}) zg$pzQB<}Ek<)+?0I%4|2JL11?>irK(dw+zw#l>KdumLKy(w#x1m;wPcC8crz7xJSB z7@*pTkjwH+N_c2+bZ}vG#Vz6| zSXqss+#xMEKFB6+10IvTT4KI6b)gmo8FL`fReE zP$XKo&S0W7xe}oL-fgf+^j#1kFeq?#{1|otYTb~4K4f4tUxgeB2>BeS{4KbvEj?rq zGAMZ=5&{k<`&@J)us}veuq)e|PX&GyN>Xz**OS|N3dkJ@;>Z*jz0gYKVsw#E`XwPd zOEV-42e?=2#uIrsquNa%!L$3HIWsrQg=4Iif4 zf4Zmh>C>-#iNzfXenrXa_fIX*>;1}53jKxq5~?>4jeKT$&Mr3?`cCTN)))J($&S5pN_8QFlgDIXIIa74QQU@jqdN?N~o$yrf1Az_Ycj;j4IKP27@5UG|h!j1v`F?~jC}H}AF=>iu zpsWK_D7Xv@%0wu=!R3iuu%_xX5I#m2LXI;4h%=xk^f!fpB7;)9%aXI`f$O2@hCL-z z(xZeVB0=k6UP}XBM4MP=07enN05Q@9;ob!U3m1a>>zBMaw={Jj3$(=%>j3iMxsp80c6 z50#$+Tv!ccj{{^tT5K2-)VEq2Gq@?H#w>Umxmob*Iz#DqLU~RM4;SDg|9Zu3?r!;? z4GzD3?-=?G5k_V9?a{Yw*v858M|gt*;C3g49eN7%YATJ;=pFC{#ggm~6*y-O1pq^5 z(s%NWF!r>LR7BW@tx2))f9rHyA39x|NkjR#>6$(1LR^5?}r&pKiuCGuAoz?4o;gt zvS^rKQlQ**|yHwJ2t$Dwg71mT;3 zq_kC=!LZrOl6igK?_lYr2@Z;48}S0p4r#1;qIFZ(_9jj#O?<72sE~#2=7RW@fT%ks zwcsU-UB`ml9%REZ)K(%QBq-mnI?yn148`mOG1@z7iX74sq$n9bz924ySd0uVwHRfU zXvG2HhZYztkcOH2`_v!>z;g63`|wMTE8$DY_a+b^jUWp5S972N|CJF36N3#Hw*k57 zS%M@Z0U%uFev8ewXV>$tf!G6@d-1yU?P0`^b9$Ro$K$z^%?V~-8fv*vbJGAe9a;#3 zM_Pi2db5G5Q-A6k;GEtZv$lQFFLB<{jslsMcr=-A!fp$xF=&4VG=c6_$qow-ytCa5_Bq!rojL0LW8kqOspL4A`K8qL5P zLHoh%S&?Bxe))Vwbuxflu{#H0tDeY^E45shqdH)w!-X;CZ#4X&M<0F`RwwP~4BTPq zkA4_ZRn-RD|C9kcF@UoSd3s`O$7$sYNe>+vL_*PP;(2tojgZ=yFr~#i0vi3po7Hjn zGkMC)7y4w-FjBLPncray7&Fq}7%c1J77k+SE;?C+T>86yeqvVE6r7H&8bLUvwM0Ho zF{T;iHhGTKvn`|m8YDu3MPdW$Ebu6egw|?~AzshhV=@bFM}NPlDctfO0BWfb53IbF z(gK_1C&Q}da8H-(4tzmp73M!C(9_5w`9`C6|1}%|_YoROPFA zo%SmgO<`hku#zLQCHu6N}YxVPHFRnT{{J=(7sMK_GIO4d3_-b zX4NJqm0diS8Dkj-SxE@$+rW>$G+0-bG0DcjS`NuVsky-)qnL5$AXvs{PeY?PHoo9n zdUs3A-V(#*vu8dNYUv`=xHS04y2FeHKn zyqZ>m**Z~$tqj+2B)Eq{YWPilV>(VUxTNn{N4##?)!lj4d(k3aU>YL{lnH9f3))54 zUvarn`x7I5(5p3OhXJN&R$lzjZ@Qjn)Ghl`dERbk$h4itH*@Kv#+jYoN7-E*2g9;B zt%XGWd&$9hQ>4%`Z%Qq$hhx7U@N=x47JtkVeCNjYx#?nWSsJJOQQSm9eO(^Xrr(+D z9@gH=_x3!|7}}3VhxX8XTHflsCtIh*sNaqAyz1t4`CgV!(`ia~>8{I?(Y<+{cZ1L~ zT2|Bq6mBQ4%yj9TZvuHoelc09hu+jfk4gR1rRJr?;Jegm*;`jB8x2``ZE4A6QMJb} z;JmwmYf0a1K1XDySe-Z9rF`tVX>J2^ad*A4=vJk>OkLC9k*{-md8`RP(SE*5V>1l0 zPU~4kCO!+dW55ONKt^OFw_swAEz3z}uoI%&HjZrVPejF$9iH{XonYLxw^w!`K^P-z z(zd4WYxwDAn7*i}D34cpj;FY0((D(S20=v?G^8m#UvHhI#_$Zn>X6~&a*CjMo6O`~ zut$q;v*#0ZF3s#Hw8=ItZ7|i~!g7BJ%!SFpeU;^W_UrXB*Z*g5m$voAg7@r~$YPVG z5+i4j>=z8iYn&XX1tC`br2)y^NsG$Sk_xVd^^>u|HM(0-^h@P0Z)>FR)c0(|Zlg@) zG}LSEmd;vLY>{Tz&4l21yp*t3=&$WuJ(H0)yRdBqiNv+J_nF?mXN_OK1QU~XWWJo? zjQ;pGrcSqb2tA2MJxzhDZ;GLCm$!1KI0h#m^X)9_!M-tivduEoYzijFUDPXSK*7uT z)zk5qcz4VJf`__Gr-{*Se7Y8UaV!{n-$Dh1`+C10jeEwVcm)J^Aft50=t+TIMMrX^5M+MGd zUy=)NI)66Q;Oy$^qQs@C+G|Zv58i6=`9p=Ffo6>Z7Y}HYd9g3gq+}@<$%5|{RkF2jpxjh8Eus!J}f9kGF+0&386Nsvo zeVf~l*Xa2YHF=b$?ckbxKzQQ*EbI(vmxnne;lgvhVX;VhA;bPOv|qCOoZ*HooWbqH zI+U;9?)2lZhZrL}#yw!#=!E)K|H#$Inbn(hnPSjt{@|1pnlv_@B<zmNg@m9^qUW--(Y?o4w!^Sw}==nWMtcs54fpW`JCL2p-ZC1jkLpO4MC88 zM_}emW1f3za6W)p1-aQ%*!>;4(%*CJ-tc}lEU9j;AYqcmN4 zfO~!9qpG(=MDBLdKY@AZ!=$ZF%G1(u8n;+oA+hIt;&~8Uq0O&e1`47 zK-0L_oT5C?c{Ka(t7AC+!=oufTJ(nJ%NrWt)RBg9-R`!>>P){+9ikWPg6o;0a`AOS zQ4jpX_rnK4*s({-^ubTsV!F%?rbRsk8Ge%SRP(wd?Cv)cV;%`qbKZ##2dxd~54;$2mgZ-Vi-XvgYxk?!??;Nr z$okuBTZ(uhDVf;@ZXRMHK^KYnEcV8y3>S+{3_H(2W}U8QX1g>E%lh|~-IIM*oGP-& zhaPWg9n&Ehd5_C_yPYBj&-0y=UW~nFb4%@(%kqQ$18h03eW<^?{O7F*+DiMpTd}QZ zjPO`&hyTKBpxwcYQAR3~og-(0k4*Yn`( zk~LD!UZFRg1yYxr7`1(^6TR3}XwsBpEth@u?@&IJOs4zQ#`IG~+a)(Y`;57wHQW>P zpkn}!OadVo=)zf zI?hE$@s}{6tDT2^g&2f(mJ5M0kXfHNW$u*fW8Hqz{k^*Pa0rxNhJKU1NZDgv^E!_~&IHdW8f_a-#563!9&)i9-J@BwQVmceeglAbl|S< z5^1XVSXqp$+y+W97jqng-@o!tjL-?!b+%zS?pd07dA-#|Ch1z~auu70LgA)ed)Nvu zHEb5Il5IQQ;Biw~6_4n?sxIaAHIXmix;SS%*YC*sZm*UH>;#X<9-L9VN0%DDZjA>V zw>@pLFcYxEA9gKhxN5y!jB{Ia#_oQSUewAd)GpzCNGL5SO&oPmhE!%tOr_FYt$)J@ zXmg`&^IELfPI(tNh0h$j2GCO7TR%ON6aQP$`*+^^;*BM-2(7F9M)z;()e4|quaK^K zpr}AV{{J`pv;Vdv@sErYK)pUPQbvw{rL!L?>3>OQ&om&_P?zr@frW+n<$z%$kcE+m zwfT|F1oB~kmH2VupaIwO2$_J9>%z5b{9#yy2K>M=$EcPT{lLczTrt$-DeshL_OBB5 z*Hgs_lpIyxFW%3FUU?tlQthXB_MNAA4%gtWp>MyzE93-)qrE7Cdmn%3XHo@4k%^K1 zi46^ziSdaY9f|&tJg;%KG@)Nf!jOZQ1pRF2p_BDTka`5J+Yp0n<@@mo^=PHKdq%Bo z8O904>BXs0A+jH#;_3EtrPy`+7y>_GA_ED~4g!-{Ax&WRZo34)lKue0NB;_j?~9f} z$cjVE-wlQr2Gz{o$3Y}cD8QT@N0{ah9Y}#J{X-xwDv8aHFb-xIFe2d>X7E`)41J^k z8IKwr3C<)=$aoop4<{5=e=~rv!tl2P$yItaY9GqShy)lf@{4tETrfGAUBc44_5 zC}2c_=5xYxQO^~<;@fa4vVeEY-}adtGQd7#1=wfYFt`HXoO{(u2K=RdB@UYo{sBL!ppB zwpk}5d+32d!m)lPiy=di7Ui+C;P3RYqxlmMg?!2W5+2v^{Qv^rO^*mfM3^1Wkdg?b zfT4m7=_!Di0|7=Ph-d*L677GDNKm)80rdrZj7U7TkF0($FNgsiaXv+up&09t=)ag( zx8|(?Dv~EUfO)lkp%b1AX8*&yg0M4u0(ZKakR-uULZFI$aTB^lh3b>B<+6adCt?Zc zZLXn8QeYJ)t(>_Z?Vwn_J^73x3yb3X6v<9wb|;0>tg&%=)uiHnaFiN-IR?7$G- zzK<>>%l9!c25X%mSt)4LcdnmX(AMvf;ttk26VCTqF2U0SNuFsn135n`(kO=P0?}%=d7LBsQ>+C%ZKI zRxC@V*Bn$QNeCD!Ev*t6>sSaDakezPiwT6e14z|6;#^mDtK)o(oV*6y1u|Am7wr;L z8`PPf(CWzI|G~Wo|8OsZ+#tY?MA$$|z(02+>^^oR5@5_eb|j#w!-P{}xc>8w1P(7z z08I|qUpo?kshNH3`^}z)MN^ zdq)DR|C=FL2m?-Bf{=RqUpo>{s(>Ad>_eHDc!30W$uqklXep2(kUDu9a^!yDtE22) zJgOZ%0h_{|ZgAbsI3=@%McQX@z}-As@g6{XQ=Igquk3A_0bWGy#1sOdO z=wIaPpY~Y*z&^Xw1CC-A#nf!M{~b6HOh3@i`W&|Rn;#kzUA{#2%ep)r@{;1bS;!2a zBPDtTbfo)$j?`^P-aauPs70%E6B2eW+&jFG|3`{3G>e^e!Qzrsv6G*p8Yo~$LgL>; z5@+Gl=QGo}nm=pfuh_U$m=P%6yc-PB*M@VYmzW*wyY&0WKW)d-a3Q&fTVkvH;Nn-J zFG9`v6ae>$M4!uvfM#gt$;a{_3WcUF%b20a){Hj=m}BWHgC52@dQti@HEoNfA=;ss zDa9CSuIFR`k+9Wo&?i7g`WeuXehRiAWC*h{)rEMj`H>^NUzZ?z-G?$<1&?bt3VpB# zGr#X+ARgz8(voZ8q5zs&{V-UG31}1F@LqhlAgcpO6+Sr-Sr}H<{&~w6)4)a56M$e#C0-5Vv?%4a6d?rN! z^$+>1oWenkDX067eAbx%AM#n;0IB{$sdqUwFlGuY0xRPy1ePR6NUtF`v}f!bln!_j z`jQpQ0U2+-0%PK*KKaL)m(=FPAOC4dLd8#f;6IimIQ}e2fCH8!Hl}F^#DDn^c2^%M z*giKH&Vl)EfePBKbSfrFvf_&8O9PjOOJFrg)TRGytU(jdM@u4kt7y-mOlDOKuIh*@ zth-q_&wnpnwh8NbBjEWo5zU%%=;Bj{Y;0ipm`F z1R$T`NWO84nAZx$7gltoJ<)Grmg!(R0G`BZ^X<&s)jxztevfuqRNS@DUOSo+<-RGF z6pX?+o#WH+pnZ{Eqn-%nJ>t2=JLwvCvUA&aJl2;E95sVS_br{Avvc@lZ6@!18o68B zYJ{qK`jBj^m)*XqY;s^wkPLy3+`Q>#U({MoOW1l*v6?x=yRK;4{cuSlTLE%)S?77q z`TJ`BHa!^omnsB$lJml(Jt`FA0+R|0`Y}4#wx( z7z1rnul=<7c!+1b<*cNLgx9U~X8?%T@stQz94fUidA^5Uw;!|~;5ljKJRXyITv_R# zx?FO6O%6>Cz6;>{c2v>gV#?!wp>;+}M(5=LxD7M1It&_-c=Jff*@wAmly~>jlf`PW z3rnfezO{GWM(c@m*;@YYxZ%RlXgp7Nwcmd?^Yi3AgL;@4SC`ID(mCheuPiGW)Kj57 zzP=XNmkdxZEv!uKVbwF286_j7n1hJN<*e7n%L56TY}`{2j?wJ3rUeUO&P`9ci;-S( zItdj6lzZoxC)^`X4k2SR_v~e=Osqr#MvKA64V6?h7|i)gteVrEwqIL9t{I~C4+mgA zL6Vs>Dk(u^nzG8zHP{*)<(C2bw>!G71{bQS{Nl_<)OIn&2#;Md%<5{C_4ZX(M;!C= zD_O17$#Qy~=CoX$tv4Ceu*y>(EQDq2JW0?ch3h;sNPoP0(jFu13rb%W5f*L{2@VR* zN=tfdrR8hA6Ei&%BHuli&2Bq)uYUfvC_c;MS=$rFpB zSLP$ax;@|2ZNoPQ`E&DyG{E`2vvoGQV{5VEbgmQWa^~?`7hZ(3_=>i-AlH@4a^ECH zvgT{*aS|D4^U*kO*7lV6o;Rhst4;E=1=0DaC$rmq^z+1z%yVO-SF-fQ7-xo7DGlIw^WqBR1?Rd-Um+| z<4;O4y?_0NX~%WQcX3nab+)}Q;klZ-De+s8*~ROH5&p!!AowSy?U=v;&or8>U3Y7d zsI6P#SnH(Ps{NSw>0nyPI1{qFb+HbX9?R*OB)+Pp>_v>{AZ5|>1;Zjq7`0Nzj8e^4q$z?Q1s?eAHKi^-kD99AwKNny3L<+AL=0=J%03(njGx zN#1i&Z)s+gil}yP+m|@s@jarsWmg`9U__~DF3cwxz7_?#`?%(zQSXurK6I5yys@Q+ zm#ZueN%D#G6~MqA{#E^gwnDz`0Kc=4?IrP-hpZ<9=QwmJd zv&SpDq4_2txeb@*PgQsQVZO`g0Ehnl{t?Tbnf5cHF=nQorbUOjb|j7g&k4tc%kR|@ zO6}3T%D3+Wo2g0dEoIPVqNc=WH`=X|grX%f>mRqIK-SBnL&;Up02i^j1CF z9+v7&*i7}8G96RZ1Mc}=ro zzamClhEiP(PAnkbtTzCsnENM~Hz&oh8abt>_FAqs9*ksjm~&pkV{Uj~zWc>aLh|jK zk({-)?^~F@eOgwb3IF*z*%i}K|JmmFSKykkb*NbG(v7PK{u1ld(M<4gI-7$xo}=;O zbc!pg6a1>32-68P8XAk&&+T-K6+96e^!5n+?e3DW<$46e%sZzOJ_epPhm1qlNFFo6 zcBM5CaBxmGRg>ST&TQU0j1@E;@k1vQahWDi_d=xD);KmJ$2BbziHEUF+A>;Yu*ZC6 zOM#M@x?S0!gLLye4Ucy}Y4R*^*jFo}Eq^r_Uv;o}l1d&L({WZ0O%k!5_eLWgYGPt4 z?+*s1w1}uF?cr%WDG~1;&qf~GK7wPzo-9SNj^qiLIA83Vdz~rwT)3{kJ{789WN!@8 z=%_nZ^%5vDlh52+8ON!_^mIVO`5(1P(tS?6)p{S|?MugOcyddQ9OInjYCiUtmvhj; zA*|qSRy53!Ijw^|v8jr~e>xO3MjO&0Lx2x*LW2%QV57S7GS_S1HA&PI>te^l==Rp> zex7Kne1Xp`s9D>UIG)Y(JvmKwhL>JdSFi!ZE+jAt|bxHHqd?;}+JlfED3OAg`z@aJu+niW~ zx9a>VlMUUP`;-gv`Tp!1_Qc7o^|J?&Q_pssx@f5DZ;~Qn` z;K{o??y1&fZ3GN#b9(2~ZN*Y;%C+EmxMh)1ChXDg6+`>Od5+}qls9~M@L#FPo-gvz zR=JOWoVd1hOWz+BLwdwpV%7?Q2EQ+CPL){^-=oWY;vK`tu&>zJ*y)vtFfpHq6 z^U93&-d8o~J@|yl9S5FKk5etn&fN}klJc}&!fD3zZullHK{X=c+>ofMs!lD_07Iwa z%H^q!IX2z%afF%nxk7rLamR_PQ*``sk%6@nuYA@U1EN3 z%Cb7G%5o$A>zQhY6iw!*IBMzolSVSS*Sn6;Dfnazz6ytUxAIm6rI%md^sc5R)89WS zB#*(4jLbIM8MhdRj?lT;Thu_xz#5YVZjPnidJx<^_8I}R1Ugq* zL-XoI7ws>j>MX;jF7l`X^DJUxfHr%ah| zQFl?hXMh3Ov-MMLEgU{4MA!~L7zETiM5`5}5Nh-fBhhq|qC|uh(nCuRjbiIlCWaq5iiLIO*Cff8 zq8}B=6H;R;CY>{j=Sqvdh%w8Ab)g28N6p+!2RQf2K%fHhoMtqSJfRDK?S`5+GhWx# zC$0FE#U|m#lKS)gRN$_D)OBo|5ydm9ygww9w=KsC@A zee>l)VFWB46voJWjkb)qsgv^ZgN9dX&LJ_jBm!O9Qb=mRv3A03MeJ4HuQ0tMyCKPgm!WMQGuAY>Zj$B6nCU$W)bAFK4dp* zE$U=3LCO@OcUlk@wFZBF;2R8(s9gHEUa+&iXtK-=9eU3%0mYGj%WmR?yYqT5mq!cHGI%0|sNMGZd~b7h1xX0!~bnJj0$g%otv= z1{h4JsM5)TJjD8YLOgpp^Ta9Hh(2xl>=-h4d=$orhVWnhV?73$VLz-GpP0f^cQ;Ex(_F;IN> z4z1WIN~fqkW}}Q=g5*^Kk_kA_lo3B`M1E>3*Ye-nv6yEF+rHdipiq%qFzszO&bneYT9XxtiqpveiiQ;K z3=jqp{`gsnaMiQXDE2wNcNoL8Li=+x9!jOUru;-M@og|1*zv*ojC>@&*c;AWJTmkx z6q~}hZg6c~@k)jh%j7eGqOa;&jF;21X488keFnk_nDkW`joUm-b43Ml~X>(6(C}O4~GAW~Soqm~}^M?kDXDpHBT9??0u7q^*qm7 z%ktxdcZt zvmT+{azWE~#nQP9ag8=`wWuKG_0qNeEGxoJ4!1Jf zyOd)u9I5;mC3y^JA(rtn@GkUEBh-SypMMtJWNI&4nfjY|vdo}UODph>hKm}1w27++ za1^SB(M_#<-I|gCZcV}GQTt(htXq7{^uOc4?ukBs^X%g-|_bIAlt)@kT; zLb_jJ7LnlO_{g=!)7#y?erfwet9&a#M05G3`Z5zxqSSnS<(hzQ>^ExK@37450NOVO zQaq~9=hrc83i^*Ut_j#43g@qRvRF`D@{P#_;YG>4tYprD3|nAFnS$1GKVq}SAm$>e zgdVS#%C`sH5#Q$Sr0&ZTD`40N4lL{Z4-s3R+ZX$w6nJfd9Hl&Cln_ z>6)qK3#8u@1${O0EDvzT!KLn050L*ze3&Y(&E_sTkKlPlDa%MP(qd$^nHzP?i>j#I zoJrmZe9m;yHTc-=?CB(q`|L}(ks~QgnhelnR#d0bkkaB%X0n31wwcev`-1_eehgQv z&Okxsqki^K+Bl`=+`HIgGry}oCV6_^)%cS|V{>k=K6^8sYQ@cHiTFWNrtx~YacLPq z(Ptils`4NmBJqMB4@p1egHLZwMQmuN5Ih3AhngGq;!=sw`U$Bb$j{SSc}Yt4a4psn zeaZM2Lw&^={IS_n`=T(8#xgFzh)-CKkHVK1&2oFD5B{(i3-9tovA{|NRXtYt*Btf5 z$(^4MaN!Rsuwl&DM3SZ5r$=X4SM1J10JPS~gq-jodEw+e)EroEMOpTb&fXeg(|1O%peo_|d3PFfX+2)z&e@5Jk`g`W8mp0M6)#lL zj6qxdZQB6;-Zm4g>+usW@ACb$#`WqrT-e>rYJcx4(er0max2>}Os1jnoT@qPfIw8~ zjZ{IDEJ+u9Z5>lD+xlBTC;^CtWv5sC^pMSzUpK}>MPWnQYbl9CSs-??%ff^EgVC1V zwtd&~_~)v%{#_>@>YD zCH}(83%nLru)8&nOIbJ?R-DULO@Z<+!9(LXpr5e{%_RnETg!zZ$6>h>)EBRLLq&5J zQdl4TVex2Ge^cnT9|B-dYT9SbwXfba3Ld?RG4!#6;;sDhU$xuah&PkA4QAk@O)YO) z$~^>d%7`e&Zsj#6(F`%5M;&jw)VaEyUJL2_R;OxTQ6tvR(vTRzK3!F$bo8T&z_?Sg zQg3hiU^17(<+qRR%VW*n-+u5ES3U3CrS0tP+=y!2&O7Tzrq6PJht#Cdpj=uqxy11s)r!{C z@X>u!Yol-Oil^lBW;aE9)}VoMLzZ#CUD#6HnO(~?xKw}J7y5t^JsMp;QnroWIB2;E z$z;k3`n?a!#uaRcQ&#-})lNtT z|I5UW%*V&fUp(xsM`zKgC&t#~a0}GGa$gB=ghuKW7t`}+V@96e^(=!aK#_yFK4?pf zcrRSVcq-%`&z##iIN`=R6bMdSCkMwmgV(eDoum(8oy%11W^4i>8l@BQp;nuEhA*}t zcEw{(;kYvAS;+yr<8Z>6v}fy8Zofa_qC3f6nKhhpko%n1BI@}iv8kL@XHPx)M6^tS ztk+o3uzOX<@-Ft3EV?v&S!6HJxC)<(ynv==C% zo6WL39HOOkAsHFl1Kj3^UCqvC@($4gUy+Fh;T|GQuJDx7+}1=yu{R7@Y+kPSH<78g z(vR!PHWRp`gn5RKik=$(mOIad&n4kdYcst?AI1%E{kZ#KR?EKzhjVRLXEkoY%G6{R z7_%$Q04JdO>KfCFlvH==rR&axKYWz8hLgEI<+R4c&wjOvHbx^8(KW$s#`Yw(W@z=W zAPw3B^jwEcGdy4hfOHs<4cjdAS)eG7C)0aXDvH^#iZvoi-lpohWndMq+*5ond5_>CXmK8}#86 zsU$7nbY=0-Oj}q=3d$x{K-QX zr!OH_JaiA8le!9mg_locT7cf+7d;s7=vXMz()EyC5n|>1S?2Z zFx70fzlu*y=h*Ujv>x@?Zda68U%ZhNTURk{pe-h1L(woXs5S<_>Yqhd?_~#RI4tdO zUCgnW`1a)3SX($PE9Hl7<=N8Fbd}L~`NUsW47}EWa=nk}_lpZjT=BGsBX;eXXFv;> zB_G=usou~hxkN!tSq=Vz%Cn;h(7YiZt1jj-yn8wr64^v@Y$wk%uVTwuXhW6AzqZ0}_}hno$qt;=#FieLcJ7=vbeWVw-+dSvwwm7Do(v z_TILq@c6Kvn%v7@#EOz=TH*ZuQ8{fGqptUKVt{>uC2yZYO?;}O#&5b2L>09&;a+61 zQWD2b%0qOnpRxL^3Y4cQcp5%h&J7uXlX5S2{eBQ+9{{GE{-MKg*)(z*Ck)ro*`ni_ zh2cEilH;}_e)w$p?y*x}CNJhCRzaPN7*B6&_|S*r)}?jY>>PtVWZk;%wnW)siX{9_ z|4as|YRxXH%hDMIL5qxyAc^q{A47RyC8hp#H_#~wuET6gfE5e!uN}Rmn#MKD>x*&Z z7D2*l8P5v++3Y&){7J|}c?Czz4NoF|lP3$HKFl>sVV9IMG7ZzzG_aZ8o}y>Q}Ci-c9g!z?QP{GhGs3d_RG+MfqZ5@ zACKF}`H$p_zo%nfFd|1;H|4z1m$Ft?YIJ3$hen@I^1V1Rc@mlbhldQTE0^lSsTVq8`Z5Sg4RK+(9WKAtdP_t&!`Uzk{E z%zJYq>Rhk-cpplVs?!>o%6rXUk0z1qeyrW+w>l;DuFQL^%6Boe!my<24ZdqZR4i`F zZ1%q|mK??we&F>@p-Py(>~1I2UN-k7KCTqLb?Iq=XUM8XLesK7(d_QF7P7S}wLY*{ ze%?&qFQw!#tk&G~{Lp6=UZFmyHZ9>Yh_>0b9acknBi&_emuhd zw}+=sM#c+B;?NJM|GsMWWoVOs7OTYuaILR~g#7i;JGyy)JpA^(IMyeR$82&dx{6?AxLQftVQXmH`h{7~(4=|rRd_IAn zLp**AK2Q7Yfz=z77tKwItC6#jG0*GxS(v6LmN}LtgZ$5L-|At1a6OG|YrN9>?3O4H zZ_&W6%^a2YNx1UK(NB==83+cXL#swg94AZ+_s?Ti`|qjD(5&_pWY2@qe9!T={!Wel zw(kt~4Ib*-&X3O>Fwyr@?VmJUi-i-AcPULNIe4k967A%`Lb|xjH7qedFbVK9db{4! zsN^Q8^A}4lEP7O8RDLS-R?JLwnTvmCe#MGZWOQeC@#=qQV#x0Dpkc62@(yQuw~5Fy z{8mp8k7JWkYdBYF4+rv%2~1k(n@lhL=Csl0;CpQ$J!4MJ7BYcUtO#7aSbQEKMxXcL zlQE6X`o?4YwDgCpGwqt6a0%(V1ri(uyd$pN9r!nyV8Kte19!1<3AI%5g>4}EOSrLi zg;LdQ1{QT_*)XaGj`fD#=6ejC&Mpj$(^a+9H1#q}sRTHJn^R5P zPj@ug8XedmIr4^%;*ei+T8WY)g=bu!yd9t-1WTuaUd+!?vKO@NUB4ne&4KlLprM2O z%1{K|-6JF7bI*Weam&iyQ7@%aa%xh1>1MBF1_?pF?TmK~CqsWEM65c$M*ZUBdmYW+ zIlwKt9YcISZh7Yg{jDmEPBI|bu^Z$+Z4jqI3VdH-lHAH`z1Z2lR#|UPV+N!+cXP%iPT_^qwhzc*i7gm#g-^g zzj}kTPx}-{fOWxW72m;_YsX-yeM1nsl$muMcR9D4gUj52Cv0T4Zw{t>56AcBOlMql zJ0E`(gHS=*aO|&f*m>14Mk9o6zEIjr2DcM#uX@p@NDl}smExwG+)M+f2hy z6F+aK541G-)ZGdSvZUPT+&d4@$^L9N&}2xtWPv7_kapsFxp1Iz=AP@PaE28QP)0E^ zG1+UkZ7P&m9)O;L@I0YpKaV6ZBs&e4O==QKqMTBq=&N(6KyEIY5xP~KykhJkBbt4f7rAA3q-3!@BS>6NI{)-;d9$APR<~BCAqt~GJ`Nzd$0+mQU0gCb z8c8H~YGr?iS#(71EEaT3iJ7^USqTN#$Ifn$JKSEIW9S1U$c@eR(%kF~{rI<4+b~P~ zZ|l-CD~+uGSOn{<5%i)wW&1X!DNWgZB&(?#@=Rh)1m_y}%3($dblKi%?RyPao%MRY zS+Wyi3{Kif9BK{)a*XO?aTciv;ga<5MO0Gb7+AR!L@pD(Xik$$aeZ1DnHv@Ba_>CF zz|=j02hZnKbbIP6J@#U|ucKTOa^5peQ*FBhQx={}47usJtU@Axw#mIhIQBG^10NhN zzW?-mT8nC=ls*Z_pdjY}S+&4kHRrs%U7CZg8CGAzZuU1^m6}rH{t;N{xB}Ql-ofrT zRqkU3lxo-)-Rvp`$_(e?-ia(osTP}Sg-4&tU_hZb+4SM6#*ljS{au)`e$EmsEQFa; zVK>Pw43&~dZPZ!YSAVz_FtM-bj8TpFdY!OeF_Larw7I0nH~^Qvr=&JxH>l1+z52xe z!uKV|KYZrFY)@@9lKwTIuLGBNhJqF_Mg9oMD0l-W>M6B1Up1yIf*yUgA-G4!nUl;t zNc}n(tnM|P5{^~X0N=lu$&J_Pf<>_gTj({5%0ey8!gun^G_s5?Ba@kkZB zD!>mg2DEb!(ZF_9pGdox?07|sT6XlrmtSP|_p~Cmla@<_KFpgVec0eKSA0oO?o=X7 zPe(|kR!1drc0=_%hOw9Q-%0b_`mtW-N<2|}WmshWm z5E`-^Qa*iclW+O=2(Rg&6`)s;Lm4{0VT^HKim88WxAk>rY*yU-m_RnFPs%DO2wkB%=SXKgtn{j0v zec`j#^tmOXl%*6arj~|%=byegAM*mOaz;g>-c7fdneXx_l~pWEA#;vVtV?T|m`$c6 zjMy6K{~SP|aj4D^yrw0_+=6S(zGfvDGvae)(8hqsAh|7K6>{aByfMs43U@pt7<^ z3)|?eHG#Og{|AkzZwBg3J-i3@ zu@@q}ilKn)n+v5e#+3BtVKv<-kUTZ8<*DSL@rU7OM(xMVPAbkpF`j!`afS=BkW{>B zdKCXwI2cB?=2wC76qnOA;6?<&!=Rp~F7utg)Vlt#oo}tpawu8sDfF{Y&&!&Qio>DI!I*>h}km0h1C0cxy{ zGm{JEgFyvrhr=gZvf9 zmrO48s{o47rsjPEof5yi%c}{Mopa?C(c{)OeDoj)o23~Nj3aEl>>X$+b)n$`b*Gu< zP*>V!420c>-oo8@>nLiRca4rDM;UE8N($8AfZq~vR-6e-2M#iR;*W_*0s(T9^nrCo zI=5D;1!bl><%!8W6xI6J#TM>bic{{RGO$#s#SplG%Mslk&f&s*JU=CrBw$}gRCswx zvUaBnW;MgIa?b(=!z9_Y%?2Xwg@zFroEoxTm^&dhX`p_|9Ime=+P2iE_5iKA87?=! zd+>OqP1l8jVoOruur9!hEy5V76L~OOXRPf>muulQUmz-L<8Wi?*7|3l6V221g}QB% zN73KA+VFy1)$BN`ILo-)zOd!IoUJEt08Gq*S}LApaNNbuwRDO|Y}NQGYu(Ga)!Z#n zT<`C(BGd$CaXN41S=5EWeR2UV+NV0k&b>$90({hQlLx%L@+rMNAOr^_ldp6{eqqrl zK;{FpgUy2C0g_T3V!X6MJxtRJ@3lp}`(@Z0kQ+ddc*{OI%Hc)d@o~HRR*`Z}f9;rt z&O%*5Ilwq^Y4G`f`^9K7o&DP%6y`sE8T>?X_;*Y6*SX75-URMnz_i=;|G#PXzd4ox z)9!y9%m12o|8E`35GqJZsGv!H-yy9?siA+%QU+xB6@vg1Li)jL{6>%U4JT%xV-V$) z3~R%$lmuCN8$T|L)_{j!p?Yzhv;Gp`ibkU077ZJ%iqax4EyLI zCQ-NlaV)dKMZ5eHN;3q+nH`fFJb28^Gj=ATv(@LTM@JAEJOzn$RTg z4i5we9VMPD;sD)M1hNNWB3&{3pZqLOXckG3g>Ko&Cf%8fEZ8xf#+w;hCDL2LoF4zo z0=-^7I=P51#qu^CBf9{(1F3FFkB?kn2#JV5zMl+1#(eD`9Xb~>|3JBbX?RtR#rN*- zn3SN_h;hK|;`||)cm zct8@oGvI!|J@x?#P`Yk%$AZJA~Q`})S6=j(9;GHFurM{Zw}?o0u2rMMS?Jao!G z!K9Gji6UE1Z~=N;M&LrDh9};0R~+<3&7u9@6^FrLj@oZ_nBaLLLD^Zd+z+PVvA;PU9WeU3{HN`+XMdMMM}DsdM3|LBbj+ z39<@bG5P4z8^n(#GoUJ#DUnxPi8*0ILh? zeoyb`$i?Gtmtt_#Mn1WqD+Wl8Hr=mCTG_uMX`uf^(vsu2{yA^4v6;Dmryj2RE}IE# zJ%F8UVkyXY?ai-i*dU{=T`LuYnmwp!7>$#Q&9!}dx*Ylq$H%`wy-CwV&S>t9hB*u^ zfu4neD5|7_lucIz{05tRwLlPzp00qnO_VO45h>ZMe&sG$blFk`V0D3rVDFIqr9iV@ z?A^}7DuIEm0TUS#wB6hp_bV{6&`G6k*<#NzBDV)hzZ9tyKNc7MyMH?(GCIDDEiZ0c zKb<3+q{x7t2A?8!I6_KnM5FyXr+ghD5I3TzV7R!#LJ;tv9s=_ojQqFnOmv!_(}ED1 ze!RfaJ|Q|7BC!9mx{Sd?j2Qf5b%CHx%qBN58?)bTE(EBRLkTQTF5}qS`fRhtFLc7Z zlNXK!g}3EeE9Sc|975#6Mg~OT^<9w30_juCl$nQ8UI11Xc}A+QMp{m42T{cu1Y%Nm zO}B*u!0PfuN|#4;p^FZO=F5=qjoRd0OfEtdr`3W2U{v0T4GK<_@+X$2{zZY?;p^H^ zok7YAxNVW3Ko~gdHd3er;{0BAqGTp5KhC=i-uyeeall4SJ_}t%1Q%vzVBw6bq4t^f zH&e|@Cq5IGNUty*`@Lx{$4Qtkmta;9D?h;MG77M|IDA=MoG`hR`Aw3iie5ddWm=w! zOdx^({7jU=dL(v>fC-pIEGH>0IjNapHAfKhPB>9GL|mfwTg=OmZ7Q`T*(Kmmf`&Eu z@NUvaksHcYT`q2G$*SH)^6QuRI&C4dT%e%E!evip_#?x}2Lc@`6NW`PX8ZIt>drIR zEid{#UZSq{&4{;hQid^kz+oVzkUd!1a@Ayk4H0$l+7~^mF2`m(%(c6{lAve@;8jM7 zMj{kKz}_8;{szmK-|4|TDhkw`{!(%gJttLNsFY0urXViYZY)nhZed(Df*es78+0I3 zYu&*V(EJ7%b%&PL;1z$3y6ePG136k%!)B*azDC{DfKfML2-TNXSq^(;7?|5)IP66B z=BS>3CG_)9CGK-X&u4ZyKupSp=7Yimiy+f8%haXAt%jwcxq@D@9m#wfV+-F4U zcrm6ZiCs|8P=GWJ&?=9ZZ@q8~DTm8*nemEN;PVA$OZ|xb9>>DYm@=jlgqoc3JFXNL z#0sPcDx=$krL9VCsc`CIa8Y3;2nJ;L-vVh|5wKv)$O3yNDTR79_U3(+0}W9cPyJgm;m(;Phz21PTp63-ALn0+ z3uVlT(|gwx$!RR~h?*57a2CNOvU$Cd!5(mXf>{4!)-Biz42>5#6&>2J6nT#XEcXq1 z#QfuH)(uUVNe7s9lb5WL!&oV$4at#4$f_WI%DVDA&~z* zRfQdo{`MHwLjjqd+GyDmjR1MjrTJpx+1{+R0i+-7heNg%_n-xsbxVih&v~S9^S^TM zNa!4DNcRag!NtPqwXV_IVGdIyD%PQ{c{4#yK7G7cbkr%%C3;%xU?*8{3zOt&`cXN5 zb8^~8W9j7ZaM&5nKmz4Dq0*dnJ`a)7M1>o7#_>Gtg|;7J!s1wEM}G>6f6Wr_)K)F! zS?Q~G5$I}ik9}>y+9I}MkAiMyu~!k$tyl-~e?$;=xFI1#$tU}i4c=uSBP0zu1@GkB z(qEIgLU8oE%|$%0Ca0B1EkJ7T3Qn-q2kkJe1%=i)L9bt$!SqW1raj5ZLmT5}-BQpa z@c!Y5R3-hVzpC{Y*hHW>G)`|Zzk+C{*DL3e#WC{DYQUy=k`@!jE0(yD1u_tt<0+4Q zgZI`0M_s$nfpPiTXWqpbhIJ&W(yRUAszhSSfKxlY%iR@F)D*WC9uM2i^}%+-C95Gz zQ3BcMu0^Ak=Vy3_u2R)IO0xW2Xudg}>7rkNb(fa1#)l1x8VX`_EwSa%1GV`L#Ut>> zB)hBO-@c+s;lwL0r5f`C)QZ#EPz^cm#-SKer_#L{i7J1R_fH9?c5TEopu#opoVbcB5Ujh6$z)h{{i<~%NCo+uiuja`t~7dftbV_oU& zp9kcTJ?noDT9m?KA(<_weYboS)gKjBpLan@F+Mz#xNknL>10h*x3M*>&%$uJoC@{_ z3%G`o!?pMvD-F{5ov!)W zBUtxg`qE?Wd(pP?YXJzSEGGvoH#OP0;^V!RiOpoc<6ID-g48wK(Mz$O;YFLieN;u8 zg_DEX9ECs>YoDpq(p=$;2!!Quwftn0FJDx9-D_I~R)wU&8mp&}VlJ z_zWunnzJ443?Iq}7&k7Ymz}ob*=bmmxZZ61;Ck41r#CeZ%O(6rtzybGT$DDp!wt995~=SrOA@y)v6NHAoL9!pH8(z ztD>%s^_|1d4MAxhIC2KU?4V+9dp&#R@c*)lw(nZ<+brTrsV$$&ssE!rx%*2xqNB0G z>10+-a6lWP%J@n8G9ZeE&9glt;q?xA$~?o-DpIZCP5fh6s+&HX{qROpIytrDeQazQXvT1q>=l)xd3` zd2*G`TShX(@v}_6$x`E${K-(^02-%5L09c~13PhbjiE{zm4|EPRJV0uhZ<>2n+NK+ z59>y8XP>2Ic)K~JahrJT0;hBhTkc+7Z!i)u?Fk_%V+15>&`^c=A;?`J0%5tS%wg`3 zIQR+&OJW@Af#Oe=fvcgX?nqUf9z~(kt9g6PM!|2B#F3{Il1d94^BvWBPkzVD+w2T1 zDuXlea+%*l73S;$LPk&88CI=|G*0#UavFL)+qxsdj~7H%518-bI}$^ba03Xzn?h87 zIMo$p8FQzCJO^$o!H7Rx%Qw*p9+9;0xC=UsPNmusEPL$T^N>Quf~OVLEoj7urLNtL zRN!}6tTuA4BzZQUgBGf!9}3r3uy8DnZKYAKYx@UAL-@0s203z`ncg$mK}PJooDqZu ze}bkZMjEHMp_62s*pKbCp&fTj9b%+LbnZ7Xl^G9r9G3LNyn2Y{rT*Y)rM!6AeMjN| z-!$L6dt%}&cp5s=Rc^%yR%)))v{tcv9?y2(4t%RGOMm*>^;7Q+Z54Kr7~l2 z!{{w(elQ%j`*9ie11DYjZ-ymN*O|-qY4wMH+@rL*4UuJ<{mV42@VMos7t3oC%ZSPG z)v_m-hIq#Q?W*;5%R%`CkIi2Y<*0O92%tz+s4%%LjVTR=sEYlnGX8tGYMP|W$cAR+ z>KnoMjjpm+cIwb*m;qE}GKpIQ3~Low>^CWdie}yW`&yLC+}O)?$&uCt?z2i!5@)@r z_0h_z%R)~ck_>OJi$M-`F*FVPHZFaa$t&tr;X3RntdIOs)a$swbF8^C8%Lo%$+|p@ zsKSh^wT{i+PS6;`@T?u0s1%1*nTr~`-wlYjaV1-&){hKLS99letWX2LYvdjINA zJT-M)f8XbH9^Ou(o9)hFnE1SVONyzs`I{V>TU0J_`aHl!o%p-+M|mmsCQ_!0o&MOQ zY+4kG_!t@~t0!@|g|*pp-f?oo?ke2hHA#z)zpjc=KBmDLIM?(}4xF?LuE8<)Iho0( zCJ%uy!m$)Zq((+ma5=yw|zb#A{6G;N%4sddI{@WUw0g0&n! zOwl02wxloy)HU*Ea&;4gpWAi{w1<@-ub~@->0PLs;H@t=N!mi{9|}s>UP+I3UHirp zdyR3ITpWt7(eP-G=bsjJM^WYDV7MugcMn3`EZ@>p17kVTBnh-#m zbP=9`6*PFssll6@OceWD8%$(R{Mtx&Krc`Wib<_Z8oaOnbyQLQUnqn@vArrWf55nf z@B8wkZp}e>u+vU4x~Z$Gtf;D}{Z(03U;eAMthz!^ql&hpVo(mO%T}EaAWb)_g#gkd z-^FDEAWib{hzEUyXJs#m(cPS(z)3-Mx>uSZ)q(Wfqd|H3tm@&n2ne`)yPUi`z;qsX z-y`{+WEjODWufr{zYM~*+q7Q>VHtsb5*1iqTSx@HfO=>=P7O+-5qLxtc>rnhrIaAf zUBnY13k|IEL!popC-|;HjIRodEZS}={swNq9OR7<6ds8eBXWkOOX6EVD3%K_2*>o& z^vd~!M#I0X9NNe$%0OT~d&r}ne}J<>8jazwRTt7}?1Z(Inu(i%AMYxN#E(0N}@O+7Oy73>?g= z#c?s5)-801JanKOUbqtU!fxOm-RAIs_$_=~^xZiqgwog8+HbzmrM(fzGq+K#Rue00Ntef><9tx1^e1{A zG|_yKrc#tiC``qQFVdv_MVb^Svf99dS9Q371Rt0P^M971&7dUt4yYx?NY8n~;QgFD zDR3y93}m)I4LvKfKzY%SqrgYf2mCXHnD*~sX>qLuz3=G@6%>53fQnb-jn0g>;bR-1 z;sqV{7nh9x2_-PXJiU83X=fB~68u}HxQ_+tZ#u{7#&3?0*+9UFjVD;WAg+tZ@gxJQ z)nKG*40Muj-Z|CnY9_@rsxy0Unj_u5bzGvjC4uvRvGHIlA31@4{b}bnm)|PV%S#j|yLblyxj9lL*T% z?8^w)xj``HJc3+XR;#0Y;+(t|tVMk0G#AYR6Pw=TlYr8YeDHv7+$vy@Aan&x_+e^W z2~{a1`ai%)atUDi0)+;7=WDc63RI##QT_CEyNH4fc!4!G0WfTR$N+z^gffnUEMF2X zd=C=7-ZFA1Xu-E(Lf@)9c(VT>O=Q$DxsV2>0Me9v@K3}`65}Gyt*wtv`ybMj3Ls6I zHb2*u%nY2_1;_^Wb_hf0I78$4RO0~}m3{zek_M2biZ9YMr{FGb)`CPx9oXl)a+HV! zdM%4z%XHxdAWZ;&Fn~0@iqR*iX0=*i{NoP}3Qm>sCkK!wYQhX(*M;f~YJqPj`Cp`I zyvHcM3P74D{%6vphyz54A3b=qQ~HqkciH1gS;C^GN<{K8=0-ML$}{yivQ<|{dZq|( zGTQexU?# za)3nTxJoADL}TYfP{y5+@mPuJ+`z+VD6|J?;xo&s+~ZWW9vs7voPoX;-d|}zQ7Blz~s4;~>jCsQC*FgG9L7)z(f$@((UsVa^bkkD% zr};)0`E&v=FHGf-Yc>qX-t=U!@l!WD@-KTZkSaeS@-|)P-Te;lAk02Pg7i&qUP;y{ zEZaFSk{QU$SHNq4BT|iSgHlQqb(4|8)B=|6wNafvK=@NV;VpJn$=wMsY_eQp*cnd4>ervx<~Lm`(9sxaGDqZ7 z{erkK`FTKbbu17d7ACE|h2SDsPYz>oK9u_>ZKcc51(WY4f^ouDrjEzJ_Hn(I55JnX zCvD@l6rz5~$d!f2A{5X0{C+C0|8BgbviukJc9P^RTo=jX2FEu>ZwUscT zH0tup+xqd6QPQ*F7l*(P`w0aTVce7?C7D1P$_Gg#6#n_GNJGfx0ckYxGSrJ~c^` zJNr^Gdt}sLNGX{|2Pv-4Lt$+)Tre=dEnk(Na@O9$aqMlz%!g<4W6bxb+&}VQU-^Z` z3Uqqb%x7yf&&o2SrID2}dt|^i1h*?7-sO=cBD)7CKBH&C1FD@o+u>i}bm*?7>S}(D z)Oo)pAhFsy80%5KVEb`dTSVZm_qt@^*j8_|jMwD8ThA!L#OF?}{Osw z*fvAGP+GkCdRN2Ff|HMAD!96|_FE`??^PjUMK`OFTc|kn@171a1#x)rP=w^ z!+Lecb1A)U@}jG4<8oIhYyrnj<*3d#$;A*~OD_M&U!{E+yZeLJPGCrFi>4;%78Z=M<}&a7s|@^C zX-=)&e{52mn8Jeyqr@Ak;T@sr?Z$$ot2DiFt)(MHlV{Fjy)?(E?-$kW1{~+9JHT!m zKA9uia+DvOb*~xVq=HO&=ew^aS;C1Uu+;2ph^bx_bQglIFUhAv+v>>8AC)F+Va3Heh=!REh{w~aYwvXL_3mmek+YIo^6VUx(ks~+_q{$!FzQOR|6)n) zb5I+1iBYOn)TL{@P2Z;Vc{nBB9a|K=7jaG+nevcBZNlJ;m9EITlCUJT;c}((bZ>~A zcFQ=B)_dKncbN+Avb>3)W`uYWrS<(%voN_861v+tgXDZn8cAyy1MA^i+1gs)zc7Td z6!QlMtX6F-W5r>IHPy9-VX?fj)sL}%v}@cg4$TDHXL2`lhJk0G-5E<5RgZ}Aa|Q1j zI}SZIHkHrb5DA}gTeHnvhNAuD5q6%>Q#5`_I(lH)SALB?63_jy>^C)PP~^{2O+WFAKfj~g(H=&akaZ~RlI#H2j}+ZE-Zm*X(^fI1qw^U z%pA3)*was#r*BU(EtYviYCTZXV-~|o4rld?$DxOFo7)#I9=sjyRH*p-^xVdduznFhHGl1r z=tBts=gNa|5{rcgb*8}92S;8re$MDJ)-$WV8W&uF{&~AsvB~ZZKIj&4j7$rVLN^=J zy$44KG`O`|K##LDX?VJ4&E;hXpF{IUrcYOu7r)!QCzZCmq*nP^rDvw=^-8hw;M+=1 z+0<>%c!s`dJxCUx)ty4GI6ga0Aa%WRZvmXpgxa`LB$63a?5U5d)*tM8&7OvGwowCa zCYc5?{WLWELmu9~7X#UfUG12UdIne5EuO;2cSnnLdQz8K9_NCtKa(s*j<(t0{KpYG zQUe=BGp0B(dbqrkIbMSG#xx9g4r3<$_Idr8*++< zqK>>U7uSy(Z=LaW7rQg62=g-ZUl%i_shhdoRf}8)AIr?5GE}_u<;yo_X7ecYnySRy zQ&|?=Z`Ler+pL#AXg5!n%5smz=foEi{1uL{s)jH=6R5-N3a;%N&7;(qj{4)CUX^-+ zVed8@o`8P}yrd2{TN!rdhVx_KgRx_V(UYH^Cz{P@GS3?M5VJMU!Ai$<$TCQG4}vb} zJ3pXiAt`{J*LxW*p403OU}cprd)r?((rxd|QmM%-#FANWpK&L}Ibhx6&q+QwCy$E0 ztZ?EAzllf1KEFcWtq-+A_$|3_>}&C;#OcAi=t(S2+z0NbGzFt=fBz|~n5@S6gX?8{ z8U|u+C6=@KJN_gU4k2^-|o^ zUX@Jy)}IRTn}#XA{n>j)kLOhz?v}4D;rHq{Z8u{pLuls&6k>3-)exq?Evkg)D1$lA z8Hn@vsRagYH|%O!FN=svQKnXDldQWOws~23+M<0Psi0F4s7pud3mPWgBZjAw3`{$0^iMLZ0LH^;7ERsmKlop+g_XZ}RupRBuPKvwDSCi7m zurPdMi}l+~48C9|Dkc%{Rcrq7;KMv<$mKcVLIt<@_wYyP>rRdgu;+wRqs}@zN(SaJ z+xm^?>RO%meEr9!RFkW<(qNt=SHjj-GeMRN47S?}k}Nf*y>Q$0U@mp22WTyBT$?fL zpfb2SKIV+$o3d^X@mNiZ#Rx^Is0+iQ&;FNAX-k?CO+kr>sr_wk-6*d(Y>M$2><&6Q zqIbKf7_>&BhwEdfY~?rJsEeS3o)v4#F|oMOiCXcvu9gKZo&mx;+iNUo`l{-!H71&C zL;hRL%^vrFm`cSku=rJ_0S!7%!Ty)yET-G>rxAh80|Eu?b@l4jN{#Z8$EW2#d%>HK zU>vE7i4n`IW0@|UjzN!^TT=Zqf*=obs4tx^@N^XK6^b4uE6k5Wn{Ve?OpO-ngcKQi z`7WzC)4)IH2dvLe^UN>MS^>R2YsI z=6#ZVw(6<=2uCDA+&Dw4{fdvtxwU)?)vg-U(tLd}dWfeIeV1%wuO+;#=WL+HM3lz4 zs#Q|5y~v-L=J5)lnOn+Ke&3!Bo*rgRBLpkA+AXH&rSQf|^>hwA>kX=hf(`Rx_kgH$ zeJnKB(b)n{yN8U=c_jI1LNF`Cb9dDFzG4I{|TxR9Oo@IEBpV=0V z1&l-?ScsQa+!P4$L2kveOh~L!n|w~`+75Ou9RqPblWCLQjMjyoc~I>=OX5SBdgV`( zU|T^_f7~!eFi|lMIwTh#=g=wY7b@<4f(qF-wrh_Y-N?a}zFr<*`h^iyBZF4)F){rW zel@{4&-1L0dYIwYGv%}p|E20a+)&$PPSXXB!JWxxr@ z|M>FI$$GMNfD47pLeOvO|JBR0=`u4h>Kf2dG3l`}P%#?PGE(Uo=`vE$>Fcx6(&@4? zvC`{P(Q1F`g=wki>DaUZhYHNJbbu!|S|+A{h#BC!Z5<6A{_|snO#W8|$-1{i`o8D4 zt4HdjFbG&t@XLFNi4)fW@v{E5+bDQ`rET&iBB%j%B1=``JX4(MLV?a8Z&OO2-?bM?GXIbC*(RBbPpqv;&} zyu9Gq-(zt+o{c41dp4}t9~UD-pTt;b5U4bF6$Rf$q+WWoA&;aGM_!@<9`~vtROob<)=O*ZEF;kkq>&iTppee zl1!59Kd!pX9dX|6HvIxI8pE7 zuJO8TZt^@1!Fe4ZX|p|buv~g=u5bSGBIf+uvHx^GzwvCmd^p&8{_MOOcD$Ec?QQGD z$#|>7p;?{#!{GdJv?+D=SyZ@nY|X=-o`Q9;3I2JM!s3;zVb$!<@cO`%W%02sZyc(9 z=KgX@;*`PZ!m90ZH!`ulZr-TzdL7YKZ96u2p|e`F>1AoO_ZY;#U9$AC752x_%#+FU zV6XE4YX8#f{aRm|r_0>?(6Y$;b1dt5^k~3g#2w$S;~926@>r(s)-u- zIQx9ax+rx+Tf50Zs?mT&*mx>ZgL^aXG;AspOS2=)5-N@=C7USD>=<0`-fr8E)4!!> z4RQ4rYK-J&rmHe=)&IzRdvWy@lEcj%lHmEggWq#~)jhN&^PE2~F!TI*d%IJkv%1dN zc^txPZDjb$_2z=$8BXRj4A~o zv4P2JGuu|TdRj$t2_<331-|>`?H!#*#5Ay?(7uF{n5!3Di%zzbb$XR51Dj@gI@wZK zC2^#IG=I+uTa4=ujEx%a)~9X*L$9DyV}|)ttZ+BykBjlMi@v}Tvl6vZ38ARc-P<~g z%n5omax^d!D0?8nypEAs5iWJJjm!-LW?M$~sXVOTPL`JV^EfP$$Yp^~I(dS4)EXJE zxi#jNY)c6BK@(IR$At=GxkH$R^d;jZ0tX2KL)QYr@~8O5kN9cyJ&TX=G$t!b>K3ur z-;qC;6I%40CnNfwB^ZOT*O4>D7lZKG z7=m-_nie138TKP_%2yS;z~`&LXL2URC(;D7T2+p_jCv&w(w2hTs8fyt zap!4SU9fJfu{#T-8TE`8fa>}g`ZDy)|R+k^ED!4=N3w4_f%frSN{tCf3;l+JXHH0Ke(xG+;m&6 zE0c;*$b_+LbIH>7Qr05t!IVK`h*s*>?Io0GdAC*D?WR!L7u2hZEG0_1MJkmj>b1Qr z?|;rQWBzl_%$ym$@xhsMX3p>X|Nhqh|D5yx8(w!i5PsKXX@2w}Uv7a##k=*~nKfPu zH|4mjkD2*;_^~%t9v?QjCoEOJl3h1LcTn;U{o`j&%^a3+-*U(JWqkVs%yCcd>_fAR zyH_s0!|`6I!`g6n3V*_%2UeXu_vXa?YR_BMBg(U{XO;w2GvEJ!#_ftfXBhgvJcrqQ zfQM$A`;M=lKhbt*0=IJBDz80@YM(wY4eDIg{nOf(rv67~)HJ%ka6}JzzuA1e_(Bccx1#^Jvs zI3oBTKlQSow&0ET<7o#~8cIhl;S2VF|JmLvv>5pN+~zNKy7<=d&1p-3kN}_6OTPnW z&wL8L=Y4Mv1}$-4Ie0^Se|uxBCb32Q-GQQOHjcx$3&(ymx9vUVsrShWoJsKVOv~$S zSI^EJMPE@Ru$|~p)LMycNO*Z>;NQ*u1u&I3{QB%<@@ErkO2b|YcdI{R&dRN`N&XVf zn;9&6Qt+`Ay4c4~E(iwaI-XLXuPivevi5HS4Z$1X{{}xsIL@8UUT}{caz0BHqb}NK z%Eijc$;nB_lkA@Me70>-{NVxEc46k3OLv4=5G1!TA!jPsU31g#I(FPsr+dXV0?rgi9iu5$ zY(odR&v3%MXwEeg%~9(e2kS3xRN1neU6!9a zl-{k<-aooNd$sZYUC!EZfn^R1c5ULOIaMEO+x}e^KH(3u&07Du=91j8mN`V7oqcas z+pX4N$&Bw=KFXJ_+?cWF*{~m*OB^+Weoe95?)A#vBD0D$bm$<9wVkRVw)uJ^gI2m6 zttyQCK7o_*irL}%>dJtoFX_%x>XUsPgLOCGF74XYzDV!4UwG*^x(@1lZ)IhaoM&e2 z3ESQl+HW3bNJZDYaORWPk`cF}ou2desA!j+Q1-p>VT(}>d+sAo&qKq{ZI3u)_GH76 zajogzPb@vM9piX2H#V~Zh6(UdOfM;|**`!~oz}R?&WEem)=7RD@B5^EC1}i8kiXX%01uE7g3rO5)dBm}VqeeY;DJPp| zkm#`>%hGNsKE*62L@~KzAi1eh<;DbuqufARiWHxecPu34l{7JY4i}B!M546ayK|eK z!71QrW$FmxB9w!<_w<2MXcWpJr}7Xqf*%+h#pQDNAl`&VaKggDuowt768)ZV5zOy} zP6#$!Av6hUeMiGX!4O>x9}STXhsZJ*ocR+P3Qkgz(;+_`Je^1+2P<~OrYsf@ih~S7 z)gl2JUy`aOfC5^g#Z#*aAGDN+Mxv^Exe0a6h?ok3F$>6cgH)vrmU>eF4Lqo7YGKM| ziQtp2GQxte*=QtNP}2y7V2OfY!4`x~M?X(VX|167MZVZta!aAsGJGk=gKEl z(E`vGYU(}th4!%`bb{p1(36X5v5K^>C87g_rL=EYunEZ&tr_83)tM<5DSap+*$@Gz z<&rBbA@BezqkE2}p$+G9*r!3MvJZ@7`UnMMRpn@Xc^WJf4CN(!#7LwzfXhr7fmS?A zE`Z>L8vGyN$(=SY8rW<+K}p|#zX3i{T8FB18tHT3>m^I6;n^DZoxVO*rdYaRHpl!! z7j#}!Cn87?q}}Bv3c({y2MMQp6KfkwQo5IsmBg(Ob`l#fpf9ls5vF@VRuVdiI8LZG zf}KQSflpmY!|=+!4Lqnug5k-@9_+*Zb_%fvg&2|%!agh~eK5ZFis2L2E=(VcPX>SD zLtcWSjHph2o1_qb^tvj&Mfjt+OpSYBEF*y>?xr1z0T3t16n0Zn6H6I4sg>jmiTSEa*kr4z~-~5FN;Uw9mD0_5=6ng!JWw<m#VGECJ$h^f36L1o-OQ@#OCvR4u~ zO^m@0j!|L~?e)b|uic(|1B?OSS1?{pjBHFxl^-K9zCn2lt+e6uUO1Z0adv#5S|yTd zN2bXS2QopDYVit5s9O9vae_99DV4w}UA0D8eZ&crN18(16rR2va7jb8I|Hhk-T*G6 zNL)7d6(fmD5qAb|g1a-}CN$i+^y>7(3jod+q_7il_8V|1$(^w);$XsdYZ+I>du_9` ze?ou?xH7Pg#Ai^+x5g=Ao7g?uU=9N_FkPlJA0pD0_~7VxVc5aO3Y8ww!HG|6PwVm< zR=mM0V~XC#5G;o@x!1`ARLH2G4)ZjKFZfOCN|5G_9RJ4tWxG-_DF$)tc2i2x$Am;i7xvJ(ale%3|Qx|m}mgX=@uhM!LnWuuWw zAkCZXu*fI#X%%{mz&HMh0yH$HVuU6_N#+EQ3vL6HWKR85LB)kQfge;KToIp$dH??a DKm{Lj literal 0 HcmV?d00001 diff --git a/hudi-common/src/test/resources/variant_backward_compat/variant_mor_spark.zip b/hudi-common/src/test/resources/variant_backward_compat/variant_mor_spark.zip new file mode 100644 index 0000000000000000000000000000000000000000..b1ab29842049b245aa7caef00b1f0c4d004888af GIT binary patch literal 88480 zcmb@O1yo(hviEU!cXxLU!GgQHySuv+g1fuB1a~L61a~L6JK=+w_ukx@J9oZothG6; z1$6i6UDfrgf7L#+62Kr(0PkO>QFNNW{O~{DkN|K1ob~L@^sF6qtZeLc9BlRME$EdL zApwA|O-;;h-!3OtXaGQv8$bX65R|w7l6@QN{S^YBKa@_>lXqcL;O$r3AOHXwe>xT& zyAgvSBO50xEeAWJ5iN^83nQ&QhXETcGY2apt1%}N13Lo~ErSjd0}~qqBO?m~Cj%P; z2PZodou!Ql9it8dEfe$GAG5RlvM4`I6SIT2>ko^nP?(YqU_cQX_!9NAd%8VWuSs&S z4*s#SgGNHZY!OT1i!CAHNXPfA3n1cNhFT0W`?TBJ^qIA|<;APznUT@b5Qi@_vy8`N zCad?Eo!5I;L5-J*Yukt2HgM$Y9zaG!qI~j*sqx53rf4cpxS=pmiQJ+%o)@Btcu4kaJJU8Ehz6sT_%X{Q7e|AC06OD^aYOMGMwqBP5z2V*)AZ zh!Uv@_(3_F1l?2ar;Sm%3$-69hvjyV68%BUSSoed!wg9hvYk$O&13I=$P13)W^1-N z_^Mos21uzs7fgXo7bTuF>UAr6m->0e7xSq%u3;CL!KUh)kP1V8%~ueo@l|5iiZ!*< zLUj~Psenmww<;b?nkN@rJxp&5ZuPrt;@wlKthFbY|9uMy@)BM#c{R-|>Gn#lAOFl!0e^-`FiO9so?A;477`CoJKg}ZH0G$Z^c49`h%;3~Z{ zGhX>d*~!|y59-Cw2p`okGj_!ba6*k700SkgIgt%PmuyLYRF`@C-2jo88YhrBw}WuC zXAP`ZnM{$SuE?}*i2JZ-u0|Swj1A1QYk2`C>irC-Nol}P7867!9TpfEV6t$>MAucv zE{sdB;wuLI7B9*_vqyg7F2$lN6)I*=f#G$dh4!;5rixIG>&BzQQ0OK`gTc9<#bmmL z1&0A{8*ZFNO+MfVVbBU8-43CIe`n^148TP@^<1WdCWtRZ;)wU|8HGt1>I>H0>)f)s z(pgSMP3lPhQ(>84tIK`W&oA4b&#amwm2~?*H46@8RZT$-mx!m|t9o2fB_|c@myqas zIdj$vUe+X`H6KBY1OfLVljROH>B*Kzt4ya7(;^(20%xwX0S`eVXaXTxR*1K%<#q!S z?1QnJM#R}TEJ1h}qzhi^5Bmr6bKIi@z7SXfM&`>ENc%od;h6Bunm20^$)82%U?PLH zXdhW`qQ$jn#h+P4Q3LR-9nvuL^+QI@I^TG2L;Z^b`EI^XDb<_aTbl zFO8SZ(a6g7*HeP_Em*x@fAHOZJTm@YkF4k9`rE)d|H{Br|NnvM^lfY$9USfT{$}z2 z(j096(j0Veaoy3((agsBe~#3Dd9#@RwfT(AERDWn^h){m%xr_rUW%DD}&W_&+!J-@}ffk)@-afsK`wnIoOOk)4x~gQJn* zF9)&jTQ~~;`;hb-hrO>Mq6 z^;;}(3PL+r$x$G8I?%_C-M%~M6M`WL-4x*l5w@e%T9l-a&!!-@-tZP1l+NHTu;|QL zpOT>y~G{9(6JVuw_`n!o4k{9PARh%MugzdN43GsXf5He&0Fa48rUNXg03?9mfu9=HIK z7{}3hl)tE@M05uhCUn!B8Xu}H-4LjZQ=~6gvFY9z0Z6z~2$3!NvdU2wxO&MDAqw`b zJ7fn+AgrPfOC2@_zw?sf?D9o#e$t$kBSE%&Rq(l(yIzmI>j!(kJXe=UG=EU*uihhR z`3jZQX*njMjFb&WC62xM-K0r;#I{zR1pHbMbVv-N4()EEIH8W(=ls;eFZswyAZN4u zqEJzH!@_8#QQenKi0w9^`QzV9FcH&REt7HjCsq$%_ z17H%#?eXv*v(J?-X$>IbG_qA9x3_DHEF+e{@q1ag)N5y)%v$PMj*;%|R$v8amY2#L z`z0DFAMv^cm#c6+O0;?oi4AK7S9S?Bn7e~ofhUP{`9NQ;%ZV3-g|L-{NTc-5;+=-Ko zndNU?`4f8I0C}H&&VN^tqpS1PA^cWp7Z0{SV?aqBaxmbUoL6M*fVX#<@2%ZdkG7ASW zj4tF_hw8MS7-bIR2N;L;+F_jN4|ifkR1UmmBFzE&o`fVpA`V7(oaga7VO?3sV^5h! zf?UEVd#6KeWOHir&u}>`=L=m@0`yA$cF?LOVkHoAr8$kZ6H>|T$&QhPH*S~xHA6j_SS@B#=n!;^8y?@pMkg*lRBQ>CsKcfUKFUW5i{LZmc1}MKTeZg>YE* zD1q$lXb3y!`#~1`SGtjyXs#_VsU|O!)j2{m!keEK4sK$>>d4?feL;ZnNfK>z~&GC~Xib=R{C^%)b36Nj&@(=54dHl8Z#~1JIkti8#US6G- zm!C=xH==U#VYNEGEmk`c8Y%kSzA2j2CWh)l)LDx()|eV6^2+oFi{X`?nlpQKmc}=h z+U()!+S6C=HHt&#ZQ5qS3KNFx&31%U#f-?^fj+N1d%Vx$l=J8|3Oi$3@}i$*c5b7X z!YQ}cJTPz}s_-P^7wB|oX+Tf_lRj&r7>{XQ1ut$GE+p8OM}&kc8M7EWjV_cAbz&4} zq0b;W{j5czo34T!n|1WsIGsNGyzRa8Jqny3D zKA=dSUvK;s7OB>Jy^w&Z24Wmrg?X!8a_O*?wuX%~4o;D!WV`)V;6`79Om_;r?^l_w z1ZW+=WwTc6AM1`*a8BuW9%m*`4|m6TdyA>JFix1PK&`-ZkIilcR}%Qwx#L}hqf_jT z@XePmv;}a%T8Ayk_&_3XTg9)e=P|TW6c*T`U%7Q11;DX~Yb0<9da8r`IQ+Cfv}3tm zBW@=f9L^qa2XiWNZuwP*L!=*Si-OpS4hePKF9?xfUTN3DWmm3 zYq9!V2u(wxF8r9}A&5rtIR4@okl)R@8%l9Lz8ecC9xj*I)yptHQZ@?Mf92aiO?4Ls zr^w+-%qAf3`c+vvD?n5%2V#q7s^!cBu$nQK;f~K4qJAQ7%Z1V)X9CemBkkfhqqy2BFZ8kIx;L5qcmcfY4=Q3Vc%&}uVOQu z71nTS&~r$9fo4+m{5i@l8a7Q^q-u z_WbVSZ^%>m@c@^FW1yP0!NPDGUosN+<2pR#3a!ichV<(Y5pny~hzSmX1rMbO81GtF zZnVR|*L|L!o_Fpx8jPBh(_JB_UB zp)ehmNAsmlMp7BGjED`4?F)^J#{5&5uX2q--HlXXnKNpjevDa4&lr%UZsopsk^bz9 zo3_TADk>T9k{A7^vvV8A7GART(G^DvqVr=aMGbC^hATQ__y;-%xm4EL2o2+aG(C}T zHH>&n$|)DgtBl*#xj*%@y|pDJzCVnJyTsT@9dxAR+!xP!<4#Ydy#QwkT3t9Jxv8rN zAIS?9lMk1ER5w36>;XXsZk3~VqI9ydB$oe@?X@>!g98n*cO>l7(sx|rKJJ&>g;@`n#ZIB(*bp&5{@h>35Px=D)PFTdyZb#D{CPr5{UF?@yJ#%?NayI(A@Up>q?@k@vw)55| zL6 zn;W`YfL^KG*;rO1b@Y$1-k!TQ~LE`;Lcm_B42b#_rD_C|S8dH7NQKpxAITJ4OGSxYTTIr6>U z3aNnQ;99XGsYD~qBUm@_vdyV`2~STsA(NYM+aduqb9cZI?Ie+*JfKSX`Lp@@Pke5T z^KIaz8b70%#HM#x*FZ>riuq{q&+D+4Z?E!&0v|#>a{nl*gwih`C+A zoM-UVG@doN-IdV&9o8*r&()!lTYPPtHMA!l@%mGOnOVuH*d9I|u#Bez+2Mosy+RU= zZd<#!#6LZ69TlzX97Nr%pXqqDzMf0G!}@7S;)vNqE~C2jrIfeAS)9yL1+L1J?WlOW zX0ehK&GFssm_U`W3Y15Et(x-@X`XpJl(6)Sgm8&|@?M7=JC$&Od7=T5RXfMF$jNSw zlVye)zJ^uVY%LtV^xwkzA0?r`)-(Svg*P>GaI~>^`~T_$zlo+#!WulnZy}lP-=uLi zc1G6Uhh%m(j=!hzf1on|M&p7@kdWiVBHt79kLKNRVC2jUicbdgN~zfGG;AVqambgF z-T4G?IL^?|p*d^?1hxj?H%8$%gHQ6c8s1pk!xZVsYwfP)<q+a|1*NUE-+lK_5^4mOHO$f_?8#I74W*S z+wMXRTVGFg2tfl+yR$p%CL#2GT`*9|$CkuSSAeSqI@78=(9L2T zmUqS2e_VFj^WAdedxA}PG6Sc=X+y#$r2U{t&Amk+FHhlrFI|7?w8hWHkAj9bgjYD{ zrv3~)oP2f}lNPUyg|baoJw78Jll%P>zHUbTsJ7%2d9)kRL#*njdz=Hgs{QW#!qJE& zP`r4vuMLKSGt-XNW~wCTt4++IstxsuKcI@Nm9I(tB^K8;?i9I0kHMqQ(d=XS_l*64 z>6H1Hvupv=nS0~Pytdic)XrzYtkiw;x6w*L0tC*KZEcUmV>AZ%Jz|Km*#@C5JrpE; zX}{rC67N)?c=oBfhM#OQSTR^UesZMjM~GH)0Ho&tt^vNcFWT+TKhut< z6#b5)W?4%(#W5c4R}76Q>jLz-d41O!o>_;NXZ`u*YP-tECkrj}#qC=4siVLQQ;NHo zua|A4wLr68zelr1gT-OBRB8M~QNp;SQGn9#&@9{UTXbGRn|2bWQEkq;xmW@r&w zd6x0i#+3Bq{%2l8^-|x&JqWf;zE0?fAEw0k=*5|je&YRhxh(vhE+Rgs0abgs5?>m% z$8X3C4<20*BcojL;z{Z%)`PRi^p zAnz2Lf(7>s`N-1+>LPrjm3)_DRMy1Qm@XKpv@i?LftOx|l-~>StEyxWz%lfwkg3!eO@P zPYA|YA7E)pz|sJ_?4d%oh>PuZ$?iPvb2$*qJjh8ushze;e@AFz2k%DBJX5Nk)Rmv+ z@Nafmy2}ietzZmlC!f}v?zHdE5-r%0r8mP-dyK7NyjQ7|FQDUdjo)Y(cjUja{5H*aP;UU?fd&+CA>&iR|nMzjQSE?S{L_H$U9 z1tEl=Q2dT<6|2{u^nv_PUDUgSD}0n<{6Cb}+oDxrcud&E7glyZk3lmg#uA2FS@_MX zcMn6~>)DJvsjcU6D9D#`HY~6>EpSyikS7jULD^*L@leV`*cd4 zd3U@Vt>U_A_iDnPEkrV&^z(MMk@vq`x~ylgw->d`1X%mrM?d2KCTsP+nf!mMNB^D4 z|6bPe{&SoEZ4G~n)_*OyTNyd(8R|LyjhavQSEZIes`-D70{=HS-9K5b{uKq@cM%t4 zhb-x>lnnfa10ViUnESg){=A_2zHjf>AM$W3`6=@^5#8%d1>+G%FN8%3%>5k8=7(wG zbTiDYzI6q*hT<)2Z2D&BdIYjlvLXcW!-|8WW2JQ#t&?fHFghnkZdh@|-i0UUmpHZB zq3xR_*KZ0d^z&u2mlI~oZ;oY8R@`ERbL2Xe)66T}JNPT&Iq@a9U`(ZWTjf*tGsr7i*L;2VTjb5zQSNgIg!i3auUMn4Q{`YhnLjXp@dMwVMW--Ce4JQ zVa3;r>__Rce*J0qfMQugA>dQRgjomC>g)XdCb&Ivf+(H6dEudt-FAqf3eFJw-2Un0 zg7WG55|%O^zN+)`#k%z374_wB`XT;k&W!(>G5jY5%D--7_&d9!`}ZrB-2dX6dY|b( zE@6uNXOsTtHO)`|;R61%BIjRMRR8|&{zp|$mVfQ7{$CFDi-hOZz2@>xYt3h1 zf{Qc4GlP&rFEj72IG9vU;8wP}Otd92#OLCG0~?-ipDd{4LA;dGGa@iufB6V*oJk)D z0htLo5ZWRDN=V^~6G03yibD8gGFNgN$ULF6jo z8T(03!7uD)RIw{!0=d+WnIDFThXW)vyD+64j8ZQjoCPq=r&IHlW>Q{=N8o}Q!fTVCCa?x$D*SXQ{m7Ur?f8JOy^eaIH1>u-HEb; zrx56et)nxY?V7ujNp21}5Gjn#hzKbh_aOhfmC-w>na*pUQ#bdq;CA-ZJVQuvRIdao z(1|J*a!{XW;);0p?TgUz9Ik}cpykSJPLCwo-x8-k+v>p73rN_LBgIjeTLvU!XB zFRfSTzxSY)1A+8>NB{s_MgRbV|Aq(snMT6G_P06KFK5nsYV{{&i&d-<>!X2Lv{NdI zmV?}(+%JA?UVMq9+|6cfSJa_EMb5i}TKT8_p*U&|Qud)KIfjF_awuAk^Ur*DFTvhI zyx9l%C_=uuUnsHgbwvcyW06r6Z)<`nv7exespbIy=TOgm{J}OeYm!Hl977B zsNLgNU=k5+Tr>y!K6sV|y=m`E)sa7Yju?P3L$B~s24rn$xz=(WiN z4-k^>R%wpsc@r-SQ}SKY0a5cad#r_XNr^Im8R&f}<|jOYLZAnhzz~B&A()|jUIoe@ z#2)~Fyl?O99~6WOkYJk!-A_Fc;}btL1?&sgxj0HLU&w%uNjz;nG|W0KMhjTq@aNLz` z;X_mA$&wo?=TD<&vS*LKl;-Nnx9sTDX;q?Z%2CNzA*`4jk8l*+@bl{`M#VRLD8$*ia@6ZV7atal*#sq5of&%EkF(=wNqc5O^ zayr5~&9GPH^hwXVX{zU~;Sc)A_362dirJ)t5KO=|989S}7hZi@61YRNu}$^RBW9F^ z?mG?*8!fddT2&f95@Fo;QIOE9+S3lNwa1Y&@B0(<*T<>#*Ngpg<(H?bPHWGrG3V{s zkjQf)Ue_`PnKpBT`xb+bXO0^a(nS|fCp3N59%vgU+9qdCMawV6S7!T8`#0pPQOmL! zORJ3;8SUfb+)o~zmMqW&5%5?V=dW2mTVY@KKABotu@=4(^7VR}!>DYu1^Li;kI46R zUUE4fIUXz99uA~`q@6*;WXJNkmx+jkkgeQ)Yxd+iE#R{D^38}dXZgtTO4i|N(s9yI z$2-1TZNho%GVJI98m`Uy^MUpd?P{1ZNJhWFyOpFhT%a?V%24#BOZxEjY1>)qLc;zN z?~`MW+$t)ZFAkDK5(SqrU5YKw7T^@IFw*c&7@Xw198LM%F(XgAM8?uv??j<5hnwmY z11AC_OV9mhh8k33E|Z($Z1?hX`ZB36qz&jWLM4zW-`h&=?Iuii2kn!Gq6CbC_J-kx zXAYy4gA~0m8G8^jr~_n#6=MJm0PPGkMV;%vO-*v4`4xVU4Gh4WW^YcK^b2JXGf@^Y z-M|v2!Ui)1#DQh%=?y3LtPV4-UE7&$7>gE1<>lwa4g$YN;q5Vy#Zo-qb97xKPW3Mc+ zMV^T*agBNIJ1pXl&=)B&5~LrAaedDRJ&xewvdu>#7^UAN1xzP}*WP6W?+R7h5$K8* zD?7nFg*vE-V-q6)-bZPoJ#S}e9{%Hg{;dJ4Bo^Xpf4FYWmbgRFhi8Oqm9P1AIxPu| z$Bdf@Bvzv=n06+H+j#`AIeG_nN4L#h6kBcCj&r*7WcZR9?8#B+ck3|`2oos7;twCMK(x9%1wobr1A}erL&=GNjBl7)z^su4 z)Q4x~Q@yr;LO{FO>=h-oN>QfvlHD<_Va@OwbA7zqz zz1mKS6TSmk%Ptfxf{__Jp@uY}0(isovsov?fWMs~elj+91f6?lj5Q*(m2BneP|{tBK9@$5v3Q>)*En}cG}o~7 z!pZ`TD-SaZ4wy4n_h(V29*XM(2HkrGJA?x|q3FmNDkwohyUS|fuf%CJ4vTQBkt7?$ zyq6Ucv1WqP8oj4ntvhSBzp;&`Xzd7Bah zu_TX^hf%^Rt&+Tox~Mz2F*9|MZW)GoHyh(KMl1B=)xIC~Mk}2jCG#KBBuc0xdx}C zN1sU(f%bDq#R|VmI7cuGD3Khcp-_pCDOg}0W)(-KNyyO(7@1L6mW8E-v$89*!NI&J zu`9n&es0!_vv$z2-JCHI)Li7>60%Z&a|SkyZMF&ogFyxZQeNop@Vw*1sUy~Y7@mMR zLe!0($BzNjU94!*D!tSE&=8Oxsi|`(9eU%I_Dsg{m3LuK!FE*R99YRPD=?jNF6pHX z??FwwLpj`$jO?CuH5!}z8HY1@#4nI^+FKxUf?I}i!J&pag;hk-+s?vO|FQ!MI43Mv zbWJx*<-k?k;m0_QMf1bzDY0KOCRCSd>a|RrWWq1{3;xe>S(cg?Rp>jnsr9qE*$GxbQKKxfik8ozJvdv(gJ` z@u@22mNhV-r^SFs&G#z-oUkOIap+-%25Xh|)w<5uQs4NXtb-2qv69W}vIbfb=TCE4 zk(a-I3z7{ppjpC0BqI{$BI9bYvj3Q%f-2Z-)LqK}8emLM;z1XzR^r1&n=PP4VY%QP zFc8y=6a`z*K9mAxgES&$xgWr}-z$Av!IslR=cQ-Ic2@96LQyQdh2o)tsk5*mh8R?+ z%3R7ZB!kNlP%7oVA#v`}Ld%Dc#_)mgG_Ts-d6XT(9VFOsZ@pw7cC}8#0;-xWX(o@! zp^l3;kxnXH{-H^EJ-Udg(Mk1Ie93v=$Yr`LRf^Pqj|LnAX$8NB7-qcO$^ypiscqKu zdFOHK@~h9yZLW{Iot-<=llA1UZ7#p&g@5d_`|sq1e@wxCuTTC(zwDnc>fg-^8UFC~ zyVjexXITEGf%YHN34Tx79lYFGxNoElM*r`V_8*qS@Q3HB*cli&|CF@f7x)KZ@}0D| zho*>kqCq_1>@5$XEdkUL8NU%7rCGVv2((%BR&gF<1)wd~x=T4DuS$K3+*}5FNJw`W zrNob+A7a!&*oiMaB@_CLQWBB?DhwP>TC1&fMs=xw-`hG@b@1t1CENf_p^wk&G}{EE z1%5p$bUxH_Mng8^&!6hU(RB)#gDalBFt8;#KP#n8eF@}w94S5V>rOdf-R6-ZjfUPj z#O=_6TD-1TC4?DshHc*-z>2{~VLx7y%>$fSyswL&hqA8QMuIk-x^Y z2JmZgZCC%9E?4l07_WxENfX2h1s&dYJ@XixD@V)cN3qvQN86gq5+U{!3)2blC!Z4( zILm=n;>v|SH4_t>Pgoi+2@&7lqQ+d_ty&*jf4fxI>cU%%kcFjohA(meDKH$MlfCwd&$N&#wr3ce^RwZ36Xnkm zMJg#tVPiom6{fIY7S|O;6iM;<=Lt&!w(B*~k1LD?-4DY}n#o|_zf6hN*-M-4aOJ1w zV4sXG<{4QpsbVSXwGA~2M-#xzQE5p#Ijzj=Mw-1E&X-0ubEGloS-WZ)Pai-sdMyEI z6!ZY-2LzXuz07{FZr)f1Xoo9akYr0DMG5e11oc}6iCf_UIaQ$qV&iBK?X1%m5JWjm zqkRW#t514%wKF{_jTqEf7e8;D6pXv=go`LC}-aQ z8<*S^@vPdH1i7#{0}^51YRv1kNB{F| z#az|N`U{@ttw%-8i<4~IZB3)k{qE+^r?t(&wXGtHO$}ZV0MQ0p2S`%nG$abv1DXiT#w2-? z)(4ZyiT0PCh7-*}bG4F;wO@>H;kG^rf)m&q27PAMYePe8A$}<*6tu)azA5cz;X)u* zD*=Lavm)!LI+FI5bW3@rFqAW`TxDPx0dU*dA5@_v#+hnvxHpt!#bhEAJ+SM}!3OS{ z=1S<1rDNH&g+pDwCvv1v5WH;ugiIwcB8t8W!0LOV!|!@vU<9MiD9KvZrriiC|Iqzp zYT~g$xYl1~iwy@W5}K%Zk&9fhysK77E~FeP0~8v!RZf{hJddj zext}eux1V!i5uh#Jci86HkCLxDS?O}I?`I(Z|f|;g56LUuN}6h%40!mC|jNbhR&z@ zz)}@aj-T=r;^A+hU1@Kky>bg7oOo0pHIjlMszyb@pYarojrb> z@_1|~;M*45)y3g0x82$UKj4MH(h&6`ijD7RUuzAt2{EAy(lzxfDh@5lO7ssv*EL3* z=f>0(sQIdRsShb}j8O=qT@s+U5HmC+gPP~JK>CCgsHZsMN_CE^lg@;hwDI+qUZ`R% z{*S^C4bYIZo9>8DOrXYNOu=Z@=mK2BH}XpvQ&8Rhy4;Yc8cF6tj+}*R!b$PY1Uv3y z78NoJv`#bp2vGV*e&8&RV!53YKzQ52t#b0h<5*Kr^`z_nl(RaXlh%ZRvw_a?~0~(XQD~ioh6~$ zuHK|7i_XF;RE=XW(xy1vHKt5KPKtQVvv8t5#ZSg=?o8c(&fsE=rR5ZoVG|~9HL2Ok zsI!J1_CtSDodWU857#x(g70#udJU-KuR4zroxBS>`fI3hB{YRv@YF1lJk?)C{8Y-d7n#g8EbR2?K;_vwh38f9T-|VG(~I&3mO3LW znQDfDdY5|A2~-*wucrE4@gfU~+1hzX4JY8uT=WZr@rKiI1yVDIukQ7SE+UpR4aUM4 z446p1h%QU&e9BCdPsC$r?PcCFTJ4@qBM3>SsE98DL7EG5Mf%+CWCVqltnqL#hlop{ z@x~s893@{kzGxt%2Is8aO${;od@1M>+7gY$yb06V_LpV-_DC0!N~};^4>OU4i_4x= zs~xyBExV+gz>cY5U7n7w`j}>2UTI8{*do<2=G7ZCfr`C342MLY$+o}2#>bY@Dr`E$ zphut^fwuy!tRtA5alSOqg{B$>7~px~!309ni0UG7!yl^0J$g8Fa*{2RKGwS9VRuZ| zw9Nblx3+%fmzgB#pyP=u`@okEZJxMMj&w!$%*}FPty5bcl z&teT$qvC=)9_1>FM*J>SE8g-l@O_H~W>^N}$lpKojL+y;?IQC0QTg&kcn5W_V~ZC)byvyjq!V9@xdI@VfxZ`pFT7I@N4#j=6g!UJucdH@G}_5 z8dg82lqMa=@Ng^nIya{V4fn*&?shm@dBvCRS3Cy0QAucoIN^)6p0*vPTo;gmr$!wr;naDN5hPC;uTGV?x*n9jeiVCo)ny9q?sI8r$rJZ?$t^@S&7Zvya7ELnz zEvNmSCjUVt`L_!EAG-klUF`lln*80GevjSni~5sRnBODrrs<^K0t5hn8{K~oasSW* z&(6g1PdD7Hs&hwe47 z9G*bCz^RD9>d?GmyY$0Y<5f z&3Iu-V77+|MaoAQcwc_a=XUK^P1y?JH^%jDQU`rwTmx5Yn_HfY>CD$3O%B^XlX)Gv ze_`AV1ClHkXmqx?UPcwm!!`#7M52g_ms07&Pe`U|ViRh84F&BI0M?coo5EOFYFjh= zxR{z0q_b!^_T@VGNWjVy5}bId$2J~vy62!EJqfv~C^1m3p5v?PSbkm;jXX=42I!!oMK27xHfNbG zPgPJafGaVE$P4h}Q3suwo34)BFlvSk1JSFN`(a;b5!>NsEBRCu!sayE-yaX@FtS}5 ztxVE)Z3b?h00kUKi&j~68^B~$|4z7&#S7BxzY;ERKqpQ2t_7EB;>9&!?z|q^#Gcv)AW-o!8sPI-QrtqP(o1myZXi&)kor zuzar%7fuq=-z0e{K5AJ2K^XyvNQZ7quw>Lmk2a;)@{b0_gfll62&RpeJsigH>cKr zAfyybT4r>AapF4W6|53 zFTJINon+|SDgv}=;a$2M#d?bn&=Z8iMS;cQb@Z5gi)5R?LAab8qm}ZlW=Ve4*2o+l z7YI_^ePh~_RO1NUZUgXlI>l0Yqf;%Bq*G3leQ(24`+b#B897FQgbUiVE?K- zv5QrHt^vdU(&@m9^v0eFBP^Y&j=XyIlHxJeFLdh4Vu=;@!REnLU%I;u){<8mBU`#z zi3r_>5ThOD9Aa!wIo!+;EP&XSm|+>pEbI>REhjf}WUUH5N`k+`+#GHviq$EJhF*UB zZYe?!(idM5JdGRR9_S62w}^*neMmGujTT?1XL{$wa@liNgVH39BT>7dU1y;wUrMO9 zzF#D6300rC`BGQ5y(iCBHmQn5wOo2gW-u6$(CfF`c`{&Lty_b{_Q2*61)T->E1a&e zCH)PYZghijH`%0i_%m?tM(D`Xe9i;_eb_$xhETksyh1zsL>PR+P)6+H^5U@Nxkl`R zMt%x@=W2&@h4#$H`oM=Y4zxegZItDyJy&(~nQj;6HO*$I0@*tR>3GCXlH}5x|ML|) z{X>mgJ+&HJh)HJ$D-q|DVuW8&Y4FfTm|y5rzNKzN-P-4kP7&Vdlp#^yhQJ1(W2Q?{ z(sqrQyeSXHa(PqbA}f;h?))Y@B$K^i{Pd9f0%D0T4OVkPTMU%vlCh35V2QR3toJ_QE10#mTtJRujf(bLIz z*!&IUV!8%DGq$IoP7AM-N_FNssVGf}3Qqm0n@|Qs#>?^CZLcUz+Ur>*3YXQIt&}kS zFZ3i&ggxbRyk{+4LBX;M)i1%=MSvq8*yK-z#(~-}6PYcOx-~C1Tek)U? zM$Y15HdvX8-kJ}7292ryFP-|l(<$DM0Q&B;H#&`K3PcopNG2PrGic^#KxQzke#|JH zf2UIgU#DjEz~Sk%*;&<=8jONNx1x@mQ6rL1i*^BME)acgsyTnb+eIkROqeuCf; za(rp9V%+I{o}HSqkv^k)z1Dh7OZ(D&;xZrb<>=_k)$SSHuOFuIYbWD-V~*m7hmef_ zW(EKK)0uw%lmjO#=ilp!|KM4qRV;}&U2zu8eRhZ|S;!aVt0ceiRI>TG`Xb|iv}3ULU0TtH1+{2x^U6Rqku>R=n%1sh6#72qVMd5PYvt{ zU#Nk!{@U)|xwW{_>%6`1NF$`dNxV06ro+Q_BRG`$B=rr4aMidBQx?C|hca+vG3MN> zvQs~`MRDf71ga!}Kwzzb(i5-dmcrg`8R0f)$gT`#f;h_#-rLjdLbB?Y z8)%u>V;?6Otp<5d*y)c$iqe#4j0CaVTAHH{WGcdY+`n>e#YR-PuniD}dKz@*#-(Ng zteAf{*sD>ps}g`&K7#rHzMmu>fF<1y=;<5sv&n}}t zJc(5c^^P>zV2O9-Yl#<=bO3 zGo_>H55X7^n~YeBi~g$?V$87j_N$lrC1+|(GflD%)eB$Cv(?(H!NdY3<(ZKQ#;;nQ z_RNl+@Kax(2k)yc2Rok+ZC*KvJ1sr$R-L!|$$u%}SG^VRP5-ulf8va$-^c@f?p;-P zS2%Ohaq79|Hskj6VB%p_EPZnNG(Ns|bUI$vXoEEdB$_w1;Q#7#f5-NmhMZGVKb!nB z1OWD`gi+J)3higjeP=z&MO9nmDi>}mYI+M}(ZsN$CuEy8+s|i%|E=IlIIXyU{-u1+ z_*TAu*8Sh|J-UOdM&UBF?se|PollN6V^zBRJ&E#RlteAH%24!UO2+rsr=pY8t)!h$ z@7o^&c`ej90GtGgbaIZPnoOJSeZaXQ;Uw|jNO6-Z^Hf#O76jehlep>^%d?~a-wK|& zdiDhRrhym8divC(j^l^?{5PtcI?8Ei5|;EBVe-fn#cm20+nE!?QM(K=_(7vkV+pwN zMYDtzP$iE{`hmn;DqvaR#YDgZKwAS%(Z}X-c{xrbKvHhm5a2xdHum&6z=&qCp3+tG!X+ss4=bEiD9 zPv!%A;sNspU;;b{VJL1=B6L3v42YTFkZiH8klD42Y0`b(+dvN>t#W(lJI2gK6}BIyJSn2-7tC&f{)9bgw|l~aGcz%hPDg5k z1%n@L3A&}8SBS+X3vcRwL zWd$rxeV}mA{#HB1$t{YMxnqnsOiTG+0SS{Vw5bhxR&>8p`peojST_r{XYE|$Glx96 zF3VAufUaa$N7cbfjvg|>*--;LE#v4~~jl1{5T6c#BWmwV2#VCzB_lKp!uO z*IP(Z*hpZl)zUUr!=Pb~ul0hFnEer2gUwXvi$W3t*sB>P27}Yla}6n9s!xq2x92@ zuF$BWytw*CxdlP_jk;`g_E4cCvs$<}WdKzaBnlh;{hUIWw5_{1({|xXLD?-NVL&Rc>&F zA1*h!_|sceK3y~axa|7+k#u<~4q!aKB)G?)pGTn6cSnR%Q)`TOWiMb`V_gr$5?9L3+`c}@VfB8kD7db9!76U?w<liCs z`@#a%6zgdpk14PeTQ$%BqwSrdEA8^d>)5tgv2EM7QL$MuDz=S^?TT5kZCC81V&|WF z{dV^`r_cHNyVw`|YCj_xYmD{GHRo@65Tbq^YV9qA(F9*8=SDs3Kf*Wtxq+06BayQ@ zEOqH#&!25mGz$MnZYKsCGuXF}H)>8mkPfB7(9 z-{yi|k-AtYGajgjV>EU9a|1ceFg0ZzUsuvriZM(BofJrc!R$e}luvo!DH_Q>!-Jz(v;@uRCW&iO6iq(G|1#ihyTeXJV~oG?-pg zA-avOwvSHsGIF>75`T3SzrE!8`5$+V-p49ms{W()^#3(p|7lO7MV1za)0UEl(^8&O zB7&SoeMBlHVu9M4q9sDnIf0gDgtz1tB}D9;{HXR4bheUxMCW7zlMs}Q=^r0>n(BDg z%(So%m%Ns4x{e4_K;Hii0Q33vVzW#kz`HLQ0|N6AG#i*{BKvoHn*ZINPKoG81ThJV zc_BJ@0z?Cs5D!aA2}LN$S|k?lS*+6Lk^aM;PW;23PW+$k=^C`kBm^pnPkPZSqc-{w zeM4no8`8G@nf~~O@JOZ4VSdrjj*w;FUI$6wX4aOpAhPINy~ERYlyYj0bN0ojP*=U@iV+knLp<|MMmLI6Z~lz)eT^X` z+Sb0hx9$wI?!*1XIw4aDQ|n=wm6S7)6@*fo1NI9Lklp%a^NF-;EeBd4Yz;mw=LIGK zYQcjl;njTv*i?Ss*5J%i?F#&`RX@2k+CE3?Sp66FH1*%?>6(AAr!$O9xn2HhPq!)k zW>44sVo%%U?5CJ0i3g5v5C6@c-d+`Hd_J3)YP{pqU3+tXc)pQ4fAUM_`$eCQde^5v zHD}y)kKQ@_sZTeiZTBK&BwRZy4!e0DPN_7^@R0RiV)ge}MKlXJ#)VwjK;mg#9PLQD zSn%fk)Td`E*~%j4o!b>E7RvUcUHAVoW9lEvq5ow@_D2=*zwg-DuQ1!|{*w#$pPImb zM{oH#9}Y_Zg@kG(Y_%X%yD6jrEC972rxyW~A_{Ra6a`u%c?~*(CwlIL`clYK;F0<8 z3eo%uQsD^YA~+dh3vE(W-l@x|%x=c+TJ>yeIL z{=vzKC!7>FZ(?Eb3t@WESqe%Bfp2EaQ3Ui#0Vk*e8+SE}a)OFqd!+-!Z+dA3u=>3p zE)7O8{FH?ezfJUkOQ1BM6su22gK9r==U}lT_7Y5g2L1wRkv3!*gLHQ7OR7I}j((gC zqFG!lMLyrNi)2Uwd46c0zQh9ah234{R_k!AI`dBGL{KnMkM+X9o6=xz1Gl+XpylV- z31pK6Tkm*JN@f?9uVE-UHR!RlY)-J10YDbt5?g5?Efk>+yXZ@9`e`k@jy$Q{Wwf3u zLoNIl9);`zi#$-Hm=EG+qrPNk-nNe$B|u9PIufB$`eUm-hsH35&ZhLo7Eq9%^4o1w z_N6F>uNq}?snVgg-;o>XcwW7i4_F5pjo905Mdgvpei$;Cr{#->wTwC|ShXNok*$K_ z2w>Rtl}?AMA{zuvH7~=JzmyLIYWo3W;2K(xFgA_+BLP&f+I8LGtzK6?C!Rfx_-0f9 zXN5G*kfF=x4FpXg5kF^6j8B7{4Aca^yjqeXVA&UGEmr@ivd$+6cUfj96N?z-Xb2Ws z(EoNBcmN@QcwbuIrA1HEjkrN&% zPhL+q+pT+%d9e4NdZ@o8P5-pE|LuqRb8to* z$Xs*Qf0Qx(^KdI>j=%lw{uplcFXr0)=B}abF+;>?(Ffyj3dR@%Y9w*j#p`T5FP8(@ zTHUh*hccPP(v%E*NI$TVyu|`@7z95d#E-Qd#0MFK-$C1nb41ud+KDN(i6asTo9!6} zVhYcU?R8gkZcj6PiC&4spQ=Hz-i=6+d%J$DDZv+8uNX;f3YOg9*+t* zcYJ}JnJq}`ms1hP1wnLJEWB{*UCtZawH8`Lr@aEO>q^}(-Ftl+2rv{c*h>rFlx&OF zu9K3f2NI|1ditLk)8_AtNgrY&(ggo7rC~e!Ha+>aXU4khuof_qdu>(hx=2;jfM*%U zeyD$*3)=`$xT8jINeu51X zSae*8p^d(#vox1~lZqXpELTgfxdsLyE(US)Nz(fqG;Am?zq*L^9M_LDLLOkN~tD;u4(rBn}5 zgz1AS#8%U|V!=pSW!MwaD8nhf&GIZSylg-9q}nPSV@V`P_Ii~cO&&k6VPxP)*^9_; zjLGS*jA`Ll#zgx!#$@}?jOp|@#&iJvzhO+b=EpDiyiY@XQSWhn)%x>8G~e@KStoPH zlTGW7twiFsL3m#BQDizSkdCvI2`=3Gx*79l8!qUYl`PQv>V+kbl|z>er;d}hQ%}I` z<)4uV{CgzQ&c;>6=d)sl#*Y=*kM{hw;bYy?$$^Nar4@PT12IXTjrAFo-S#_v_#2ZH z;@j}?V#Q32*0&OD$H$MUQFML_BmEn%I(fuE3CB$c`--@Yx`zAtSdnUb8@Mw}x;xN+{*myLtIY?|G5vL|FaMI^4C5@_}}hBc>mgmApNxu>G*g05YhKO{9pDVz<=pO?*8jO#Q4AJLl*wWK7>2(y$_lAvk%FM9C!Vn`w)Pz zH|_tj5ApuB4}tnyAJTF8vkw9Oy$>1qzw{wOXY%JB94`Z--+9+vW+g2E*~m|}t$>m1 zTEc`@p#VCA39#8lf?gJW??bBop$~b>vE!!MxtUYKv=BU{JS0d~m>{sy9RNjoE&-Si z$3+LWfFHZFSBI@UTcm;#B)GL> z5NRgHYP4UA+kz|6CFZ@OmHc5QDF0i`|Dq3BPyXeI^4C73-G}w}J|y_a#~>QWPx z__Zs~j*9ibKZy{g0-;lnQw;0VjJWv82ytttT@)`e8=LF?_%4p`??yG)fz1%YX|5K^ zy%^YoaRm@PX>8NS6xpk92VwzYvci8*&I^1*!HTUj=ZdjA5}XcVf2NB#HIY98JoA`f zB_;>Nu2hHx1r2rs@d6~u#KUN2A(!!O3_?a{;pC|poyjMvpi9F{B97uFPG>tp9(CxE zG;la}XjU`mk*xInkWx8ZUIPnirF4tbXe=@@5?IJ*E^@9f;eCdyL}T+r(?zpB@IUk+ zkAL_eLWu@7PWclwb0f3pvPDF_i%D#uj8*(rj{5SSz7xgfdc zSwkZTk-^A9a8*#};o8Fn;Qi*W;>{q zy245AL1Nu{Rm*j#GFFHr;Ftsq19=Aj9TCiQzJVpE&1>0$+1u00+s(oL=KJ&3&hzum zr`xqJf9*s5@eTREA8P+ka|3A2KE98&H~eVe^gqsc``7o7(EocuvyrWpv$c($qmzNt zZxikRn~4a2Zi+QQhbe540001a{%zasZ@)zQ{*C6BAEkMJi(vT2OLBhMxBq%c4g<~m zH99m*j2wTY)BoZ%;#3n-9eLpv97Wh3opO^^vK^kL0$f+JZXD$8Y6Lh?H03g%e>Ec&&A_0izg4=!nhX&^Fc}SD+c(Hm;Hij69&})B*wZCBhu=_f&*6f zF`Av)bx$e*2;My@0}{G#P__J%uNVTJc%BfsSTIZy6b!ja2!y|IfCGf{bcqTpAvF+P zkG>xWA&T%B<}sW;AT3&mYp4W}P8mT5KoN`Sjw^W8&ZUwdB~&DmO9%iVqW8)w01i3< z{d{4Ex<5ZT3ogmQgFL-7KOls^falU`aiRiGTJv0R1G{K9xhOyRq!4ug1-*bB04-sT zsvROi#E-*3h(0d~4PSn7k?+)yR{4QWl=%1|Y{e(lB zdYY)omBOpKSM;Z_ukzA0_%a$N~k)vr9`WJ$LuUOGg>RGE zG7$o)mec^4c?06)n;7`BSmZq!0o@D<5L^zos$Ccx&JZYpT&c*>i9OJf3JkvupvQ2E zd!USC;;6JsiQ$@M3vxexIZ@yR2ZZYkegLAVl_Mk^YcfSVwS!bIcg1{QUMoU~`@UM2 zz&hJ}ftMag^z=b1kT=;KB{r0yePj@%K#pK1n52rw5GZ4=ok~!_uN^huQvchV@!bj` zw=x9ux{zYf1u_)&5`$kX%`V|*c4B}}v_Q2vAbkl&Hb6%x{J;>>rq?G3@G+(|qHc!I zJhuQMmLKv%-{?@1RG?39NLGvwKrQetVJ?Mc^T<5iaW{y&xVH5V?^bA;DeBB$(bN0V zBvV@x;|69`6Ef&O0Jy;?o=)cir=}>zuN7c0V$l_1owB|dAh`&kK~ZW8=E$BWfYnQ} zE#sf#!p%g1s|6G18nN4)=ga2gmEq3QuyVQS8hV()NbP-g6fCeEb;~)&u z+Lza7kbWRSq%EX`?+=k45?yKb$|_Nh1HgwQ5$q?qv=D+Z5`e(2^`ijrOG~NbJ;)89 z6(I0OkAN@`BF9MpB8Ug+fD`Kd5Pu4E;8)+*BpxvwyQGu_U3JJg@zTmZUH!sqUdPq8 zm$!s#^#i_+v%Q#xHfKc=_e3_hSe1dt4yhLdPed|@TDpU(bzA!7nJI!IZM`{u_JeER zx?=PyBoca$K$n%%2;}E`a>QwDSGkB_=x?c#0O)ORl1PCX_$?Mh!5c>W5q&usW+a6fjH)p|S^`9OUA81H&!8~Lvjf>66k9W7D9@51pdpBh zIk}PFX06vfY?5RzTIoaqTfu%8WMP-!{ zFpuufyzr_MXgwP-gKCSwNEJtVpttwK3Em)-w`-Ox*C?=CB1-x+A1Uu6oofW(MF+{Q zm5ygdbS~By+VNk#)!|Y#vZAeZ7uKggA!75|h!f1!0{bOTH#mb90tGnlRi3Bsl&64O4XQ>nNQy@Nux)nc?yt6Wx~Xg7rIVequL>g+-H5`G(|$7 z_%p#ifRdg&nK34RgQ214`b_F30JRDwi_YnHxO8&MsME^Z%B7`bgU$LCgbX5~n<$>x zKMcZ8+RQ1XmT-a9Ep%L-)p!uaoD);|hhW@Dk^X#vk2DM*dI~%oE9DCqmei+^RU<#c zJ)$DmHV5jFnl-%9Q%|#8eM&^xFSU8i$xRET(VkERL8}y_It3@)q~Z}$!q*0oT_8|9 z!EXBSNLC@xo{-P$%QRoaQ+x@#ejtPv?=RPQ`SXJT3thH!DkVyw`3ut*0?do_S!)&a zj9J{NI}|ZMg%iIju1{u3HJ0=Q>q;;!xv9BM`^;MfXivH%0Wy%1f50Ttz1dob(_ zlkauvIGrtNc&M+ibIAbnRr*i7xYh!8(0w7#l3pwvpX}Ivc-sH=`nsO@>WY2lW_EaX z?`*y%`>kg}^rToPElaaX2|X?;>cryWQd09(f1+aw)?B=9!LHl|70tnZs1=g(eflzX zqnOx0?q@jR9rb~a`Nk&uHnjeZ4$WMKOx3uzpcKsy>2*w#v!nS3xd-{$$4XPp#7HzW z(bl+jB=XiHXu$Qx_5Af+{WKeuDv|XY8L1WLZI2~U%A71DPlur93JPJbN-qhyC(hwt z@psD7_tI0+R`-73UpzX#i}l;&%~sc)E_%0oyxLs0Y-(!oOc!&go_nsd8efX9`spHY z!OAKe?0<}=kmFYIoeL7O8s?7Q2U(R}hZY2m@xt+9|YCbJ(2XzY; z2-G0kj-R(>v5wXExHB!!9~b4b<1~%mYNLbGE3ENLm@7&IhaNRpBilY{catu`S83++ zPMM#$)!Z!VmxOJX4w13UYQVn6myMG^wOzVmVaf6_-&vrVyEs=~mDQo+5Q&nG9`beV zCMe!_HtulCx?y#G*Pi2GznTtASY2Ixc^l8 z7H7!lNfFxze$MIjPth}I8{!+^qorL#U{%-M3y3GtSh7YbydQn@5~uR>>R1h`C2fq4 z00zDUk;_#))1?nd@wU&8D4+>u;qiZrfQLxEXg&Dn%kKsk_wN_DHokTa&568ucYj0T%d1=GT zlgh$xo#Cu4_-0I1U*2Tcs61Da4l@YVAzJ;|FtyMjT@rF5^NGP`enHe?%N$G-+a;>y zDiN5$tH;-v?Qr_4sMITP?`f((AeoLS``Apz-ClnpBp{8bZo>%Of+Zc>O2(2RG{iw~ zO&znYo2JQ-$@1i?g3|uA+^D~;&m^~mHC-sEMM7@TT>9a5=`dA6>1nqQ^tH)mUyUVc zcV=}V(v5i1Q=9Mfl+-x1av=T~uRjfX@X&puU!7M=y|u$eKu2Wo)&yMHRq6hOIxn(j z&b=*At3UejFjrP&RL9A|cW+%OyRX_*IWoH4@WlGMXLrqy183spN>mG_4yWQXix2nL zZqiO_@68yc$s}a5_{2D{s&M2;1rt3hlzBAvJ|FL4^@m!_q~eqgwj+*>9XBGy5aLTP};YvkX%N>Sqxv+t-UzNaW? z^2X|^xM`oC0h%2c6j3ea+a9^-?4KE1KfBlNo?H|h z6grk~C2}R^SU%TgznI$1^VyD4(k899JwBwSxYu^nh6Ijn9F4fxu!^xaio&_teqBxE zzWA2M5ZOKn)c?d-+o`egq8@139;}!*+=;($u_P3r9^vx$^FN+ z_6JKLxPib(loC@~F2~zNiyu*wmy2tB!V4W#IL-&7A=VUb=p8f;SWoU&ehz}r*VpOY z>A*e8_grBStS_b)2w1XE}8|7xBx1 zwpr1QTjl*?Jf8ir?wVg=*g1k#yd|9l(MA$IcmvTssSiJy;*OubVrbz@uzgWpCL{W{<5@^1}K&&$B^rRxvy)ipV09b!ct=Qd-A zDCQGBaQeYfUg04w@#SvB+t)2*o#_!bD;CBNm5Fa!c6c0nIqzQPMEww3RXx+bmWt3| z>D&m*s|4rUSGNyD)r-%S+Fy$wHhNPsNkVOdq6A!?ljBzqSnJpGq`l2B)+s~sK85&o2$h;YhJlK zkG4X2qofqS2AbD-=cU-%FihNIB&%RC(~h+Nk}oQYv>&V}_t_xnO+2iT#BjiY%9SSSGG_rx!clJ%}wzrO$b!%PuT7 zVWD;0$pWae-l}fOa-pEsFQX)}>k5NA#rZU^%QwI0}$@UWNrq}KfNkc zf8WJSX6bC_&|JY-yeRL!-TF4k2N(S2V03$gPM55w%et{LAm+ZwmK~JIS9?o}g#L-ex8HsvY@9ktPXL&afr5Qb#cRknIF;PU+aaBRZnkc*5(v~11b2xXJJR&{|OdG|Bo!}?-xeDUop+Z{^y0!{~}BMm4%_fRW|us z3Md5m0zm-^qfAl;i170pBcsm}rpCk?lNyr}0HHPJq9qoQe*s;UM+8j{BborMr$j5+ z>H+P`^{+VS0Y&XQeP>~Ui#2n1>kH9OUKb=x`&k}IZ&@DW%~P&F9G&J&r;N66FXUjK zodJC^Q%uXfjYbC|m>R1wENzT<$5)q4o-y`FTFK*NM8y#rXtcsE>?pFWyZiMSZq`5t z9>}-+V(9Tn4Q%Hc13>l^J~T0>0Dqnk3?NSUp#X|cJVL%hBidnf z2X`wdh8QAH2or#s4x+;$5$gK`hzNW7TXS(K5_WSF()4iS(W2$;a3i5AIYqS9Eu}~c zR|v%@qClH~Q>x}CiGoy13)fmx#F@n>-6M*V2l*fdMMPZ*#)Qyy%K}MBBnaEXcoGHb z1rz|F5?&pc`UIwz`b%yO?x$Cq-_iN@=tn)r(yRnOAXIdJ&MMIF7G+QjMb*22=murx z?1gd~G$`rzRSXOv5Em}@l}FA@s0}rM{DcV>ZVZt_ zo)$~pbTHf>D2y`bCkjKLB_d2?&C&(|o&_;|3_fruUUTzrDYOH{w zD%grj$TkoBT--P-c7a6>tBim+=AHu>!V4W5CemCgrEo=v2ccc6sq{UFz@TQQie@Ja zkQ4)wVoBh;Ac#?0xv(4i1+!dOSJ#jUBP4!drVgE?Ja8Gs1+oAUNUMgIzdTHmC_f4) zh*W`Huc=@SAQK2NAYFB^d`BuyNDM!Sl3f{i24h;Qp{FrnJ+!hH~`!Wb+gE<%=w`(m3_Ap^cq+J$xzOdQ4X8lJE2!%gL zSQEl7B&H{GESNtD65?=EJE#(aW5*Y#h;JP!o`xt_MO*eyF9RPZIfS56$^u|NS|G}{ z1276XU@)zu_6nQ?kZTX|0e$&Fm=MrC25@=ew-p4k6YR&oSRUSkHCVh#s0?DGAg>l- zr6rya2CApae?(xd!tQFPTu+1R{KX6Q$HC`Xm4^%w_EoqmNb=gMpCHT`5!W75z1;dI z2wV6G!deovkF?DOOmYT0hE;PW3RFdm4W(BY83hihARgR?EXz^=Or2N4-?$K8Uy zq=ru*>szS-C)ZSBka^yaSJ$Oso=q-K&Vem{pky4JP>9aDxN)`;^K~8gHhuah2g{O7 z)k^rCg8}`8gUysC1H0D*Z8<${oZ-&;-G_P)|Md;35ADy&HV%H843Q%Uy$pj+4 zE99dLccZk{l8nzT@b#qRnx}I~$XqI%($o)U64+DVz>JDEX3&R3BN1BL-}QWI!pJBA zu@iVwiz^v@T>ZE*Wgdf`%*?HM$XtARE%G_=1A0m$Y6DXM1b=3NIZ18`=3>Mpi?5iz z$eQG-c^D=+4j>!EYy*aD=@GH72*@;sPDg2bkqaySGb*yy363!Y2yrL@Ih*0?rhsP$K6(6>wb};LJhgR6s69u2&2CJk%9hQ z)P+>z%yT*qvkkC{8N}Y+l~D-Wj&RK2b*6aqRmVdmO`v#PAjXgq@&N<)(<^JJj27hC zp`LMEc(Pt9E`#P|7)n%=$-rSG6zj3GVN->%lC=kg8WKj1gG-GYSVY{<=N2$wr6q*L zgGFC~>!h#)xLP#ArzcXqKd5KWLiyI@hfsuI18gwjE%ky#;S6RSDO?_W6ofFST?`OZb0eh%GAF==P047NAm8L?LSYr*Kk6#d$^tPaRNGMKGa+|A3#Llrc93dJ zlBH*(=or%lW?SWuMano*0G6~sWuOFd#)?5nX&cHJ(gtJt+I0exh5|}qN5!sl08;2& z$2H_^{d!_!ArolL9)m86KprSUDh7gz2$VW=P`$}NbF@p6Cb$$ps8nEXC_e`+p~eeZ zXY%q6tDCw!?cTe7wn_DVArw4h@zn4P_Y2DE@-7!4dbW$4=CG0v?0&=1d#qboiuipf zlCXhFDXo$Zfil%7$r1-WSbmTwgYZ{fcpunfs4hq@T5GlVW9b2f0=5YmC64QjdO-Y8 zfP^kJJCqWoey3msH(CW#V-_RompKejQom5J7^(1*p8a1anB%V$OmGtHor2|ey&(Oh zV5Qb8^v3$kSC;6fBT=^e%hvFCee)vIn$zteid&2|JyfMM~efZ5vuL7T)l&KD6@Q*_35hVy?So&GY6tKb};o zf|0`6*kH2q+(&VJ3e=5x#j9YjMDAFa&dyys{yudCm5n;m%FCne&GsdK(01&4I<;}A zo$!e$$UE#+Vq1qyN?>};^Wof08yC2(Nam`*yN?O>S;nS3yI7ZR{W<01TBe6<`tjjm zqs;l~1!6puL$$a2at{@gmEw);9tz57w*%Rd@lMp&x8U2Y@1TbdWi7m=qeBK>q_;in53N9TU)rA-8nyp9z9Ygy@rY+&z&BTC5^R3z~WY6qSdVxk4ROm ziR#j|H-&OAtW@xXC6^pxjp}MSP&j$}nf_28hGeyJy~f4vK%QsMz-}TwXb*{>x@%to-{sAlH;>yJYIZ`^Pyue z%FEPZ965(Np4qwI8YcT;me@GZk~XTq^;FqOe!cc`!YXHih87#gxjSi>+}r5l)8y=m z>F_>XdafiX)V{eYTA+aYv*ewKar3UsRamzj-%=}*G0Gj=2m@WPxofJDWdWj|)8X2> zfc$MIt(sikl=9}Eo!Da-k4*KczE(%E*wUo@J5vt=S- z{H!TSdDi`-I~&b|+)O8gik>S<`K+be8xBCz)ZC37lAH_;RQ-?pbt&xJ@4hB-3Jucs zH@pDjSy;(tq?mqOe`o0Mm3Ei(p#$ARbPU_U?d_=x zpmCVOoke?VFtAu>2-!v^gbwc3`E8+vz|`{OPD&0Y`00lnnZRPsa;cg(=1@}0oXtu^ zrv^=P?#7JSVJVf=RymYxQ5pPjt$m5-rB|g>ORB9wg?VYN*;Zo)kvlda*oC-< zc=~8o`K?y2!^4Mh==kMB5t9p1khc%e5$KNnj|n^TH@pou7HUsftGHDAvLP$ORacmplU8<_@@fq;!dvgWDZN$_sfX8Oqt0GnP0POm>Z$JV^-D(p095mgT;KR z=V7kd9afu=Yt9Ls`iSbnW!-IlWEw{fflW1-3^{rnlJK#077d}JM|pXjd>btmLE@(t#P@aUf0Ak#*zC>#4xh`alL7zeoTWhI zbsdquuLGpf*;$?P$E^PN6q5Vxg*jrM#)8?Sp|r&MGkHS7EVWh! z>N?ktZjR~LxStuU!glw3`)=Z#47s9q>$sj&-V zd%kfVdA?`g^FCeDK~LA)R_hpHc%$kgJHh$#eM}E>Qm0!z{IOQUk1w}o(OMt5?wZgG zU#YT+j5ZTuYCYV~BuApp+3nvD76Yo59G*{UY_~6*dhLgw?USZo_bJPSm+OJgbc|J= zTy{Y`7tOt{fBb;f*ef2{E6H4_aCH`k)orvf)JRT~0naU?IRSc5QXOkX=GtfvsXrT* zyPuF4_otfA?04m?;ag{YTDWhS!{cH^p6OV##r=57kae|v@KoihUCtd(AG%oIqdW%m z4QJB-Qest!L2Yf}dR@C^nlWzkakaWMRrw=E`WRCjOh$dE4{3{;UhzuS&GW(74ES1k zan_C3+F3h1WgFYeX8mC4B#)AQG0*FOZz`uIe)clnR~a6JFC8LZInJNs^k{te~yE zK6F)Q)}upf6EA?~uaVQQz{z!JEcNW=9AdW$AkwZ9We#+5U#udra1naM7a!&J4bb4B zFTk+7x$u7nZWRHelg+}8(ZMC(oM1muyNH*0TS%>&AmyDTpK_>yQlN{F#3Pl&(9X#9 z($Q2>&G(UWu+|?Ha7pP8x7kJR*6qLfq*L!0TCfrrU;rwLKY_QJn4E`P#lrP+KNvZ{ z98&?h<^W<|sq>MJ6pukyHN&=mZ_K<&U~Wco>S5p2*_A6li%OI-S*(?}1!paC`-Fq@ zo4XiWV<_=3S-UXdc7AbiA791D26GcaYZ|?cY4ckRzbgy=N&VaH!1)zXmF+l!n#zZ# z%6WWh-8|m%?Dzdr4|6GHz{$xNReth5X(PApNl9{EXdL6Sm2YTQb-h=s9P__Uj^-6 zqrgL5tDITOX_F$_J*bF>>DWHFm76Y)m{h0ZxWXn7W6^Tx^(`j~lQ|Q3a@J;zfC6i@j?(5}T#O$cXyoHMeEj8VI zIYXpRP{bCFTV7hvhhEy*_Z8&>u1)P%K>vwFFLI87Q*__}12@meDI#j>FJIn@avXB# zf4uqt{0YB)CL3BjXzxnss8@V`tKjf|I2g#df|_r-Mgh{?qqIj zqxZYQ_s1iK|MTEso>2xUUwRmTuJ$U3d%t|M0#LHd461PfXcKX&xHn?js77tqDXNv!tW4TS>N*D&&)~d0cxOr0}UrKLZ}EsM3k2o zcJU_|E(Pr+#2hqZL}o-zf;wxGUo8birJ?nODn>2!h3slVJ|mb24+8Q^AL%yEM=C_0R>VKN zXOaGXRBX``68>WkqG1M2uUiNJ!lfN8LCt534g^3k0&a37k_U3ISWl2R`ykIv1VA9t zfZPyYvFdVQv>{=Ly%|OjNBbNElFt&#;!_NlV1J5v{ZMfg;xv%~-AI5#wT0q_GMw*X z-gk^wiFYv%j7~V050VR!np6R)kKrP8#|J4Xrx*aTF#849<`Rk(Sd4Qc;9blE!~v`; zh%VPy9}QOLz`Lf#L`DmX9aH}S7)`*PAQ~(m1V=%=iy>c%Fa;6{@e#qINU6?BP#r{9 zvY0Wj^y85DqoUIYKl>;`sQ+>jh31s$^_I}uQT@FyBaAvUS*L_A`$C>6zJdrKpFd4E*9g?;q%|-7g+~4==0UuRc_zPzc?W?I55J0e zZr@Kqe-rcWNY9rW9Lm#BUGsm6d4lZ)yuXWilh76X*9zhZG!}H7KgB%%E|RL>#k_Sv zoL|JeR9uuHG5Y?IG4g1Tti3Qq40g9R;oi^hVqOT+^soivvvB;om?xi&@Gj;}{U+wU z#L8Rs{VwLQ=|TyUD+4ImSpsT1GVkkie+XL!z5I)qXDA!?i3|H^90B-g|1Rb^aJ`FpoOOBnj|vdxtazw)1Ys=Jh2p%!yG-|HWUwIN!axc39%O}v zd!B-a7@xfu9|Fkt#QL`3@3MJ5%la_r)1tGcDTRw4fyQ9Qb*C(ie}FS26aFswhzJk~ zp7Q~P1CL+=j+{5l*OLJOg3HNXm>6~aV$J3Zv^Z|Q4%9weROwkk z9^BUnLOd^d7mDmUfMC8Z&m5Gc{g@b&)%uXACJ^fR_6Y8@Ge?LayE9Emk0b4S-!Z{h zucQjWY$=_v!QPa!kH-GfrU;kYaSOBzfa6DJs5!-5>rg_?)tf%GPq>J*lmVdl1QbH| zn1MR8tbVbyyTpNiig~l{xfI1p0I&op`cw5np8Q$6ehz*2;3t4#E*UYTueRjW6=<5r zQVoNXu(O`Vm^%Z-VsiJ-oZkGu07?7?9WzCfnP)oIaNI;jS4xuPk_vJT195;$Jj#^< z5pYI^0@79?hJ0pkX_jU4$DR)-q|_+NO##}O>$I>Ii8fz(*H>@~t>G(K#e`SvR+lAC z1;xa;%9X4du3EIG^n)4ZAXN}U5h45pbRnT&2#eVXVsy6D9XVvgg^9GKRq!GZQ{y5V z9FMt`Iz9mMB1r~BNEy!sVf1$+uo}ZF0RXWu=y*+vK)UBo(Vj z9v*W#JkGA9_XcGYR_Jl|fn`Xmqj8qR)xWpT4uc+h&7qqBPm=m1s+F$4FK|^ zlO%MtJX+@#m=kt-D=tV3Jrm3=i`NVz`z!d~sec6@K4392Q-v@JHau&d6YM6Ux@x1M zN17oRlvo!mP(4jh{N-anHxQ6{I4%Ce`V9Cxku*l;I(;XSJbm0Kc}m%EEqSsSi%Qq# zAxy@b+_}#{5u*Ep{7NPRQ;xd}KAGP^qR8T1T5Ll$Vd z5L_rOvsQIE`5x$xe-I>uK&2<|raj@;i0W|!T^YqASuy3YHk2Ki_M``{WeyhnjYx8E zTdV{Pjw9;nMG^PjXTX9Qx{LzvIYV=xHg*B0_>u||p?qiTxOUj**K0_c5at}hjn402 zb$ut25rwK>=)jGo6x+}hsQRO{W~I?qc&W@S;V7Qq8-oC%pNp_Q=UUi8P+qdE_$?Jw zbmOj-iyZDt^C3~oqJws+8ye&MM3P9fOXDC>R)bMT3J!-~g_gtK_SNI>h75gY7Xzf# z%t>hhenXNvQw)BvZViJ{ zUxRBQg-GRQ#=l&m$Kl|C*1^3PS-c}j?%$APw}>$wG%6E12hC=pfkx2jyuQt@kP?Sd zs2xxu-q>1iyNQ>v5;=>ptic&&0XnOJp*B%2yd?;!JOSS*s7H*x5YDF5=G~qRph!0e z3}6&pLP$#`t`-@@2YigUm50ny<=Q}TW+Nzl9p%+pmF3lPmnafjth{RWN84T4ie{%Y zPS2A>=J-rkC$bl{G0{N~>yI4tLNKdzKvm>+!z&HgLSTK2oi0;%E?=)|>RhuKL(^jd11_EI%owrD>T^>lnQbs1m5x*%UpRx^yzdw?(a!@_Q&T-Ulj*9m?k~A3oJ?`9&JU`NdTz*xXfv?5)_H%UG2pHw;BJ#*XXAHm<9& zuX2hXzo61&Voq8yeP26uwY%6oE5R8~1y+hrJAZ|>c{IofH{=?jPm=- zo2YAA+U@L2Y3E7rL=y8Xb0Q5zx|^%{u5#IGMn*C`>8|u>GfwljiK7>HE>DU=zNmY6y*N`S#6CjMwSAh7Fb3kBMngaQUsw4{+CtyBFe}UXhJWaX%2J zxSp`a=44+uz=&oNM6V&ur=2x*46U37gWC7*zTuROofO&DQ`eJr!dzBp;Iz|pJb(00 zij=bL!)7gzEYTk-zb}REO}cT5XM$eko;Yo9{#v|=VB$2TR&3%jjEKIqCpz)Fe4LktYJ=e!$mzb!4Mz<>Pcb zB)okWvMJz76VAfId#F>b#9;MkiFz8|h^3abZJ6;^C|Sek`OvZD=CFSJWm5`D!(;DN z-K*+M{V-Mv*rs(6F4Q9(-4@!Xrr@p%BOE)8^2MKdfp^&}*Su5F3HWbE51B5@wTT6~^Jp)7J9ZN^!TZ z%bZEh-v$BGNVG zS`#}m8GUXu)6>!6b?SIu%F+Nw%AZ5L;Ts=sHT4ifSQE-wpItN^;-0=YMUp6JiC8Qp zJ%wpF&e|{@l3C_U#GE4$u!R(UZnB~7@ zvP(3AE2gIE`1qiAt-Q!fY`dcyeR!1s)&{@YtD;TCc|B&4NarvsYnhp$lBNFYxY}Nk zwlE-cr8qN=f%1q4`;}KCibI;jgiP06B~iGv?WC?|-66oOdqQ?j0yzi|+Nnc9x6ZBo3s+q$TgTb-|Y}$VdCTYjonD#*$dZeP_l=?WC?%f?5@hZ>(lF#*L2T{ zfmi+A+gEOu8hiQG65NxAjWeD~@3Sw0d6Ac6+)vk9?-OOYQEzgJMWa1~x9hs2_U<~_ z>4gEk3R{kDzzFb3;*l_+QWs*l2lfp@VOX}f&<-Iyi&1Ks)z z{`SPULuFW&(REetbm=pP!5y#@w&|TxyG_L2(I_09`>2#5%^J&atqIG~UZuGne#-8N zb|u4_db_@Q!*KTw8d=?1E%THPN~@%h2f$;zx0y)qB04f+Dfq7a(C zeKAVMzVm7+?Cd8y)XF6S7JQwuM2e<+uu9Ej&`s(>u#of zR#>O_i)&1n8G5Zn&(%0>m5~I|FNUcyU00>gI$!Q{tgB2`H2W~vskyYWcyfJ!5nIMr zURx36Sjmi0q9WQ1RLzeq=_x6=>^_Q+;yfnyC4{?mX0V zv6*-!`b+ip7cVs)y}LtcI^}rA_^pyU5?-P&)4hQYSv^ujmz{j~&Mi$RczRrh#5PUl znqCdA_bB+>J{L8vov?LjK|H5#I|a`1@ow!$p(Q*32-PiT|f2!?~|1{kk-+s6J^Ri|M?(_aU8Vage zvPPRA3@!Dd*=o*5HK&=f70A0xGAhgpb#$CD9i?i)l5*n|d!r>Ox`S@ME`8<(yvaV^ zf`u}@#JZECHvO~dbm8X#PoZzdSjkvL1y5kIb8OzMAwag&a;v>T3XcDI^k8OCQGB>X z)vkKUp2%c{J-yNbGxyG{|)4-^uXfs#ldH-0%)^GEqdrsUFFdpmWYzt|^ zz2#_w+s6}i)8)E)YY?4=$K9Y$?G>&{x5@FS&})lUYJa_B$SnzruC$V!LB1q(phde< zUg0%hmD$Z)Y8BSwmM0RSyK27m^MyPX zxFHBTHVjQLh~Qg(I*S&6KyhxEO) zs8Y}f)f}phAlO}^#u~nGK8MySohh?<3$@cTW8c(w^e}x!7~Kgq3vCk zOs8$!!J&%AI6}RKDS@DAwveDSc@hjfanJ!%NP*OfVTyD(0Y=)GWMT!r3K5~-An^^K zgg|fte&?zuB;+{!P9TU$VyhVD#x@WwOWx=BLtPu z6~q9Mh0ECD*B||op1)Ecv4lub@Rt;LY8>AKb;SghM1?3MyuVm;35IqRBd$MT7$;C{ z@tc0YYy$K7y_>q>Eg>dNC_fOINK`^Zo1Y!0uK3JG6KAwS!oB-w6Z%iLb1_39EQt}U+HVDYrz?T(x_^%aMbS3u#CjNU_ z48(^)e3Ok9tg2I3n}T-{DqQjA5Bwg>92NyZDVDP!FoG8nl2UZ1S?#bi_)vlE+4(dm zb`Z!hlus){cusvH@?0)wg>sNsX5C6*H=3(@2yy*wW9mX7n7PSHbkcUeh54sYVnqLq za=x8EID>lxVL<-&I8m=66m0sz!;*gkS!Jt2AR-jPAj;+i;7YnQkKut~6bp2-7o=^> zK0!kNO}3@BTvFD>x&A1%=6N1@E6crVGdJmR08QcRGr<0b?l6$|MF?Ga^_JsuTrZfh^kJKsMrUAZv9qqTu!oHJX3ZM-GMy z*uGnv*lNBIt!5sYfVse*u`W{*reE%NZ{jKOw=@hPNs2`%f@pY@CISKvQ#Oss&cox~89RGI{Jsed zEbX)IoGY0sd)u{4<4I*=qbP-_q^0Q48-udXtK29Z4rgpAC2Lb>_{R(=#iD-TE<|#f z)9NpW6U*ErfwhRSVd2@t#x073uM`s(*|S@pmXs9|cjPV?wPL$&6`Sc!X8z0JcK(YP z2=k8^$mpiSDNBJ01ylTA16hNt5{<7wb|ismko437IOHG}0^Py?zYAoANWPMI@G#KA zZ^I;hRYnLI|B8WZQIvMdk>iUu+OAT!ZkJsHu^GXy1)G;w2~$5^st5QdPVe$2Q{rr> zS!=m*^G^Fw3`u@75``rQX+*fIHTI{zytrT}vEU!d{fK{K@sO@?AeVp%A@*B2PKw7t z$`_drX?`6OO@a9<2KEe970jpHmyCIf6bq_o&t91o&gG0K{T7pwAlKhiT?Ugv@a4+* z5(6Emt1(sn_OYk`T_1aZQBoogLPHuec#LH6ki}Q~jF6MHZ1}g24G34{%zR;)!*xmw z;1$gd#t;UZ{CbPnZT2e)hQ-7ghgVJ5Or56a)vH>$;pxZ>>WKm)NrCtAr+Z{%parQq z_`D*ke!l%8QNoACLdhiQVk<-i9;|GuiYzzgXSLD5w(sGm3%9DjI~r|oQDyooCN>`s ziND2)U$?ZmRuj&p2+F}t9fVAUPWIy8cX3v z*{}lTpE=)x?tNJ}*+Zt3T+pAU(*<-Aep)>zLgWk;=^HAAmba!ZN|vS8eLol{5(7PSntX{2fPzu_@)E)pEr)_3fhfNRqcnIHm&VaRg&N6xFAv}SuVP?6Bh;S# zCxxxqaN)rGj#5=Y>Unz$_eqx}n-HxT@R%Pahg}y?dtP^m`5HV16N}$-gLIVydS?t^ z|A6!Nz$}z;&99rSGB8)p;1@R-^H%lfTrVHPmg2VhPxTH;2ya zB@h*N*JBnJNsTVc8%u3Z#`D1+tc=A`dbkN$^A`>S!`of6wz%b938u6>#oR?q=#7jh zID7!T){(NbHZ=^a{S$3mmR;?XU*>7yTSE9d%_Fw~@DQf2x~H~NTHl~Q(pE1Xo{%<1 z(xE+GL47o>ajswDeGcyRI}R^- z9XoQZ2swvL>U|Gd{K7MZ$KKXI!u-v31(c`~g z6_=H9;aDoYd|;Xpnc<%qJ_!|ue~IrJxu+5H=lBi@9?}5y*Mk;!+JbRU7R#C#G+c$A z0%@K_vzu#+yK9?YsQV#}YL_0i?EK!6d^v`X(B@Rb*a+ZmoaYs%@7ar_s$qj%^Zcw} zsME1y_|>zPg}_$SOhG`vQ}xBmUe!}mR<{Y3_`Zver7V4+=gxmTT(41L<>IehR>x&XRiGV=U5ZHKNVVt+&JI4x9(AAWQp#vSJKdV4lrHu#P2vzMafLI z2nXbh^?i*qjx~X!{d$2<|4@UKb5r71&TOe2eB~D&lZ~xLA91?Zfjz$TS9|$G{eulW z>GP$di(y#%q4!0!*L|IIX2C5^wL1#NyCKfp%Xc{#*@dhq!*@>HN|7jc&54ei0;7Jx zXmp=_ocA-llaCyd*P)RCfH$R^Rfjx7ceeDT)=sLh%&`~t*25nK=|Snn18cAQ&&ii)Vt#3i!33KW@!$5;fi^LF?$vi%%l!LJ_wPd%7yeGKyY>Vb&_D3tWB` zZZGz=`}vS&A-8Ax{ybRb>yp1>hm$+!8>%>~@_2OtC?`F<_Q>r(Kin?@emF2_Rr$Jj z72jFFUv}pC4Ozf`?%z2|H8*K_`crqN-dp=xCy%oD zT$mgnPp8$o6^-FCtGXFD#*Y;hP~JJOirnX2mVvkh8PzZxDi2LOUkIzp z&oPb4NlSUTS*MNAtj=y2)xW9;R?x4D~s zL&M;Y;5>CW`T6O0Ycc9B`K8vAtaMMkSw?h`nI;c5zgz$vj_f!oqnza(qHaZd$giaf zVDv8QO!GeMhtnssE#VX1hx`X*HdzawCm_=CV}z+^s+~7poF_*V$#7+bEqlx~8yUYx zUgZES@htm72|lLr7em=wkHVhJRWKP(f3!~D*TjRx+T2YxR;!7m720KOFLsfWxGyVG z*#X5+?-z8(jK z=!vWi?9rgN?a_nA#c-1@DYvEW>FtgKr968*dvwZHtGg`!b9wbU-Dn^q7wM>f&eXsJUBiPQqvIG8-z86bJ5eoHAYe5KU}j^4OY1-I#Z7uMB58Z&hniOT9t&y6Oftf zvwZ3nN$r-8pY-1n4I@1>h4mHhmlI}Nk{}UvjfCs_qV-bMI@=ixCVy2#mU&>Vy|1(0 z3~X>}tmR*lW!#_OIQKukbn9fq)vuE7*-l(v=-mw3?B!%jQF(uywnY2bkNk{MJggfr zw*(q|UY>`UTH{=QxoAkSI~az@ihUPtTW;aPBzg^?Bs)Z&TIEJ@pSr81OgT&cj*sOy zJZsZ)FMl#}TFKgf7KAMtU z-paFj-=Kd}uMJo)@9YwgG1eW+abdZ{~62r%r~#| zt*qR9V9dLE=g8v`t26elHKlg{kag})M}Tw#+Trt&2qfC#OQRlfCd$2G+4NHHsBOiB zKs;old|*>PyG&DLz0SkUhCn>|;g%z~7SG)q-9u%Ce+alK8YcXGhk408)13HF{#B!% zg=()_n~Q9iMn=O}xlT{+=~a}utGAzl*?vP9^HGPfH=1fXh}R8?B3&Z1yKa4122+n= zImKnh#^@gaCtu?*z}7UJzO5+J)tRby8(ojMlJ|)HqXPPBarc1|awx(Sp4*?k^k=b? zL;H_L_vuklG(UOd$Kp*r!XtQ8V?7EE@3t#CQ@8p+?1Kl{kR>fASFKVuOq}U=+M9?* zx2%tydGhng!s~&Zy2a?)P2*SZ_34C=(A>TEpHzG2TveJW?w3vqDpRBfi7J~?M+vjk z2kg$T2>`sJ!?O~+ z=j7TJSE@3#UhK}|+Ofl1REOGa8()XEeKXf*$w0?bUlwn$eR%od7WTVc<$~>!jEQ>A zUQ}#bySvHKhx?&fuDH!PKncE`y8YJ)k^}i?^8oGA&QHSRu$Qh(ls213rjS$b*Q^Sk zUH2m}&#})L`BCV{*EAHFieqi$7hR@(C!3q=b3mD8o}+w|w5I_X12>b$T^oBt-f zg%$h0+JbFG5D?Xvak|H;_4LJ!RGX=Hh@a+S`Od|lSxdkh`Et=Ue1T#_nMJN+s`5r= zE5bF5-JmfPMo3SYq-J^J4BM*Mni;lrzIA=%fBNvo4;zUcq&W8h{NE7W-*VS)F0f+y zOO%uOKZ5A~rxm;ZRPKJQ*!@%P{wvD;-<%J-&;(aU`C74q^f&x22zu6xB5GHXmp5BB zTcF-c1lI=4-P%@nIsSu20zpCW`*%wPWs#cL5HSR=l;4bs-=Pf!1qZ|!fDmhVSa&v3 zSAq(S@;)CwZ|u9^JHKY;r+5!srg)Fm7o4vh?G}>q2d${Dy?fg*(w~Vx5P@0}VEn8iro=-9fDYn+)wh4Z_@#(f1|SnDBlJSVR7%iA zg7g{U^;DBcs$x42z>6nAS5NiEEHc_*B3)}NLmD9ZAyJTh%Z9i#HNcFxi4izm+1ep1 z(}%VrxrN0{cunrjWl*{qj@uFTQK#(-NFz1r6#x~1R`u*5hJY@b?n%xQ29m}EE1+@( zkt6V=4CFeVC&I=@^H0uYkdN4AB65j#BCqx>BfvOAiRWk)67$3pGnX3!0P9K~kOw88 zVh!LT;0Y0AEVc+@3<~QDmc0-$0_E=iLMc8J*2W0&M-cgDPvWQsUVgbaPRmz61OgHP zWd{N3{`-jmk_-Z_w;yC4hz`de6vu*4+gUFTts7BD)s-7B_}WheLS(*fq^+p}rVMQW zSAt&&3Ov?!Bzl`w0qak9ANRcwHi8!tQepV&2%YF;$TET5h=pu0c82eBZ&y=H**6A6 z1O;eriimIsGy0|+9vIg&TA?pWM_dyFtTbtZO8O^qq2MW$03lwR&2gexSr zO$X|PiogpU8dtM@3oU_HumN=h7FC*6NDN<3Pl)HF;7py33?JZO(5WAHE-fL!c&RG_ zE(3#hrXdC=`bZjB!0AF?i0ejoXIK|wtY=)NaP*^zXqK_a6!RqK4m9wse!oFW#<#B3 zR8-=N(g`WBjlX_TI`F{A8v|W>l0wL?oNDSda)!73a!9OdZuvIb=?yyL5+PK_6%=CD1*$mwrK9>QzaZXO} zfXV3{oVh~!ul$Sa%}WMrAG-!_qScVSM!8c2F0M5L9mBeSC6h2Pt?xRSF{rT+eh3)* zi8RPdD*g%Sx6CJ%``ml2?V|HX8|h=_uwZQtDWtr5vka(@^*Cm;3hWFEDAqKIQP zZje_vhxGBMZ}gyjvXm)>uNoPaR?JUcD*v`XoI@uTh%ksRAQ=@ZY^$Z7++XVIO({4@ zK>!~}zVF|V?;d>GmKpGacJGQ)|G8L9w-zoJP|#ek zJtJEzm{d%U)k}Kr>(*w_<{SQPKJ-|Xh>8IyU9lGI3WBCWYlK&p2{?#&8_aK2)fch$ z1*DG$$2^)&Z)H=7J{|tH5V`G~I&O|v6h~R0kT3I>I4-EZykaq-SiIlu?>lG)tVCj9 z*%Msxi9|%KL%ToT_~Y?3ADn4GSwd-~^P?T{x<=f&t&oeiT7OonV>vAmfB2M~8XqB@ z=?2b%3tWGe;Zg%s(R6x?3jg7Bn-UBwe|PNW8%;o4fki+axe%d* zhJ6K*oQ=R|9V&6qt0&GC@{AVe=nm`YCP0}9pdBe7#`D^Ce<^Jw3frs^oN(?Q9i4ug z;WHJY@uPkB_1=(@8~oyQgM`cyzlp3?lQ(TDrwbZyFr~;^3=H;{lHfQGK)f74PaOnW zNk%+KTXs(=skMexUB;zW(}jCG9~ZX`3_g{OVw7=V1;skixTo9dWt;QU8Yu2FF2(%^ zO7gy8NC(P%w+G(V8YkmQ>Gp$MiQ|=GOLUgdy#Bdy)&4{iyNL?&O*MB4K&SJZZ~odDgKV-wGfc3 z>=k#*kpqqcig)2)IT_PWLrcjNi?1gDwE-=LPVD|=>nt8xnVG$TTS3_Xi+s~-heKK; zar8TfNQl4)3YSb~xK2g4oj3v3oJk`M4lJ=F=GQ?C!`VFlV6ZRr_;+yZRBzzO0V|Qy z8hu2s*f}U&%A~?&8|XteK9(fLM1(%Yk(qVjb;}uXt{`RMQwt<9e+rAQBiAI>v9N8~Y7pB=H z)t0_cS+g%}fPyCaR?(hIn@pz^BHs~LRChahS@2o9Vpo!5O#}Y(x9@AvHY%i?#@d7a zu4LLc#pq!k6Wu38aGTZ)?QTZS_#a#VF;YcB#Wrwm+<^>y^ z*Zw>8nZvnuuvF2-(vnffvPvnit@xV$fFyac9{ek>bF)u()ZCPmzj#3!Zwetf4pDL2 z&`0#@3?^I#k#1zD^nx;RvE$Gnm^;1|R`b*I8n&z&+q0IB*Malqwuc%tr;7!v^^bBS zFQpad03@>HYehzl?N+|cUD3R@ zjsqg_4X`HGsY_9|{K*rj>&q_o%W|%PCeO+QEsaW)SbP%%4c`5wC%ZoP!eG%gaGS$% zL(bpAE`@tkKm%f;O|VI`;vA?cLc<+|Z-fTC~HuNTdK3#kQ{AdTe<5!DKy0 z4nW~1=kka<2i4Q8@PP98tOVXS%2vsJ0R5vOl$9;*l7XOX zm~|UyaU&fCq9Rj<_6Cj)Z$`I>NpMDnvq4+v=wbzO8CoKgn7k5f~9 zha!{cm~wR*iM1D=_d_Z_Pc&8G%P)Og4>TVM>Z5du&6?@UF=5+PPStPTw623RLij3& zjU9~ADw+}&+JDcwUm9hj`1Id&H>03mZ@hO+Y5#!gh>+N@R;aQ6RN?+ZDJ^L_n(xxL zUUhl(>+7mlVW~|ZHMTyRD)o-SJ>gtsD+;N+}<6_ zFd1^#ES$RC?qSa96U08bS-0K_dAlTM*f-f-e9r1SKGiv+w%VC)rHW(o?A~o3bo3hI_@%m56?_z=jz-#T^eT zp^{adx@i6iWg=~tpew=7srHX#+B9e78*Jez>a5MzRsjCvnf;zZlYp868bb!@3A8mY!OVC3+54kReSgc zl!T6HJcFAY-~EQ`;{m4$9~cU~^dh>`==sptP5&AKL@^sCrO>&z0|D6DEG>SU=jo?53w$*g$2;N0P5$V97;rlo5}D6co$@|9=T z`O*+LnA-2H;mK6zMHf3S?cmu(T>KbLqH}eW za={t*L=S#ur|uQY$o8b!bxaNCc#$T4|Cz!;K{wdFb%Q*ejLzk;^Y-=iJn&{0j>-pH zuJUJFEcad`R2f(xSAQAj#YE^y6>etpqF6b=b!(43v%fr#`lz8Ydw-+#X8xu>GvQBx z7Z=fC;amVN{V~1t&d``Bpc9#kPoCc;?8bqosVk|%Zyhbu1u%Dyj}P}7E7?23k%gyn zzxtrDt_{e7h@A9h7SGYu03aMY?NrKt-l@^#Qe;lYZP}gbfu)5+_2V_1t#YmFVEigo;5eRtY4;!!-_4R3<}yQNaeYsbA)4`qeIc?z6Mn(Dlds5aZ#PGA5)r`1>+v9i~udI2e1>5|Ql8JZ3! z`>EW(6Vn^Q+odv7HsfCEP1&fkCc5N{7e^C`kw2k~Av!j)89Plq9bDiex{lqm0ihYn zo3-1X&yUh-1vLC!24l~*S!_pt@JCoADV~-J13;0J3|P&SjzVov zvtC9*uKVZuLp1e6pi1*RyCOpQE$%y!-J-(UY`DzO&5Wk15ho4$L7?P3b$Rzh zY-$B7i+t@(&pDe4fF%PewCFzUxG4E6ZA}k#W##5B%4I2o^oO1r$GIcQ<4e4UANL(@ zwZZ9$7`*ReNT!L~43`Irn?eCnKXvB`!xNcp!?u(s4wIAAo7roVl1fSUIR;27w|F=R z6-RVfyv~5+U;R(RuMhd&i`82KE5l0cEDuw~OWG^pxVAV}*NqkzA#|M!-%%%gEUj7h ztht9Q%AL86(zW=??2d-XgUVsDMt3Hd=}=y9F)ZxuqI7L`6f4q=vT$%7=kNqaV_f?9 zE+3R7KO|l9mkyA*IKN(=q2`0XrFees)u(q3O$e23?A!9{aQ&=porzUzIUr}P%i?!H zS_4p2(vUTXrPGq$n0ACA$ovGdfkLEt%>j$d27A55X)mJ+{o8eEN+p|`gt zK71Qh*7Zh-68W1q41^ZI;zB!n_?8I%JRV}J(8%XU;!u#b<>dYNIOvfaj<$BveQi?P z`7m}@>&7|L*l#AwdK=G=5O^m-gGIg9*Vhad&7>N0W2bQ{WrH<cXqCP3*SkstNg(cj z^t9-ml85V#Q#V^*r!LSPAHS(ZwSch}@qPDU=@G~Fbf4)m$55}Suv&>HTdBrfZnW~a z66rEd7-0OZPT6Z`JdA)xw#S?T14%>H)n?cE^)kwM7${X3c& zM|d}ZC=ikov)466Jp#!9go^CM%uN^rspYpeIJxm35`V-}?O1Ir#BK+~9y=1r-BjG` zzF1MD^A3grCzx;%!!2lTN*ZDXpaGKYP=rqOBp~I$K*g@Foc0u|)Hgq6%2?r24??_$ zWYIgdM4Mm5DWMRDi0btLJg9T^{*czWWj~o zCMYy<#La2LnMND1G~KZf)y|Be`QZ!w=mrUg^=}{Bi9fu5NI2yxuA?Fng7n&EVMiGd zfh4h%(Z>@bNXd!YTkw$t+J9+$BnWzp?0hVpO3Y=R9LVtq3{*XYD941zeIzLa960gF zF;LnZAkt6sLd`FoB(%Q7fvs0`L@>y}WeC5^E#qBmv{+TC26QC90AWbH?Z|(O`|Vt~ z=Ysy;gbW=O)tOHwIuhnVXr6B%>w}f+HaywV`c3>@h6JR*5L^@*5%R(Wbu{0v*o_GU zOs+9O`kCS2{HJ2Cmbz?cwmMJ`=EW&;l$5>(jmQsyY8kMhPGS@-24NDAZO{jV(I(|@ zh)BQnk!&V>l|&t*0;Bt5+5|R|zfGmU_=^%j=|GKBUXcw+f>TWTMgdL9$dI5*zd@#S zf6@Uhs5b-%;N7DEMG+}v2)JGJMU&jy(5G}F3n>p01H*~b7LB9Oins}3;aeB@>6a)6 z&e{OFmG3ehDR8Z;AIY*Y5<%jwn_oX1b7wF~fX5elgQ*&Rlj(g+B?OH)xscH(a26t| zM2Y@p<_STXVQ`;i|dx-1#$f+pjPR5T?Att_7+m27M_rBpxF^ohmljg+!FH{Wg`S3?r^;&fo{KL zI&wlN&MT%uC{sE!Okyad(xf0@798PbBW_9VZ%tc?N?6jp87AOXef(9^>LB7#1&eOP zB#^!N_6|14>7z*7qyEK%UDAG56mm>X^*{f?gZ`57+%6Dbc(6_}3+8MPkHDNfjB@ug zxHrrqx3E*B4n{{HvTU4SOO)55H)N>>`yYwV#kNhM;I8Od&kD_y$>dT&3QtEX_DkY} z*ylBFln#Y6&Xbl^Z8M}W13Iy&@4E|;TTX@flK7f6GdIa#En;ozc{a6hiz?tNRmDZl z?AB)`t%bxrxXXpD8n0V5XSQi_%aCh{;s_Cd0@{&KFboy!1qj;uNlG2ClBEk%BFcHp zh-XQ$O^%Y>irgiEJV}zm&66q0U@>Vz1=M4pWq=^&`<3qnQAw2{fp&NZiGouVSSUaR z$RIs&gN7lv)Uhwj%=xF@i#g&}9I{!~aHHI8|HiZ%<#M!K$ni?n zg{scQLJ3kbbYi|3eZ-QI38IuPCgK?*RX#zCl5?ys{u%pKn_8y{+;aL%96%|N7Mzj< zd|~3w1H1ygef|H?!Pc%gZR^E0y0u8TfV{tSP_kGslN>z~0g0}=J+m4XiRhaXQ)}XH zDVm<2%H`jAfT+(@9pvN31$H2s1#=n`^#fb^fYbwtD`o~%0K86h&j^1qmJRe`GcJku z@{y?S&5OCYq=6n#oUja8FgcP2?S@eiql#5EdCzi%%N|!q#a`9rV{wiB_X8QO!6gK#M;IqoP{4Gu6 z7X-En`y(%#=iApW!9Q?NwE<}3d!9dM=9k20+)V(hjEHA7 z*;^OKOk4UnCG1P$V|i9tAS}*c$kO&}#ZjP>yon-Ki^#?(CJMJrPpoGP@%MVHAneWi zEEyyyS;g{?@TB_721Wm6gWMa@AmK`)-x-o%QX&2xxyTT&XY`|nb|CXe;Gw!usNenM zT-@(!!pLD@tDIMf4GkB-k`M*vGPOm9QdfwX{mTZQ{<1+b%6S)+I!vjo*O^hW=-%1P zLgYJHsHP3e?Ido~BotzguOk=Qq7h;L9=Wh}G@DKCVfi|85lLq9?~w~CzmxUzI+WH% zALpiqqFFW?Gq~VAicnFmf9CyaD1XTq3i3_d>{{oJ7mUoCuVN^_3&r)38!T4(EX#C* z9tH$N%5?b+I^gBqr3i@W?Xv`^g_@8yYWcSW!LL{p#Tjyxa-_BW_Mq}gP?BIxkjn*7 zp`x=PGRe>t!dZe0Dt@K3z#a&uf`3z57nLSol3d_mh(4l&*-6WxBuF4akl=l%FNrUP z1}U`gFB%+8Ym)l1tD8; zZbtjm^eBiD5pA`@7JJKT&la2kSG0#T53A1X85(gQ&kedgKGUS9)HG~YIJM?-7gPbW8MBqY7o z``2S7TYzuXgl;6p#VR@ z;4*)`_}7;Byh55NXd`lo#DBZfJKthB&pp0^mZYx}PF##`jI8Ob$+hWEs0~V$bYBiV z2q?A?+0ssYcb%JS81rsvVejiO6#nJy{W05n{=OnzB~NR`m8pI+*BQr7Dt|P|G5u@B zUY^&TjQg@_0Zz4+SG?B6tkXJeFZP=F>?RSw7X1ef%8niP{<-S7F@E`F3u?6NAgbie z+`K&a0G<7`c%{7ZfrX`E@@mv87Fy*-OvR&v>{2Vau53~qeAuTdg?-03wn2p-+34aK z*b1y5F|mDxo6vI~RBx!Xw6CgIj#-_a0sn)HpWjXSBG#85W^Zr{8VAj4b*MI@+Vz+T zXCvG6sh^n+#QBfI%M&%to}4>^*4XQ`yk9&%yS0pHaR(R9-5*2aiarQI1xeMB(b|CI zI4^CbSI$ZXmW#3e*9pcnoa&axrvgF|NIm&Kb-`*rt7WT+PA^XF?@^&Pwbw=wN!Hk& z)P1l`>TCoXfJ8v*9{N#wE%QuTE&9tAFFRYU&D#TYz`UiKsx9AZ|M0TK_$e)XF|F3i zNpITii0xY-DK#e9l-i?|MmNFW^-nFyp|}?zr)y1cC6F$<%^3TM zhdYo@ewGgu%&0$US8R}}%o(ry?EKPwn1L&QGDjL=v)45=S@TOt9CR|Vxp*jzMjw%p z`Pw+b9!F9T1FEu^=`cO8aj|WhokD}Wk9#ZTz=$2;!HSJ9(TtBAnJ0o*t?&{+R|?jB zIt}6CCmd&LwXGkXip2ff(&x(8e?-C*5Q=LW)Hn~=OHV+nMo`ng9>H_*bN!5rB~B-4k+h@78$tZ znX;PGt-pr9z;%<1Rdwr_5+PGI(Hu^R`7Zszg1!sO_-f%6V{;HQ0BTP56pG#xqNxNtG^j+w#3o$2v%h**lBP;+J zoddu2*fLPdMh0>B2pv28{l+6z0}M|d9&q2>$q35C<>cgf_bBVP#X`yM@068?TANsqEp`vEWx*K8x=2sdbcnMKy1wCayDO z2k&g&+`}l-C&I$2cQV*jo~PB;_i>Xvn-OzUA ziiDeRnCRfPZrWDN?xwbN6skGT`UU`8J+qLC>||2D zJQv1maYv)3k~)yvMz>&D9^Wmdmwk&-qB8vAe*9EVtHaak+j~~@y~w6FJM**16-v@# zJog8er-7BJhF)(S$?nDCrt|FvHuxK&Dm_jY!VTgn!@Orh9lwTW^`qxk{tXWo%DMh# zGc9LD0Bve;gJSN~*lYC3RL{&>r|NNPa0u)NpEG$>G@8laT{?4pZ!+T@1&dXHQ*6sb zevBkHKYODjttLBg-7uAh{=2M!Yv23JDnxely|}xcrm13j+$*_O($L3cFS?V^6sB%^ z$k(zm9Lh??sMq{NCIZ>n+vYjtf1GoBaG~ zN>>!^+Mf>V9X>~cQ_CJZJsr7ScsS2jE1L1agwoCh+U(5X64vDp8?0@1+LAHF6>nM2 zN+&5EU3ai$r&@IoX>1g1x09v~&gkWEtZx|;UFNfovL~I~N0WG3%k#+%fV7e}+&%1vgKJzu{Xn({KmG<>-m;@&00Pu#kaR9n?2*Xv{K{`VV8m$%Rx z+xEOGr$lZ5ZO<>~JwFn~~2OM{%k0t{a3aBfaHEm~xuJ z6E2u;R`&~79bPfN_AS#{gsX%oHQn6(inF%=Oz#=-mkc;9P6l2(+oN@j>TOUUB9 z@~3m}m5V1`y_6`$- z-(3_ni}t$vtpKBIS@OG9qbZXT&c`EA0D$H@!DI#YZ#~K5Oa^t^(#00c}Cjo!LKBuLh4MCU!evmp(kTtIq}j{Ipq( zcY|f>>7N9GlEnB$HqECV1iC3T{N3hL_pgm>_qUIda5Uj9OmA-5mt#v_MH#SKBTl1;9oqQq zljhqyMwisEpAC1ieSU5Htme3NT7)zTZ^_I{(c=J&d>*IHS*)kF!)l8O`!p+9j|t4Q zODsng{|e1dOQx_IlnG0;Op>;EHN-4GX2u$nq@L{sAlb9>W%9p!SGMoV$jfKwxC{Jxq7&A)Z)*Q<9`VnE{@#-P2Mp3Oa(>;( zU|{^85FrDd+26nXCzSR60&M@QP&VnAef|}Hh(8}wYw<_p`8-B#WTqA4N*bim$fB85 z{yCNwIwm(E=_o*$^D$ZIot={Dn$ z-d5JBzBRZL59Zsqg?I`4&wb-6XQb{p@-(cca3dNpWS=-u?xH^&%6cvZ&5ac0TJ>$V z3M6ckgr6yhBvK*f=HRBjd3v1=@gJ{9b~_%}>wZmMrFS9$kp zMsK~daoGXzW$uH*Dt0x%KrGNEsoLe&)J7jibiOwDrM9@2ABryPQ`_wz*SXpkZd?S% zDbc||-{6X#J~3~3!@h-wwj_EmePi?bQ6x&_vxg$(Q#W>raYxi4P$=4F6GeIKr8HvT z18Lh+$)EBL801R(8BBqqd1_4UhSaII1@yet1VQARO34fdw;I|Xu>CL|*csTFKX?F) zFM!da)$>?4;uDcaZt%MB=T?tc;G5D9qGrCI$k9i{erp9kM+qgP5{Yk@v#+woIwdf_N+;>angnGY_#!fL>``4SDbn)+Yq^^2^{=s9R zwCl%cGt@hQo8(gAIhXqux18Pz$;gJv+}&&Lo)B=7=IR9jD~HPEr>t!?IB(jf;uqbUe1xY#v%&O%B#5V+Yrxh% z8L>I!K_6TADm36tcYeSvm1cN^;Z>;0x?kKUEp=f(x)y!=s-h|iIbp4d+wi$`L4`(< zbc}%QfV5V!XGi(RP;CXe+T7ucx2US&^uhfBH-_eJ&pX*`qmir9Ltns-uVF|xr&hKx zaoP}0c^Gah&-ysjkFzz?j@7XjlKF_i>E;G$nM0G_y?3{q!9lAtXBNYSdzK&~Gv&re@|AyNw+`$pa*(~V6WH_@c`ZWHKUie_Q^!6;X+lk*MTe~lD4Z1p3-`JK+9xjLRC~pS1T#ubD0hF3 z^=}|oDO*^hc2rKv;-BkhyS9Bg*BZWmaE8gh%&nr$uuKS)!r`)+bZZ+)Gvr-uBTdsA zKR0VdG$oMHFchDe!~|omInXnrp$Ge^qjRKHMxVK(2>z5!?H7kp!%Tu24{s^Q?O>ig z&2fMI063lYbrRKIm*e1MmQ`DD+og|$I7Ph#V-w>U#-rvIuti}N0%!i7$dY+R3w1H~;OXoX&kEP<<%$va-Il%`;b3GS$cug?q}+0Q{fu5V>^q`_11 z?Ko(oy3ywt>eTOcMQsi7=inz147?VCo>Cmzpm%pH@jMwcr2IY7doCF5?rR&I&aPnt8c$L%F6hvThIS7mk_~Z)|TIc}1T+WQ-NblC=x8jz_8X zNu$uu^)JNr$J7nk3_J5>i?b80B=-$!tKy{lhf5F0rigIbN9j9QW4Q#gR7mbm(`WEG zC2%AbpPBcPdUU%q#R4&Rgtg)}HEW}G;}1cvuYVeik!tVFv4|>itb&;?_Ib$VOqRpnP#aVBw z`)r@}b+Wg>kWgtORYUUNb(hqrv5$r#z#F59SSrOd5sBZ-u3Np_x4h%-2_=yrV0FEu zry1;*WevC-JUap1z>=~2F^~Gap;kPeL9L~8-$Z{S4>#{2<7)tI`bfc|#7Dx+(A03d zi}TO)#h0~%UoLR^xZ^VmO9Dj1c-n;hUAYf(cr2Y2@`tn9gkEX6yae9$S#5P^dJeC> zM;?;Rx;{TqBRemZZfM6~^MO5I@;=v5Cypvok7sY78nifU8(Q%(-pV4{3T5a&X;9zc z?u~4KB#On2gb{h-d)NX=^CFuUF|ZxON~A#R%w8D+a$u+c-0W{9hb~p zh}QEdqrJSbtSbm@a7sRMIr7ANBU*=vK!3eVx?9N=bP#rjcKF%HBbk!}uZnpSI)zOd zpKYC>q*~=ZV>Vsx-F%NVx1vLMD2GI+r@pn7LzNSJ&&|rXPyOz^>gix@tz}{8w0IZa zA(_Xg{jlM<%91L<8{vp8`i{3w=WSkaSR*m7Ui9Ler%ffdC^lB75 zVye%*H^3`c^6%AAE}Lg1c#6!UjI`6x@2}#Y-ycz~VU86r{89~MYQ+WlkqO*lu*9%i zfKnt|-PuD{T2}OM&-!sLR(AVss|>Pqs_RDc`L{lXY}+r4!o0kN0UT8rGTl#@U!4@%53$w=iF|}l zBrcVniI7^MGDI0kxYk;U=8H8!m`1tqdYPYci2>S!d0}3qu+~i5n~iZCV$=7^_YWU; z*FzE$Mu#mgm#0myht1H|3skzURy?M`kRA*O&sE=O9uL{6{@%&nk;zT(fDLkI8Rz^S zw3isor2azWeaNLiNdlq_M2E^&OzM1}mtDYz>=g+by$=R^TXp(%u&2*i_UvJFKG z88Ezd(i=KOMDQbT!hGeBlA{RR#catVxd=2u{US`1>hTef-sFe1UGh`Y;q$}P_dUI= z<0Kfiluoh5-QIBprky;@!rrh?>wcFM;JAz|oJuwG27KiLWHG2jih?DvGFU5yH8@-m z{)Ny1^T_IKi(Wmzp48Ab$nhj;C8S-bCd?)W2;kB4*K-2qFp+UOA*i{Cb{krRQIbfQ z<%-#I1c(x{qT3*rJ(nBm%cxgr9swfLsm)p}!eDe+{ClTnxAt z3FtQiV?qTF-C1-3BneGd1j?wf@5&SbrYrkxR^iCcPz(-6yKlzv-+=2u^F^v)2XkyPC5wNds zycL=~!#I1p45~hNdFfR=A0bJzxK{uLI0_V^G*B-n5P1hN>Is7&J@h582PE2B*>^}` zaeXXnr|)w@4m@NcyGPnNXMl>7jPH7-3H-^>4qrMb#a4mJm~vWs_Ch7m7^zeID6g}L zoT%#hy$9f4QOG3q1&jxvp; zD84Da0N2l)dCKf))L>2hE{e2ML9t=_MVdT}s!!pK)cGm7FNgwl8BUT%{Mj&-%hb#( z5G!BCTMHD0SO*)ZiLn!SzQ#P+_6<&VG!XKL`kwz$P^bC?s8i zK&Q{s`^wXh%JFBQeOgZjS&@q~KHfEjI7a0IfjkvGb`JTpul`;V(xX`mVXA@sS@3t7 zoHEV)QgT^*E?5cF{OA4lk|C+bOU@>uo|hfHqeUx>po9(PkMN0Ind0#Y2}%PCDh(OT zIKPhIN$1ZqzEYN@BNhs%qTByr{v<_ZL8hoIW7 z^&*FpEF%DuKB=puP?xb%gx|FDs#qa0!$n+UAf!M5)6G(My{M#8P{6T7P(=Qfgrt8@ zHwPo4rk?@-@@<6DlF2bQJRWt}c`UkF_X#JiJh_PHZ120em^jx9?@nJx73Q`Bn^Q5@ zb0HHl>qp&21vq2U5=7K9+*MhJQs19kFz1+Q9EyL5_pWx5X0D4vlW&{qu@)T!@EasV zn1#NeN(GGdq)Gr|G^f`H=$*M1JEK+JNhJ8>1iB_6 z#Gv^vWz6OrGm{)Uur*?d6+_1CA~|$YoS2{^RD*|Xr6o^52|4SUYC$msq5166 z{7rzldE6|d{&1VFa~cXOp+I%twLS z%PAzN938o+9zOkCu3Y>zxy2gE5f3i1O*!%}W6bXNo!`b7ekba*z{sNX7d&hpelgUI z-;>RYD)qo?Kn8vs>5x!JoO;s%oTq%6QFQkJaWhCEmI*hUfy~A%GChbr77$b~braPj z>%x{x9;ZF0wJ$b36o-I4DpGA1?Dd|Y;K`UtW!io) z)lqgDJjY&u{w?(Q7|>lD^970ffMGfboVxv9ML{ly(y68m9A(ujKp0bGZVAbGZFbso zKI^^{Www|%?$v}3g>DiCtVa!8`=<_tFb^xWCmf=6j#~mL-nj%&hk|DjwxtOx_c>kZ zq-?amkO0mw4=BB!4H}HgwCpcojMP|sz^U@FhuI2j;gnb5Z(&SBm7||mJdx=&i|t$H z9?j4Z9!pMvYHY58TrrTi?+7+-+Ub~H5}*!6IKJ(#I+XaHU##uT5=%u>=Yxw%Ye5iT zyCk8rYK9I%a!7?F-yy1#5Jn9(X=|>}B+I(6d&WH16hIB6E5s+nvnwMOvg1jK-9`9yriC9YnOL zt@&CLn4kzr{f9FHnrkb}I`=D?hA)13FZn~I`S%k$<^i61X2GO;E1{MC$)Q0f8dxtr}!2ua@!54=U+4~@5&Y=B2Pq3w!Pj%U@+)YWm z7fL;PFfK{+U+A$orkr9kGv^7`XWN^;i7&3ceT8O)L($OTeUXPqcOdJ;cs(^r+-heW z_;oN{KxHjzJwLiTD7#F<)A;gQ;%&0Q^K_^GS}~OfA9`=hn4*JCO-FSy<+C!SLmEX`BMavZ4{S?DtqR|=G)=h3dWxJk*mL^bZ#3}u8+GiE8S$SH^(IJA#(B# zfdGC4;t^flB*~hs@hDLB$g_^Nhcqq!uTh0dI_nf_=S}h{A8vF*-)i%VMIyag5~(;> zXj$t^3=St#W~+6FiNC(!LuRBi+FrAxbhn1ba(P2!J*c>qwkNJgKpldzoVj%YzI^Q` zIhF1yu8uQhV*`!1Dr9-dpm}S(lV<`FCWD)JueMz|JRng>C+ppnHL*1J@jF_eHM>^0 zyXX(--jsvh;q7*}&UW&u>rI?>Foy0p+|3n&N4cqY_4mA25IlTYs&Z|A#|Y=e_H-~kQgtDV^h~@W(GL@h4Gr$IdR)$l zqbi@?`WF7cg+Ka`aQk$!`Y;01w4uA?B;dH@bx(KeX5?)A6(48vp^DS0rL1h7oMfIZ zi5PFzEY?ySh1U3N(;KaMlxP*!%QELopr*3f|7o8?FZ)CN3 zoNC_6I&yyH<3GW)QMLu5KFfVBzOQ_8@bijhiBg?&W34X`wHYTkA1&U8SoS{1lCX@R ztApFhSZv>276mIuFP5$wRqxoKx70tXgPz@Tcr843&Sx8EyBe9yUhDO|-%YR!)8E7( zKZs%!_*wO4zYo50AZ#&*Djy8)Fupn@6Rn26G_{{!#`SUGVYk|fzV2`XgC69Jp#o)v zqab)&@yH?zQGB+6G$fudr2b0pBPp&!>qd9tlDrzCW@n8rS$Emt(9NBH$U`^L7g1eJ zO^<8saP(YM>1|TyGl_WDe5r$1>vruc9A%@!PNRXvqL7-kurqK#fNcc7dp9RnrfkE}l;B{cd3dR9^H{4E_z?#w?=~2chzlMyp27CBnE#ta>88mKZ`UNniefcR zJ$iXPTgQc^41h+D4(4ob$`fS`-Xa;3RgA&PtGF0H&`o(cQW}~xdrVbthgLEAL%rkH z@wmpYd;(jW;~w&7Xja?NPdnFpj&8?%7S*3$$zt(((TMTe->jcVX>q<>;I|2AsgQKE z;3Gd?IDhSItWS_zb?t)hLX?Dk>GQCV--0@9=B;b#%saef_Q&5Gr52jM~l{67t#~9a^UvXKnZSafdKZ zayUtMuW3_mRWkgd6**LOPynOcFq-i~=NLVi-poz_qnYo*H}!MWfDd=sI-2e9^H3O+ z42U<*y>X+Dxcijgkb71uH|@+WQp>U#D;$#QYm>HTl}~Z9tjjv5SG8@>3|YDNb5yK6 z?#n@e!pu!qC+ki7P*;9cq$w}AP1nh~f5u@rgb+K9(^TF*&x6VtD;u|+>#gvpTOdDz zd{vHtV)UwIK2mbDR!255``3O}_A&im-))CP{YFQyH(Qd0?|d!+PVRk8QK?Kvvjh7_ zHqln@*c7jIN5k#vKK8~HAn@1xL=Kdxb|*-fMygx*t~dtrQ3{&4U7d1sCdogIy#cG<(9XEG zTzuJ!HOx`QSX-Y1BR+lJ&uGWbyVP=T>v&!K`Tg39_A3e#FTz^Aw5{``&D>CP`8l`? zXWMf-?^h2z&VY%ItB!sqPn+f$%4YNM@h+zL(WFOE^yDr$nX~jwmrg~e<@36+!znKn z<@%+SgRT?0>^%ksAJnV{DP4s`=YFo!>T@a!g{mPP9II!;k?aFu^87j7C|!%6OkJjX zGiitE88294_*^HL&d0V8j^nguR(!jw{2STSBfq#Rp6}8Djy~hgbAz^ff#SO|UUV40 zUkPCr3^TUX-VYZXUiKV<9M10fw?lTGcffZ7xLEJlm;-KQ-VD(bl3qvWAN`23T4cdc zm|SU-GZz2NgYuUY1t2V*c2}nde?$0JfyVE-SH&lZIy^ue+V3bgrT_n!>AzNG(Eptk z2AG)rJHPFJGx7T0^V{Ha{-^l1-(z(&=rETAj8hMA6a2p#>;E~o?f20Bhuk(rS~2P` zBh(|~Q;NgVG~!a?)Z+UJ@UqhKmlETO^vv?~H4TjPAVh)Fh$djfAjC{n`>*Tk<8d+L z>q?p+jy+&(6QqqfBz=o9F#m@3ej8|@O5os@0cg+m|0UYvU}OCIoCDC_AMMOvXz#an z=J(CX73LEV=pE3fPoSuP^Y@Q@Hc;U)hk?ov84$Ea#1%B91m!$xXdtu@c5yQLwDEvc zC?tw7n3nITwxXH?0*!uZOUH@@w8|9{!sXvr%S_8Dtkuvg23$^0#(aK@XVe~pT(78aQy5er-;exx1uso{c;-%wY%DG;EldtgPYg~hzpff3aKFSU zruF3R?n4eG3jTq66&wJ#*ZrSxFX?Z%*Fyh@6KR`|W{?0ggv_oQ6plnmfVxM-w+mnE zNr)UJ+#iEnm?saLmxm-a@p-pTfD)KYOit(v8AsI)lsOVn{^!p`!Yxuk3bCkP@{iQq z`Z!VqB&r1!K*NOXK#$Rm3Xz~HVia&x(@198SiDJ){dX<`2+P0i>?GHff@sS%{>FejE*eOKJYJNaO z6KMU&5iAl3hECFeHB}yFj}tI7hh&)(Cr*b=@=vaq<`h6IhoHjcL2g6dlfeg_2xoO|n-VE9vTh8_wE3Dm#8 ztToh6DARcHaT;Za0&_qNpqxQuOaVs$g|PzzGwTr+peHj%Cu)2kwE{xUZ3ujb*=>PU z3ls=xlCD~7(x_k#^O*5AMn#{sIr_^uLqMu|D?p6ciH5uo?rusaJQXZNU~6U}9gH34 zXLQ}&jEDdtMGrk&a-1a1`tx9{)~cJim+02LJ)4zx-Eo4*{U>GLP~?(tDt`IEBuJxxCji9`%G;WTtL0!fy$a|2Ar=E zR_(`5^~seMqAr(YVt%0)eV+0=IN~zLkuxOzUvzKnKjTykb?D3F$qdb2H(Q) zeB$8`2hB-`56Y+p>ke70Q}`(_li$T8XBCsw$DvdyO$sROOoS4|WcbB=5lAxy%%cVi z?R6}Zx4%_0MP}(=g=L`^&uNbKWfa(fM1krVXvkra^Y=5FT7vb6w7@5okXv%zs3;qZ zJJ0IQeleGXNrK~Xasn2KKA9>p9{`P)`=2$Kg62f&X@*j;B3p_qVD%8!v{@HEFi*gu zMt#QyHP?hKZn%r>Q9w2eqwR9OD9xM*mb>d$c?r9{n z6in!TpGm>_Y&v?*Zd{c#U86RB`=VEBe&sTQ3Ykz(oeO?WbBmP@SxBrFY+iy@JHu&# zq~X(e?0AU0MB_J^pR3+n=0?6}<6NPJL;w0=KJ=x;SGc@p#?fI*zmFU;cADlZqN|w| z3sns+0a;3YXbfU52)J}fAAG}stATFK1eaWy+u|I_PiAUdIAgxas2Id?gZ5oZ#Z2;| z=1MuQsDOaW4uZu)Eec=<6a1QL?BSypFuWd;{ZAwELjBK-!_g3M7S!2|=5hE{1>NSH z`w=KXb5!)nAjhO;n^3o5p{!6?b^Slp#T6Y&mYcV*6ttdXS-_woK@OaInE4-0{? zn%JO2w@foPN+ZQ^hNLEuL#>p~Kg`yBmdJjQaEcc-GSfZ+^@#srbA{D%HF=e^=Lpb2 z&;CmXJpnI$&zMpO&_U1qt%LqSLfWJX>0mkv6KKNuGS z+A=2D65hxnNh-Z4mA{dW#LWQ9nC5wC5zE$hU|TPL&L*c;AfSYsjBq(T1N^eAwqnrv z8$H`~Ze>#GK6_|c)KAvkED`)WR1LUB#pQO1g+OUkU#k6a{m32?=>1?}J%l)|5D0)| zrs@cNn#8gvX7>>%P|M|PnVml;&zDfRa;vnAB;CKiw&>HMaA;8F?7nfM6EzqUW?*Pb zcOo0}&sp>1%O=%XOXtnm%v)U+>{9fG*n=Ke;1*qYotf?VCM)b9RFb-G=g*j@2a_%j zm+4%eVFXPEdM(?Mf z3Mi0baPZ9AhG2xLsdS8O5VF||8djE6Q_D^ItHT zmoR#B@7bMEBURpvjET@r0+gOZkMtqK=jTI}f%f)9&zqI=%21dvJnc$xT~eORK+8y| zjSR7jmb-%YsbGgh7FqnPE|zzgp=Su0ZYfD#qsxmW-)Lta+`E~K;7RoE#Dw8{6S`Ba z2O`&&oQ9X@>Xu}0eC>{nJ-i0$%GV0Dn4-XPzV;ioV#N4&@E7Os5FebcQ_RCWqd>OJ5MFV*SFR9%}x+zE)N2l5jV@9+6 zA(1k%ZYs(~3JCK16h=SK&R%9&p~BnoP7{&c%f^lTEo*z0b~ENQ{M{(5^{t#}Mt6_= zbC=B9#k=;nKUiP(Pj+GOwV-T0w$~4yN@H$G6U)!!iH34}_Kma0dBSH2FG9ghz@2tI zvNcKVE|TKWYIMpz=pILY@Sgfp{$aD--wcqzW7I3|DDZa!; z&FN%Dj`BP*+n&h-X)99GxTfRz@*Dp?f;*U|@oA_1%gVb$yElC&k6nlQbt%Z$ zP#=h@)A@^;m{VQv?2mnv?(CuCU~L$jeQc!WzVk!<_i#MPHZG>RvqYLI$|rG0>dk?h z!lPvf9Wxl!FXfYmnH2!@IrHdLxn{I?V&`*ogMa$iEJ1!osgC?Xcpwb;{gZ7~yLiN` zM0S1Gw$TTDmor7PJPFP_dDq6;2P;d@r-;j(zTr>V75=ex4av-w$46A~OYFE9K9 z<(s{Ud*dpJ-V|sHSi+X5Z4(_lx-lB7-N|RsZdB0sRppEnS0~)tWv})&KSsHC_pY92 z*3(zKB|6z}D1Bt^;prQ=SWA5hN*}3*rg!XOUZ|oo-|B?c!}`P>nCM&-kEPq0+n;NP za(%ctPPq5tPx6LgWK2X`FS^p~Hk)dEr1r(#22Oo}WeZb9)sE_OmdTllHKnFXpza>u zSNFX#sQm8(o-EY@Ta(u>+8wsq3IwlS*tYOK96bs3p-YOwnL5<{U>JPOM+* zKU`ldV~}XqZUCLVUWZFOQ7!!#$L6MnK%3yACm0?_YEC)l-0_J#b7n_Pv98K>UJj4nIl$3P}c+$@H)9PC3EP7LoVsQT*@axqQaXv35u zS{yO{2?9e{D(P?-%gF*;Sxlf*_lu?nva zo`sv|(PEsRXWjEV_Y#I5rrrA|s_W?i`+Dwg=u6yGJ#8Ny_DemomS4OlH-5UX9u~vS z5J+Aq`>mHk%;6mC!b^`Ouv4m9{(7xr92Q9pc71!yRI*>QKT4IP6Xj~)Oe*y`JF!OE zn!hoqu^%u2o(Y59|dx_91+Sz)OwSs`z-V^7e z0xZL8*F#et0=f3Oj)fal;S_mkW>oi8Gc>pGeL8IQExBl<_bJg4Q^{=SsAf~KY@~iD z>1XQ>gKasKF+c6hAJF)%q-&expn+F!A+x;s9{t8#oY%1>z$+SP?PnN;nYT)J?0zk_ z8^_nU>Q=`aVR*H6Ku31?Ez(zb8L2zA$I+{86TYG%uW~#iD;O12!ZZrkIVF0XAy7VM5r-nI?BYv72UC_b)I2e>lxXmAIlxd z!U!l!y)UxK!sD>wgp-DEVKT>_)*TlPy}O^?ZXhnCpZV+*T{{-X88$qnI~rUL;v#7s z{41MPY%%aU>^j0FZSAMy*Xm7-yJ}=R^y()LvyRHA@4iv7y05wL24}3PfJLhEWeT#s zdOMy)V$9Z%NSd_J{t8#L-OWE*Ap z5`J#nuypHBW7!x(slV3^Yj4&f#g=8M<*vqR#dlU1qghH$4qvqz z=jxp^^4ebgK(_0EjDyi;aF;Ifiu$Ig%`fGaM5L#v(UU@}B5ZUX{Uw_`AUIeT|A6ep z(XwD?dlW_zo*GXf-uo(ZUmR*HsyapKV&CHg zW{gCe9WpMqewx#R+V3f_>*dqIWx!_rwfj4Bbggq%f(gpjLD~Fdr*C(Z` za?{iw>bc8Z@%G1Q`a34(Kx>&ZL$Wl0?*07Om-~zEb)*$mFEMb64E@5y&Hv^Y5jFE~ zjgR77Crx_RXWoC&z2Ev3a*$81n*h>e^Z$@E{aXeZfHcvw(J(R5|9@tXEiFN;^YMf6 z7Z_xM+vXyw$ji&CgqEnW()iJnpfJXb2aJ;-A(e;R>k9MP@y8OOm1vZf=n0oAYn6ZZ z-WEZ#P!`gfQK2$F@L-A$r~$LQd+{j_c9CHp8TWeFICwa3f8e3^^zURx*SJlA^ygjO zow8*)JprK(DoFbd8)KXsx z774|m!i4L8BvL6z5__tBAps~2#r>p3Um;1*`?g>NMTSB!NzOZKQ2LOON#$`sw#0fp z^hZPk#Xx_jkSVIlf#D2zVxynEKh?k*Q{lDN0jJVInRES8mjI-r6DzAh0Y?*L$mRoS$b{mo3+!jTt#m`l9&$3pg+t7cQ6`jgrr3Ip{bL*YC` z$yxTo_KN+EAWJYQ{zTM|@dI@|4X8NmCr(5#b&j)d8y!%V9TX@`AafgPisH7kgdDtF z1`sY7o~#xHn!YcI6!HhEuX)8PKS8oL43KCn1%KZ9{f=ntq~+i4l6+uzq1vJe5{w)z zApdAqfZJG&TIh4Li{2OGNsojmhGma<*#HAVC9d9GX zz8QawXAj9l45S)R{lttH5Eru!PcVf`!4vo0j{y;K&8fB`Fro}sKzJJ->_E@(P#{l< z+Vuli0isl7r^r9Q%;Bg6q zKzNv*J_d;qByUd5RV51MfC4}CpZqa{Klx+YnWy}jz^0#k`!FZqp@7{MVH2o&+iM^w zG>m@ajAJf-=a2QBEC+t##f6Y`v1*?#7%zIVu|#+I@>^X( z`&a%L;=kpOIsaCd2mtcO0O}I)Wh*s`5?iQ7&Q58pd7=$dw~l5`DNTH>%BYaVotDD* z)qtpbXSLvE%RQ&Yyk6XIWhiAtaRh&>ONjnbmoO-ENQ(^^XbC9eMIfcdMK(E%aVd6^ z0P!G;3r9#MEChiL<{`1}11Ww2W?|6wnihuA_TvLp{T`x+Ee0@}TlTSDYV|1{!`_sHzs}y70^%pv7mL_f-Ar>MtL6Y-F z>?jnVjv9_?uOOKmA8zSO%{WeG7Lb9JJtQ#$^2fA*MM3Ik0JLd0S4hfa-JgM$7TDwq zkk<5FVu5-6rqc?oBNplaDI5Y^Cb&};ziGqw+fP3idYqd+0hPqlR|=ulgQUyX0=&i* z6`G~o(hPrljnVtWoxC1vV)!)jsl+-T<_CTXhS(bfdkLGjbq53+I3CZ*{6(91;Ov3` zv?&2Vn|}SKO{eVcRcX`o(V`D8dbtKy4>PEj7);c;#^-0ZSej6UglfShB^ovIUoDU% zdQ>YxO02pkToIftbn4N2vUb`EK%c&_Uw;~%K+(P z`pk{MUK}}Eal5#(l_`4I)kKn}e~U|&Dhc}!K2*NRDOths3-cu6rf9DctbA zv}ga+N&pBut+Ig<@PCT8aZN%15%{cQ&MH{ypj%~mO+j5rnW<;3;J4ZdZr$q8F~Gi% z51ugpg-8kVn3;f*-Ys1)z7^Kk)OY1GGbN0|wu)`rG*pCfy$6+})yJu|g><3S@-%e@ zfgE(WR21YFBG9_xnVp`X*YpF@Re_ZtLd9aUZ*q4q;_%Ua>h15f&4BbVyuZ@Ns1JxF zqOV8+>0?Z;iE1ha?W8H1kJCC6N(LExXrd0}cCv-h_Rxhvvy|i7g}Xn@Iw|d-@AQch##SSr+OCsWX;htuv-n)RZ_jL1uE4& zB1K^>$Mhhj7sGl}jKR4Z=7SMT+Sp8r5K@*8E0dChoy$Yp1k=eE14On3J0YGmNpV@Q zV$!#HYHe-e7N=%E#rv8t)x)E@s)C=%%4P(ia3wX%%QN2s z#ZUU7U@;O49s6YkLtXd7Wre&vvpV3; zE7heXXF6k;mEvP3?Qy;M^D1$<&j!!!RFPRbm`vn;1^H4?H0}q3&;81J*a2E#l;mrt z0<0EFWbn}n;&Z*!AgIkb3#S>qThfwjPP_U2z`V3cH?-!PXMyYJsiEO5WPFi0arAO! z7xwky1lh$(Q$tGz@)AXJSzU!_AN!rm;_HhdF7>z%K@5h!LM3i~5dw}T?#D~{>u8-c z2>8CX+R~KAI(MJW;84f;Q`YKu`{$VXhcLJE`sHfysWhEt93R~2ud@ZO>5|0G7Z`=0 zhuzGc%G~A#W|!EV4w{+GTd%Y|ldcl6x$=pJje}OZBeimtR#%kv>Gk|ajrthUQSMNS z=e3v#We+(j-Oh4?GX%k1Og0tQ>yPrcZgfoX+St~@7wZiUdtSwwp*+5!gW^0v>M7^q zxW!15aG`?M+*!qY2ag7hr6mtQ=~7SE4s1k`7bL%Emi1B`K)Y*r;=C-+t6)}n5vUs6 z%z3QreX2uxfGro~Z!RL-+pd@BR(^7M+ytzGPmOZ!o6Hi0x*WU4nP}7sE`+X8&d5IS zfpc7P@Akr!zjj`|h#Ez zF)>~BGz@TJ{E!&3bT4_Mc03tdp=$p1M0QXct(HAo|$IO@ZmXvG#DMd-qYhO77yF`qxjY%KK~hE`F$>I z9S-g-8UKBi*8Z$%)14f;0gLRef@eE+i~Md1$LB0M2h1k`!{a^+Et3?lmGx(UN@04} z`&&fuD6&lW~PhT}tNTp)`(Bb{KlVYDqb2!5r$SJEbeS}PS5iSw}f1{8=v zN7Wfiy54h+Wo&ko=9Z=?f(DPhzIy#(0Y>EFj5elnVoa=09weu{-8FHRjIYY{YB~ZB zZp5Tl!OhS_a2FNq+ma`qmMTY+%Jbu0B@+)k(^H+z)thJQ`P{X(+K2ebeTtWcj&^*z zm;1_WxS5uN8yl5e!=WE&G=Ar1}IH*lrl?cv=5Dy?|3OC z8F74uDR)DI4~1~h_FrzZ2svwyE*~>+5m<-de6qY>At3hbz?;0Inh!Y!Ek8HJ-?=$E z%F!jq31ZuCp11H;8E%lOOmq2rD}R->msyE|{k(1>JlB}oj?M{p?>}dP$Qxl%vb7g; zjp+V7@ovtY3U6Z-Y84m#p0g=JsM=^%Empt56ejueOv6=G;YyC4FX?`toGQG0xVhPI z%|;A2K=FiJ+%TEEmUd~vwMjybn-Bdk9(ifXD(PY9$V{eK_bgGjIgGwCvUXgm^lU|m zb$61oQ`hErBQlWe^MLUByZPz&4H*w6+4KDk`X!IIj_qa#-tMgGZ0zIpc%Oob?y&~% zJG5s8(nz%ptSg{~iy4lft2R4stRQJM*NVgW&xwV`ZtL}wTzf^$4rS&7FE3;mpaI3Q z?eB$qU^>yQFQ@h>qWxCSMwuRO6B8C)Muk}1vWYG0I+y+3*C-z)&zW(v#I4VBb;(s4 ziiIZ1wO*yM-J8BZQ20;xxJY(APe{%5)k|HilWySo2q}6+zM}q&2*q14+SJW3PG7QO z-@mOkETe3pV*XSUy7Jy|f2qHvL38m_o%r$dG*d0>*Za%&#+@SD;kVOn_i%@0{U03R z$(gGTTm(*y11HSIc#oqRdlsAbMSz^*uVB%PS0C?H;u+ct!)g6btxONwCq6nV=T>en z0?P-6C+}0hFG3SoeGJ+K&5xembXz6-RHx=hH!-}5GRG}LqR%31#*-$>nnpq3w6TL6 z51P7lR|;ntFM{z@U@+ZZQyjqgPhd{w!lA!RZVbQO)lIJ!P}sjJ_Z3!iAG(HIguy2k zoeds}YB_swPB*+d4`aZsx3#75puppP>>HeR73E~04f&K(d3Wk3-_o>h2bHgZyv;)$ z9>iCxsIvX+czl&tW+~qp2X9tXu^~Fp+=uya?;w8NK3&=xPP5NeW!Xng>}&3@1F?3+#G$9-f)aB$*~gVLj@0=gZw3_F#DV* z)0cEr5V}FsY|SEaE;aA0oJEpqzV_||IG;*ra{u1{>OdDn6HbE0@ zG^ToxM8vxc>8L)q2rD0-cVPG-l0MIac z#@~)&02gDVVPas{{g3j?KjaMX#_gPp9RK-lHO>F~+N=$j2-E}IPnRo3>N09z{pt_5 ztFubY=KF0PbW!No2-A?c;2~CdwH)WQ5E$paL%y?QG?n1Bg#kG%Mab!73pu$3R+i`X-CFK4Rbko74cdY-1U@DFsHH(J2J1qKZYC@NkBO9u`L?ENS@>c!Vtxr(P> z#RcEVs9|}1xPa$oDyRKIfmYO)HuKcZxm2aSHdoJzJHL7N(>F~bDRSrevf5_dodC@Q zYH%tpv+Hp5Hj(@2Z0+Xg>m2t{R^Qr%2gjTJb1n>H=vRmz=^9kR69{w&Tkoq?+Dv2= zW#o+}FRw!uenUL74dyZ#G-!(fftQC6c;4(BKLgtHaQNjOl8x|XJBMy013gf7e1#t0 zbb0TZYY(!Y`oB3ePfQii@?K!48*AB0HeH-JWIwJHXTKeUpJb)h7c@D&9cHIEoj5#b zy%uBJ_*f4ZVJ9_a&N(h@%6cE2-}p2-{G8Z<5q#@*yV0zkd_0e4^}aiPSbufeG3oP< z=D9tt2wk&0I7r>zQH9^IS0P<0+)3QA`0Dfg96*z~>@Z;QvD|gMlD56Mb^dD<@9exf zfOq-fI(#Iw?dx*Ay^*oW&w)-4iDsI%hxfB^m#P)D(`CD*^}#{28{99pO5C|&6mqv<-6<^uZOcGoov`HSKFmU z7^+Mbax7>zACm_VlgOhoI$mcAF^C&opRdX{o2zZk)^)6GY-!vyfd;juH6_yurs`^z zcs80T6%*i%D@s;8V#?Qn-=fUu8ZcyPI@wu$@-V$vBR24a@V_pg9X<|ap4*xESV4dGJC$cKb6rLm9ya3Ba1)i=%aJtOmJSn{3{F>(79XWp zx(Qep)wnym>m#Yw1ojF+RU=lbQ_R)ru2KuH(r#j|uaHa_WDYm0O9=4J4@s--;4d#8 z!E3=DFOnGGR42PqSMLs+wLXeRLB72+5E)*#-*Xj~w&h$r=C$%&Orp=Dj`7ZM&v9`= z@ZhTPy@uEGgKux|6IUTPMS#UbJmNby68jYL@0rORLTP6DBb`B8j}EdbxvGxn;9MMC zU|odjxhA%c_RvV*o7me=e^Efy$p-3c-^VibMfTQG*MfieXtPUeNe&jrfvABw9wdTecHRVY<(`T2YF(PvpWb4G9CD=ZE9XH zow@R>;C>1oruSD7Z?7#y@>;SNNvscZqL03YHtjH&{F;dFkXfraj}pmB#92E&R+rxOB|Oidyf5G`(m;<<09Wo zhcaw^>ekUlh`&*!bsjY7#-5GwvVF5NQm&@HnkkKvh{O`cz^A&sSfNk%x;(YLvfE>H zst~OngFno!2o*JWa+hmmkHCv|5CMLbnVl7T%{?sk!)DxW!|QflAGX5>PCO*au?!F?X)u)bjGh;~P;{Mw>5z_H_WfTbWx z#@BF~D2W$kGC!C4+9b6xOlrO_KM%5JiMuvKjC;M`Dj#f#(3i-6UzNI|)J68Poq8z5 zQ8je{9=y2VVtE=uOM7>4BK4LuJ6&YzeOSb{fN>noe@-k)W@NIpt>?qV5-cfr)~Crf z2&3gf$>y-qsM_ph{WN<|hw8WflrN##-P+#Vl~Ec;?lJ^&==H_A>=wGF#``J?{xlZvm&QtK>|d=#rEq{R?cZCOm02UmU%P<6+g^1YtAG{5f3}_@ zQ*pp!?A&TH|2Xt1g_-NmQ}raSbxuEtR#Z9v-T9w)9T+;a@c(E@Spr^tdtOuZ>>oz} z_bClYs$c)K^+s=8ME>j*VcIMKgkU`Zue*?d1Mx%0<`DF`RAsn;#z%JH!r!mRjx^Qx zr~lu7*eT;*q`CeY-C*s%hW_ajJ!NY*;lH}lJ5`kb*P+@Xcj%!1?vjGM{*LpnzW;Sk zPeJ|n_`g${0^W??tnHk^f9(NSpHd$yd}waaC{j+w4EQqEW&j!3zczYhH!BiA zv&YOr8*pMA_o_qx`5->C3pD6ITmRU3%vSkxvc4lsUYkRF%21Fj2b>dI1)dJx>=#OK z8Hn?Uzb{`;`%Ws+pQjZs#sRwibL%NMYIzxA2ni`EDXADGOC)P)yWLXV-*;vj=9_(npZ=NtIq!3xcYD6?eCIu7gWHhsV*FV? z$-E4CsLVZ&?$ENMWQVw4f7ac>T{6$YZJhTndeN_H<48XE^0KoUR;?g?W>p52sz<5l ztQ!dVasG>=&7w0qWn?<1^hVWu8uLN7r>W_AgGFgx;2)_M(y4V#4xarEPqMR9E>NFb zzg+v2N^N}F^!V@WOR2R@j~nZDZtwq5G~;Be1~mt(c7^C@`;A5R2AY+ZR!N~yc?u|$ zBS+zg5l}fodXZ^d2T+uHCOY~C!(WNec4pI!85@E;C*qa%yghvOk;l{e>u)D0dYE)S zvQd5WY}Tq3N~TUZr>!d2DjP4?Ei~J8TFbD|n)sXih7iZ7W!PU-rT!adb?@k1t%;HH zxR<_@?reNvr6z$rrv9VN(sgBcUrF)B!T|rA+yjFFiCHa!*B^ek@sQkjjnq(oBWC_! zhDFqL$s5l9YVDZj+2v}cKIcvt)!XQ%^}A(LOckvOIQL~2cXgdTd9PElDCWVyezZkW zMBt^e*Y1`VWUt>$Cmvjz3=X*8N-=ohWgmAtDNn8*yo{{V(Vbv%0rxS~`|vWHj0{fm zeAUV4_I@~pbIPkz^iFN?^YfTqQV^qD zRa3_;wU_jcc&MGLj@J*kD1Sxv;;fY0p_PMHp1H0@C-fsdwrzdiWJ2ouMN8v3SPsX> zcvNJ1IeajhGtk|3Po=9v);g*B(X$KH;8C?sFOFD@T% zga3S2+p@$Ks}q>Hyl+cNN6q*(_>7k&pCf`l_Exquy;qbGi!Li@J9am#bf$RRtyxkp zP+m&@C(w)0jV&#@f#;5t*kvuA93^kKWcj3L(eq#9OPzL0ZF4=-;A-{lTdZvgapARu zFlrF4`j~WfyNXrOg75{|GK&9wd!Jh@9o>3%PRNlc|J4oMMw^ww{DX^{ohoiMc8L`S ztM0+act29OBOCBf@!LJKwo&^iE1MI~Z;GmWa>{{HYOG)VQSWx6CB^o1$-W1Z9#+|w zJPAtkKQ4|f`22+$soM@F=M@qM6{tCx#hQ3=Uw!ky!S_c~j?3wNskHhW*&2Sr$0D_% zW|QmG;BWmcuZc@~Feb{eYXY>)$)ud;trf8gKQ#O0A0GIwy5@B+5x-6SP2jSGKl6fX z%^#c2Bfs_ka^%z-`*C}&4Y)6A{H*ki@*_R$tBUd4&1WPgEnAYmd9B4~Y+P_+;p>g> zW^^wti(O{5e=y!dW|gU&(IdAz*yj_^?mY7a<>G;PteGo5rF(3_(qiKri=*<}`|o<9 z>zb9vklj*?_xEoxd)A2i{WRV;t>)Ut+_=u%@N0F|bDS3)mh|o%n}Yv4GCM+LLCE^v z1L3~M#)m}*%sxNa%Osk)rWn-~XYK@XsK;MDE_%~0Rj1=C; z7D^o4O`Jcq!%bsvf=69J^UL&P!d4;yTrJ6@hz+%V#W*fAs2{SnXm7`K03XwsCWdJe0j_3JjXf?{8Q4Apb?8 zQtwWw!{fG6 zk0a&B9Jrw@cmJeDr-bF+oOh2S3GwsbZ9$cdFpyu{hVv}NWQX+25PORyD^P9B?CU{S6 zCwR@O)Gj%u?UQP2c_rK|@swYx`_V&xqT4+O4g!{z*%-bTKN>t!ghF zBe7*|@8t53lhy54Uz5W^yy7Ho^q<`2yV35M40)BN!^9~aH_H$Gb-W|{tm5h9Ht+Z~ z+v9x?$GlW~uKCVsc2mt?re;S{vnTaLhElVSMlW{!{(Ae&tnK?&OhWbGjW>SHC>fZ( zc7>SuGzrclSSO&AezwSQsQv}W&p+M#2yWgK7m|;k3mNP#Y|~wDCxw#u=*iX}=+{Nw z{*px@@1gCsqP8srE#3_149B7CqEHw2t>k!J(VE!si)o;V>4UuzENEf?AcvRIc-^?l z5J2R{HN-jHIEMb7A0GlZ6RQPX-=F98F$8aST!4!^&I1{9%b#rIDAwvSIkiiGR(nAB z+@H3YJIv9Tz?R#liwk$oj_}`hYo{5I*}M( zKObKl*r>yic^jlt`uz~b3Ea`p-wcS-3(~v{VkkJ0FP%&sZjsB`%5=!ecivXqb_9rI z1PvLn2XdM_cNoM7W{Zyxg-r2t6YZvqAbUkZ*~1L&t^SNoAqmxoY@@*fOz?Z94WA>p zi)L6IGJJx!Ve(c_9LY@}XL`oMaj5~>`M}c-g+Ri25xjAH@|+%;NX89CKlouYWC+Rg zwP0(RJCj3(;3WW3J>79$6t}HDBocvw@xy`LV=@KjPD@A*pNeCx05N6q_PL+}+y_hi zz-dRXBIMGw*gU%6Q>^Z}(Xe;f+6 z4x%SU6gO+p??Z08o z@F_SgA&+A?1y2?M7i)M!;l!0D&W(W4-%uz-A>;#{BmhmG1Tw|PFOW|i8TQMdwWEbK z@C$}5UR0F@G6=W^3eBLKl?6Mw-1BTGkQj82hDuxt0SFoyfRW0^#SoYuBLssCF&0%v!UA@; zrf_~4fv_}yDCic>@Q?uHo%!@qgGd+V8VfJqH)siU%R`o-9$?Jco8h9%wK)&>%B5@a zVlqKbN(7HRNdAA`ro9dMMOZ2p#}(Pts51r=mRmVp58TobO_XA@$@_cp#?iqz3t4ugrDPBdmuINYji`N~9Yte^cXfu7c0>S6MFhy3Kmh>`SC#9MPYV+`CaWEgVM3^WE>a69 z88}&u4NRbp$9sE*b27kt$gv(m0mXRYzzN+gI5&z&OJ8_F_Mc$3xB7VPAbGoxDQ*CgZmpB2;tXL;Ep$;$_i(R0zGzxzJVgtv&J z#}HCS*BVeLt^jC{IV;c|;HZ!QF~kEX^8G^yC{4W6fr6)Qv~iGpp1KVOyQ2#F5ZI)l zjRbXpWU*h&NP!}ZvKJ2qP};0eu#e2OWDJ9xUUyZFanhWz%}(F;KpzqTB#!~1g5jxK z9pm2(yz9#rhSISj;NcyHOb3HN7|t`$(Kp2Mt`Nh-SziCwOnM>;70L!QI(ei(*}{<4 zx)|Q2&5D=@rjqh!6v>x;k3%zR0i!F~fa>Ot6ewF_Gvy0ll$I}aM!5=*VMe*<%h|gl z1&S~_D*1wK%fyiYn?Km9kKMh|00~zgWMw|&c3OPWs}G;0B2(XxezX8S!uD(kME@%&B$&Byh za`P0^7r_8JuAUmE4jvuNjLu?EWOS^-&N4`vKW~Ur;X%=KWc23HEfg}|U4f4Sks1b1 zdS<35=bOkB!45=r>FBA(H1`?MBtYSUrvovgQ&kWXM3K-_jYoJf)qkd0K>(cq9Ugv{ zjM<4%GkKun_-2`47UB25nE6<)&dH*HXL<#C8%!{S@Ihl{gduY{Ar$Sz7F4|%$BqF& Xjx|9_8rg@o2LIB*rf$nTB+vc_K1~%_ literal 0 HcmV?d00001 diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestVariantCrossEngineCompatibility.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestVariantCrossEngineCompatibility.java new file mode 100644 index 0000000000000..d22d5fae7ab51 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestVariantCrossEngineCompatibility.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table; + +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.utils.FlinkMiniCluster; +import org.apache.hudi.utils.TestTableEnvs; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Integration test for cross-engine compatibility - verifying that Flink can read Variant tables written by Spark 4.0. + */ +@ExtendWith(FlinkMiniCluster.class) +public class ITTestVariantCrossEngineCompatibility { + + @TempDir + Path tempDir; + + /** + * Helper method to verify that Flink can read Spark 4.0 Variant tables. + * Variant data is represented as ROW in Flink. + */ + private void verifyFlinkCanReadSparkVariantTable(String tablePath, String tableType, String testDescription) throws Exception { + TableEnvironment tableEnv = TestTableEnvs.getBatchTableEnv(); + + // Create a Hudi table pointing to the Spark-written data + // In Flink, Variant is represented as ROW + // NOTE: value is a reserved keyword + String createTableDdl = String.format( + "CREATE TABLE variant_table (" + + " id INT," + + " name STRING," + + " v ROW<`value` BYTES, metadata BYTES>," + + " ts BIGINT," + + " PRIMARY KEY (id) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'hudi'," + + " 'path' = '%s'," + + " 'table.type' = '%s'" + + ")", + tablePath, tableType); + + tableEnv.executeSql(createTableDdl); + + // Query the table to verify Flink can read the data + TableResult result = tableEnv.executeSql("SELECT id, name, v, ts FROM variant_table ORDER BY id"); + List rows = CollectionUtil.iteratorToList(result.collect()); + + // Verify we got the expected row (after Spark 4.0 delete operation, only 1 row remains) + assertEquals(1, rows.size(), "Should have 1 row after delete operation in Spark 4.0 (" + testDescription + ")"); + + Row row = rows.get(0); + assertEquals(1, row.getField(0), "First column should be id=1"); + assertEquals("row1", row.getField(1), "Second column should be name=row1"); + assertEquals(1000L, row.getField(3), "Fourth column should be ts=1000"); + + // Verify the variant column is readable as a ROW with binary fields + Row variantRow = (Row) row.getField(2); + assertNotNull(variantRow, "Variant column should not be null"); + + byte[] valueBytes = (byte[]) variantRow.getField(0); + byte[] metadataBytes = (byte[]) variantRow.getField(1); + + // Expected byte values from Spark 4.0 Variant representation: {"updated": true, "new_field": 123} + byte[] expectedValueBytes = new byte[]{0x02, 0x02, 0x01, 0x00, 0x01, 0x00, 0x03, 0x04, 0x0C, 0x7B}; + byte[] expectedMetadataBytes = new byte[]{0x01, 0x02, 0x00, 0x07, 0x10, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x64, 0x6E, 0x65, 0x77, 0x5F, 0x66, 0x69, 0x65, 0x6C, 0x64}; + + assertArrayEquals(expectedValueBytes, valueBytes, + String.format("Variant value bytes mismatch (%s). Expected: %s, Got: %s", + testDescription, + Arrays.toString(StringUtils.encodeHex(expectedValueBytes)), + Arrays.toString(StringUtils.encodeHex(valueBytes)))); + + assertArrayEquals(expectedMetadataBytes, metadataBytes, + String.format("Variant metadata bytes mismatch (%s). Expected: %s, Got: %s", + testDescription, + Arrays.toString(StringUtils.encodeHex(expectedMetadataBytes)), + Arrays.toString(StringUtils.encodeHex(metadataBytes)))); + + tableEnv.executeSql("DROP TABLE variant_table"); + } + + @Test + public void testFlinkReadSparkVariantCOWTable() throws Exception { + // Test that Flink can read a COW table with Variant data written by Spark 4.0 + Path cowTargetDir = tempDir.resolve("cow"); + HoodieTestUtils.extractZipToDirectory("variant_backward_compat/variant_cow.zip", cowTargetDir, getClass()); + String cowPath = cowTargetDir.resolve("variant_cow").toString(); + verifyFlinkCanReadSparkVariantTable(cowPath, "COPY_ON_WRITE", "COW table"); + } + + @Test + public void testFlinkReadSparkVariantMORTableWithAvro() throws Exception { + // Test that Flink can read a MOR table with AVRO record type and Variant data written by Spark 4.0 + Path morAvroTargetDir = tempDir.resolve("mor_avro"); + HoodieTestUtils.extractZipToDirectory("variant_backward_compat/variant_mor_avro.zip", morAvroTargetDir, getClass()); + String morAvroPath = morAvroTargetDir.resolve("variant_mor_avro").toString(); + verifyFlinkCanReadSparkVariantTable(morAvroPath, "MERGE_ON_READ", "MOR table with AVRO record type"); + } + + @Test + public void testFlinkReadSparkVariantMORTableWithSpark() throws Exception { + // Test that Flink can read a MOR table with SPARK record type and Variant data written by Spark 4.0 + Path morSparkTargetDir = tempDir.resolve("mor_spark"); + HoodieTestUtils.extractZipToDirectory("variant_backward_compat/variant_mor_spark.zip", morSparkTargetDir, getClass()); + String morSparkPath = morSparkTargetDir.resolve("variant_mor_spark").toString(); + verifyFlinkCanReadSparkVariantTable(morSparkPath, "MERGE_ON_READ", "MOR table with SPARK record type"); + } +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java index 4f8d88d0f6d91..ce68c4a50a189 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java @@ -250,6 +250,10 @@ private Type convertField(String fieldName, HoodieSchema schema, Type.Repetition break; case UNION: return convertUnion(fieldName, schema, repetition, schemaPath); + case VARIANT: + // Variant is represented as a record with value and metadata binary fields + // Convert the variant schema's fields to Parquet types + return new GroupType(repetition, fieldName, convertFields(schema.getFields(), schemaPath)); default: throw new UnsupportedOperationException("Cannot convert Avro type " + type); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 0612dae85b266..2b7ade43817e3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -520,12 +520,18 @@ class HoodieSparkSqlWriterInternal { // if table has undergone upgrade, we need to reload table config tableMetaClient.reloadTableConfig() tableConfig = tableMetaClient.getTableConfig - // Convert to RDD[HoodieRecord] - val hoodieRecords = Try(HoodieCreateRecordUtils.createHoodieRecordRdd( + // Convert to RDD[HoodieRecord] and force type immediately + val hoodieRecords: JavaRDD[HoodieRecord[_]] = Try(HoodieCreateRecordUtils.createHoodieRecordRdd( HoodieCreateRecordUtils.createHoodieRecordRddArgs(df, writeConfig, parameters, avroRecordName, avroRecordNamespace, writerSchema, processedDataSchema, operation, instantTime, preppedSparkSqlWrites, preppedSparkSqlMergeInto, preppedWriteOperation, tableConfig))) match { - case Success(recs) => recs + // CAST EXPLANATION: + // This cast is required due to Scala/Java generic variance mismatch. + // 1. Java returns JavaRDD[HoodieRecord[_ <: Object]] (wildcard maps to Object bound) + // 2. Scala expects JavaRDD[HoodieRecord[_]] (existential type maps to Any) + // 3. JavaRDD is Invariant in Scala, so these types are not compatible without a cast. + // Please do not remove, even if IDE marks it as redundant. + case Success(recs) => recs.asInstanceOf[JavaRDD[HoodieRecord[_]]] case Failure(e) => throw new HoodieRecordCreationException("Failed to create Hoodie Spark Record", e) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index 9b361ac3d79c3..a7cb677f866ac 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} +import org.apache.hudi.SparkAdapterSupport.sparkAdapter import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.ConfigUtils @@ -289,4 +290,3 @@ object CreateHoodieTableCommand { } } } - diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/schema/TestVariantDataType.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/schema/TestVariantDataType.scala new file mode 100644 index 0000000000000..0d830fde66a21 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/schema/TestVariantDataType.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.dml.schema + +import org.apache.hudi.HoodieSparkUtils +import org.apache.hudi.common.testutils.HoodieTestUtils +import org.apache.hudi.common.util.StringUtils + +import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase + + +class TestVariantDataType extends HoodieSparkSqlTestBase { + + test(s"Test Table with Variant Data Type") { + // Variant type is only supported in Spark 4.0+ + assume(HoodieSparkUtils.gteqSpark4_0, "Variant type requires Spark 4.0 or higher") + + Seq("cow", "mor").foreach { tableType => + withRecordType()(withTempDir { tmp => + val tableName = generateTableName + // Create a table with a Variant column + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | v variant, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}' + | tblproperties ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // Insert data with Variant values using parse_json (Spark 4.0+) + spark.sql( + s""" + |insert into $tableName + |values + | (1, 'row1', parse_json('{"key": "value1", "num": 1}'), 1000), + | (2, 'row2', parse_json('{"key": "value2", "list": [1, 2, 3]}'), 1000) + """.stripMargin) + + // Verify the data by casting Variant to String for deterministic comparison + checkAnswer(s"select id, name, cast(v as string), ts from $tableName order by id")( + Seq(1, "row1", "{\"key\":\"value1\",\"num\":1}", 1000), + Seq(2, "row2", "{\"key\":\"value2\",\"list\":[1,2,3]}", 1000) + ) + + // Test Updates on Variant column, MOR will generate logs + spark.sql( + s""" + |update $tableName + |set v = parse_json('{"updated": true, "new_field": 123}') + |where id = 1 + """.stripMargin) + + checkAnswer(s"select id, name, cast(v as string), ts from $tableName order by id")( + Seq(1, "row1", "{\"new_field\":123,\"updated\":true}", 1000), + Seq(2, "row2", "{\"key\":\"value2\",\"list\":[1,2,3]}", 1000) + ) + + // Test Delete + spark.sql(s"delete from $tableName where id = 2") + + checkAnswer(s"select id, name, cast(v as string), ts from $tableName order by id")( + Seq(1, "row1", "{\"new_field\":123,\"updated\":true}", 1000) + ) + }) + } + } + + test(s"Test Backward Compatibility: Read Spark 4.0 Variant Table in Spark 3.x") { + // This test only runs on Spark 3.x to verify backward compatibility + assume(HoodieSparkUtils.isSpark3, "This test verifies Spark 3.x can read Spark 4.0 Variant tables") + + withTempDir { tmpDir => + // Test COW table - record type does not affect file metadata for COW, only need one test + HoodieTestUtils.extractZipToDirectory("variant_backward_compat/variant_cow.zip", tmpDir.toPath, getClass) + val cowPath = tmpDir.toPath.resolve("variant_cow").toString + verifyVariantBackwardCompatibility(cowPath, "cow", "COW table") + + // Test MOR table with AVRO record type + HoodieTestUtils.extractZipToDirectory("variant_backward_compat/variant_mor_avro.zip", tmpDir.toPath, getClass) + val morAvroPath = tmpDir.toPath.resolve("variant_mor_avro").toString + verifyVariantBackwardCompatibility(morAvroPath, "mor", "MOR table with AVRO record type") + + // Test MOR table with SPARK record type + HoodieTestUtils.extractZipToDirectory("variant_backward_compat/variant_mor_spark.zip", tmpDir.toPath, getClass) + val morSparkPath = tmpDir.toPath.resolve("variant_mor_spark").toString + verifyVariantBackwardCompatibility(morSparkPath, "mor", "MOR table with SPARK record type") + } + } + + /** + * Helper method to verify backward compatibility of reading Spark 4.0 Variant tables in Spark 3.x + */ + private def verifyVariantBackwardCompatibility(resourcePath: String, tableType: String, testDescription: String): Unit = { + val tableName = generateTableName + + // Create a Hudi table pointing to the saved data location + // In Spark 3.x, we define the Variant column as a struct with binary fields since Variant type is not available + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | v struct, + | ts long + |) using hudi + |location '$resourcePath' + |tblproperties ( + | primaryKey = 'id', + | tableType = '$tableType', + | preCombineField = 'ts' + |) + """.stripMargin) + + // Verify we can read the basic columns + checkAnswer(s"select id, name, ts from $tableName order by id")(Seq(1, "row1", 1000)) + + // Read and verify the variant column as a struct with binary fields + val rows = spark.sql(s"select id, v from $tableName order by id").collect() + assert(rows.length == 1, s"Should have 1 row after delete operation in Spark 4.0 ($testDescription)") + assert(rows(0).getInt(0) == 1, "First column should be id=1") + assert(!rows(0).isNullAt(1), "Variant column should not be null") + + val variantStruct = rows(0).getStruct(1) + assert(variantStruct.size == 2, "Variant struct should have 2 fields: value and metadata") + + val valueBytes = variantStruct.getAs[Array[Byte]](0) + val metadataBytes = variantStruct.getAs[Array[Byte]](1) + + // Expected byte values from Spark 4.0 Variant representation: {"updated": true, "new_field": 123} + val expectedValueBytes = Array[Byte](0x02, 0x02, 0x01, 0x00, 0x01, 0x00, 0x03, 0x04, 0x0C, 0x7B) + val expectedMetadataBytes = Array[Byte](0x01, 0x02, 0x00, 0x07, 0x10, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x64, 0x6E, 0x65, 0x77, 0x5F, 0x66, 0x69, 0x65, 0x6C, 0x64) + + assert(valueBytes.sameElements(expectedValueBytes), + s"Variant value bytes mismatch ($testDescription). " + + s"Expected: ${StringUtils.encodeHex(expectedValueBytes).mkString("Array(", ", ", ")")}, " + + s"Got: ${StringUtils.encodeHex(valueBytes).mkString("Array(", ", ", ")")}") + + assert(metadataBytes.sameElements(expectedMetadataBytes), + s"Variant metadata bytes mismatch ($testDescription). " + + s"Expected: ${StringUtils.encodeHex(expectedMetadataBytes).mkString("Array(", ", ", ")")}, " + + s"Got: ${StringUtils.encodeHex(metadataBytes).mkString("Array(", ", ", ")")}") + + // Verify we can select all columns without errors + assert(spark.sql(s"select * from $tableName").count() == 1, "Should be able to read all columns including variant") + + spark.sql(s"drop table $tableName") + } +} diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index 4f2bbb4b5b75c..e2342008f4617 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -26,6 +26,8 @@ import org.apache.hudi.common.table.cdc.HoodieCDCFileSplit import org.apache.hudi.common.util.JsonUtils import org.apache.hudi.spark.internal.ReflectUtil +import org.apache.parquet.schema.Type +import org.apache.parquet.schema.Type.Repetition import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -34,7 +36,7 @@ import org.apache.spark.sql.FileFormatUtilsForFileGroupReader.applyFiltersToPlan import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate} +import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate, SpecializedGetters} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -45,7 +47,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.HoodieFormatTrait import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.sources.{BaseRelation, Filter} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.types.UTF8String @@ -53,6 +55,7 @@ import org.apache.spark.unsafe.types.UTF8String import java.time.ZoneId import java.util.TimeZone import java.util.concurrent.ConcurrentHashMap +import java.util.function.{BiConsumer, Consumer} import scala.collection.JavaConverters._ @@ -176,4 +179,36 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { Dataset.ofRows(sqlContext.sparkSession, applyFiltersToPlan(logicalRelation, requiredSchema, resolvedSchema, relation.fileFormat.asInstanceOf[HoodieFormatTrait].getRequiredFilters)) } + + override def getVariantDataType: Option[DataType] = { + // Spark 3.x does not support VariantType + None + } + + override def isDataTypeEqualForParquet(requiredType: DataType, fileType: DataType): Option[Boolean] = { + // Spark 3.x does not support VariantType, so return None to use default logic + None + } + + override def isVariantType(dataType: DataType): Boolean = { + // Spark 3.x does not support VariantType + false + } + + override def createVariantValueWriter( + dataType: DataType, + writeValue: Consumer[Array[Byte]], + writeMetadata: Consumer[Array[Byte]] + ): BiConsumer[SpecializedGetters, Integer] = { + throw new UnsupportedOperationException("Spark 3.x does not support VariantType") + } + + override def convertVariantFieldToParquetType( + dataType: DataType, + fieldName: String, + fieldSchema: HoodieSchema, + repetition: Repetition + ): Type = { + throw new UnsupportedOperationException("Spark 3.x does not support VariantType") + } } diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala index 58ed3eb5b88c9..172ca45f6a4e0 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala @@ -28,6 +28,7 @@ import org.apache.hudi.spark.internal.ReflectUtil import org.apache.hudi.storage.StorageConfiguration import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.Type.Repetition import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -36,7 +37,7 @@ import org.apache.spark.sql.FileFormatUtilsForFileGroupReader.applyFiltersToPlan import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate} +import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate, SpecializedGetters} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -49,7 +50,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{HoodieFormatTrait, Pa import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, Filter} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{BinaryType, DataType, StructType, VariantType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.types.UTF8String @@ -57,6 +58,7 @@ import org.apache.spark.unsafe.types.UTF8String import java.time.ZoneId import java.util.TimeZone import java.util.concurrent.ConcurrentHashMap +import java.util.function.{BiConsumer, Consumer} import scala.collection.JavaConverters._ @@ -196,4 +198,86 @@ abstract class BaseSpark4Adapter extends SparkAdapter with Logging { storageConf.getBoolean(SQLConf.CASE_SENSITIVE.key, sqlConf.caseSensitiveAnalysis), getRebaseSpec("CORRECTED")) } + + override def getVariantDataType: Option[DataType] = { + Some(VariantType) + } + + override def isDataTypeEqualForParquet(requiredType: DataType, fileType: DataType): Option[Boolean] = { + /** + * Checks if a StructType is the physical representation of VariantType in Parquet. + * VariantType is stored in Parquet as a struct with two binary fields: "value" and "metadata". + */ + def isVariantPhysicalSchema(structType: StructType): Boolean = { + if (structType.fields.length != 2) { + false + } else { + val fieldMap = structType.fields.map(f => (f.name, f.dataType)).toMap + fieldMap.contains("value") && fieldMap.contains("metadata") && + fieldMap("value") == BinaryType && fieldMap("metadata") == BinaryType + } + } + + // Handle VariantType comparisons + (requiredType, fileType) match { + case (_: VariantType, s: StructType) if isVariantPhysicalSchema(s) => Some(true) + case (s: StructType, _: VariantType) if isVariantPhysicalSchema(s) => Some(true) + case _ => None // Not a VariantType comparison, use default logic + } + } + + override def isVariantType(dataType: DataType): Boolean = { + import org.apache.spark.sql.types.VariantType + dataType.isInstanceOf[VariantType] + } + + override def createVariantValueWriter( + dataType: DataType, + writeValue: Consumer[Array[Byte]], + writeMetadata: Consumer[Array[Byte]] + ): BiConsumer[SpecializedGetters, Integer] = { + import org.apache.spark.sql.types.VariantType + + if (!dataType.isInstanceOf[VariantType]) { + throw new IllegalArgumentException(s"Expected VariantType but got $dataType") + } + + (row: SpecializedGetters, ordinal: Integer) => { + val variant = row.getVariant(ordinal) + writeValue.accept(variant.getValue) + writeMetadata.accept(variant.getMetadata) + } + } + + override def convertVariantFieldToParquetType( + dataType: DataType, + fieldName: String, + fieldSchema: HoodieSchema, + repetition: Repetition + ): org.apache.parquet.schema.Type = { + import org.apache.parquet.schema.{PrimitiveType, Types} + import org.apache.spark.sql.types.VariantType + + if (!dataType.isInstanceOf[VariantType]) { + throw new IllegalArgumentException(s"Expected VariantType but got $dataType") + } + + // Determine if this is a shredded variant + val isShredded = fieldSchema match { + case variant: HoodieSchema.Variant => variant.isShredded + case _ => false + } + + // For shredded variants, the value field is OPTIONAL (nullable) + // For unshredded variants, the value field is REQUIRED + val valueRepetition = if (isShredded) Repetition.OPTIONAL else Repetition.REQUIRED + + // VariantType is always stored in Parquet as a struct with separate value and metadata binary fields. + // This matches how the HoodieRowParquetWriteSupport writes variant data. + // Note: We intentionally omit 'typed_value' for shredded variants as this writer only accesses raw binary blobs. + Types.buildGroup(repetition) + .addField(Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, valueRepetition).named("value")) + .addField(Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Repetition.REQUIRED).named("metadata")) + .named(fieldName) + } } diff --git a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark4_0Adapter.scala b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark4_0Adapter.scala index cbf0b2de8bb44..0e0b5be82f591 100644 --- a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark4_0Adapter.scala +++ b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark4_0Adapter.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.adapter import org.apache.hudi.Spark40HoodieFileScanRDD import org.apache.hudi.common.schema.HoodieSchema -import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql._ diff --git a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index 8623d24c6f580..7bd4c54588c1f 100644 --- a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.internal.LegacyBehaviorPolicy import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import java.math.BigDecimal import java.nio.ByteBuffer @@ -218,6 +218,37 @@ private[sql] class AvroDeserializer(rootAvroType: Schema, val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale) updater.setDecimal(ordinal, decimal) + case (RECORD, VariantType) if avroType.getProp("logicalType") == "variant" => + // Validation & Pre-calculation with fail fast logic + val valueField = avroType.getField("value") + val metadataField = avroType.getField("metadata") + + if (valueField == null || metadataField == null) { + throw new IncompatibleSchemaException(incompatibleMsg + + ": Variant logical type requires 'value' and 'metadata' fields") + } + + val valueIdx = valueField.pos() + val metadataIdx = metadataField.pos() + + // Variant types are stored as records with "value" and "metadata" binary fields + // Deserialize them back to VariantVal + (updater, ordinal, value) => + val record = value.asInstanceOf[IndexedRecord] + + val valueBuffer = record.get(valueIdx).asInstanceOf[ByteBuffer] + val valueBytes = new Array[Byte](valueBuffer.remaining) + valueBuffer.get(valueBytes) + valueBuffer.rewind() + + val metadataBuffer = record.get(metadataIdx).asInstanceOf[ByteBuffer] + val metadataBytes = new Array[Byte](metadataBuffer.remaining) + metadataBuffer.get(metadataBytes) + metadataBuffer.rewind() + + val variant = new VariantVal(valueBytes, metadataBytes) + updater.set(ordinal, variant) + case (RECORD, st: StructType) => // Avro datasource doesn't accept filters with nested attributes. See SPARK-32328. // We can always return `false` from `applyFilters` for nested records. diff --git a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala index 968cc8ff02c67..c6e131fd26ac1 100644 --- a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala +++ b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala @@ -22,6 +22,7 @@ import org.apache.avro.Conversions.DecimalConversion import org.apache.avro.LogicalTypes.{LocalTimestampMicros, LocalTimestampMillis, TimestampMicros, TimestampMillis} import org.apache.avro.Schema.Type import org.apache.avro.Schema.Type._ +import org.apache.avro.generic.GenericData import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed, Record} import org.apache.avro.util.Utf8 import org.apache.spark.internal.Logging @@ -221,6 +222,32 @@ private[sql] class AvroSerializer(rootCatalystType: DataType, java.util.Arrays.asList(result: _*) } + case (VariantType, RECORD) if avroType.getProp("logicalType") == "variant" => + // Fail fast if schema is mismatched + val valueField = avroType.getField("value") + val metadataField = avroType.getField("metadata") + + if (valueField == null || metadataField == null) { + throw new IncompatibleSchemaException(errorPrefix + + s"Avro schema with 'variant' logical type must have 'value' and 'metadata' fields. " + + s"Found: ${avroType.getFields.asScala.map(_.name()).mkString(", ")}") + } + + // Pre-calculation: Cache indices for performance + val valueIdx = valueField.pos() + val metadataIdx = metadataField.pos() + + // Variant types are stored as records with "value" and "metadata" binary fields + // This matches the schema created in SchemaConverters.toAvroType + (getter, ordinal) => + val variant = getter.getVariant(ordinal) + val record = new GenericData.Record(avroType) + + // Use positional access in serialization loop + record.put(valueIdx, ByteBuffer.wrap(variant.getValue)) + record.put(metadataIdx, ByteBuffer.wrap(variant.getMetadata)) + record + case (st: StructType, RECORD) => val structConverter = newStructConverter(st, avroType, catalystPath, avroPath) val numFields = st.length diff --git a/hudi-spark-datasource/hudi-spark4.0.x/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowParquetWriteSupportVariant.java b/hudi-spark-datasource/hudi-spark4.0.x/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowParquetWriteSupportVariant.java new file mode 100644 index 0000000000000..d989d5e9a050b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark4.0.x/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowParquetWriteSupportVariant.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage.row; + +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.VariantVal; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; + +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; +import static org.apache.parquet.schema.Type.Repetition.REQUIRED; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** + * Tests for Variant type support in {@link HoodieRowParquetWriteSupport}. + * Verifies that Spark VariantType data is correctly written to Parquet groups with 'metadata' and 'value' binary fields, + * respecting shredded vs unshredded schemas. + */ +public class TestHoodieRowParquetWriteSupportVariant { + + @TempDir + public File tempDir; + + /** + * Tests that an Unshredded Variant (defined by HoodieSchema) is written as: + *

+   * group {
+   * required binary metadata;
+   * required binary value;
+   * }
+   * 
+ */ + @Test + public void testWriteUnshreddedVariant() throws IOException { + + // Setup Hoodie Schema: Unshredded + HoodieSchema.Variant variantSchema = HoodieSchema.createVariant("v", null, "Unshredded variant"); + HoodieSchema recordSchema = HoodieSchema.createRecord("record", null, null, + Collections.singletonList(HoodieSchemaField.of("v", variantSchema, null, null))); + + // Spark Schema: Uses actual VariantType to trigger the correct path in WriteSupport + StructType sparkSchema = new StructType(new StructField[] { + new StructField("v", DataTypes.VariantType, false, Metadata.empty()) + }); + + // Setup Data + // Note: This relies on the test environment having a constructible VariantVal + VariantVal variantVal = getSampleVariantVal(); + InternalRow row = new GenericInternalRow(new Object[] {variantVal}); + + // Write to Parquet + File outputFile = new File(tempDir, "unshredded.parquet"); + writeRows(outputFile, sparkSchema, recordSchema, row); + + // Verify Parquet Structure + MessageType parquetSchema = readParquetSchema(outputFile); + GroupType vGroup = parquetSchema.getType("v").asGroupType(); + + assertEquals(2, vGroup.getFieldCount(), "Unshredded variant should have exactly 2 fields"); + + // Metadata (Required) + Type metaField = vGroup.getType("metadata"); + assertEquals(BINARY, metaField.asPrimitiveType().getPrimitiveTypeName()); + assertEquals(REQUIRED, metaField.getRepetition()); + + // Value (Required for Unshredded) + Type valueField = vGroup.getType("value"); + assertEquals(BINARY, valueField.asPrimitiveType().getPrimitiveTypeName()); + assertEquals(REQUIRED, valueField.getRepetition(), "Unshredded variant value must be REQUIRED"); + } + + /** + * Tests that a Shredded Variant (defined by HoodieSchema) is written as: + *
+   * group {
+   * required binary metadata;
+   * optional binary value;
+   * }
+   * 
+ * Note: Even though typed_value is not populated, the schema must indicate 'value' is OPTIONAL. + */ + @Test + public void testWriteShreddedVariant() throws IOException { + // Setup Hoodie Schema: Shredded + HoodieSchema.Variant variantSchema = HoodieSchema.createVariantShredded("v", null, "Shredded variant", null); + HoodieSchema recordSchema = HoodieSchema.createRecord("record", null, null, + Collections.singletonList(HoodieSchemaField.of("v", variantSchema, null, null))); + + // Spark Schema: Uses actual VariantType + StructType sparkSchema = new StructType(new StructField[] { + new StructField("v", DataTypes.VariantType, false, Metadata.empty()) + }); + + // Setup Data + VariantVal variantVal = getSampleVariantVal(); + InternalRow row = new GenericInternalRow(new Object[] {variantVal}); + + // Write to Parquet + File outputFile = new File(tempDir, "shredded.parquet"); + writeRows(outputFile, sparkSchema, recordSchema, row); + + // Verify Parquet Structure + MessageType parquetSchema = readParquetSchema(outputFile); + GroupType vGroup = parquetSchema.getType("v").asGroupType(); + + // Metadata (Required) + Type metaField = vGroup.getType("metadata"); + assertEquals(BINARY, metaField.asPrimitiveType().getPrimitiveTypeName()); + assertEquals(REQUIRED, metaField.getRepetition()); + + // Value (OPTIONAL for Shredded) + Type valueField = vGroup.getType("value"); + assertEquals(BINARY, valueField.asPrimitiveType().getPrimitiveTypeName()); + assertEquals(OPTIONAL, valueField.getRepetition(), "Shredded variant value must be OPTIONAL"); + + // Verify typed_value is omitted (as implementation skips it) + boolean hasTypedValue = vGroup.getFields().stream().anyMatch(f -> f.getName().equals("typed_value")); + assertFalse(hasTypedValue, "typed_value field should be omitted in this writer implementation"); + } + + private void writeRows(File outputFile, StructType sparkSchema, HoodieSchema recordSchema, InternalRow row) throws IOException { + Configuration conf = new Configuration(); + TypedProperties props = new TypedProperties(); + props.setProperty(HoodieWriteConfig.AVRO_SCHEMA_STRING.key(), recordSchema.toString()); + + // Create config and write support + HoodieConfig hoodieConfig = new HoodieConfig(props); + HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport( + conf, sparkSchema, Option.empty(), hoodieConfig); + + try (ParquetWriter writer = new ParquetWriter<>(new Path(outputFile.getAbsolutePath()), writeSupport)) { + writer.write(row); + } + } + + private MessageType readParquetSchema(File file) throws IOException { + Configuration conf = new Configuration(); + ParquetMetadata metadata = ParquetFileReader.readFooter(conf, new Path(file.getAbsolutePath())); + return metadata.getFileMetaData().getSchema(); + } + + /** + * Helper method to create a VariantVal with sample data derived from Spark 4.0 investigation. + * Represents JSON: {"a":1,"b":"2","c":3.3,"d":4.4} + */ + private static VariantVal getSampleVariantVal() { + // Metadata: [01 04 00 01 02 03 04 61 62 63 64] + byte[] metadata = new byte[] {0x01, 0x04, 0x00, 0x01, 0x02, 0x03, 0x04, 0x61, 0x62, 0x63, 0x64}; + // Value: [02 01 03 00 06 20 01 2C 00 00 00] + byte[] value = new byte[] {0x02, 0x01, 0x03, 0x00, 0x06, 0x20, 0x01, 0x2C, 0x00, 0x00, 0x00}; + return new VariantVal(value, metadata); + } +} From a1f6bd7ef6e8fe5bdbc2a9667dbd29aaddf47874 Mon Sep 17 00:00:00 2001 From: voon Date: Tue, 27 Jan 2026 10:40:55 +0800 Subject: [PATCH 2/5] Make VariantLogicalType compare against singleton --- .../java/org/apache/hudi/common/schema/HoodieSchemaType.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaType.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaType.java index 81b7f94ace0d9..bcc1b4079f3e6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaType.java @@ -154,7 +154,7 @@ public static HoodieSchemaType fromAvro(Schema avroSchema) { return DATE; } else if (logicalType == LogicalTypes.uuid()) { return UUID; - } else if (logicalType instanceof VariantLogicalType) { + } else if (logicalType == VariantLogicalType.variant()) { return VARIANT; } } From 3fd0ae308e6acf76eb754e7d3594e016a564c630 Mon Sep 17 00:00:00 2001 From: voon Date: Wed, 28 Jan 2026 19:02:42 +0800 Subject: [PATCH 3/5] Address comments --- .../row/HoodieRowParquetWriteSupport.java | 11 +--- .../avro/HoodieSparkSchemaConverters.scala | 1 + .../HoodieParquetFileFormatHelper.scala | 62 +++++++++---------- .../metadata/HoodieTableMetadataUtil.java | 2 +- .../parquet/avro/TestAvroSchemaConverter.java | 47 ++++++++++++++ .../command/CreateHoodieTableCommand.scala | 2 +- 6 files changed, 82 insertions(+), 43 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index e0273b4f52b0b..21eba65261060 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -286,18 +286,13 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { // Maps VariantType to a group containing 'metadata' and 'value' fields. // This ensures Spark 4.0 compatibility and supports both Shredded and Unshredded schemas. // Note: We intentionally omit 'typed_value' for shredded variants as this writer only accesses raw binary blobs. - final byte[][] variantBytes = new byte[2][]; // [0] = value, [1] = metadata BiConsumer variantWriter = SparkAdapterSupport$.MODULE$.sparkAdapter().createVariantValueWriter( dataType, - valueBytes -> variantBytes[0] = valueBytes, - metadataBytes -> variantBytes[1] = metadataBytes + valueBytes -> consumeField("value", 0, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(valueBytes))), + metadataBytes -> consumeField("metadata", 1, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(metadataBytes))) ); return (row, ordinal) -> { - variantWriter.accept(row, ordinal); - consumeGroup(() -> { - consumeField("value", 0, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(variantBytes[0]))); - consumeField("metadata", 1, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(variantBytes[1]))); - }); + consumeGroup(() -> variantWriter.accept(row, ordinal)); }; } else if (dataType instanceof DecimalType) { return (row, ordinal) -> { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala index 161b4770796ef..a3bb13ff75753 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieSparkSchemaConverters.scala @@ -253,6 +253,7 @@ object HoodieSparkSchemaConverters extends SparkAdapterSupport { } // VARIANT type (Spark >4.x only), which will be handled via SparkAdapter + // TODO: Check if internalSchema will throw any errors here: #18021 case HoodieSchemaType.VARIANT => sparkAdapter.getVariantDataType match { case Some(variantType) => SchemaType(variantType, nullable = false) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala index 6bb9dffcc4e03..496a87071aa96 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala @@ -19,15 +19,16 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.hadoop.conf.Configuration import org.apache.hudi.HoodieSparkUtils + +import org.apache.hadoop.conf.Configuration import org.apache.parquet.hadoop.metadata.FileMetaData import org.apache.spark.sql.HoodieSchemaUtils -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, Cast, CreateNamedStruct, CreateStruct, Expression, GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, NamedLambdaVariable, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampNTZType} -trait HoodieParquetFileFormatHelperTrait { +object HoodieParquetFileFormatHelper { def buildImplicitSchemaChangeInfo(hadoopConf: Configuration, parquetFileMetaData: FileMetaData, @@ -60,38 +61,38 @@ trait HoodieParquetFileFormatHelperTrait { // Check if adapter can handle this comparison (e.g., VariantType in Spark 4.0+) val adapterResult = HoodieSparkUtils.sparkAdapter.isDataTypeEqualForParquet(requiredType, fileType) if (adapterResult.isDefined) { - return adapterResult.get - } - - (requiredType, fileType) match { - case (requiredType, fileType) if requiredType == fileType => true + adapterResult.get + } else { + (requiredType, fileType) match { + case (requiredType, fileType) if requiredType == fileType => true - // prevent illegal cast - case (TimestampNTZType, LongType) => true + // prevent illegal cast + case (TimestampNTZType, LongType) => true - case (ArrayType(rt, _), ArrayType(ft, _)) => - // Do not care about nullability as schema evolution require fields to be nullable - isDataTypeEqual(rt, ft) + case (ArrayType(rt, _), ArrayType(ft, _)) => + // Do not care about nullability as schema evolution require fields to be nullable + isDataTypeEqual(rt, ft) - case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => - // Likewise, do not care about nullability as schema evolution require fields to be nullable - isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) + case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => + // Likewise, do not care about nullability as schema evolution require fields to be nullable + isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) - case (StructType(requiredFields), StructType(fileFields)) => - // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations - val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) + case (StructType(requiredFields), StructType(fileFields)) => + // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations + val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) - // Need to match by name instead of StructField as name will stay the same whilst type may change - val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) - val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + // Need to match by name instead of StructField as name will stay the same whilst type may change + val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) - // Sorting ensures that the same field names are being compared for type differences - requiredFilteredFields.zip(fileFilteredFields).forall { - case (requiredField, fileFilteredField) => - isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) - } + // Sorting ensures that the same field names are being compared for type differences + requiredFilteredFields.zip(fileFilteredFields).forall { + case (requiredField, fileFilteredField) => + isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) + } - case _ => false + case _ => false + } } } @@ -205,8 +206,3 @@ trait HoodieParquetFileFormatHelperTrait { } } -/** - * Default object implementing HoodieParquetFileFormatHelperTrait. - * Can be used directly in Spark 3.x and 4.x environments. - */ -object HoodieParquetFileFormatHelper extends HoodieParquetFileFormatHelperTrait diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 90c09cc94668b..31da3124e98c4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -2061,7 +2061,7 @@ private static boolean isColumnTypeSupportedV2(HoodieSchema schema) { // Check for precision and scale if the schema has a logical decimal type. // VARIANT (unshredded) type is excluded because it stores semi-structured data as opaque binary blobs, // making min/max statistics meaningless - // TODO: For shredded, we are able to store colstats, explore that + // TODO: For shredded, we are able to store colstats, explore that: #17988 return type != HoodieSchemaType.RECORD && type != HoodieSchemaType.MAP && type != HoodieSchemaType.ARRAY && type != HoodieSchemaType.ENUM && type != HoodieSchemaType.VARIANT; diff --git a/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java index 50977e573b6c1..f9c80548a12ec 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java +++ b/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -860,6 +860,53 @@ public void testUUIDTypeWithParquetUUID() throws Exception { "message myrecord {\n" + " required fixed_len_byte_array(16) uuid (UUID);\n" + "}\n"); } + @Test + public void testUnshreddedVariantType() throws Exception { + HoodieSchema variant = HoodieSchema.createVariant(); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myvariant", variant, null, null))); + testAvroToParquetConversion( + schema, + "message myrecord {\n" + + " required group myvariant {\n" + + " required binary value;\n" + + " required binary metadata;\n" + + " }\n" + + "}\n"); + } + + @Test + public void testShreddedVariantType() throws Exception { + HoodieSchema variant = HoodieSchema.createVariantShredded(HoodieSchema.create(HoodieSchemaType.INT)); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myvariant", variant, null, null))); + testAvroToParquetConversion( + schema, + "message myrecord {\n" + + " required group myvariant {\n" + + " optional binary value;\n" + + " required binary metadata;\n" + + " required int32 typed_value;\n" + + " }\n" + + "}\n"); + } + + @Test + public void testOptionalVariantType() throws Exception { + HoodieSchema variant = HoodieSchema.createVariant(); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList( + HoodieSchemaField.of("myvariant", HoodieSchema.createNullable(variant), null, HoodieSchema.NULL_VALUE))); + testAvroToParquetConversion( + schema, + "message myrecord {\n" + + " optional group myvariant {\n" + + " required binary value;\n" + + " required binary metadata;\n" + + " }\n" + + "}\n"); + } + @Test public void testAvroFixed12AsParquetInt96Type() throws Exception { HoodieSchema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/fixedToInt96.avsc"); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index a7cb677f866ac..9b361ac3d79c3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} -import org.apache.hudi.SparkAdapterSupport.sparkAdapter import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.ConfigUtils @@ -290,3 +289,4 @@ object CreateHoodieTableCommand { } } } + From 8756466024f485a9e028229489681336039d5a7c Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 29 Jan 2026 10:43:49 +0800 Subject: [PATCH 4/5] Address comments 2 --- .../org/apache/spark/sql/adapter/BaseSpark4Adapter.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala index 172ca45f6a4e0..0bf5c44d43230 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala @@ -27,7 +27,7 @@ import org.apache.hudi.common.util.JsonUtils import org.apache.hudi.spark.internal.ReflectUtil import org.apache.hudi.storage.StorageConfiguration -import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.{MessageType, Type} import org.apache.parquet.schema.Type.Repetition import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging @@ -254,7 +254,7 @@ abstract class BaseSpark4Adapter extends SparkAdapter with Logging { fieldName: String, fieldSchema: HoodieSchema, repetition: Repetition - ): org.apache.parquet.schema.Type = { + ): Type = { import org.apache.parquet.schema.{PrimitiveType, Types} import org.apache.spark.sql.types.VariantType From 6aadb0a2794a38a3bdf69fe91665574adcc0cfa3 Mon Sep 17 00:00:00 2001 From: voon Date: Thu, 29 Jan 2026 18:34:22 +0800 Subject: [PATCH 5/5] Address comments 3 --- .../io/storage/row/HoodieRowParquetWriteSupport.java | 4 ++-- .../apache/spark/sql/adapter/BaseSpark4Adapter.scala | 12 +++--------- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 21eba65261060..771047a78bc48 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -288,8 +288,8 @@ private ValueWriter makeWriter(HoodieSchema schema, DataType dataType) { // Note: We intentionally omit 'typed_value' for shredded variants as this writer only accesses raw binary blobs. BiConsumer variantWriter = SparkAdapterSupport$.MODULE$.sparkAdapter().createVariantValueWriter( dataType, - valueBytes -> consumeField("value", 0, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(valueBytes))), - metadataBytes -> consumeField("metadata", 1, () -> recordConsumer.addBinary(Binary.fromReusedByteArray(metadataBytes))) + valueBytes -> consumeField("value", 0, () -> recordConsumer.addBinary(Binary.fromConstantByteArray(valueBytes))), + metadataBytes -> consumeField("metadata", 1, () -> recordConsumer.addBinary(Binary.fromConstantByteArray(metadataBytes))) ); return (row, ordinal) -> { consumeGroup(() -> variantWriter.accept(row, ordinal)); diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala index 0bf5c44d43230..bb5f4f9a6cb73 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala @@ -27,7 +27,7 @@ import org.apache.hudi.common.util.JsonUtils import org.apache.hudi.spark.internal.ReflectUtil import org.apache.hudi.storage.StorageConfiguration -import org.apache.parquet.schema.{MessageType, Type} +import org.apache.parquet.schema.{MessageType, PrimitiveType, Type, Types} import org.apache.parquet.schema.Type.Repetition import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging @@ -227,7 +227,6 @@ abstract class BaseSpark4Adapter extends SparkAdapter with Logging { } override def isVariantType(dataType: DataType): Boolean = { - import org.apache.spark.sql.types.VariantType dataType.isInstanceOf[VariantType] } @@ -236,9 +235,7 @@ abstract class BaseSpark4Adapter extends SparkAdapter with Logging { writeValue: Consumer[Array[Byte]], writeMetadata: Consumer[Array[Byte]] ): BiConsumer[SpecializedGetters, Integer] = { - import org.apache.spark.sql.types.VariantType - - if (!dataType.isInstanceOf[VariantType]) { + if (!isVariantType(dataType)) { throw new IllegalArgumentException(s"Expected VariantType but got $dataType") } @@ -255,10 +252,7 @@ abstract class BaseSpark4Adapter extends SparkAdapter with Logging { fieldSchema: HoodieSchema, repetition: Repetition ): Type = { - import org.apache.parquet.schema.{PrimitiveType, Types} - import org.apache.spark.sql.types.VariantType - - if (!dataType.isInstanceOf[VariantType]) { + if (!isVariantType(dataType)) { throw new IllegalArgumentException(s"Expected VariantType but got $dataType") }