From f72a622cbf948f50bf930e098c4c4329ee4e7298 Mon Sep 17 00:00:00 2001 From: Bo Cui Date: Tue, 12 Jul 2022 07:42:32 +0800 Subject: [PATCH] [HUDI-4098] Support HMS for flink HoodieCatalog [HUDI-4098] Support HMS for flink HoodieCatalog [HUDI-4098] Support HMS for flink HoodieCatalog [minor] following 4152, refactor the clazz about plan selection strategy (#6060) [HUDI-4367] Support copyToTable on call (#6054) [HUDI-4098] Support HMS for flink HoodieCatalog [HUDI-4098] Support HMS for flink HoodieCatalog [HUDI-4098] Support HMS for flink HoodieCatalog [HUDI-4335] Bug fixes in AWSGlueCatalogSyncClient post schema evolution. (#5995) * fix for updateTableParameters which is not excluding partition columns and updateTableProperties boolean check * Fix - serde parameters getting overrided on table property update * removing stale syncConfig [HUDI-4276] Reconcile schema-inject null values for missing fields and add new fields (#6017) * [HUDI-4276] Reconcile schema-inject null values for missing fields and add new fields. * fix comments Co-authored-by: public (bdcee5037027) [HUDI-3500] Add call procedure for RepairsCommand (#6053) [HUDI-2150] Rename/Restructure configs for better modularity (#6061) - Move clean related configuration to HoodieCleanConfig - Move Archival related configuration to HoodieArchivalConfig - hoodie.compaction.payload.class move this to HoodiePayloadConfig [MINOR] Bump xalan from 2.7.1 to 2.7.2 (#6062) Bumps xalan from 2.7.1 to 2.7.2. --- updated-dependencies: - dependency-name: xalan:xalan dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> [HUDI-4324] Remove use_jdbc config from hudi sync (#6072) * [HUDI-4324] Remove use_jdbc config from hudi sync * Users should use HIVE_SYNC_MODE instead [HUDI-3730][RFC-55] Improve hudi-sync classes design and simplify configs (#5695) * [HUDI-4146] RFC for Improve Hive/Meta sync class design and hierarchies Co-authored-by: jian.feng Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com> [HUDI-4323] Make database table names optional in sync tool (#6073) * [HUDI-4323] Make database table names optional in sync tool * Infer from these properties from the table config [MINOR] Update RFCs status (#6078) --- conf/hudi-defaults.conf.template | 2 +- docker/demo/config/hoodie-incr.properties | 3 +- docker/demo/sparksql-incremental.commands | 2 + .../aws/sync/AWSGlueCatalogSyncClient.java | 34 +- .../apache/hudi/cli/commands/SparkMain.java | 4 +- .../commands/TestArchivedCommitsCommand.java | 6 +- .../hudi/cli/commands/TestCommitsCommand.java | 9 +- .../cli/commands/TestCompactionCommand.java | 6 +- .../hudi/client/BaseHoodieWriteClient.java | 17 +- .../hudi/config/HoodieArchivalConfig.java | 194 ++++ .../apache/hudi/config/HoodieCleanConfig.java | 297 ++++++ .../hudi/config/HoodieCompactionConfig.java | 397 +------- .../hudi/config/HoodiePayloadConfig.java | 20 + .../hudi/config/HoodieStorageConfig.java | 16 +- .../apache/hudi/config/HoodieWriteConfig.java | 97 +- .../HoodieBackedTableMetadataWriter.java | 16 +- .../action/commit/HoodieMergeHelper.java | 2 +- .../transaction/TestTransactionManager.java | 8 +- .../hudi/config/TestHoodieWriteConfig.java | 14 +- .../org/apache/hudi/HoodieSparkUtils.scala | 9 +- .../hudi/client/TestClientRollback.java | 12 +- .../client/TestHoodieClientMultiWriter.java | 30 +- .../functional/TestHoodieBackedMetadata.java | 73 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 34 +- .../functional/TestHoodieMetadataBase.java | 22 +- .../TestHoodieMetadataBootstrap.java | 8 +- .../hbase/TestSparkHoodieHBaseIndex.java | 8 +- .../hudi/io/TestHoodieTimelineArchiver.java | 30 +- .../org/apache/hudi/table/TestCleaner.java | 43 +- .../functional/TestCleanPlanExecutor.java | 75 +- ...arkCopyOnWriteTableArchiveWithReplace.java | 8 +- ...stHoodieSparkMergeOnReadTableRollback.java | 10 +- .../hudi/testutils/HoodieClientTestBase.java | 9 +- .../org/apache/hudi/avro/HoodieAvroUtils.java | 42 +- .../common/config/HoodieCommonConfig.java | 7 + .../common/table/TableSchemaResolver.java | 6 + .../log/AbstractHoodieLogRecordReader.java | 4 +- .../exception/HoodieCatalogException.java | 35 +- .../schema/action/InternalSchemaMerger.java | 10 +- .../utils/AvroSchemaEvolutionUtils.java | 74 +- .../schema/utils/InternalSchemaUtils.java | 7 +- .../apache/hudi/avro/TestHoodieAvroUtils.java | 30 + .../util/TestDFSPropertiesConfiguration.java | 2 +- .../utils/TestAvroSchemaEvolutionUtils.java | 78 +- .../external-config/hudi-defaults.conf | 2 +- .../java/HoodieJavaWriteClientExample.java | 4 +- .../spark/HoodieWriteClientExample.java | 4 +- hudi-flink-datasource/hudi-flink/pom.xml | 25 +- .../hudi/configuration/FlinkOptions.java | 6 - .../configuration/HadoopConfigurations.java | 42 + .../compact/CompactionPlanSourceFunction.java | 31 +- .../sink/compact/FlinkCompactionConfig.java | 26 +- .../sink/compact/HoodieFlinkCompactor.java | 53 +- .../MultiCompactionPlanSourceFunction.java | 90 -- .../strategy/CompactionPlanStrategies.java | 74 ++ ...ategy.java => CompactionPlanStrategy.java} | 16 +- .../InstantCompactionPlanSelectStrategy.java | 50 - .../MultiCompactionPlanSelectStrategy.java | 42 - .../SingleCompactionPlanSelectStrategy.java | 43 - .../hudi/sink/utils/HiveSyncContext.java | 2 - .../hudi/streamer/FlinkStreamerConfig.java | 4 - .../table/catalog/HoodieCatalogFactory.java | 34 +- .../catalog/HoodieCatalogFactoryOptions.java | 47 + .../hudi/table/catalog/HoodieCatalogUtil.java | 99 ++ .../hudi/table/catalog/HoodieHiveCatalog.java | 908 ++++++++++++++++++ .../table/catalog/TableOptionProperties.java | 220 +++++ .../catalog/TypeInfoLogicalTypeVisitor.java | 236 +++++ .../hudi/table/format/FilePathUtils.java | 4 + .../org/apache/hudi/util/StreamerUtil.java | 44 +- .../org.apache.flink.table.factories.Factory | 2 +- .../compact/ITTestHoodieFlinkCompactor.java | 48 +- ...y.java => TestCompactionPlanStrategy.java} | 59 +- .../hudi/table/ITTestHoodieDataSource.java | 41 + .../catalog/TestHoodieCatalogFactory.java | 67 ++ .../table/catalog/TestHoodieCatalogUtils.java | 67 ++ .../table/catalog/TestHoodieHiveCatalog.java | 150 +++ .../src/test/resources/hive-site.xml | 52 + .../test-catalog-factory-conf/hive-site.xml | 58 ++ .../testsuite/HoodieTestSuiteWriter.java | 6 +- hudi-kafka-connect/demo/config-sink-hive.json | 1 - .../connect/writers/KafkaConnectConfigs.java | 1 - .../writers/KafkaConnectWriterProvider.java | 19 +- .../java/org/apache/hudi/DataSourceUtils.java | 5 +- .../org/apache/hudi/DataSourceOptions.scala | 19 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 24 +- .../org/apache/hudi/HoodieWriterUtils.scala | 1 - .../apache/spark/sql/hudi/DeDupeType.scala | 28 + .../spark/sql/hudi/DedupeSparkJob.scala | 245 +++++ .../apache/spark/sql/hudi/SparkHelpers.scala | 134 +++ .../procedures/CopyToTableProcedure.scala | 128 +++ .../command/procedures/HoodieProcedures.scala | 6 + .../RepairAddpartitionmetaProcedure.scala | 89 ++ .../RepairCorruptedCleanFilesProcedure.scala | 86 ++ .../RepairDeduplicateProcedure.scala | 86 ++ .../RepairMigratePartitionMetaProcedure.scala | 112 +++ .../RepairOverwriteHoodiePropsProcedure.scala | 89 ++ .../UpgradeOrDowngradeProcedure.scala | 4 +- .../test/resources/table-config.properties | 21 + .../apache/hudi/TestHoodieSparkUtils.scala | 4 +- .../apache/spark/sql/hudi/TestSpark3DDL.scala | 68 +- .../procedure/TestCopyToTableProcedure.scala | 353 +++++++ .../hudi/procedure/TestRepairsProcedure.scala | 507 ++++++++++ .../org/apache/hudi/hive/HiveSyncConfig.java | 5 - .../hudi/hive/HiveSyncConfigHolder.java | 11 +- .../hudi/hive/HoodieHiveSyncClient.java | 32 +- .../hudi/hive/ddl/HiveQueryDDLExecutor.java | 2 +- .../apache/hudi/hive/ddl/JDBCExecutor.java | 2 +- .../HiveSyncGlobalCommitParams.java | 2 + .../apache/hudi/hive/TestHiveSyncTool.java | 2 +- .../hudi/sync/common/HoodieSyncConfig.java | 24 +- .../common/util/Parquet2SparkSchemaUtils.java | 71 ++ .../sync/common/TestHoodieSyncConfig.java | 55 ++ .../utilities/HoodieDropPartitionsTool.java | 9 +- .../utilities/deltastreamer/DeltaSync.java | 2 +- .../functional/TestHoodieDeltaStreamer.java | 20 +- ...estHoodieDeltaStreamerWithMultiWriter.java | 18 +- .../sources/TestHoodieIncrSource.java | 7 +- .../testutils/UtilitiesTestBase.java | 13 +- pom.xml | 2 +- rfc/README.md | 16 +- rfc/rfc-55/hudi-sync-class-diagram.png | Bin 0 -> 418052 bytes rfc/rfc-55/hudi-sync-flows.png | Bin 0 -> 139211 bytes rfc/rfc-55/rfc-55.md | 157 +++ 123 files changed, 5663 insertions(+), 1283 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java rename hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java => hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java (53%) delete mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java rename hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/{CompactionPlanSelectStrategy.java => CompactionPlanStrategy.java} (81%) delete mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java delete mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java delete mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java create mode 100644 hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java rename hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/{TestCompactionPlanSelectStrategy.java => TestCompactionPlanStrategy.java} (74%) create mode 100644 hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java create mode 100644 hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java create mode 100644 hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java create mode 100644 hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml create mode 100644 hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala create mode 100644 hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java create mode 100644 rfc/rfc-55/hudi-sync-class-diagram.png create mode 100644 rfc/rfc-55/hudi-sync-flows.png create mode 100644 rfc/rfc-55/rfc-55.md diff --git a/conf/hudi-defaults.conf.template b/conf/hudi-defaults.conf.template index 175dbaf23d739..fbcedb3f18b9c 100644 --- a/conf/hudi-defaults.conf.template +++ b/conf/hudi-defaults.conf.template @@ -20,7 +20,7 @@ # Example: # hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 -# hoodie.datasource.hive_sync.use_jdbc true +# hoodie.datasource.hive_sync.mode jdbc # hoodie.datasource.hive_sync.support_timestamp false # hoodie.index.type BLOOM # hoodie.metadata.enable false diff --git a/docker/demo/config/hoodie-incr.properties b/docker/demo/config/hoodie-incr.properties index 80f474b1e7716..47bfc95b53cbe 100644 --- a/docker/demo/config/hoodie-incr.properties +++ b/docker/demo/config/hoodie-incr.properties @@ -28,5 +28,6 @@ hoodie.deltastreamer.source.hoodieincr.path=/docker_hoodie_sync_valid_test hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt=true # hive sync hoodie.datasource.hive_sync.table=docker_hoodie_sync_valid_test_2 +hoodie.datasource.hive_sync.mode=jdbc hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000 -hoodie.datasource.hive_sync.partition_fields=partition \ No newline at end of file +hoodie.datasource.hive_sync.partition_fields=partition diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index 9ec586e49d854..3d7da63703c2c 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -47,6 +47,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor"). option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor"). option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). + option(HiveSyncConfigHolder.HIVE_SYNC_MODE.key(), "jdbc"). option(HiveSyncConfigHolder.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). option(HiveSyncConfigHolder.HIVE_USER.key(), "hive"). option(HiveSyncConfigHolder.HIVE_PASS.key(), "hive"). @@ -79,6 +80,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor_bs"). option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor_bs"). option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). + option(HiveSyncConfigHolder.HIVE_SYNC_MODE.key(), "jdbc"). option(HiveSyncConfigHolder.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). option(HiveSyncConfigHolder.HIVE_USER.key(), "hive"). option(HiveSyncConfigHolder.HIVE_PASS.key(), "hive"). diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java index f322ef79cafe0..79651154ed63b 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java @@ -64,9 +64,9 @@ import java.util.stream.Collectors; import static org.apache.hudi.aws.utils.S3Utils.s3aToS3; -import static org.apache.hudi.common.util.MapUtils.nonEmpty; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.common.util.MapUtils.isNullOrEmpty; import static org.apache.hudi.hive.util.HiveSchemaUtil.getPartitionKeyType; import static org.apache.hudi.hive.util.HiveSchemaUtil.parquetSchemaToMapSchema; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; @@ -193,11 +193,11 @@ public void dropPartitions(String tableName, List partitionsToDrop) { */ @Override public void updateTableProperties(String tableName, Map tableProperties) { - if (nonEmpty(tableProperties)) { + if (isNullOrEmpty(tableProperties)) { return; } try { - updateTableParameters(awsGlue, databaseName, tableName, tableProperties, true); + updateTableParameters(awsGlue, databaseName, tableName, tableProperties, false); } catch (Exception e) { throw new HoodieGlueSyncException("Fail to update properties for table " + tableId(databaseName, tableName), e); } @@ -210,10 +210,7 @@ public void updateTableSchema(String tableName, MessageType newSchema) { try { Table table = getTable(awsGlue, databaseName, tableName); Map newSchemaMap = parquetSchemaToMapSchema(newSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); - List newColumns = newSchemaMap.keySet().stream().map(key -> { - String keyType = getPartitionKeyType(newSchemaMap, key); - return new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); - }).collect(Collectors.toList()); + List newColumns = getColumnsFromSchema(newSchemaMap); StorageDescriptor sd = table.getStorageDescriptor(); sd.setColumns(newColumns); @@ -258,15 +255,7 @@ public void createTable(String tableName, try { Map mapSchema = parquetSchemaToMapSchema(storageSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); - List schemaWithoutPartitionKeys = new ArrayList<>(); - for (String key : mapSchema.keySet()) { - String keyType = getPartitionKeyType(mapSchema, key); - Column column = new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); - // In Glue, the full schema should exclude the partition keys - if (!config.getSplitStrings(META_SYNC_PARTITION_FIELDS).contains(key)) { - schemaWithoutPartitionKeys.add(column); - } - } + List schemaWithoutPartitionKeys = getColumnsFromSchema(mapSchema); // now create the schema partition List schemaPartitionKeys = config.getSplitStrings(META_SYNC_PARTITION_FIELDS).stream().map(partitionKey -> { @@ -419,6 +408,19 @@ public void deleteLastReplicatedTimeStamp(String tableName) { throw new UnsupportedOperationException("Not supported: `deleteLastReplicatedTimeStamp`"); } + private List getColumnsFromSchema(Map mapSchema) { + List cols = new ArrayList<>(); + for (String key : mapSchema.keySet()) { + // In Glue, the full schema should exclude the partition keys + if (!config.getSplitStrings(META_SYNC_PARTITION_FIELDS).contains(key)) { + String keyType = getPartitionKeyType(mapSchema, key); + Column column = new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); + cols.add(column); + } + } + return cols; + } + private enum TableType { MANAGED_TABLE, EXTERNAL_TABLE, diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 41357453510da..e94c38bd16af8 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -35,7 +35,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieBootstrapConfig; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieSavepointException; @@ -538,7 +538,7 @@ private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, Stri private static HoodieWriteConfig getWriteConfig(String basePath, Boolean rollbackUsingMarkers, boolean lazyCleanPolicy) { return HoodieWriteConfig.newBuilder().withPath(basePath) .withRollbackUsingMarkers(rollbackUsingMarkers) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(lazyCleanPolicy ? HoodieFailedWritesCleaningPolicy.LAZY : + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(lazyCleanPolicy ? HoodieFailedWritesCleaningPolicy.LAZY : HoodieFailedWritesCleaningPolicy.EAGER).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index d822ad658920b..b936202bd0d80 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -30,7 +30,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestUtils; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.client.HoodieTimelineArchiver; @@ -72,7 +73,8 @@ public void init() throws Exception { // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .forTable("test-trip-table").build(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index b23c6fd150dc0..e03699f66e2cb 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -37,7 +37,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.client.HoodieTimelineArchiver; @@ -212,7 +213,8 @@ public void testShowArchivedCommits(boolean enableMetadataTable) throws Exceptio // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -266,7 +268,8 @@ public void testShowArchivedCommitsWithMultiCommitsFile(boolean enableMetadataTa // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java index 17c1002f6b0dd..bc5ba168e3ed8 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java @@ -39,7 +39,8 @@ import org.apache.hudi.common.testutils.CompactionTestUtils; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieSparkTable; @@ -166,7 +167,8 @@ private void generateArchive() throws IOException { // Generate archive HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .forTable("test-trip-table").build(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 961965353b7ff..24400c5eda90a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -39,6 +39,7 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; +import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -276,15 +277,21 @@ private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCom TableSchemaResolver schemaUtil = new TableSchemaResolver(table.getMetaClient()); String historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata().orElse(""); FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(table.getMetaClient()); - if (!historySchemaStr.isEmpty()) { - InternalSchema internalSchema = InternalSchemaUtils.searchSchema(Long.parseLong(instantTime), - SerDeHelper.parseSchemas(historySchemaStr)); + if (!historySchemaStr.isEmpty() || Boolean.parseBoolean(config.getString(HoodieCommonConfig.RECONCILE_SCHEMA.key()))) { + InternalSchema internalSchema; Schema avroSchema = HoodieAvroUtils.createHoodieWriteSchema(new Schema.Parser().parse(config.getSchema())); - InternalSchema evolvedSchema = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(avroSchema, internalSchema); + if (historySchemaStr.isEmpty()) { + internalSchema = AvroInternalSchemaConverter.convert(avroSchema); + internalSchema.setSchemaId(Long.parseLong(instantTime)); + } else { + internalSchema = InternalSchemaUtils.searchSchema(Long.parseLong(instantTime), + SerDeHelper.parseSchemas(historySchemaStr)); + } + InternalSchema evolvedSchema = AvroSchemaEvolutionUtils.reconcileSchema(avroSchema, internalSchema); if (evolvedSchema.equals(internalSchema)) { metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(evolvedSchema)); //TODO save history schema by metaTable - schemasManager.persistHistorySchemaStr(instantTime, historySchemaStr); + schemasManager.persistHistorySchemaStr(instantTime, historySchemaStr.isEmpty() ? SerDeHelper.inheritSchemas(evolvedSchema, "") : historySchemaStr); } else { evolvedSchema.setSchemaId(Long.parseLong(instantTime)); String newSchemaStr = SerDeHelper.toJson(evolvedSchema); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java new file mode 100644 index 0000000000000..32bccc3a3d18f --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java @@ -0,0 +1,194 @@ +/* + * 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.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import javax.annotation.concurrent.Immutable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Archival related config. + */ +@Immutable +@ConfigClassProperty(name = "Archival Configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Configurations that control archival.") +public class HoodieArchivalConfig extends HoodieConfig { + + public static final ConfigProperty AUTO_ARCHIVE = ConfigProperty + .key("hoodie.archive.automatic") + .defaultValue("true") + .withDocumentation("When enabled, the archival table service is invoked immediately after each commit," + + " to archive commits if we cross a maximum value of commits." + + " It's recommended to enable this, to ensure number of active commits is bounded."); + + public static final ConfigProperty ASYNC_ARCHIVE = ConfigProperty + .key("hoodie.archive.async") + .defaultValue("false") + .sinceVersion("0.11.0") + .withDocumentation("Only applies when " + AUTO_ARCHIVE.key() + " is turned on. " + + "When turned on runs archiver async with writing, which can speed up overall write performance."); + + public static final ConfigProperty MAX_COMMITS_TO_KEEP = ConfigProperty + .key("hoodie.keep.max.commits") + .defaultValue("30") + .withDocumentation("Archiving service moves older entries from timeline into an archived log after each write, to " + + " keep the metadata overhead constant, even as the table size grows." + + "This config controls the maximum number of instants to retain in the active timeline. "); + + public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty + .key("hoodie.archive.delete.parallelism") + .defaultValue(100) + .withDocumentation("Parallelism for deleting archived hoodie commits."); + + public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty + .key("hoodie.keep.min.commits") + .defaultValue("20") + .withDocumentation("Similar to " + MAX_COMMITS_TO_KEEP.key() + ", but controls the minimum number of" + + "instants to retain in the active timeline."); + + public static final ConfigProperty COMMITS_ARCHIVAL_BATCH_SIZE = ConfigProperty + .key("hoodie.commits.archival.batch") + .defaultValue(String.valueOf(10)) + .withDocumentation("Archiving of instants is batched in best-effort manner, to pack more instants into a single" + + " archive log. This config controls such archival batch size."); + + public static final ConfigProperty ARCHIVE_MERGE_FILES_BATCH_SIZE = ConfigProperty + .key("hoodie.archive.merge.files.batch.size") + .defaultValue(10) + .withDocumentation("The number of small archive files to be merged at once."); + + public static final ConfigProperty ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES = ConfigProperty + .key("hoodie.archive.merge.small.file.limit.bytes") + .defaultValue(20L * 1024 * 1024) + .withDocumentation("This config sets the archive file size limit below which an archive file becomes a candidate to be selected as such a small file."); + + public static final ConfigProperty ARCHIVE_MERGE_ENABLE = ConfigProperty + .key("hoodie.archive.merge.enable") + .defaultValue(false) + .withDocumentation("When enable, hoodie will auto merge several small archive files into larger one. It's" + + " useful when storage scheme doesn't support append operation."); + + /** + * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead + */ + @Deprecated + public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key(); + /** + * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead + */ + @Deprecated + public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key(); + /** + * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead + */ + @Deprecated + public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = COMMITS_ARCHIVAL_BATCH_SIZE.key(); + /** @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead */ + @Deprecated + private static final String DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue(); + /** + * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead + */ + @Deprecated + private static final String DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue(); + /** + * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead + */ + @Deprecated + private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = COMMITS_ARCHIVAL_BATCH_SIZE.defaultValue(); + + private HoodieArchivalConfig() { + super(); + } + + public static HoodieArchivalConfig.Builder newBuilder() { + return new HoodieArchivalConfig.Builder(); + } + + public static class Builder { + + private final HoodieArchivalConfig archivalConfig = new HoodieArchivalConfig(); + + public HoodieArchivalConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.archivalConfig.getProps().load(reader); + return this; + } + } + + public HoodieArchivalConfig.Builder fromProperties(Properties props) { + this.archivalConfig.getProps().putAll(props); + return this; + } + + public HoodieArchivalConfig.Builder withAutoArchive(Boolean autoArchive) { + archivalConfig.setValue(AUTO_ARCHIVE, String.valueOf(autoArchive)); + return this; + } + + public HoodieArchivalConfig.Builder withAsyncArchive(Boolean asyncArchive) { + archivalConfig.setValue(ASYNC_ARCHIVE, String.valueOf(asyncArchive)); + return this; + } + + public HoodieArchivalConfig.Builder archiveCommitsWith(int minToKeep, int maxToKeep) { + archivalConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep)); + archivalConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveMergeFilesBatchSize(int number) { + archivalConfig.setValue(ARCHIVE_MERGE_FILES_BATCH_SIZE, String.valueOf(number)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveMergeSmallFileLimit(long size) { + archivalConfig.setValue(ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES, String.valueOf(size)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveMergeEnable(boolean enable) { + archivalConfig.setValue(ARCHIVE_MERGE_ENABLE, String.valueOf(enable)); + return this; + } + + public HoodieArchivalConfig.Builder withArchiveDeleteParallelism(int archiveDeleteParallelism) { + archivalConfig.setValue(DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE, String.valueOf(archiveDeleteParallelism)); + return this; + } + + public HoodieArchivalConfig.Builder withCommitsArchivalBatchSize(int batchSize) { + archivalConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE, String.valueOf(batchSize)); + return this; + } + + public HoodieArchivalConfig build() { + archivalConfig.setDefaults(HoodieArchivalConfig.class.getName()); + return archivalConfig; + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java new file mode 100644 index 0000000000000..7b665f9b2b876 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java @@ -0,0 +1,297 @@ +/* + * 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.config; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; + +import javax.annotation.concurrent.Immutable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; +import java.util.stream.Collectors; + +/** + * Clean related config. + */ +@Immutable +@ConfigClassProperty(name = "Clean Configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Cleaning (reclamation of older/unused file groups/slices).") +public class HoodieCleanConfig extends HoodieConfig { + + public static final ConfigProperty AUTO_CLEAN = ConfigProperty + .key("hoodie.clean.automatic") + .defaultValue("true") + .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit," + + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage" + + " growth is bounded."); + + public static final ConfigProperty ASYNC_CLEAN = ConfigProperty + .key("hoodie.clean.async") + .defaultValue("false") + .withDocumentation("Only applies when " + AUTO_CLEAN.key() + " is turned on. " + + "When turned on runs cleaner async with writing, which can speed up overall write performance."); + + public static final ConfigProperty CLEANER_COMMITS_RETAINED = ConfigProperty + .key("hoodie.cleaner.commits.retained") + .defaultValue("10") + .withDocumentation("Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits " + + "(scheduled). This also directly translates into how much data retention the table supports for incremental queries."); + + public static final ConfigProperty CLEANER_HOURS_RETAINED = ConfigProperty.key("hoodie.cleaner.hours.retained") + .defaultValue("24") + .withDocumentation("Number of hours for which commits need to be retained. This config provides a more flexible option as" + + "compared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group," + + " corresponding to commits with commit times older than the configured number of hours to be retained are cleaned."); + + public static final ConfigProperty CLEANER_POLICY = ConfigProperty + .key("hoodie.cleaner.policy") + .defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) + .withDocumentation("Cleaning policy to be used. The cleaner service deletes older file slices files to re-claim space." + + " By default, cleaner spares the file slices written by the last N commits, determined by " + CLEANER_COMMITS_RETAINED.key() + + " Long running query plans may often refer to older file slices and will break if those are cleaned, before the query has had" + + " a chance to run. So, it is good to make sure that the data is retained for more than the maximum query execution time"); + + public static final ConfigProperty CLEAN_TRIGGER_STRATEGY = ConfigProperty + .key("hoodie.clean.trigger.strategy") + .defaultValue(CleaningTriggerStrategy.NUM_COMMITS.name()) + .withDocumentation("Controls how cleaning is scheduled. Valid options: " + + Arrays.stream(CleaningTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); + + public static final ConfigProperty CLEAN_MAX_COMMITS = ConfigProperty + .key("hoodie.clean.max.commits") + .defaultValue("1") + .withDocumentation("Number of commits after the last clean operation, before scheduling of a new clean is attempted."); + + public static final ConfigProperty CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty + .key("hoodie.cleaner.fileversions.retained") + .defaultValue("3") + .withDocumentation("When " + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, " + + " the minimum number of file slices to retain in each file group, during cleaning."); + + public static final ConfigProperty CLEANER_INCREMENTAL_MODE_ENABLE = ConfigProperty + .key("hoodie.cleaner.incremental.mode") + .defaultValue("true") + .withDocumentation("When enabled, the plans for each cleaner service run is computed incrementally off the events " + + " in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full" + + " table for each planning (even with a metadata table)."); + + public static final ConfigProperty FAILED_WRITES_CLEANER_POLICY = ConfigProperty + .key("hoodie.cleaner.policy.failed.writes") + .defaultValue(HoodieFailedWritesCleaningPolicy.EAGER.name()) + .withDocumentation("Cleaning policy for failed writes to be used. Hudi will delete any files written by " + + "failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before " + + "every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)"); + + public static final ConfigProperty CLEANER_PARALLELISM_VALUE = ConfigProperty + .key("hoodie.cleaner.parallelism") + .defaultValue("200") + .withDocumentation("Parallelism for the cleaning operation. Increase this if cleaning becomes slow."); + + public static final ConfigProperty ALLOW_MULTIPLE_CLEANS = ConfigProperty + .key("hoodie.clean.allow.multiple") + .defaultValue(true) + .sinceVersion("0.11.0") + .withDocumentation("Allows scheduling/executing multiple cleans by enabling this config. If users prefer to strictly ensure clean requests should be mutually exclusive, " + + ".i.e. a 2nd clean will not be scheduled if another clean is not yet completed to avoid repeat cleaning of same files, they might want to disable this config."); + + public static final ConfigProperty CLEANER_BOOTSTRAP_BASE_FILE_ENABLE = ConfigProperty + .key("hoodie.cleaner.delete.bootstrap.base.file") + .defaultValue("false") + .withDocumentation("When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is " + + " cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the" + + " table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap " + + " base files are also physically deleted, to comply with data privacy enforcement processes."); + + + /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ + @Deprecated + public static final String CLEANER_POLICY_PROP = CLEANER_POLICY.key(); + /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ + @Deprecated + public static final String AUTO_CLEAN_PROP = AUTO_CLEAN.key(); + /** @deprecated Use {@link #ASYNC_CLEAN} and its methods instead */ + @Deprecated + public static final String ASYNC_CLEAN_PROP = ASYNC_CLEAN.key(); + /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ + @Deprecated + public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = CLEANER_FILE_VERSIONS_RETAINED.key(); + /** + * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead + */ + @Deprecated + public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key(); + /** + * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead + */ + @Deprecated + public static final String CLEANER_INCREMENTAL_MODE = CLEANER_INCREMENTAL_MODE_ENABLE.key(); + /** + * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead + */ + @Deprecated + public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.key(); + /** + * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead + */ + @Deprecated + public static final String CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.key(); + /** + * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead + */ + @Deprecated + public static final String DEFAULT_CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.defaultValue(); + /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ + @Deprecated + private static final String DEFAULT_CLEANER_POLICY = CLEANER_POLICY.defaultValue(); + /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ + @Deprecated + public static final String FAILED_WRITES_CLEANER_POLICY_PROP = FAILED_WRITES_CLEANER_POLICY.key(); + /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ + @Deprecated + private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY = FAILED_WRITES_CLEANER_POLICY.defaultValue(); + /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ + @Deprecated + private static final String DEFAULT_AUTO_CLEAN = AUTO_CLEAN.defaultValue(); + /** + * @deprecated Use {@link #ASYNC_CLEAN} and its methods instead + */ + @Deprecated + private static final String DEFAULT_ASYNC_CLEAN = ASYNC_CLEAN.defaultValue(); + /** + * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead + */ + @Deprecated + private static final String DEFAULT_INCREMENTAL_CLEANER = CLEANER_INCREMENTAL_MODE_ENABLE.defaultValue(); + /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ + @Deprecated + private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = CLEANER_FILE_VERSIONS_RETAINED.defaultValue(); + /** @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead */ + @Deprecated + private static final String DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue(); + /** + * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead + */ + @Deprecated + private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.defaultValue(); + + private HoodieCleanConfig() { + super(); + } + + public static HoodieCleanConfig.Builder newBuilder() { + return new HoodieCleanConfig.Builder(); + } + + public static class Builder { + + private final HoodieCleanConfig cleanConfig = new HoodieCleanConfig(); + + public HoodieCleanConfig.Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.cleanConfig.getProps().load(reader); + return this; + } + } + + public HoodieCleanConfig.Builder fromProperties(Properties props) { + this.cleanConfig.getProps().putAll(props); + return this; + } + + public HoodieCleanConfig.Builder withAutoClean(Boolean autoClean) { + cleanConfig.setValue(AUTO_CLEAN, String.valueOf(autoClean)); + return this; + } + + public HoodieCleanConfig.Builder withAsyncClean(Boolean asyncClean) { + cleanConfig.setValue(ASYNC_CLEAN, String.valueOf(asyncClean)); + return this; + } + + public HoodieCleanConfig.Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) { + cleanConfig.setValue(CLEANER_INCREMENTAL_MODE_ENABLE, String.valueOf(incrementalCleaningMode)); + return this; + } + + public HoodieCleanConfig.Builder withCleaningTriggerStrategy(String cleaningTriggerStrategy) { + cleanConfig.setValue(CLEAN_TRIGGER_STRATEGY, cleaningTriggerStrategy); + return this; + } + + public HoodieCleanConfig.Builder withMaxCommitsBeforeCleaning(int maxCommitsBeforeCleaning) { + cleanConfig.setValue(CLEAN_MAX_COMMITS, String.valueOf(maxCommitsBeforeCleaning)); + return this; + } + + public HoodieCleanConfig.Builder withCleanerPolicy(HoodieCleaningPolicy policy) { + cleanConfig.setValue(CLEANER_POLICY, policy.name()); + return this; + } + + public HoodieCleanConfig.Builder retainFileVersions(int fileVersionsRetained) { + cleanConfig.setValue(CLEANER_FILE_VERSIONS_RETAINED, String.valueOf(fileVersionsRetained)); + return this; + } + + public HoodieCleanConfig.Builder retainCommits(int commitsRetained) { + cleanConfig.setValue(CLEANER_COMMITS_RETAINED, String.valueOf(commitsRetained)); + return this; + } + + public HoodieCleanConfig.Builder cleanerNumHoursRetained(int cleanerHoursRetained) { + cleanConfig.setValue(CLEANER_HOURS_RETAINED, String.valueOf(cleanerHoursRetained)); + return this; + } + + public HoodieCleanConfig.Builder allowMultipleCleans(boolean allowMultipleCleanSchedules) { + cleanConfig.setValue(ALLOW_MULTIPLE_CLEANS, String.valueOf(allowMultipleCleanSchedules)); + return this; + } + + public HoodieCleanConfig.Builder withCleanerParallelism(int cleanerParallelism) { + cleanConfig.setValue(CLEANER_PARALLELISM_VALUE, String.valueOf(cleanerParallelism)); + return this; + } + + public HoodieCleanConfig.Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) { + cleanConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLE, String.valueOf(cleanBootstrapSourceFileEnabled)); + return this; + } + + public HoodieCleanConfig.Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) { + cleanConfig.setValue(FAILED_WRITES_CLEANER_POLICY, failedWritesPolicy.name()); + return this; + } + + public HoodieCleanConfig build() { + cleanConfig.setDefaults(HoodieCleanConfig.class.getName()); + HoodieCleaningPolicy.valueOf(cleanConfig.getString(CLEANER_POLICY)); + return cleanConfig; + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index 4003a07de7f0b..d1d0e6726173b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -22,11 +22,6 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; -import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; -import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy; @@ -47,57 +42,9 @@ @ConfigClassProperty(name = "Compaction Configs", groupName = ConfigGroups.Names.WRITE_CLIENT, description = "Configurations that control compaction " - + "(merging of log files onto a new base files) as well as " - + "cleaning (reclamation of older/unused file groups/slices).") + + "(merging of log files onto a new base files).") public class HoodieCompactionConfig extends HoodieConfig { - public static final ConfigProperty AUTO_ARCHIVE = ConfigProperty - .key("hoodie.archive.automatic") - .defaultValue("true") - .withDocumentation("When enabled, the archival table service is invoked immediately after each commit," - + " to archive commits if we cross a maximum value of commits." - + " It's recommended to enable this, to ensure number of active commits is bounded."); - - public static final ConfigProperty ASYNC_ARCHIVE = ConfigProperty - .key("hoodie.archive.async") - .defaultValue("false") - .sinceVersion("0.11.0") - .withDocumentation("Only applies when " + AUTO_ARCHIVE.key() + " is turned on. " - + "When turned on runs archiver async with writing, which can speed up overall write performance."); - - public static final ConfigProperty AUTO_CLEAN = ConfigProperty - .key("hoodie.clean.automatic") - .defaultValue("true") - .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit," - + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage" - + " growth is bounded."); - - public static final ConfigProperty ASYNC_CLEAN = ConfigProperty - .key("hoodie.clean.async") - .defaultValue("false") - .withDocumentation("Only applies when " + AUTO_CLEAN.key() + " is turned on. " - + "When turned on runs cleaner async with writing, which can speed up overall write performance."); - - public static final ConfigProperty CLEANER_COMMITS_RETAINED = ConfigProperty - .key("hoodie.cleaner.commits.retained") - .defaultValue("10") - .withDocumentation("Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits " - + "(scheduled). This also directly translates into how much data retention the table supports for incremental queries."); - - public static final ConfigProperty CLEANER_HOURS_RETAINED = ConfigProperty.key("hoodie.cleaner.hours.retained") - .defaultValue("24") - .withDocumentation("Number of hours for which commits need to be retained. This config provides a more flexible option as" - + "compared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group," - + " corresponding to commits with commit times older than the configured number of hours to be retained are cleaned."); - - public static final ConfigProperty CLEANER_POLICY = ConfigProperty - .key("hoodie.cleaner.policy") - .defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) - .withDocumentation("Cleaning policy to be used. The cleaner service deletes older file slices files to re-claim space." - + " By default, cleaner spares the file slices written by the last N commits, determined by " + CLEANER_COMMITS_RETAINED.key() - + " Long running query plans may often refer to older file slices and will break if those are cleaned, before the query has had" - + " a chance to run. So, it is good to make sure that the data is retained for more than the maximum query execution time"); - public static final ConfigProperty INLINE_COMPACT = ConfigProperty .key("hoodie.compact.inline") .defaultValue("false") @@ -130,62 +77,6 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("Controls how compaction scheduling is triggered, by time or num delta commits or combination of both. " + "Valid options: " + Arrays.stream(CompactionTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); - public static final ConfigProperty CLEAN_TRIGGER_STRATEGY = ConfigProperty - .key("hoodie.clean.trigger.strategy") - .defaultValue(CleaningTriggerStrategy.NUM_COMMITS.name()) - .withDocumentation("Controls how cleaning is scheduled. Valid options: " - + Arrays.stream(CleaningTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); - - public static final ConfigProperty CLEAN_MAX_COMMITS = ConfigProperty - .key("hoodie.clean.max.commits") - .defaultValue("1") - .withDocumentation("Number of commits after the last clean operation, before scheduling of a new clean is attempted."); - - public static final ConfigProperty CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty - .key("hoodie.cleaner.fileversions.retained") - .defaultValue("3") - .withDocumentation("When " + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, " - + " the minimum number of file slices to retain in each file group, during cleaning."); - - public static final ConfigProperty CLEANER_INCREMENTAL_MODE_ENABLE = ConfigProperty - .key("hoodie.cleaner.incremental.mode") - .defaultValue("true") - .withDocumentation("When enabled, the plans for each cleaner service run is computed incrementally off the events " - + " in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full" - + " table for each planning (even with a metadata table)."); - - public static final ConfigProperty MAX_COMMITS_TO_KEEP = ConfigProperty - .key("hoodie.keep.max.commits") - .defaultValue("30") - .withDocumentation("Archiving service moves older entries from timeline into an archived log after each write, to " - + " keep the metadata overhead constant, even as the table size grows." - + "This config controls the maximum number of instants to retain in the active timeline. "); - - public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty - .key("hoodie.archive.delete.parallelism") - .defaultValue(100) - .withDocumentation("Parallelism for deleting archived hoodie commits."); - - public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty - .key("hoodie.keep.min.commits") - .defaultValue("20") - .withDocumentation("Similar to " + MAX_COMMITS_TO_KEEP.key() + ", but controls the minimum number of" - + "instants to retain in the active timeline."); - - public static final ConfigProperty COMMITS_ARCHIVAL_BATCH_SIZE = ConfigProperty - .key("hoodie.commits.archival.batch") - .defaultValue(String.valueOf(10)) - .withDocumentation("Archiving of instants is batched in best-effort manner, to pack more instants into a single" - + " archive log. This config controls such archival batch size."); - - public static final ConfigProperty CLEANER_BOOTSTRAP_BASE_FILE_ENABLE = ConfigProperty - .key("hoodie.cleaner.delete.bootstrap.base.file") - .defaultValue("false") - .withDocumentation("When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is " - + " cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the" - + " table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap " - + " base files are also physically deleted, to comply with data privacy enforcement processes."); - public static final ConfigProperty PARQUET_SMALL_FILE_LIMIT = ConfigProperty .key("hoodie.parquet.small.file.limit") .defaultValue(String.valueOf(104857600)) @@ -202,11 +93,6 @@ public class HoodieCompactionConfig extends HoodieConfig { + " Hudi will search commits in the reverse order, until we find a commit that has totalBytesWritten " + " larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * this_threshold)"); - public static final ConfigProperty CLEANER_PARALLELISM_VALUE = ConfigProperty - .key("hoodie.cleaner.parallelism") - .defaultValue("200") - .withDocumentation("Parallelism for the cleaning operation. Increase this if cleaning becomes slow."); - // 500GB of target IO per compaction (both read and write public static final ConfigProperty TARGET_IO_PER_COMPACTION_IN_MB = ConfigProperty .key("hoodie.compaction.target.io") @@ -227,13 +113,6 @@ public class HoodieCompactionConfig extends HoodieConfig { + "compaction during each compaction run. By default. Hudi picks the log file " + "with most accumulated unmerged data"); - public static final ConfigProperty PAYLOAD_CLASS_NAME = ConfigProperty - .key("hoodie.compaction.payload.class") - .defaultValue(OverwriteWithLatestAvroPayload.class.getName()) - .withDocumentation("This needs to be same as class used during insert/upserts. Just like writing, compaction also uses " - + "the record payload class to merge records in the log against each other, merge again with the base file and " - + "produce the final record to be written after compaction."); - public static final ConfigProperty COMPACTION_LAZY_BLOCK_READ_ENABLE = ConfigProperty .key("hoodie.compaction.lazy.block.read") .defaultValue("true") @@ -247,13 +126,6 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. " + "If this config is set to true, the reader reads the logfile in reverse direction, from pos=file_length to pos=0"); - public static final ConfigProperty FAILED_WRITES_CLEANER_POLICY = ConfigProperty - .key("hoodie.cleaner.policy.failed.writes") - .defaultValue(HoodieFailedWritesCleaningPolicy.EAGER.name()) - .withDocumentation("Cleaning policy for failed writes to be used. Hudi will delete any files written by " - + "failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before " - + "every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)"); - public static final ConfigProperty TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = ConfigProperty .key("hoodie.compaction.daybased.target.partitions") .defaultValue("10") @@ -290,39 +162,8 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("The average record size. If not explicitly specified, hudi will compute the " + "record size estimate compute dynamically based on commit metadata. " + " This is critical in computing the insert parallelism and bin-packing inserts into small files."); - - public static final ConfigProperty ALLOW_MULTIPLE_CLEANS = ConfigProperty - .key("hoodie.clean.allow.multiple") - .defaultValue(true) - .sinceVersion("0.11.0") - .withDocumentation("Allows scheduling/executing multiple cleans by enabling this config. If users prefer to strictly ensure clean requests should be mutually exclusive, " - + ".i.e. a 2nd clean will not be scheduled if another clean is not yet completed to avoid repeat cleaning of same files, they might want to disable this config."); - - public static final ConfigProperty ARCHIVE_MERGE_FILES_BATCH_SIZE = ConfigProperty - .key("hoodie.archive.merge.files.batch.size") - .defaultValue(10) - .withDocumentation("The number of small archive files to be merged at once."); - - public static final ConfigProperty ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES = ConfigProperty - .key("hoodie.archive.merge.small.file.limit.bytes") - .defaultValue(20L * 1024 * 1024) - .withDocumentation("This config sets the archive file size limit below which an archive file becomes a candidate to be selected as such a small file."); - - public static final ConfigProperty ARCHIVE_MERGE_ENABLE = ConfigProperty - .key("hoodie.archive.merge.enable") - .defaultValue(false) - .withDocumentation("When enable, hoodie will auto merge several small archive files into larger one. It's" - + " useful when storage scheme doesn't support append operation."); - - /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ - @Deprecated - public static final String CLEANER_POLICY_PROP = CLEANER_POLICY.key(); - /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ - @Deprecated - public static final String AUTO_CLEAN_PROP = AUTO_CLEAN.key(); - /** @deprecated Use {@link #ASYNC_CLEAN} and its methods instead */ - @Deprecated - public static final String ASYNC_CLEAN_PROP = ASYNC_CLEAN.key(); + + /** @deprecated Use {@link #INLINE_COMPACT} and its methods instead */ @Deprecated public static final String INLINE_COMPACT_PROP = INLINE_COMPACT.key(); @@ -335,39 +176,6 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #INLINE_COMPACT_TRIGGER_STRATEGY} and its methods instead */ @Deprecated public static final String INLINE_COMPACT_TRIGGER_STRATEGY_PROP = INLINE_COMPACT_TRIGGER_STRATEGY.key(); - /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ - @Deprecated - public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = CLEANER_FILE_VERSIONS_RETAINED.key(); - /** - * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead - */ - @Deprecated - public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key(); - /** - * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead - */ - @Deprecated - public static final String CLEANER_INCREMENTAL_MODE = CLEANER_INCREMENTAL_MODE_ENABLE.key(); - /** - * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead - */ - @Deprecated - public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key(); - /** - * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead - */ - @Deprecated - public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key(); - /** - * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead - */ - @Deprecated - public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = COMMITS_ARCHIVAL_BATCH_SIZE.key(); - /** - * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead - */ - @Deprecated - public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.key(); /** * @deprecated Use {@link #PARQUET_SMALL_FILE_LIMIT} and its methods instead */ @@ -418,16 +226,6 @@ public class HoodieCompactionConfig extends HoodieConfig { */ @Deprecated public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = COPY_ON_WRITE_RECORD_SIZE_ESTIMATE.defaultValue(); - /** - * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead - */ - @Deprecated - public static final String CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.key(); - /** - * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead - */ - @Deprecated - public static final String DEFAULT_CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.defaultValue(); /** * @deprecated Use {@link #TARGET_IO_PER_COMPACTION_IN_MB} and its methods instead */ @@ -446,12 +244,6 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #COMPACTION_STRATEGY} and its methods instead */ @Deprecated public static final String DEFAULT_COMPACTION_STRATEGY = COMPACTION_STRATEGY.defaultValue(); - /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ - @Deprecated - public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue(); - /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ - @Deprecated - public static final String PAYLOAD_CLASS_PROP = PAYLOAD_CLASS_NAME.key(); /** @deprecated Use {@link #COMPACTION_LAZY_BLOCK_READ_ENABLE} and its methods instead */ @Deprecated public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = COMPACTION_LAZY_BLOCK_READ_ENABLE.key(); @@ -464,33 +256,11 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #COMPACTION_REVERSE_LOG_READ_ENABLE} and its methods instead */ @Deprecated public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue(); - /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */ - @Deprecated - private static final String DEFAULT_CLEANER_POLICY = CLEANER_POLICY.defaultValue(); - /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ - @Deprecated - public static final String FAILED_WRITES_CLEANER_POLICY_PROP = FAILED_WRITES_CLEANER_POLICY.key(); - /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */ - @Deprecated - private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY = FAILED_WRITES_CLEANER_POLICY.defaultValue(); - /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */ - @Deprecated - private static final String DEFAULT_AUTO_CLEAN = AUTO_CLEAN.defaultValue(); - /** - * @deprecated Use {@link #ASYNC_CLEAN} and its methods instead - */ - @Deprecated - private static final String DEFAULT_ASYNC_CLEAN = ASYNC_CLEAN.defaultValue(); /** * @deprecated Use {@link #INLINE_COMPACT} and its methods instead */ @Deprecated private static final String DEFAULT_INLINE_COMPACT = INLINE_COMPACT.defaultValue(); - /** - * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead - */ - @Deprecated - private static final String DEFAULT_INCREMENTAL_CLEANER = CLEANER_INCREMENTAL_MODE_ENABLE.defaultValue(); /** @deprecated Use {@link #INLINE_COMPACT_NUM_DELTA_COMMITS} and its methods instead */ @Deprecated private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = INLINE_COMPACT_NUM_DELTA_COMMITS.defaultValue(); @@ -500,30 +270,6 @@ public class HoodieCompactionConfig extends HoodieConfig { /** @deprecated Use {@link #INLINE_COMPACT_TRIGGER_STRATEGY} and its methods instead */ @Deprecated private static final String DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY = INLINE_COMPACT_TRIGGER_STRATEGY.defaultValue(); - /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */ - @Deprecated - private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = CLEANER_FILE_VERSIONS_RETAINED.defaultValue(); - /** @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead */ - @Deprecated - private static final String DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue(); - /** @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead */ - @Deprecated - private static final String DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue(); - /** - * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead - */ - @Deprecated - private static final String DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue(); - /** - * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead - */ - @Deprecated - private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = COMMITS_ARCHIVAL_BATCH_SIZE.defaultValue(); - /** - * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead - */ - @Deprecated - private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.defaultValue(); /** @deprecated Use {@link #TARGET_PARTITIONS_PER_DAYBASED_COMPACTION} and its methods instead */ @Deprecated public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = TARGET_PARTITIONS_PER_DAYBASED_COMPACTION.key(); @@ -555,31 +301,6 @@ public Builder fromProperties(Properties props) { return this; } - public Builder withAutoArchive(Boolean autoArchive) { - compactionConfig.setValue(AUTO_ARCHIVE, String.valueOf(autoArchive)); - return this; - } - - public Builder withAsyncArchive(Boolean asyncArchive) { - compactionConfig.setValue(ASYNC_ARCHIVE, String.valueOf(asyncArchive)); - return this; - } - - public Builder withAutoClean(Boolean autoClean) { - compactionConfig.setValue(AUTO_CLEAN, String.valueOf(autoClean)); - return this; - } - - public Builder withAsyncClean(Boolean asyncClean) { - compactionConfig.setValue(ASYNC_CLEAN, String.valueOf(asyncClean)); - return this; - } - - public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) { - compactionConfig.setValue(CLEANER_INCREMENTAL_MODE_ENABLE, String.valueOf(incrementalCleaningMode)); - return this; - } - public Builder withInlineCompaction(Boolean inlineCompaction) { compactionConfig.setValue(INLINE_COMPACT, String.valueOf(inlineCompaction)); return this; @@ -595,57 +316,6 @@ public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy com return this; } - public Builder withCleaningTriggerStrategy(String cleaningTriggerStrategy) { - compactionConfig.setValue(CLEAN_TRIGGER_STRATEGY, cleaningTriggerStrategy); - return this; - } - - public Builder withMaxCommitsBeforeCleaning(int maxCommitsBeforeCleaning) { - compactionConfig.setValue(CLEAN_MAX_COMMITS, String.valueOf(maxCommitsBeforeCleaning)); - return this; - } - - public Builder withCleanerPolicy(HoodieCleaningPolicy policy) { - compactionConfig.setValue(CLEANER_POLICY, policy.name()); - return this; - } - - public Builder retainFileVersions(int fileVersionsRetained) { - compactionConfig.setValue(CLEANER_FILE_VERSIONS_RETAINED, String.valueOf(fileVersionsRetained)); - return this; - } - - public Builder retainCommits(int commitsRetained) { - compactionConfig.setValue(CLEANER_COMMITS_RETAINED, String.valueOf(commitsRetained)); - return this; - } - - public Builder cleanerNumHoursRetained(int cleanerHoursRetained) { - compactionConfig.setValue(CLEANER_HOURS_RETAINED, String.valueOf(cleanerHoursRetained)); - return this; - } - - public Builder archiveCommitsWith(int minToKeep, int maxToKeep) { - compactionConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep)); - compactionConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep)); - return this; - } - - public Builder withArchiveMergeFilesBatchSize(int number) { - compactionConfig.setValue(ARCHIVE_MERGE_FILES_BATCH_SIZE, String.valueOf(number)); - return this; - } - - public Builder withArchiveMergeSmallFileLimit(long size) { - compactionConfig.setValue(ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES, String.valueOf(size)); - return this; - } - - public Builder withArchiveMergeEnable(boolean enable) { - compactionConfig.setValue(ARCHIVE_MERGE_ENABLE, String.valueOf(enable)); - return this; - } - public Builder compactionSmallFileSize(long smallFileLimitBytes) { compactionConfig.setValue(PARQUET_SMALL_FILE_LIMIT, String.valueOf(smallFileLimitBytes)); return this; @@ -671,26 +341,11 @@ public Builder approxRecordSize(int recordSizeEstimate) { return this; } - public Builder allowMultipleCleans(boolean allowMultipleCleanSchedules) { - compactionConfig.setValue(ALLOW_MULTIPLE_CLEANS, String.valueOf(allowMultipleCleanSchedules)); - return this; - } - - public Builder withCleanerParallelism(int cleanerParallelism) { - compactionConfig.setValue(CLEANER_PARALLELISM_VALUE, String.valueOf(cleanerParallelism)); - return this; - } - public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) { compactionConfig.setValue(COMPACTION_STRATEGY, compactionStrategy.getClass().getName()); return this; } - public Builder withPayloadClass(String payloadClassName) { - compactionConfig.setValue(PAYLOAD_CLASS_NAME, payloadClassName); - return this; - } - public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) { compactionConfig.setValue(TARGET_IO_PER_COMPACTION_IN_MB, String.valueOf(targetIOPerCompactionInMB)); return this; @@ -701,11 +356,6 @@ public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBefo return this; } - public Builder withArchiveDeleteParallelism(int archiveDeleteParallelism) { - compactionConfig.setValue(DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE, String.valueOf(archiveDeleteParallelism)); - return this; - } - public Builder withMaxDeltaSecondsBeforeCompaction(int maxDeltaSecondsBeforeCompaction) { compactionConfig.setValue(INLINE_COMPACT_TIME_DELTA_SECONDS, String.valueOf(maxDeltaSecondsBeforeCompaction)); return this; @@ -736,49 +386,8 @@ public Builder withPreserveCommitMetadata(boolean preserveCommitMetadata) { return this; } - public Builder withCommitsArchivalBatchSize(int batchSize) { - compactionConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE, String.valueOf(batchSize)); - return this; - } - - public Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) { - compactionConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLE, String.valueOf(cleanBootstrapSourceFileEnabled)); - return this; - } - - public Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) { - compactionConfig.setValue(FAILED_WRITES_CLEANER_POLICY, failedWritesPolicy.name()); - return this; - } - public HoodieCompactionConfig build() { compactionConfig.setDefaults(HoodieCompactionConfig.class.getName()); - // validation - HoodieCleaningPolicy.valueOf(compactionConfig.getString(CLEANER_POLICY)); - - // Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some - // commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull - int minInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP)); - int maxInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP)); - int cleanerCommitsRetained = - Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED)); - ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep, - String.format( - "Increase %s=%d to be greater than %s=%d.", - HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), maxInstantsToKeep, - HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep)); - ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained, - String.format( - "Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull " - + "missing data from few instants.", - HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep, - HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), cleanerCommitsRetained)); - - boolean inlineCompact = compactionConfig.getBoolean(HoodieCompactionConfig.INLINE_COMPACT); - boolean inlineCompactSchedule = compactionConfig.getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT); - ValidationUtils.checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or " - + "schedule inline compaction (%s) can be enabled. Both can't be set to true at the same time. %s, %s", HoodieCompactionConfig.INLINE_COMPACT.key(), - HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), inlineCompact, inlineCompactSchedule)); return compactionConfig; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java index 2989d8c2b36d9..2a05752aa6e1e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import java.io.File; import java.io.FileReader; @@ -52,6 +53,20 @@ public class HoodiePayloadConfig extends HoodieConfig { .withDocumentation("Table column/field name to derive timestamp associated with the records. This can" + "be useful for e.g, determining the freshness of the table."); + public static final ConfigProperty PAYLOAD_CLASS_NAME = ConfigProperty + .key("hoodie.compaction.payload.class") + .defaultValue(OverwriteWithLatestAvroPayload.class.getName()) + .withDocumentation("This needs to be same as class used during insert/upserts. Just like writing, compaction also uses " + + "the record payload class to merge records in the log against each other, merge again with the base file and " + + "produce the final record to be written after compaction."); + + /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ + @Deprecated + public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue(); + /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */ + @Deprecated + public static final String PAYLOAD_CLASS_PROP = PAYLOAD_CLASS_NAME.key(); + private HoodiePayloadConfig() { super(); } @@ -86,6 +101,11 @@ public Builder withPayloadEventTimeField(String payloadEventTimeField) { return this; } + public HoodiePayloadConfig.Builder withPayloadClass(String payloadClassName) { + payloadConfig.setValue(PAYLOAD_CLASS_NAME, payloadClassName); + return this; + } + public HoodiePayloadConfig build() { payloadConfig.setDefaults(HoodiePayloadConfig.class.getName()); return payloadConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java index ba3888863d557..fc1798f206fbc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java @@ -119,16 +119,16 @@ public class HoodieStorageConfig extends HoodieConfig { .withDocumentation("Whether to use dictionary encoding"); public static final ConfigProperty PARQUET_WRITE_LEGACY_FORMAT_ENABLED = ConfigProperty - .key("hoodie.parquet.writelegacyformat.enabled") - .defaultValue("false") - .withDocumentation("Sets spark.sql.parquet.writeLegacyFormat. If true, data will be written in a way of Spark 1.4 and earlier. " - + "For example, decimal values will be written in Parquet's fixed-length byte array format which other systems such as Apache Hive and Apache Impala use. " - + "If false, the newer format in Parquet will be used. For example, decimals will be written in int-based format."); + .key("hoodie.parquet.writelegacyformat.enabled") + .defaultValue("false") + .withDocumentation("Sets spark.sql.parquet.writeLegacyFormat. If true, data will be written in a way of Spark 1.4 and earlier. " + + "For example, decimal values will be written in Parquet's fixed-length byte array format which other systems such as Apache Hive and Apache Impala use. " + + "If false, the newer format in Parquet will be used. For example, decimals will be written in int-based format."); public static final ConfigProperty PARQUET_OUTPUT_TIMESTAMP_TYPE = ConfigProperty - .key("hoodie.parquet.outputtimestamptype") - .defaultValue("TIMESTAMP_MICROS") - .withDocumentation("Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files."); + .key("hoodie.parquet.outputtimestamptype") + .defaultValue("TIMESTAMP_MICROS") + .withDocumentation("Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files."); public static final ConfigProperty HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty .key("hoodie.hfile.compression.algorithm") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index d18238fa4b6b8..4d07097c07c88 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -89,6 +89,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY; + /** * Class storing configs for the HoodieWriteClient. */ @@ -1148,31 +1150,31 @@ public boolean populateMetaFields() { * compaction properties. */ public HoodieCleaningPolicy getCleanerPolicy() { - return HoodieCleaningPolicy.valueOf(getString(HoodieCompactionConfig.CLEANER_POLICY)); + return HoodieCleaningPolicy.valueOf(getString(CLEANER_POLICY)); } public int getCleanerFileVersionsRetained() { - return getInt(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED); + return getInt(HoodieCleanConfig.CLEANER_FILE_VERSIONS_RETAINED); } public int getCleanerCommitsRetained() { - return getInt(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED); + return getInt(HoodieCleanConfig.CLEANER_COMMITS_RETAINED); } public int getCleanerHoursRetained() { - return getInt(HoodieCompactionConfig.CLEANER_HOURS_RETAINED); + return getInt(HoodieCleanConfig.CLEANER_HOURS_RETAINED); } public int getMaxCommitsToKeep() { - return getInt(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP); + return getInt(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP); } public int getMinCommitsToKeep() { - return getInt(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP); + return getInt(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP); } public int getArchiveMergeFilesBatchSize() { - return getInt(HoodieCompactionConfig.ARCHIVE_MERGE_FILES_BATCH_SIZE); + return getInt(HoodieArchivalConfig.ARCHIVE_MERGE_FILES_BATCH_SIZE); } public int getParquetSmallFileLimit() { @@ -1192,7 +1194,7 @@ public int getCopyOnWriteRecordSizeEstimate() { } public boolean allowMultipleCleans() { - return getBoolean(HoodieCompactionConfig.ALLOW_MULTIPLE_CLEANS); + return getBoolean(HoodieCleanConfig.ALLOW_MULTIPLE_CLEANS); } public boolean shouldAutoTuneInsertSplits() { @@ -1200,43 +1202,43 @@ public boolean shouldAutoTuneInsertSplits() { } public int getCleanerParallelism() { - return getInt(HoodieCompactionConfig.CLEANER_PARALLELISM_VALUE); + return getInt(HoodieCleanConfig.CLEANER_PARALLELISM_VALUE); } public int getCleaningMaxCommits() { - return getInt(HoodieCompactionConfig.CLEAN_MAX_COMMITS); + return getInt(HoodieCleanConfig.CLEAN_MAX_COMMITS); } public CleaningTriggerStrategy getCleaningTriggerStrategy() { - return CleaningTriggerStrategy.valueOf(getString(HoodieCompactionConfig.CLEAN_TRIGGER_STRATEGY)); + return CleaningTriggerStrategy.valueOf(getString(HoodieCleanConfig.CLEAN_TRIGGER_STRATEGY)); } public boolean isAutoClean() { - return getBoolean(HoodieCompactionConfig.AUTO_CLEAN); + return getBoolean(HoodieCleanConfig.AUTO_CLEAN); } public boolean getArchiveMergeEnable() { - return getBoolean(HoodieCompactionConfig.ARCHIVE_MERGE_ENABLE); + return getBoolean(HoodieArchivalConfig.ARCHIVE_MERGE_ENABLE); } public long getArchiveMergeSmallFileLimitBytes() { - return getLong(HoodieCompactionConfig.ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES); + return getLong(HoodieArchivalConfig.ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES); } public boolean isAutoArchive() { - return getBoolean(HoodieCompactionConfig.AUTO_ARCHIVE); + return getBoolean(HoodieArchivalConfig.AUTO_ARCHIVE); } public boolean isAsyncArchive() { - return getBoolean(HoodieCompactionConfig.ASYNC_ARCHIVE); + return getBoolean(HoodieArchivalConfig.ASYNC_ARCHIVE); } public boolean isAsyncClean() { - return getBoolean(HoodieCompactionConfig.ASYNC_CLEAN); + return getBoolean(HoodieCleanConfig.ASYNC_CLEAN); } public boolean incrementalCleanerModeEnabled() { - return getBoolean(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE_ENABLE); + return getBoolean(HoodieCleanConfig.CLEANER_INCREMENTAL_MODE_ENABLE); } public boolean inlineCompactionEnabled() { @@ -1280,7 +1282,7 @@ public Boolean getCompactionReverseLogReadEnabled() { } public int getArchiveDeleteParallelism() { - return getInt(HoodieCompactionConfig.DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE); + return getInt(HoodieArchivalConfig.DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE); } public boolean inlineClusteringEnabled() { @@ -1321,7 +1323,7 @@ public int getAsyncClusterMaxCommits() { } public String getPayloadClass() { - return getString(HoodieCompactionConfig.PAYLOAD_CLASS_NAME); + return getString(HoodiePayloadConfig.PAYLOAD_CLASS_NAME); } public int getTargetPartitionsPerDayBasedCompaction() { @@ -1329,11 +1331,11 @@ public int getTargetPartitionsPerDayBasedCompaction() { } public int getCommitArchivalBatchSize() { - return getInt(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE); + return getInt(HoodieArchivalConfig.COMMITS_ARCHIVAL_BATCH_SIZE); } public Boolean shouldCleanBootstrapBaseFile() { - return getBoolean(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLE); + return getBoolean(HoodieCleanConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLE); } public String getClusteringUpdatesStrategyClass() { @@ -1342,7 +1344,7 @@ public String getClusteringUpdatesStrategyClass() { public HoodieFailedWritesCleaningPolicy getFailedWritesCleanPolicy() { return HoodieFailedWritesCleaningPolicy - .valueOf(getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY)); + .valueOf(getString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY)); } /** @@ -2117,6 +2119,8 @@ public static class Builder { private boolean isIndexConfigSet = false; private boolean isStorageConfigSet = false; private boolean isCompactionConfigSet = false; + private boolean isCleanConfigSet = false; + private boolean isArchivalConfigSet = false; private boolean isClusteringConfigSet = false; private boolean isOptimizeConfigSet = false; private boolean isMetricsConfigSet = false; @@ -2284,6 +2288,18 @@ public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) { return this; } + public Builder withCleanConfig(HoodieCleanConfig cleanConfig) { + writeConfig.getProps().putAll(cleanConfig.getProps()); + isCleanConfigSet = true; + return this; + } + + public Builder withArchivalConfig(HoodieArchivalConfig cleanConfig) { + writeConfig.getProps().putAll(cleanConfig.getProps()); + isArchivalConfigSet = true; + return this; + } + public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) { writeConfig.getProps().putAll(clusteringConfig.getProps()); isClusteringConfigSet = true; @@ -2517,6 +2533,10 @@ protected void setDefaults() { writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isCompactionConfigSet, HoodieCompactionConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + writeConfig.setDefaultOnCondition(!isCleanConfigSet, + HoodieCleanConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); + writeConfig.setDefaultOnCondition(!isArchivalConfigSet, + HoodieArchivalConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultOnCondition(!isClusteringConfigSet, HoodieClusteringConfig.newBuilder().withEngineType(engineType) .fromProperties(writeConfig.getProps()).build()); @@ -2587,10 +2607,10 @@ private void autoAdjustConfigsForConcurrencyMode(boolean isLockProviderPropertyS if (WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value() .equalsIgnoreCase(writeConcurrencyMode)) { // In this case, we assume that the user takes care of setting the lock provider used - writeConfig.setValue(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + writeConfig.setValue(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name()); LOG.info(String.format("Automatically set %s=%s since optimistic concurrency control is used", - HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name())); } } @@ -2602,9 +2622,34 @@ private void validate() { Objects.requireNonNull(writeConfig.getString(BASE_PATH)); if (writeConfig.getString(WRITE_CONCURRENCY_MODE) .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value())) { - ValidationUtils.checkArgument(!writeConfig.getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY) + ValidationUtils.checkArgument(!writeConfig.getString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY) .equals(HoodieFailedWritesCleaningPolicy.EAGER.name()), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY"); } + + HoodieCleaningPolicy.valueOf(writeConfig.getString(CLEANER_POLICY)); + // Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some + // commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull + int minInstantsToKeep = Integer.parseInt(writeConfig.getStringOrDefault(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP)); + int maxInstantsToKeep = Integer.parseInt(writeConfig.getStringOrDefault(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP)); + int cleanerCommitsRetained = + Integer.parseInt(writeConfig.getStringOrDefault(HoodieCleanConfig.CLEANER_COMMITS_RETAINED)); + ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep, + String.format( + "Increase %s=%d to be greater than %s=%d.", + HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), maxInstantsToKeep, + HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep)); + ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained, + String.format( + "Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull " + + "missing data from few instants.", + HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep, + HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), cleanerCommitsRetained)); + + boolean inlineCompact = writeConfig.getBoolean(HoodieCompactionConfig.INLINE_COMPACT); + boolean inlineCompactSchedule = writeConfig.getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT); + ValidationUtils.checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or " + + "schedule inline compaction (%s) can be enabled. Both can't be set to true at the same time. %s, %s", HoodieCompactionConfig.INLINE_COMPACT.key(), + HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), inlineCompact, inlineCompactSchedule)); } public HoodieWriteConfig build() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index e36adf6be53d3..f4ee3fc9f2424 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -58,6 +58,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; @@ -255,20 +257,24 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) .withSchema(HoodieMetadataRecord.getClassSchema().toString()) .forTable(tableName) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + // we will trigger cleaning manually, to control the instant times + .withCleanConfig(HoodieCleanConfig.newBuilder() .withAsyncClean(writeConfig.isMetadataAsyncClean()) - // we will trigger cleaning manually, to control the instant times .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) + .build()) + // we will trigger archive manually, to ensure only regular writer invokes it + .withArchivalConfig(HoodieArchivalConfig.newBuilder() .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep) - // we will trigger compaction manually, to control the instant times + .withAutoArchive(false) + .build()) + // we will trigger compaction manually, to control the instant times + .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()) - // we will trigger archive manually, to ensure only regular writer invokes it - .withAutoArchive(false) // by default, the HFile does not keep the metadata fields, set up as false // to always use the metadata of the new record. .withPreserveCommitMetadata(false) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index 3e2d8abdd7466..5d1a55453d162 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -100,7 +100,7 @@ public void runMerge(HoodieTable>, HoodieData params = new HashMap<>(3); - params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), "1"); - params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "5"); - params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2"); + params.put(HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1"); + params.put(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "5"); + params.put(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2"); if (withAlternative) { params.put("hoodie.avro.schema.externalTransformation", "true"); } else { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index 7a8f8a1580d97..49827166258ba 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -21,7 +21,6 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hudi.avro.HoodieAvroUtils.rewriteRecord import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.model.HoodieRecord @@ -39,8 +38,10 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructField, StructType} - import java.util.Properties + +import org.apache.hudi.avro.HoodieAvroUtils + import scala.collection.JavaConverters._ object HoodieSparkUtils extends SparkAdapterSupport { @@ -162,11 +163,11 @@ object HoodieSparkUtils extends SparkAdapterSupport { if (rows.isEmpty) { Iterator.empty } else { + val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr) val transform: GenericRecord => GenericRecord = if (sameSchema) identity else { - val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr) - rewriteRecord(_, readerAvroSchema) + HoodieAvroUtils.rewriteRecordDeep(_, readerAvroSchema) } // Since caller might request to get records in a different ("evolved") schema, we will be rewriting from diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index f6315eec7d211..629b16cdb88f0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -38,7 +38,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; @@ -80,7 +80,7 @@ public class TestClientRollback extends HoodieClientTestBase { */ @Test public void testSavepointAndRollback() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder() + HoodieWriteConfig cfg = getConfigBuilder().withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build(); try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); @@ -214,7 +214,7 @@ public void testRollbackCommit() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withRollbackUsingMarkers(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); @@ -329,7 +329,7 @@ public void testFailedRollbackCommit( .enable(enableMetadataTable) .build() ) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); @@ -436,7 +436,7 @@ public void testAutoRollbackInflightCommit() throws Exception { // Set Failed Writes rollback to LAZY HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()).build(); HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); @@ -530,7 +530,7 @@ public void testRollbackWithRequestedRollbackPlan(boolean enableMetadataTable, b .enable(enableMetadataTable) .build() ) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index 3aeca0f275891..268674e78d87a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -33,6 +33,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -100,9 +102,11 @@ public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws E properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) - .withAutoArchive(false).withAutoClean(false).build()) + .withAutoClean(false).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .withAutoArchive(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests .withMarkersType(MarkerType.DIRECT.name()) @@ -192,9 +196,11 @@ public void testMultiWriterWithInsertsToDistinctPartitions(HoodieTableType table properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20"); HoodieWriteConfig cfg = getConfigBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withInlineCompaction(false) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withMaxNumDeltaCommitsBeforeCompaction(2) .build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) @@ -265,9 +271,12 @@ private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType t properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); // Disabling embedded timeline server, it doesn't work with multiwriter HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false) - .withInlineCompaction(false).withAsyncClean(true) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withAutoClean(false) + .withAsyncClean(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(2).build()) .withEmbeddedTimelineServerEnabled(false) // Timeline-server-based markers are not used for multi-writer tests @@ -402,7 +411,8 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests @@ -453,7 +463,8 @@ public void testHoodieClientMultiWriterAutoCommitForConflict() throws Exception properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "100"); HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests @@ -536,7 +547,8 @@ public void testHoodieClientMultiWriterAutoCommitNonConflict() throws Exception properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "100"); HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) // Timeline-server-based markers are not used for multi-writer tests diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 1a618a01dfe24..6d410ded1ccca 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -82,11 +82,13 @@ import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLockConfig; -import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.storage.HoodieHFileReader; @@ -476,7 +478,13 @@ public void testMetadataTableArchival() throws Exception { .archiveCommitsWith(3, 4) .retainCommits(1) .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).retainCommits(1).build()).build(); + .withCleanConfig(HoodieCleanConfig.newBuilder() + .retainCommits(1) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(2, 3) + .build()) + .build(); initWriteConfigAndMetatableWriter(writeConfig, true); AtomicInteger commitTime = new AtomicInteger(1); @@ -637,8 +645,9 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws I initPath(); int maxCommits = 1; HoodieWriteConfig cfg = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits) + .build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) @@ -1172,8 +1181,15 @@ public void testManualRollbacks(final boolean populateMateFields) throws Excepti .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction) .withPopulateMetaFields(populateMateFields) .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) - .retainCommits(1).retainFileVersions(1).withAutoClean(false).withAsyncClean(true).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .retainCommits(1) + .retainFileVersions(1) + .withAutoClean(false) + .withAsyncClean(true) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) + .build()) .build(); initWriteConfigAndMetatableWriter(writeConfig, true); @@ -1399,10 +1415,13 @@ public void testColStatsPrefixLookup() throws IOException { // disable small file handling so that every insert goes to a new file group. HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) .withRollbackUsingMarkers(false) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withAutoClean(false).retainCommits(1).retainFileVersions(1) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(0) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(true) .withMetadataIndexColumnStats(true) @@ -1612,7 +1631,7 @@ public void testMetadataMultiWriter() throws Exception { properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) @@ -1676,8 +1695,9 @@ public void testMultiWriterForDoubleLocking() throws Exception { properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(true).retainCommits(4).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(true).retainCommits(4) + .build()) .withAutoCommit(false) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) @@ -1853,9 +1873,12 @@ public void testCleaningArchivingAndCompaction() throws Exception { .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) .archiveCommitsWith(40, 60).retainCommits(1) .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4) + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER) - .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(false).build()) + .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(false) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(2, 4).build()) .build(); List records; @@ -2006,8 +2029,8 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(false, true, false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) .withProperties(properties) @@ -2034,10 +2057,8 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte // set hoodie.table.version to 2 in hoodie.properties file changeTableVersion(HoodieTableVersion.TWO); - writeConfig = getWriteConfigBuilder(true, true, false) - .withRollbackUsingMarkers(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + writeConfig = getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) .withProperties(properties) @@ -2119,7 +2140,7 @@ public void testDeletePartitions() throws Exception { int maxCommits = 1; HoodieWriteConfig cfg = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) @@ -2285,13 +2306,13 @@ private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts) { HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER); - return builder - .withCompactionConfig( + return builder.withCompactionConfig( HoodieCompactionConfig.newBuilder() .compactionSmallFileSize(smallFileSize) // Set rollback to LAZY so no inflights are deleted - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .insertSplitSize(insertSplitSize).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withStorageConfig( HoodieStorageConfig.newBuilder() .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) @@ -2307,8 +2328,8 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex. .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withWriteStatusClass(MetadataMergeWriteStatus.class) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy) - .compactionSmallFileSize(1024 * 1024).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index cebf3145bfd28..8ba459b77226d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -71,12 +71,13 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodiePreCommitValidatorConfig; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodiePreCommitValidatorConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieCorruptedDataException; @@ -360,7 +361,7 @@ public void testPreCommitValidationWithMultipleInflights() throws Exception { .withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + 500) .build(); HoodieWriteConfig config = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) .withPreCommitValidatorConfig(validatorConfig) .build(); @@ -386,7 +387,7 @@ public void testPreCommitValidationWithMultipleInflights() throws Exception { .withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + numRecords) .build(); config = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) .withPreCommitValidatorConfig(validatorConfig) .build(); String instant2 = HoodieActiveTimeline.createNewInstantTime(); @@ -921,7 +922,7 @@ private void testUpsertsUpdatePartitionPath(IndexType indexType, HoodieWriteConf .setTimelineLayoutVersion(VERSION_0) .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); // Set rollback to LAZY so no inflights are deleted - hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + hoodieWriteConfig.getProps().put(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name()); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); @@ -2606,17 +2607,16 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String if (!populateMetaFields) { builder.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.SIMPLE).build()); } - return builder - .withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(smallFileSize) - // Set rollback to LAZY so no inflights are deleted - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) - .insertSplitSize(insertSplitSize).build()) - .withStorageConfig( - HoodieStorageConfig.newBuilder() - .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) - .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) + return builder.withCompactionConfig(HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(smallFileSize) + // Set rollback to LAZY so no inflights are deleted + .insertSplitSize(insertSplitSize).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) + .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) .withMergeAllowDuplicateOnInserts(mergeAllowDuplicateInserts) .withProps(props) .build(); @@ -2636,7 +2636,7 @@ protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient meta private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) { return getConfigBuilder() .withEmbeddedTimelineServerEnabled(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(cleaningPolicy) .withAutoClean(false).build()) .withTimelineLayoutVersion(1) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index 2e387be54452a..29c653daee61a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -36,9 +36,11 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; @@ -338,9 +340,11 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesClea .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withAutoCommit(autoCommit) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(policy) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withAutoClean(false).retainCommits(1).retainFileVersions(1) + .build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() @@ -390,16 +394,20 @@ protected HoodieWriteConfig getMetadataWriteConfig(HoodieWriteConfig writeConfig .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) .withSchema(HoodieMetadataRecord.getClassSchema().toString()) .forTable(writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + // we will trigger cleaning manually, to control the instant times + .withCleanConfig(HoodieCleanConfig.newBuilder() .withAsyncClean(writeConfig.isMetadataAsyncClean()) - // we will trigger cleaning manually, to control the instant times .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) - .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep) - // we will trigger compaction manually, to control the instant times + .build()) + // we will trigger archival manually, to control the instant times + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep).build()) + // we will trigger compaction manually, to control the instant times + .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()).build()) .withParallelism(parallelism, parallelism) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java index bdbc9e72d3f4a..8531030a5cc24 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java @@ -26,7 +26,8 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.log4j.LogManager; @@ -275,7 +276,10 @@ private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception private HoodieWriteConfig getWriteConfig(int minArchivalCommits, int maxArchivalCommits) throws Exception { return HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java index 87bcad04bc85e..407fb8de0e812 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java @@ -32,6 +32,8 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieHBaseIndexConfig; import org.apache.hudi.config.HoodieIndexConfig; @@ -477,9 +479,9 @@ private JavaRDD generateAndCommitRecords(SparkRDDWriteClient write public void testHbaseTagLocationForArchivedCommits() throws Exception { // Load to memory Map params = new HashMap(); - params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), "1"); - params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "3"); - params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2"); + params.put(HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1"); + params.put(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "3"); + params.put(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2"); HoodieWriteConfig config = getConfigBuilder(100, false, false).withProps(params).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index d412052c2dbbf..4f41c4a44d9fb 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -45,6 +45,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -181,13 +183,14 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, init(tableType); HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits) - .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).withFailedWritesCleaningPolicy(failedWritesCleaningPolicy).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() .withArchiveMergeEnable(enableArchiveMerge) .withArchiveMergeFilesBatchSize(archiveFilesBatch) .withArchiveMergeSmallFileLimit(size) - .withFailedWritesCleaningPolicy(failedWritesCleaningPolicy) - .build()) + .archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata) @@ -566,7 +569,8 @@ public void testArchiveCommitSavepointNoHole(boolean enableMetadataTable) throws init(); HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 5).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -716,7 +720,8 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -881,9 +886,9 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) - .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() - .withRemoteServerPort(timelineServicePort).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -940,7 +945,8 @@ public void testArchiveInflightClean(boolean enableMetadataTable) throws Excepti HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) @@ -1146,8 +1152,8 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { // Test configs where metadata table has more aggressive archival configs than the compaction config HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .retainCommits(1).archiveCommitsWith(2, 4).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 4).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index b8545b0f63809..8da877940b36b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -70,6 +70,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; @@ -261,11 +262,15 @@ public void testMultiClean() { HoodieWriteConfig writeConfig = getConfigBuilder() .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() .withEnableBackupForRemoteFileSystemView(false).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .allowMultipleCleans(false) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withAutoClean(false).retainCommits(1).retainFileVersions(1) + .build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) .withEmbeddedTimelineServerEnabled(false).build(); int index = 0; @@ -334,8 +339,9 @@ private void testInsertAndCleanByVersions( throws Exception { int maxVersions = 2; // keep upto 2 versions for each file HoodieWriteConfig cfg = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + .retainFileVersions(maxVersions).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) .build(); @@ -503,7 +509,7 @@ private void testInsertAndCleanByCommits( throws Exception { int maxCommits = 3; // keep upto 3 commits from the past HoodieWriteConfig cfg = getConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) @@ -579,7 +585,7 @@ private void testFailedInsertAndCleanByCommits( HoodieWriteConfig cfg = getConfigBuilder() .withAutoCommit(false) .withHeartbeatIntervalInMs(3000) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) @@ -732,8 +738,8 @@ public void testCleanEmptyInstants() throws Exception { HoodieWriteConfig.newBuilder() .withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -787,8 +793,9 @@ public void testCleanWithReplaceCommits() throws Exception { .withMetadataConfig(HoodieMetadataConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1) .withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .retainCommits(2).build()) .build(); HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); @@ -1140,7 +1147,7 @@ public void testCleanMarkerDataFilesOnRollback() throws Exception { public void testCleaningWithZeroPartitionPaths() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1164,7 +1171,7 @@ public void testCleaningWithZeroPartitionPaths() throws Exception { public void testKeepLatestCommitsWithPendingCompactions() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); // Deletions: @@ -1188,7 +1195,7 @@ public void testKeepLatestVersionsWithPendingCompactions(boolean retryFailure) t HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build()) .build(); // Deletions: @@ -1212,8 +1219,8 @@ public void testCleanPreviousCorruptedCleanFiles() throws IOException { HoodieWriteConfig.newBuilder() .withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); String commitTime = makeNewCommitTime(1, "%09d"); @@ -1241,7 +1248,7 @@ public void testRerunFailedClean(boolean simulateMetadataFailure) throws Excepti .withMetadataConfig(HoodieMetadataConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1) .withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1317,7 +1324,7 @@ private void testInsertAndCleanFailedWritesByVersions( HoodieWriteConfig cfg = getConfigBuilder() .withAutoCommit(false) .withHeartbeatIntervalInMs(3000) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java index bd015baec9dbb..9fcac64c002f1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java @@ -37,7 +37,7 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; @@ -77,14 +77,14 @@ public class TestCleanPlanExecutor extends TestCleaner { @Test public void testInvalidCleaningTriggerStrategy() { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withIncrementalCleaningMode(true) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) - .withCleanBootstrapBaseFileEnabled(true) - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2) - .withCleaningTriggerStrategy("invalid_strategy").build()) - .build(); + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withIncrementalCleaningMode(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withCleanBootstrapBaseFileEnabled(true) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2) + .withCleaningTriggerStrategy("invalid_strategy").build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()).build(); Exception e = assertThrows(IllegalArgumentException.class, () -> runCleaner(config, true), "should fail when invalid trigger strategy is provided!"); assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.table.action.clean.CleaningTriggerStrategy.invalid_strategy")); } @@ -108,18 +108,15 @@ public void testKeepLatestCommits( boolean simulateFailureRetry, boolean simulateMetadataFailure, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig( - HoodieMetadataConfig.newBuilder() - .withAssumeDatePartitioning(true) - .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .retainCommits(2) - .withMaxCommitsBeforeCleaning(2).build()) - .build(); + .withMaxCommitsBeforeCleaning(2) + .build()).build(); HoodieTestTable testTable = HoodieTestTable.of(metaClient); String p0 = "2020/01/01"; @@ -274,7 +271,7 @@ public void testKeepLatestFileVersions() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); @@ -353,7 +350,7 @@ public void testKeepLatestFileVersionsWithBootstrapFileClean() throws Exception HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanBootstrapBaseFileEnabled(true) .withCleanerParallelism(1) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) @@ -453,18 +450,15 @@ public void testKeepLatestFileVersionsWithBootstrapFileClean() throws Exception @Test public void testKeepLatestFileVersionsMOR() throws Exception { - HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig( - HoodieMetadataConfig.newBuilder() - .withAssumeDatePartitioning(true) - // Column Stats Index is disabled, since these tests construct tables which are - // not valid (empty commit metadata, invalid parquet files) - .withMetadataIndexColumnStats(false) - .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) - .build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true) + // Column Stats Index is disabled, since these tests construct tables which are + // not valid (empty commit metadata, invalid parquet files) + .withMetadataIndexColumnStats(false) + .build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1) + .build()).build(); HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); HoodieTestTable testTable = HoodieTestTable.of(metaClient); @@ -497,18 +491,14 @@ public void testKeepLatestFileVersionsMOR() throws Exception { @Test public void testKeepLatestCommitsMOR() throws Exception { - HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig( - HoodieMetadataConfig.newBuilder() - .withAssumeDatePartitioning(true) + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true) // Column Stats Index is disabled, since these tests construct tables which are // not valid (empty commit metadata, invalid parquet files) - .withMetadataIndexColumnStats(false) - .build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) - .build(); + .withMetadataIndexColumnStats(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) + .build(); HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); HoodieTestTable testTable = HoodieTestTable.of(metaClient); @@ -552,11 +542,12 @@ public void testKeepXHoursWithCleaning( boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanConfig(HoodieCleanConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS).cleanerNumHoursRetained(2).build()) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS).cleanerNumHoursRetained(2) + .build()) .build(); HoodieTestTable testTable = HoodieTestTable.of(metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java index b4d6aefa71fe6..baff4ebac8752 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java @@ -27,7 +27,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; @@ -54,8 +55,9 @@ public class TestHoodieSparkCopyOnWriteTableArchiveWithReplace extends SparkClie public void testDeletePartitionAndArchive(boolean metadataEnabled) throws IOException { HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.COPY_ON_WRITE); HoodieWriteConfig writeConfig = getConfigBuilder(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).retainCommits(1).build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) .build(); try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(DEFAULT_PARTITION_PATHS)) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index c33f0bcc2c5aa..2f0e585ec90a0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -43,6 +43,7 @@ import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -556,7 +557,7 @@ void testRestoreWithCleanedUpCommits() throws Exception { // trigger clean. creating a new client with aggresive cleaner configs so that clean will kick in immediately. cfgBuilder = getConfigBuilder(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) // Timeline-server-based markers are not used for multi-rollback tests .withMarkersType(MarkerType.DIRECT.name()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); @@ -977,10 +978,13 @@ private long getNumLogFilesInLatestFileSlice(HoodieTableMetaClient metaClient, H private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean rollbackUsingMarkers) { HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(autoCommit).withRollbackUsingMarkers(rollbackUsingMarkers) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withAutoClean(false) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024L) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(3) - .withAutoClean(false) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()); + .build()); return cfgBuilder.build(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 16fd48af6c014..900674a677588 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -38,10 +38,11 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; -import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.index.SparkHoodieIndexFactory; @@ -146,8 +147,8 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType in .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withWriteStatusClass(MetadataMergeWriteStatus.class) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy) - .compactionSmallFileSize(1024 * 1024).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 66066040275bf..fa65461bfdb0e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -745,15 +745,18 @@ public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, * b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this transformed schema * * @param oldRecord oldRecord to be rewritten + * @param oldAvroSchema old avro schema. * @param newSchema newSchema used to rewrite oldRecord * @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema) * @param fieldNames track the full name of visited field when we travel new schema. * @return newRecord for new Schema */ - private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema, Map renameCols, Deque fieldNames) { + private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldAvroSchema, Schema newSchema, Map renameCols, Deque fieldNames) { if (oldRecord == null) { return null; } + // try to get real schema for union type + Schema oldSchema = getActualSchemaFromUnion(oldAvroSchema, oldRecord); switch (newSchema.getType()) { case RECORD: if (!(oldRecord instanceof IndexedRecord)) { @@ -761,39 +764,32 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSch } IndexedRecord indexedRecord = (IndexedRecord) oldRecord; List fields = newSchema.getFields(); - Map helper = new HashMap<>(); - + GenericData.Record newRecord = new GenericData.Record(newSchema); for (int i = 0; i < fields.size(); i++) { Schema.Field field = fields.get(i); String fieldName = field.name(); fieldNames.push(fieldName); if (oldSchema.getField(field.name()) != null) { Schema.Field oldField = oldSchema.getField(field.name()); - helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); + newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); } else { String fieldFullName = createFullName(fieldNames); - String[] colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\."); - String lastColNameFromOldSchema = colNamePartsFromOldSchema[colNamePartsFromOldSchema.length - 1]; + String fieldNameFromOldSchema = renameCols.getOrDefault(fieldFullName, ""); // deal with rename - if (oldSchema.getField(field.name()) == null && oldSchema.getField(lastColNameFromOldSchema) != null) { + if (oldSchema.getField(field.name()) == null && oldSchema.getField(fieldNameFromOldSchema) != null) { // find rename - Schema.Field oldField = oldSchema.getField(lastColNameFromOldSchema); - helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); - } - } - fieldNames.pop(); - } - GenericData.Record newRecord = new GenericData.Record(newSchema); - for (int i = 0; i < fields.size(); i++) { - if (helper.containsKey(i)) { - newRecord.put(i, helper.get(i)); - } else { - if (fields.get(i).defaultVal() instanceof JsonProperties.Null) { - newRecord.put(i, null); + Schema.Field oldField = oldSchema.getField(fieldNameFromOldSchema); + newRecord.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames)); } else { - newRecord.put(i, fields.get(i).defaultVal()); + // deal with default value + if (fields.get(i).defaultVal() instanceof JsonProperties.Null) { + newRecord.put(i, null); + } else { + newRecord.put(i, fields.get(i).defaultVal()); + } } } + fieldNames.pop(); } return newRecord; case ARRAY: @@ -1028,4 +1024,8 @@ public GenericRecord next() { } }; } + + public static GenericRecord rewriteRecordDeep(GenericRecord oldRecord, Schema newSchema) { + return rewriteRecordWithNewSchema(oldRecord, newSchema, Collections.EMPTY_MAP); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java index cc62bcc32824f..917cfe621f11e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java @@ -36,6 +36,13 @@ public class HoodieCommonConfig extends HoodieConfig { .defaultValue(false) .withDocumentation("Enables support for Schema Evolution feature"); + public static final ConfigProperty RECONCILE_SCHEMA = ConfigProperty + .key("hoodie.datasource.write.reconcile.schema") + .defaultValue(false) + .withDocumentation("When a new batch of write has records with old schema, but latest table schema got " + + "evolved, this config will upgrade the records to leverage latest table schema(default values will be " + + "injected to missing fields). If not, the write batch would fail."); + public static final ConfigProperty SPILLABLE_DISK_MAP_TYPE = ConfigProperty .key("hoodie.common.spillable.diskmap.type") .defaultValue(ExternalSpillableMap.DiskMapType.BITCASK) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 4f99926887692..5fc989e2e5185 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -23,6 +23,7 @@ import org.apache.avro.Schema.Field; import org.apache.avro.SchemaCompatibility; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -261,6 +262,11 @@ private MessageType getTableParquetSchemaFromDataFile() { } } + public static MessageType convertAvroSchemaToParquet(Schema schema, Configuration hadoopConf) { + AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(hadoopConf); + return avroSchemaConverter.convert(schema); + } + private Schema convertParquetSchemaToAvro(MessageType parquetSchema) { AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(metaClient.getHadoopConf()); return avroSchemaConverter.convert(parquetSchema); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 9687136444eeb..16a264e06ddcd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -57,8 +57,8 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; +import java.util.Collections; import java.util.Deque; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -380,7 +380,7 @@ private void processDataBlock(HoodieDataBlock dataBlock, Option keySpec Option schemaOption = getMergedSchema(dataBlock); while (recordIterator.hasNext()) { IndexedRecord currentRecord = recordIterator.next(); - IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), new HashMap<>()) : currentRecord; + IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), Collections.emptyMap()) : currentRecord; processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN, this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName)); totalLogRecords.incrementAndGet(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java similarity index 53% rename from hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java rename to hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java index 23b6708ff304c..0ac8c8d7b6554 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/AllPendingCompactionPlanSelectStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieCatalogException.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,20 +16,29 @@ * limitations under the License. */ -package org.apache.hudi.sink.compact.strategy; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; +package org.apache.hudi.exception; /** - * Select all pending compaction plan to compact + *

+ * Exception thrown for Hoodie Catalog errors. + *

*/ -public class AllPendingCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - return pendingCompactionTimeline.getInstants().collect(Collectors.toList()); +public class HoodieCatalogException extends RuntimeException { + + public HoodieCatalogException() { + super(); } + + public HoodieCatalogException(String message) { + super(message); + } + + public HoodieCatalogException(String message, Throwable t) { + super(message, t); + } + + public HoodieCatalogException(Throwable t) { + super(t); + } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java index bcea9b957b3ea..cd9bae0541cdc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -68,10 +68,7 @@ public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchem } public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) { - this.fileSchema = fileSchema; - this.querySchema = querySchema; - this.ignoreRequiredAttribute = ignoreRequiredAttribute; - this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema; + this(fileSchema, querySchema, ignoreRequiredAttribute, useColumnTypeFromFileSchema, true); } /** @@ -151,14 +148,15 @@ private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldFie Types.Field fieldFromFileSchema = fileSchema.findField(fieldId); String nameFromFileSchema = fieldFromFileSchema.name(); String nameFromQuerySchema = querySchema.findField(fieldId).name(); + String finalFieldName = useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema; Type typeFromFileSchema = fieldFromFileSchema.type(); // Current design mechanism guarantees nestedType change is not allowed, so no need to consider. if (newType.isNestedType()) { return Types.Field.get(oldField.fieldId(), oldField.isOptional(), - useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, newType, oldField.doc()); + finalFieldName, newType, oldField.doc()); } else { return Types.Field.get(oldField.fieldId(), oldField.isOptional(), - useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc()); + finalFieldName, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java index e57fce4357b25..520a6b9ec75b2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java @@ -33,37 +33,33 @@ * Utility methods to support evolve old avro schema based on a given schema. */ public class AvroSchemaEvolutionUtils { + /** - * Support evolution from a new avroSchema. - * Now hoodie support implicitly add columns when hoodie write operation, - * This ability needs to be preserved, so implicitly evolution for internalSchema should supported. - * - * @param evolvedSchema implicitly evolution of avro when hoodie write operation - * @param oldSchema old internalSchema - * @param supportPositionReorder support position reorder - * @return evolution Schema + * Support reconcile from a new avroSchema. + * 1) incoming data has missing columns that were already defined in the table –> null values will be injected into missing columns + * 2) incoming data contains new columns not defined yet in the table -> columns will be added to the table schema (incoming dataframe?) + * 3) incoming data has missing columns that are already defined in the table and new columns not yet defined in the table -> + * new columns will be added to the table schema, missing columns will be injected with null values + * 4) support nested schema change. + * Notice: + * the incoming schema should not have delete/rename semantics. + * for example: incoming schema: int a, int b, int d; oldTableSchema int a, int b, int c, int d + * we must guarantee the column c is missing semantic, instead of delete semantic. + * @param incomingSchema implicitly evolution of avro when hoodie write operation + * @param oldTableSchema old internalSchema + * @return reconcile Schema */ - public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema, Boolean supportPositionReorder) { - InternalSchema evolvedInternalSchema = AvroInternalSchemaConverter.convert(evolvedSchema); + public static InternalSchema reconcileSchema(Schema incomingSchema, InternalSchema oldTableSchema) { + InternalSchema inComingInternalSchema = AvroInternalSchemaConverter.convert(incomingSchema); // do check, only support add column evolution - List colNamesFromEvolved = evolvedInternalSchema.getAllColsFullName(); - List colNamesFromOldSchema = oldSchema.getAllColsFullName(); - List diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromEvolved.contains(f)).collect(Collectors.toList()); + List colNamesFromIncoming = inComingInternalSchema.getAllColsFullName(); + List colNamesFromOldSchema = oldTableSchema.getAllColsFullName(); + List diffFromOldSchema = colNamesFromOldSchema.stream().filter(f -> !colNamesFromIncoming.contains(f)).collect(Collectors.toList()); List newFields = new ArrayList<>(); - if (colNamesFromEvolved.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) { - // no changes happen - if (supportPositionReorder) { - evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name()))); - return new InternalSchema(newFields); - } - return oldSchema; - } - // try to find all added columns - if (diffFromOldSchema.size() != 0) { - throw new UnsupportedOperationException("Cannot evolve schema implicitly, find delete/rename operation"); + if (colNamesFromIncoming.size() == colNamesFromOldSchema.size() && diffFromOldSchema.size() == 0) { + return oldTableSchema; } - - List diffFromEvolutionSchema = colNamesFromEvolved.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList()); + List diffFromEvolutionSchema = colNamesFromIncoming.stream().filter(f -> !colNamesFromOldSchema.contains(f)).collect(Collectors.toList()); // Remove redundancy from diffFromEvolutionSchema. // for example, now we add a struct col in evolvedSchema, the struct col is " user struct " // when we do diff operation: user, user.name, user.age will appeared in the resultSet which is redundancy, user.name and user.age should be excluded. @@ -77,29 +73,27 @@ public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, // find redundancy, skip it continue; } - finalAddAction.put(evolvedInternalSchema.findIdByName(name), name); + finalAddAction.put(inComingInternalSchema.findIdByName(name), name); } - TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldSchema); + TableChanges.ColumnAddChange addChange = TableChanges.ColumnAddChange.get(oldTableSchema); finalAddAction.entrySet().stream().forEach(f -> { String name = f.getValue(); int splitPoint = name.lastIndexOf("."); String parentName = splitPoint > 0 ? name.substring(0, splitPoint) : ""; String rawName = splitPoint > 0 ? name.substring(splitPoint + 1) : name; - addChange.addColumns(parentName, rawName, evolvedInternalSchema.findType(name), null); + // try to infer add position. + java.util.Optional inferPosition = + colNamesFromIncoming.stream().filter(c -> + c.lastIndexOf(".") == splitPoint + && c.startsWith(parentName) + && inComingInternalSchema.findIdByName(c) > inComingInternalSchema.findIdByName(name) + && oldTableSchema.findIdByName(c) > 0).sorted((s1, s2) -> oldTableSchema.findIdByName(s1) - oldTableSchema.findIdByName(s2)).findFirst(); + addChange.addColumns(parentName, rawName, inComingInternalSchema.findType(name), null); + inferPosition.map(i -> addChange.addPositionChange(name, i, "before")); }); - InternalSchema res = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange); - if (supportPositionReorder) { - evolvedInternalSchema.getRecord().fields().forEach(f -> newFields.add(oldSchema.getRecord().field(f.name()))); - return new InternalSchema(newFields); - } else { - return res; - } - } - - public static InternalSchema evolveSchemaFromNewAvroSchema(Schema evolvedSchema, InternalSchema oldSchema) { - return evolveSchemaFromNewAvroSchema(evolvedSchema, oldSchema, false); + return SchemaChangeUtils.applyTableChanges2Schema(oldTableSchema, addChange); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java index a784b409b8f2f..c799c236d0db0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java @@ -273,7 +273,7 @@ public static String createFullName(String name, Deque fieldNames) { * * @param oldSchema oldSchema * @param newSchema newSchema which modified from oldSchema - * @return renameCols Map. (k, v) -> (colNameFromNewSchema, colNameFromOldSchema) + * @return renameCols Map. (k, v) -> (colNameFromNewSchema, colNameLastPartFromOldSchema) */ public static Map collectRenameCols(InternalSchema oldSchema, InternalSchema newSchema) { List colNamesFromWriteSchema = oldSchema.getAllColsFullName(); @@ -281,6 +281,9 @@ public static Map collectRenameCols(InternalSchema oldSchema, In int filedIdFromWriteSchema = oldSchema.findIdByName(f); // try to find the cols which has the same id, but have different colName; return newSchema.getAllIds().contains(filedIdFromWriteSchema) && !newSchema.findfullName(filedIdFromWriteSchema).equalsIgnoreCase(f); - }).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> e)); + }).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> { + int lastDotIndex = e.lastIndexOf("."); + return e.substring(lastDotIndex == -1 ? 0 : lastDotIndex + 1); + })); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java index bd0254da3dc6e..f2c02d627f131 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java @@ -27,12 +27,14 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -98,6 +100,12 @@ public class TestHoodieAvroUtils { + "{\"name\":\"student\",\"type\":{\"name\":\"student\",\"type\":\"record\",\"fields\":[" + "{\"name\":\"firstname\",\"type\":[\"null\" ,\"string\"],\"default\": null},{\"name\":\"lastname\",\"type\":[\"null\" ,\"string\"],\"default\": null}]}}]}"; + private static String SCHEMA_WITH_NESTED_FIELD_RENAMED = "{\"name\":\"MyClass\",\"type\":\"record\",\"namespace\":\"com.acme.avro\",\"fields\":[" + + "{\"name\":\"fn\",\"type\":\"string\"}," + + "{\"name\":\"ln\",\"type\":\"string\"}," + + "{\"name\":\"ss\",\"type\":{\"name\":\"ss\",\"type\":\"record\",\"fields\":[" + + "{\"name\":\"fn\",\"type\":[\"null\" ,\"string\"],\"default\": null},{\"name\":\"ln\",\"type\":[\"null\" ,\"string\"],\"default\": null}]}}]}"; + @Test public void testPropsPresent() { Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(EXAMPLE_SCHEMA)); @@ -342,4 +350,26 @@ public void testGetNestedFieldSchema() throws IOException { assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(rec3.getSchema(), "student.firstname")); assertEquals(Schema.create(Schema.Type.STRING), getNestedFieldSchemaFromWriteSchema(nestedSchema, "student.firstname")); } + + @Test + public void testReWriteAvroRecordWithNewSchema() { + Schema nestedSchema = new Schema.Parser().parse(SCHEMA_WITH_NESTED_FIELD); + GenericRecord rec3 = new GenericData.Record(nestedSchema); + rec3.put("firstname", "person1"); + rec3.put("lastname", "person2"); + GenericRecord studentRecord = new GenericData.Record(rec3.getSchema().getField("student").schema()); + studentRecord.put("firstname", "person1"); + studentRecord.put("lastname", "person2"); + rec3.put("student", studentRecord); + + Schema nestedSchemaRename = new Schema.Parser().parse(SCHEMA_WITH_NESTED_FIELD_RENAMED); + Map colRenames = new HashMap<>(); + colRenames.put("fn", "firstname"); + colRenames.put("ln", "lastname"); + colRenames.put("ss", "student"); + colRenames.put("ss.fn", "firstname"); + colRenames.put("ss.ln", "lastname"); + GenericRecord studentRecordRename = HoodieAvroUtils.rewriteRecordWithNewSchema(rec3, nestedSchemaRename, colRenames); + Assertions.assertEquals(GenericData.get().validate(nestedSchemaRename, studentRecordRename), true); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java index 465739340dc86..a122f414f93d9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestDFSPropertiesConfiguration.java @@ -185,7 +185,7 @@ public void testLoadGlobalConfFile() { DFSPropertiesConfiguration.refreshGlobalProps(); assertEquals(5, DFSPropertiesConfiguration.getGlobalProps().size()); assertEquals("jdbc:hive2://localhost:10000", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.jdbcurl")); - assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.use_jdbc")); + assertEquals("jdbc", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.mode")); assertEquals("false", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.datasource.hive_sync.support_timestamp")); assertEquals("BLOOM", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.index.type")); assertEquals("true", DFSPropertiesConfiguration.getGlobalProps().get("hoodie.metadata.enable")); diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java index 3850ef07b90a3..6126c479c6154 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java @@ -38,6 +38,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -45,6 +46,17 @@ public class TestAvroSchemaEvolutionUtils { + String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\"," + + "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\"," + + "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\"," + + "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}]," + + "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\"," + + "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," + + "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\"," + + "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," + + "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}}," + + "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}"; + @Test public void testPrimitiveTypes() { Schema[] avroPrimitives = new Schema[] { @@ -146,16 +158,6 @@ public void testArrayType() { @Test public void testComplexConvert() { - String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\"," - + "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\"," - + "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\"," - + "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}]," - + "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\"," - + "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," - + "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\"," - + "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," - + "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}}," - + "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}"; Schema schema = new Schema.Parser().parse(schemaStr); InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), @@ -284,7 +286,7 @@ public void testReWriteRecordWithTypeChanged() { .updateColumnType("col6", Types.StringType.get()); InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName()); - GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap<>()); + GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, Collections.emptyMap()); Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newRecord), true); } @@ -349,9 +351,26 @@ public void testReWriteNestRecord() { ); Schema newAvroSchema = AvroInternalSchemaConverter.convert(newRecord, schema.getName()); - GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap<>()); + GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, Collections.emptyMap()); // test the correctly of rewrite Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newAvroRecord), true); + + // test rewrite with rename + InternalSchema internalSchema = AvroInternalSchemaConverter.convert(schema); + // do change rename operation + TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(internalSchema); + updateChange + .renameColumn("id", "idx") + .renameColumn("data", "datax") + .renameColumn("preferences.feature1", "f1") + .renameColumn("preferences.feature2", "f2") + .renameColumn("locations.value.lat", "lt"); + InternalSchema internalSchemaRename = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); + Schema avroSchemaRename = AvroInternalSchemaConverter.convert(internalSchemaRename, schema.getName()); + Map renameCols = InternalSchemaUtils.collectRenameCols(internalSchema, internalSchemaRename); + GenericRecord avroRecordRename = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, avroSchemaRename, renameCols); + // test the correctly of rewrite + Assertions.assertEquals(GenericData.get().validate(avroSchemaRename, avroRecordRename), true); } @Test @@ -395,7 +414,7 @@ public void testEvolutionSchemaFromNewAvroSchema() { ); evolvedRecord = (Types.RecordType)InternalSchemaBuilder.getBuilder().refreshNewId(evolvedRecord, new AtomicInteger(0)); Schema evolvedAvroSchema = AvroInternalSchemaConverter.convert(evolvedRecord, "test1"); - InternalSchema result = AvroSchemaEvolutionUtils.evolveSchemaFromNewAvroSchema(evolvedAvroSchema, oldSchema); + InternalSchema result = AvroSchemaEvolutionUtils.reconcileSchema(evolvedAvroSchema, oldSchema); Types.RecordType checkedRecord = Types.RecordType.get( Types.Field.get(0, false, "id", Types.IntType.get()), Types.Field.get(1, true, "data", Types.StringType.get()), @@ -419,4 +438,37 @@ public void testEvolutionSchemaFromNewAvroSchema() { ); Assertions.assertEquals(result.getRecord(), checkedRecord); } + + @Test + public void testReconcileSchema() { + // simple schema test + // a: boolean, b: int, c: long, d: date + Schema schema = create("simple", + new Schema.Field("a", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BOOLEAN)), null, JsonProperties.NULL_VALUE), + new Schema.Field("b", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.INT)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("d", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE)); + // a: boolean, c: long, c_1: long, d: date + Schema incomingSchema = create("simpleIncoming", + new Schema.Field("a", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BOOLEAN)), null, JsonProperties.NULL_VALUE), + new Schema.Field("a1", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c1", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("c2", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.LONG)), null, JsonProperties.NULL_VALUE), + new Schema.Field("d", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE), + new Schema.Field("d1", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE), + new Schema.Field("d2", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))), null, JsonProperties.NULL_VALUE)); + + Schema simpleCheckSchema = new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"simpleReconcileSchema\",\"fields\":[{\"name\":\"a\",\"type\":[\"null\",\"boolean\"],\"default\":null}," + + "{\"name\":\"b\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"a1\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"c\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"c1\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"c2\",\"type\":[\"null\",\"long\"],\"default\":null}," + + "{\"name\":\"d\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}," + + "{\"name\":\"d1\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}," + + "{\"name\":\"d2\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}],\"default\":null}]}"); + + Schema simpleReconcileSchema = AvroInternalSchemaConverter.convert(AvroSchemaEvolutionUtils + .reconcileSchema(incomingSchema, AvroInternalSchemaConverter.convert(schema)), "simpleReconcileSchema"); + Assertions.assertEquals(simpleReconcileSchema, simpleCheckSchema); + } } diff --git a/hudi-common/src/test/resources/external-config/hudi-defaults.conf b/hudi-common/src/test/resources/external-config/hudi-defaults.conf index 1133adb4d7735..2e4c3a5d75429 100644 --- a/hudi-common/src/test/resources/external-config/hudi-defaults.conf +++ b/hudi-common/src/test/resources/external-config/hudi-defaults.conf @@ -20,7 +20,7 @@ # Example: hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000 -hoodie.datasource.hive_sync.use_jdbc true +hoodie.datasource.hive_sync.mode jdbc hoodie.datasource.hive_sync.support_timestamp false hoodie.index.type BLOOM hoodie.metadata.enable true diff --git a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java index 4890a6529a52c..6e20ee1190661 100644 --- a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java +++ b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -27,7 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.examples.common.HoodieExampleDataGenerator; @@ -85,7 +85,7 @@ public static void main(String[] args) throws Exception { .withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2).forTable(tableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); HoodieJavaWriteClient client = new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(hadoopConf), cfg); diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java index 1afc180531a16..299fe992fa00a 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.examples.common.HoodieExampleDataGenerator; @@ -99,7 +99,7 @@ public static void main(String[] args) throws Exception { .withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2).forTable(tableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg); // inserts diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml index 5ad323f93442d..4a22206e06c32 100644 --- a/hudi-flink-datasource/hudi-flink/pom.xml +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -269,7 +269,30 @@ - + + ${hive.groupid} + hive-metastore + ${hive.version} + provided + + + javax.transaction + jta + + + javax.transaction + transaction-api + + + javax.mail + mail + + + org.eclipse.jetty.aggregate + * + + + diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 7dc5deb791d4d..b1a3372e08930 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -755,12 +755,6 @@ private FlinkOptions() { .defaultValue(false) .withDescription("Assume partitioning is yyyy/mm/dd, default false"); - public static final ConfigOption HIVE_SYNC_USE_JDBC = ConfigOptions - .key("hive_sync.use_jdbc") - .booleanType() - .defaultValue(true) - .withDescription("Use JDBC when hive synchronization is enabled, default true"); - public static final ConfigOption HIVE_SYNC_AUTO_CREATE_DB = ConfigOptions .key("hive_sync.auto_create_db") .booleanType() diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java index 8666151fe492d..db696a322ed71 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java @@ -23,6 +23,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.Path; +import java.io.File; +import java.util.ArrayList; +import java.util.List; import java.util.Map; /** @@ -54,6 +57,45 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf(Configuration c return hadoopConf; } + /** + * Returns a new hadoop configuration that is initialized with the given hadoopConfDir. + * + * @param hadoopConfDir Hadoop conf directory path. + * @return A Hadoop configuration instance. + */ + public static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) { + if (new File(hadoopConfDir).exists()) { + List possiableConfFiles = new ArrayList(); + File coreSite = new File(hadoopConfDir, "core-site.xml"); + if (coreSite.exists()) { + possiableConfFiles.add(coreSite); + } + File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml"); + if (hdfsSite.exists()) { + possiableConfFiles.add(hdfsSite); + } + File yarnSite = new File(hadoopConfDir, "yarn-site.xml"); + if (yarnSite.exists()) { + possiableConfFiles.add(yarnSite); + } + // Add mapred-site.xml. We need to read configurations like compression codec. + File mapredSite = new File(hadoopConfDir, "mapred-site.xml"); + if (mapredSite.exists()) { + possiableConfFiles.add(mapredSite); + } + if (possiableConfFiles.isEmpty()) { + return null; + } else { + org.apache.hadoop.conf.Configuration hadoopConfiguration = new org.apache.hadoop.conf.Configuration(); + for (File confFile : possiableConfFiles) { + hadoopConfiguration.addResource(new Path(confFile.getAbsolutePath())); + } + return hadoopConfiguration; + } + } + return null; + } + /** * Creates a Hive configuration with configured dir path or empty if no Hive conf dir is set. */ diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java index fe55089988e53..883ba8bd114cd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.util.collection.Pair; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.configuration.Configuration; @@ -28,8 +29,7 @@ import org.slf4j.LoggerFactory; import java.util.List; - -import static java.util.stream.Collectors.toList; +import java.util.stream.Collectors; /** * Flink hudi compaction source function. @@ -53,18 +53,12 @@ public class CompactionPlanSourceFunction extends AbstractRichFunction implement protected static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class); /** - * Compaction instant time. - */ - private final String compactionInstantTime; - - /** - * The compaction plan. + * compaction plan instant -> compaction plan */ - private final HoodieCompactionPlan compactionPlan; + private final List> compactionPlans; - public CompactionPlanSourceFunction(HoodieCompactionPlan compactionPlan, String compactionInstantTime) { - this.compactionPlan = compactionPlan; - this.compactionInstantTime = compactionInstantTime; + public CompactionPlanSourceFunction(List> compactionPlans) { + this.compactionPlans = compactionPlans; } @Override @@ -74,11 +68,14 @@ public void open(Configuration parameters) throws Exception { @Override public void run(SourceContext sourceContext) throws Exception { - List operations = this.compactionPlan.getOperations().stream() - .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); - LOG.info("CompactionPlanFunction compacting " + operations + " files"); - for (CompactionOperation operation : operations) { - sourceContext.collect(new CompactionPlanEvent(compactionInstantTime, operation)); + for (Pair pair : compactionPlans) { + HoodieCompactionPlan compactionPlan = pair.getRight(); + List operations = compactionPlan.getOperations().stream() + .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); + LOG.info("CompactionPlanFunction compacting " + operations + " files"); + for (CompactionOperation operation : operations) { + sourceContext.collect(new CompactionPlanEvent(pair.getLeft(), operation)); + } } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java index 02041690f1dec..449b06846156c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java @@ -20,10 +20,10 @@ import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategy; import com.beust.jcommander.Parameter; import org.apache.flink.configuration.Configuration; -import org.apache.hudi.sink.compact.strategy.SingleCompactionPlanSelectStrategy; /** * Configurations for Hoodie Flink compaction. @@ -102,7 +102,7 @@ public class FlinkCompactionConfig extends Configuration { @Parameter(names = {"--seq"}, description = "Compaction plan execution sequence, two options are supported:\n" + "1). FIFO: execute the oldest plan first;\n" + "2). LIFO: execute the latest plan first, by default LIFO", required = false) - public String compactionSeq = SEQ_LIFO; + public String compactionSeq = SEQ_FIFO; @Parameter(names = {"--service"}, description = "Flink Compaction runs in service mode, disable by default") public Boolean serviceMode = false; @@ -111,21 +111,21 @@ public class FlinkCompactionConfig extends Configuration { description = "Min compaction interval of async compaction service, default 10 minutes") public Integer minCompactionIntervalSeconds = 600; - @Parameter(names = {"--select-strategy"}, description = "The strategy define how to select compaction plan to compact.\n" - + "1). SingleCompactionPlanSelectStrategy: Select first or last compaction plan." - + "2). MultiCompactionPlanSelectStrategy: Select first or last n compaction plan (n is defined by compactionPlanMaxSelect)." - + "3). AllPendingCompactionPlanSelectStrategy: Select all pending compaction plan" - + "4). InstantCompactionPlanSelectStrategy: Select the compaction plan that instant is specified by compactionPlanInstant") - public String compactionPlanSelectStrategy = SingleCompactionPlanSelectStrategy.class.getName(); + @Parameter(names = {"--plan-select-strategy"}, description = "The strategy define how to select compaction plan to compact.\n" + + "1). num_instants: select plans by specific number of instants, it's the default strategy with 1 instant at a time;\n" + + "3). all: Select all pending compaction plan;\n" + + "4). instants: Select the compaction plan by specific instants") + public String compactionPlanSelectStrategy = CompactionPlanStrategy.NUM_INSTANTS; - @Parameter(names = {"--select-max-number"}, description = "Max number of compaction plan would be selected in compaction." + @Parameter(names = {"--max-num-plans"}, description = "Max number of compaction plan would be selected in compaction." + "It's only effective for MultiCompactionPlanSelectStrategy.") - public Integer compactionPlanMaxSelect = 10; + public Integer maxNumCompactionPlans = 1; - @Parameter(names = {"--select-instant"}, description = "Specify the compaction plan instant to compact and you can specify more than" - + "one instant in a time by using comma." - + "It's only effective for InstantCompactionPlanSelectStrategy.") + @Parameter(names = {"--target-instants"}, description = "Specify the compaction plan instants to compact,\n" + + "Multiple instants are supported by comma separated instant time.\n" + + "It's only effective for 'instants' plan selection strategy.") public String compactionPlanInstant; + @Parameter(names = {"--spillable_map_path"}, description = "Default file path prefix for spillable map.", required = false) public String spillableMapPath = HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index f56b5a2f0fb1d..e2d2972a0de43 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -18,11 +18,6 @@ package org.apache.hudi.sink.compact; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; import org.apache.hudi.async.HoodieAsyncTableService; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; @@ -31,11 +26,10 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.sink.compact.strategy.CompactionPlanSelectStrategy; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.util.CompactionUtil; import org.apache.hudi.util.StreamerUtil; @@ -228,9 +222,8 @@ private void compact() throws Exception { } // fetch the instant based on the configured execution sequence - HoodieTimeline timeline = table.getActiveTimeline(); - List requested = ((CompactionPlanSelectStrategy) ReflectionUtils.loadClass(cfg.compactionPlanSelectStrategy)) - .select(timeline.filterPendingCompactionTimeline(), cfg); + HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + List requested = CompactionPlanStrategies.getStrategy(cfg).select(pendingCompactionTimeline); if (requested.isEmpty()) { // do nothing. LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option"); @@ -240,7 +233,7 @@ private void compact() throws Exception { List compactionInstantTimes = requested.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); compactionInstantTimes.forEach(timestamp -> { HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(timestamp); - if (timeline.containsInstant(inflightInstant)) { + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { LOG.info("Rollback inflight compaction instant: [" + timestamp + "]"); table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); @@ -254,13 +247,11 @@ private void compact() throws Exception { try { return Pair.of(timestamp, CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp)); } catch (IOException e) { - throw new HoodieException(e); + throw new HoodieException("Get compaction plan at instant " + timestamp + " error", e); } }) // reject empty compaction plan - .filter(pair -> !(pair.getRight() == null - || pair.getRight().getOperations() == null - || pair.getRight().getOperations().isEmpty())) + .filter(pair -> validCompactionPlan(pair.getRight())) .collect(Collectors.toList()); if (compactionPlans.isEmpty()) { @@ -270,7 +261,6 @@ private void compact() throws Exception { } List instants = compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList()); - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); for (HoodieInstant instant : instants) { if (!pendingCompactionTimeline.containsInstant(instant)) { // this means that the compaction plan was written to auxiliary path(.tmp) @@ -297,34 +287,19 @@ private void compact() throws Exception { } table.getMetaClient().reloadActiveTimeline(); - // use side-output to make operations that is in the same plan to be placed in the same stream - // keyby() cannot sure that different operations are in the different stream - DataStream source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans)) + env.addSource(new CompactionPlanSourceFunction(compactionPlans)) .name("compaction_source") - .uid("uid_compaction_source"); - - SingleOutputStreamOperator operator = source.rebalance() + .uid("uid_compaction_source") + .rebalance() .transform("compact_task", TypeInformation.of(CompactionCommitEvent.class), new ProcessOperator<>(new CompactFunction(conf))) .setParallelism(compactionParallelism) - .process(new ProcessFunction() { - @Override - public void processElement(CompactionCommitEvent event, ProcessFunction.Context context, Collector out) { - context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event); - } - }) - .name("group_by_compaction_plan") - .uid("uid_group_by_compaction_plan") + .addSink(new CompactionCommitSink(conf)) + .name("compaction_commit") + .uid("uid_compaction_commit") .setParallelism(1); - compactionPlans.forEach(pair -> - operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class))) - .addSink(new CompactionCommitSink(conf)) - .name("clean_commits " + pair.getLeft()) - .uid("uid_clean_commits_" + pair.getLeft()) - .setParallelism(1)); - env.execute("flink_hudi_compaction_" + String.join(",", compactionInstantTimes)); } @@ -342,4 +317,8 @@ public void shutDown() { shutdownAsyncService(false); } } + + private static boolean validCompactionPlan(HoodieCompactionPlan plan) { + return plan != null && plan.getOperations() != null && plan.getOperations().size() > 0; + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java deleted file mode 100644 index 8a8c3f6b4eeb3..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/MultiCompactionPlanSourceFunction.java +++ /dev/null @@ -1,90 +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.hudi.sink.compact; - -import static java.util.stream.Collectors.toList; - -import java.util.List; -import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.model.CompactionOperation; -import org.apache.hudi.common.util.collection.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Flink hudi compaction source function. - * - *

This function read the compaction plan as {@link CompactionOperation}s then assign the compaction task - * event {@link CompactionPlanEvent} to downstream operators. - * - *

The compaction instant time is specified explicitly with strategies: - * - *

    - *
  • If the timeline has no inflight instants, - * use {@link org.apache.hudi.common.table.timeline.HoodieActiveTimeline#createNewInstantTime()} - * as the instant time;
  • - *
  • If the timeline has inflight instants, - * use the median instant time between [last complete instant time, earliest inflight instant time] - * as the instant time.
  • - *
- */ -public class MultiCompactionPlanSourceFunction extends AbstractRichFunction implements SourceFunction { - - protected static final Logger LOG = LoggerFactory.getLogger(MultiCompactionPlanSourceFunction.class); - - /** - * compaction plan instant -> compaction plan - */ - private final List> compactionPlans; - - public MultiCompactionPlanSourceFunction(List> compactionPlans) { - this.compactionPlans = compactionPlans; - } - - @Override - public void open(Configuration parameters) throws Exception { - // no operation - } - - @Override - public void run(SourceContext sourceContext) throws Exception { - for (Pair pair : compactionPlans) { - HoodieCompactionPlan compactionPlan = pair.getRight(); - List operations = compactionPlan.getOperations().stream() - .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); - LOG.info("CompactionPlanFunction compacting " + operations + " files"); - for (CompactionOperation operation : operations) { - sourceContext.collect(new CompactionPlanEvent(pair.getLeft(), operation)); - } - } - } - - @Override - public void close() throws Exception { - // no operation - } - - @Override - public void cancel() { - // no operation - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java new file mode 100644 index 0000000000000..662dcabda3220 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategies.java @@ -0,0 +1,74 @@ +/* + * 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.sink.compact.strategy; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.sink.compact.FlinkCompactionConfig; +import org.apache.hudi.util.CompactionUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.stream.Collectors; + +/** + * Factory clazz for CompactionPlanStrategy. + */ +public class CompactionPlanStrategies { + private static final Logger LOG = LoggerFactory.getLogger(CompactionPlanStrategies.class); + + private CompactionPlanStrategies() { + } + + public static CompactionPlanStrategy getStrategy(FlinkCompactionConfig config) { + switch (config.compactionPlanSelectStrategy.toLowerCase(Locale.ROOT)) { + case CompactionPlanStrategy.ALL: + return pendingCompactionTimeline -> pendingCompactionTimeline.getInstants().collect(Collectors.toList()); + case CompactionPlanStrategy.INSTANTS: + return pendingCompactionTimeline -> { + if (StringUtils.isNullOrEmpty(config.compactionPlanInstant)) { + LOG.warn("None instant is selected"); + return Collections.emptyList(); + } + List instants = Arrays.asList(config.compactionPlanInstant.split(",")); + return pendingCompactionTimeline.getInstants() + .filter(instant -> instants.contains(instant.getTimestamp())) + .collect(Collectors.toList()); + }; + case CompactionPlanStrategy.NUM_INSTANTS: + return pendingCompactionTimeline -> { + List pendingCompactionPlanInstants = pendingCompactionTimeline.getInstants().collect(Collectors.toList()); + if (CompactionUtil.isLIFO(config.compactionSeq)) { + Collections.reverse(pendingCompactionPlanInstants); + } + int range = Math.min(config.maxNumCompactionPlans, pendingCompactionPlanInstants.size()); + return pendingCompactionPlanInstants.subList(0, range); + }; + default: + throw new UnsupportedOperationException("Unknown compaction plan strategy: " + + config.compactionPlanSelectStrategy + + ", supported strategies:[num_instants,instants,all]"); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategy.java similarity index 81% rename from hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanSelectStrategy.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategy.java index a41fcef198139..e209ff53391fc 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanSelectStrategy.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/CompactionPlanStrategy.java @@ -18,17 +18,21 @@ package org.apache.hudi.sink.compact.strategy; -import java.util.List; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; + +import java.util.List; /** - * CompactionRangeStrategy + * Compaction plan selection strategy. */ -public interface CompactionPlanSelectStrategy { +public interface CompactionPlanStrategy { + String ALL = "all"; + String INSTANTS = "instants"; + String NUM_INSTANTS = "num_instants"; + /** - * Define how to select compaction plan to compact + * Define how to select compaction plan to compact. */ - List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config); + List select(HoodieTimeline pendingCompactionTimeline); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java deleted file mode 100644 index 45382b70c4def..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/InstantCompactionPlanSelectStrategy.java +++ /dev/null @@ -1,50 +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.hudi.sink.compact.strategy; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; -import org.apache.hudi.sink.compact.HoodieFlinkCompactor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Specify the compaction plan instant to compact - */ -public class InstantCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkCompactor.class); - - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - if (StringUtils.isNullOrEmpty(config.compactionPlanInstant)) { - LOG.warn("None instant is selected"); - return Collections.emptyList(); - } - List instants = Arrays.asList(config.compactionPlanInstant.split(",")); - return pendingCompactionTimeline.getInstants() - .filter(instant -> instants.contains(instant.getTimestamp())) - .collect(Collectors.toList()); - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java deleted file mode 100644 index ee0e93653f87d..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/MultiCompactionPlanSelectStrategy.java +++ /dev/null @@ -1,42 +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.hudi.sink.compact.strategy; - -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; -import org.apache.hudi.util.CompactionUtil; - -/** - * Select multi compaction plan to compact - */ -public class MultiCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - List pendingCompactionPlanInstants = pendingCompactionTimeline.getInstants().collect(Collectors.toList()); - if (CompactionUtil.isLIFO(config.compactionSeq)) { - Collections.reverse(pendingCompactionPlanInstants); - } - int range = Math.min(config.compactionPlanMaxSelect, pendingCompactionPlanInstants.size()); - return pendingCompactionPlanInstants.subList(0, range); - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java deleted file mode 100644 index 7ca939866ceec..0000000000000 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/strategy/SingleCompactionPlanSelectStrategy.java +++ /dev/null @@ -1,43 +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.hudi.sink.compact.strategy; - -import java.util.Collections; -import java.util.List; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.sink.compact.FlinkCompactionConfig; -import org.apache.hudi.util.CompactionUtil; - -/** - * Select one compaction plan to compact - */ -public class SingleCompactionPlanSelectStrategy implements CompactionPlanSelectStrategy { - @Override - public List select(HoodieTimeline pendingCompactionTimeline, FlinkCompactionConfig config) { - Option compactionPlanInstant = CompactionUtil.isLIFO(config.compactionSeq) - ? pendingCompactionTimeline.lastInstant() - : pendingCompactionTimeline.firstInstant(); - if (compactionPlanInstant.isPresent()) { - return Collections.singletonList(compactionPlanInstant.get()); - } - return Collections.emptyList(); - } -} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index e34adac580f70..cceab5a6157c9 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -43,7 +43,6 @@ import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; -import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; @@ -105,7 +104,6 @@ public static Properties buildSyncConfig(Configuration conf) { props.setPropertyIfNonNull(HIVE_TABLE_SERDE_PROPERTIES.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES)); props.setPropertyIfNonNull(META_SYNC_PARTITION_FIELDS.key(), String.join(",", FilePathUtils.extractHivePartitionFields(conf))); props.setPropertyIfNonNull(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME)); - props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC))); props.setPropertyIfNonNull(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(conf.getBoolean(FlinkOptions.METADATA_ENABLED))); props.setPropertyIfNonNull(HIVE_IGNORE_EXCEPTIONS.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS))); props.setPropertyIfNonNull(HIVE_SUPPORT_TIMESTAMP_TYPE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP))); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index e9574dd52bedd..1083754ca204b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -320,9 +320,6 @@ public class FlinkStreamerConfig extends Configuration { @Parameter(names = {"--hive-sync-assume-date-partitioning"}, description = "Assume partitioning is yyyy/mm/dd, default false") public Boolean hiveSyncAssumeDatePartition = false; - @Parameter(names = {"--hive-sync-use-jdbc"}, description = "Use JDBC when hive synchronization is enabled, default true") - public Boolean hiveSyncUseJdbc = true; - @Parameter(names = {"--hive-sync-auto-create-db"}, description = "Auto create hive database if it does not exists, default true") public Boolean hiveSyncAutoCreateDb = true; @@ -419,7 +416,6 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS, config.hiveSyncPartitionFields); conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME, config.hiveSyncPartitionExtractorClass); conf.setBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION, config.hiveSyncAssumeDatePartition); - conf.setBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC, config.hiveSyncUseJdbc); conf.setBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB, config.hiveSyncAutoCreateDb); conf.setBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS, config.hiveSyncIgnoreExceptions); conf.setBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX, config.hiveSyncSkipRoSuffix); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java index 8ab632ba5a5aa..77e9df8fdd56f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java @@ -18,6 +18,8 @@ package org.apache.hudi.table.catalog; +import org.apache.hudi.exception.HoodieCatalogException; + import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.catalog.Catalog; @@ -30,8 +32,8 @@ import java.util.HashSet; import java.util.Set; +import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION; import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH; -import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE; /** * A catalog factory impl that creates {@link HoodieCatalog}. @@ -52,21 +54,35 @@ public Catalog createCatalog(Context context) { FactoryUtil.createCatalogFactoryHelper(this, context); helper.validate(); - return new HoodieCatalog( - context.getName(), - (Configuration) helper.getOptions()); + if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("hms")) { + return new HoodieHiveCatalog( + context.getName(), + helper.getOptions().get(HoodieCatalogFactoryOptions.DEFAULT_DATABASE), + helper.getOptions().get(HoodieCatalogFactoryOptions.HIVE_CONF_DIR), + helper.getOptions().get(HoodieCatalogFactoryOptions.INIT_FS_TABLE)); + } else if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("dfs")) { + return new HoodieCatalog( + context.getName(), + (Configuration) helper.getOptions()); + } else { + throw new HoodieCatalogException("hoodie catalog supports only the hms and dfs modes."); + } } @Override public Set> requiredOptions() { - Set> options = new HashSet<>(); - options.add(CATALOG_PATH); - options.add(DEFAULT_DATABASE); - return options; + return Collections.emptySet(); } @Override public Set> optionalOptions() { - return Collections.emptySet(); + final Set> options = new HashSet<>(); + options.add(HoodieCatalogFactoryOptions.DEFAULT_DATABASE); + options.add(PROPERTY_VERSION); + options.add(HoodieCatalogFactoryOptions.HIVE_CONF_DIR); + options.add(HoodieCatalogFactoryOptions.MODE); + options.add(CATALOG_PATH); + options.add(HoodieCatalogFactoryOptions.INIT_FS_TABLE); + return options; } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java new file mode 100644 index 0000000000000..9ee4f1eb4b2c2 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java @@ -0,0 +1,47 @@ +/* + * 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.catalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.table.catalog.CommonCatalogOptions; + +/** {@link ConfigOption}s for {@link HoodieHiveCatalog}. */ +public class HoodieCatalogFactoryOptions { + public static final String DEFAULT_DB = "default"; + public static final String HIVE_SITE_FILE = "hive-site.xml"; + + public static final ConfigOption DEFAULT_DATABASE = + ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) + .stringType() + .defaultValue(DEFAULT_DB); + + public static final ConfigOption HIVE_CONF_DIR = + ConfigOptions.key("hive-conf-dir").stringType().noDefaultValue(); + + public static final ConfigOption MODE = + ConfigOptions.key("mode").stringType().defaultValue("dfs"); + + public static final ConfigOption INIT_FS_TABLE = + ConfigOptions.key("init.fs.table").booleanType().defaultValue(true); + + private HoodieCatalogFactoryOptions() { + + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java new file mode 100644 index 0000000000000..b90bffe17b5c3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java @@ -0,0 +1,99 @@ +/* + * 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.catalog; + +import org.apache.hudi.configuration.HadoopConfigurations; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; + +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; +import static org.apache.hudi.table.catalog.HoodieCatalogFactoryOptions.HIVE_SITE_FILE; + +/** + * Utilities for Hoodie Catalog. + */ +public class HoodieCatalogUtil { + private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalogUtil.class); + + /** + * Returns a new {@code HiveConf}. + * + * @param hiveConfDir Hive conf directory path. + * @return A HiveConf instance. + */ + public static HiveConf createHiveConf(@Nullable String hiveConfDir) { + // create HiveConf from hadoop configuration with hadoop conf directory configured. + Configuration hadoopConf = HadoopConfigurations.getHadoopConfiguration(hiveConfDir); + if (isNullOrWhitespaceOnly(hiveConfDir) || hadoopConf == null) { + hadoopConf = HadoopConfigurations.getHadoopConf(new org.apache.flink.configuration.Configuration()); + } + + // ignore all the static conf file URLs that HiveConf may have set + HiveConf.setHiveSiteLocation(null); + HiveConf.setLoadMetastoreConfig(false); + HiveConf.setLoadHiveServer2Config(false); + HiveConf hiveConf = new HiveConf(hadoopConf, HiveConf.class); + + LOG.info("Setting hive conf dir as {}", hiveConfDir); + + if (hiveConfDir != null) { + Path hiveSite = new Path(hiveConfDir, HIVE_SITE_FILE); + if (!hiveSite.toUri().isAbsolute()) { + // treat relative URI as local file to be compatible with previous behavior + hiveSite = new Path(new File(hiveSite.toString()).toURI()); + } + try (InputStream inputStream = hiveSite.getFileSystem(hadoopConf).open(hiveSite)) { + hiveConf.addResource(inputStream, hiveSite.toString()); + // trigger a read from the conf so that the input stream is read + isEmbeddedMetastore(hiveConf); + } catch (IOException e) { + throw new CatalogException( + "Failed to load hive-site.xml from specified path:" + hiveSite, e); + } + } else { + // user doesn't provide hive conf dir, we try to find it in classpath + URL hiveSite = + Thread.currentThread().getContextClassLoader().getResource(HIVE_SITE_FILE); + if (hiveSite != null) { + LOG.info("Found {} in classpath: {}", HIVE_SITE_FILE, hiveSite); + hiveConf.addResource(hiveSite); + } + } + return hiveConf; + } + + /** + * Check whether the hive.metastore.uris is empty + */ + public static boolean isEmbeddedMetastore(HiveConf hiveConf) { + return isNullOrWhitespaceOnly(hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS)); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java new file mode 100644 index 0000000000000..33e3d4460a66f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java @@ -0,0 +1,908 @@ +/* + * 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.catalog; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieCatalogException; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.sync.common.util.ConfigUtils; +import org.apache.hudi.table.format.FilePathUtils; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogPropertiesUtil; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.PrincipalType; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase.ALTER_DATABASE_OP; +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; +import static org.apache.hudi.configuration.FlinkOptions.PATH; +import static org.apache.hudi.table.catalog.HoodieCatalogFactoryOptions.DEFAULT_DB; +import static org.apache.hudi.table.catalog.TableOptionProperties.COMMENT; +import static org.apache.hudi.table.catalog.TableOptionProperties.PK_COLUMNS; +import static org.apache.hudi.table.catalog.TableOptionProperties.PK_CONSTRAINT_NAME; +import static org.apache.hudi.table.catalog.TableOptionProperties.SPARK_SOURCE_PROVIDER; + +/** + * A catalog implementation for Hoodie based on MetaStore. + */ +public class HoodieHiveCatalog extends AbstractCatalog { + private static final Logger LOG = LoggerFactory.getLogger(HoodieHiveCatalog.class); + + private final HiveConf hiveConf; + private final boolean iniTable; + private IMetaStoreClient client; + + public HoodieHiveCatalog(String catalogName, String defaultDatabase, String hiveConf, boolean iniTable) { + this(catalogName, defaultDatabase, HoodieCatalogUtil.createHiveConf(hiveConf), iniTable, false); + } + + public HoodieHiveCatalog(String catalogName, String defaultDatabase, HiveConf hiveConf, boolean iniTable, boolean allowEmbedded) { + super(catalogName, defaultDatabase == null ? DEFAULT_DB : defaultDatabase); + this.hiveConf = hiveConf; + this.iniTable = iniTable; + if (!allowEmbedded) { + checkArgument( + !HoodieCatalogUtil.isEmbeddedMetastore(this.hiveConf), + "Embedded metastore is not allowed. Make sure you have set a valid value for " + + HiveConf.ConfVars.METASTOREURIS.toString()); + } + LOG.info("Created Hoodie Catalog '{}' in hms mode", catalogName); + } + + @Override + public void open() throws CatalogException { + if (this.client == null) { + try { + this.client = Hive.get(hiveConf).getMSC(); + } catch (Exception e) { + throw new HoodieCatalogException("Failed to create HiveMetaStoreClient", e); + } + LOG.info("Connected to Hive metastore"); + } + if (!databaseExists(getDefaultDatabase())) { + throw new HoodieCatalogException( + String.format( + "Configured default database %s doesn't exist in catalog %s.", + getDefaultDatabase(), getName())); + } + } + + @Override + public void close() throws CatalogException { + if (client != null) { + client.close(); + client = null; + LOG.info("Close connection to Hive metastore"); + } + } + + public HiveConf getHiveConf() { + return hiveConf; + } + + // ------ databases ------ + + @Override + public List listDatabases() throws CatalogException { + try { + return client.getAllDatabases(); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to list all databases in %s", getName()), e); + } + } + + public Database getHiveDatabase(String databaseName) throws DatabaseNotExistException { + try { + return client.getDatabase(databaseName); + } catch (NoSuchObjectException e) { + throw new DatabaseNotExistException(getName(), databaseName); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to get database %s from %s", databaseName, getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + Database hiveDatabase = getHiveDatabase(databaseName); + + Map properties = new HashMap<>(hiveDatabase.getParameters()); + + properties.put(SqlCreateHiveDatabase.DATABASE_LOCATION_URI, hiveDatabase.getLocationUri()); + + return new CatalogDatabaseImpl(properties, hiveDatabase.getDescription()); + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + return client.getDatabase(databaseName) != null; + } catch (NoSuchObjectException e) { + return false; + } catch (TException e) { + throw new HoodieCatalogException( + String.format( + "Failed to determine whether database %s exists or not", databaseName), + e); + } + } + + @Override + public void createDatabase( + String databaseName, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + checkNotNull(database, "database cannot be null"); + + Map properties = database.getProperties(); + + String dbLocationUri = properties.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI); + + Database hiveDatabase = + new Database(databaseName, database.getComment(), dbLocationUri, properties); + + try { + client.createDatabase(hiveDatabase); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), hiveDatabase.getName()); + } + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to create database %s", hiveDatabase.getName()), e); + } + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + try { + client.dropDatabase(name, true, ignoreIfNotExists, cascade); + } catch (NoSuchObjectException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (InvalidOperationException e) { + throw new DatabaseNotEmptyException(getName(), name); + } catch (TException e) { + throw new HoodieCatalogException(String.format("Failed to drop database %s", name), e); + } + } + + @Override + public void alterDatabase( + String databaseName, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + checkNotNull(newDatabase, "newDatabase cannot be null"); + + // client.alterDatabase doesn't throw any exception if there is no existing database + Database hiveDB; + try { + hiveDB = getHiveDatabase(databaseName); + } catch (DatabaseNotExistException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + return; + } + + try { + client.alterDatabase(databaseName, alterDatabase(hiveDB, newDatabase)); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to alter database %s", databaseName), e); + } + } + + private static Database alterDatabase(Database hiveDB, CatalogDatabase newDatabase) { + Map newParams = newDatabase.getProperties(); + String opStr = newParams.remove(ALTER_DATABASE_OP); + if (opStr == null) { + // by default is to alter db properties + opStr = SqlAlterHiveDatabase.AlterHiveDatabaseOp.CHANGE_PROPS.name(); + } + String newLocation = newParams.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI); + SqlAlterHiveDatabase.AlterHiveDatabaseOp op = + SqlAlterHiveDatabase.AlterHiveDatabaseOp.valueOf(opStr); + switch (op) { + case CHANGE_PROPS: + hiveDB.setParameters(newParams); + break; + case CHANGE_LOCATION: + hiveDB.setLocationUri(newLocation); + break; + case CHANGE_OWNER: + String ownerName = newParams.remove(DATABASE_OWNER_NAME); + String ownerType = newParams.remove(DATABASE_OWNER_TYPE); + hiveDB.setOwnerName(ownerName); + switch (ownerType) { + case SqlAlterHiveDatabaseOwner.ROLE_OWNER: + hiveDB.setOwnerType(PrincipalType.ROLE); + break; + case SqlAlterHiveDatabaseOwner.USER_OWNER: + hiveDB.setOwnerType(PrincipalType.USER); + break; + default: + throw new CatalogException("Unsupported database owner type: " + ownerType); + } + break; + default: + throw new CatalogException("Unsupported alter database op:" + opStr); + } + // is_generic is deprecated, remove it + if (hiveDB.getParameters() != null) { + hiveDB.getParameters().remove(CatalogPropertiesUtil.IS_GENERIC); + } + return hiveDB; + } + + // ------ tables ------ + + private Table checkHoodieTable(Table hiveTable) { + if (!hiveTable.getParameters().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi") + && !hiveTable.getParameters().getOrDefault(SPARK_SOURCE_PROVIDER, "").equalsIgnoreCase("hudi")) { + throw new HoodieCatalogException(String.format("the %s is not hoodie table", hiveTable.getTableName())); + } + return hiveTable; + } + + private boolean isFlinkTable(Table hiveTable) { + return hiveTable.getParameters().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi"); + } + + private org.apache.avro.Schema getLatestTableSchema(String path) { + if (StringUtils.isNullOrEmpty(path) || !StreamerUtil.tableExists(path, hiveConf)) { + return null; + } + + try { + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(path, hiveConf); + return new TableSchemaResolver(metaClient).getTableAvroSchema(false); + } catch (Exception e) { + LOG.warn("Error while resolving the latest table schema", e); + } + return null; + } + + @VisibleForTesting + public Table getHiveTable(ObjectPath tablePath) throws TableNotExistException { + try { + Table hiveTable = client.getTable(tablePath.getDatabaseName(), tablePath.getObjectName()); + return checkHoodieTable(hiveTable); + } catch (NoSuchObjectException e) { + throw new TableNotExistException(getName(), tablePath); + } catch (TException e) { + throw new HoodieCatalogException(String.format("Failed to get table %s from Hive metastore", tablePath.getObjectName())); + } + } + + private Table translateSparkTable2Flink(ObjectPath tablePath, Table hiveTable) { + if (!isFlinkTable(hiveTable)) { + try { + Map parameters = hiveTable.getParameters(); + parameters.putAll(TableOptionProperties.translateSparkTableProperties2Flink(hiveTable)); + String path = hiveTable.getSd().getLocation(); + parameters.put(PATH.key(), path); + if (!parameters.containsKey(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) { + Path hoodieTablePath = new Path(path); + boolean hiveStyle = Arrays.stream(FSUtils.getFs(hoodieTablePath, hiveConf).listStatus(hoodieTablePath)) + .map(fileStatus -> fileStatus.getPath().getName()) + .filter(f -> !f.equals(".hoodie") && !f.equals("default")) + .anyMatch(FilePathUtils::isHiveStylePartitioning); + parameters.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), String.valueOf(hiveStyle)); + } + client.alter_table(tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); + } catch (Exception e) { + throw new HoodieCatalogException("Failed to update table schema", e); + } + } + return hiveTable; + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + Table hiveTable = getHiveTable(tablePath); + hiveTable = translateSparkTable2Flink(tablePath, hiveTable); + String path = hiveTable.getSd().getLocation(); + Map parameters = hiveTable.getParameters(); + Schema latestTableSchema = getLatestTableSchema(path); + org.apache.flink.table.api.Schema schema; + if (latestTableSchema != null) { + org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder() + .fromRowDataType(AvroSchemaConverter.convertToDataType(latestTableSchema)); + String pkConstraintName = parameters.get(PK_CONSTRAINT_NAME); + if (!StringUtils.isNullOrEmpty(pkConstraintName)) { + builder.primaryKeyNamed(pkConstraintName, StringUtils.split(parameters.get(PK_COLUMNS), ",")); + } + schema = builder.build(); + } else { + LOG.warn("{} does not have any hoodie schema, and use hive table schema to infer the table schema", tablePath); + schema = TableOptionProperties.convertTableSchema(hiveTable); + } + return CatalogTable.of(schema, parameters.get(COMMENT), + TableOptionProperties.getFieldNames(hiveTable.getPartitionKeys()), parameters); + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkNotNull(table, "table cannot be null"); + + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + + if (!table.getOptions().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi")) { + throw new HoodieCatalogException(String.format("The %s is not hoodie table", tablePath.getObjectName())); + } + + if (table instanceof CatalogView) { + throw new HoodieCatalogException("Hoodie catalog does not support to CREATE VIEW."); + } + + try { + boolean isMorTable = table.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), + FlinkOptions.TABLE_TYPE.defaultValue()).equalsIgnoreCase(FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + Table hiveTable = instantiateHiveTable(tablePath, table, inferTablePath(tablePath, table), isMorTable); + //create hive table + client.createTable(hiveTable); + //init hoodie metaClient + initTableIfNotExists(tablePath, (CatalogTable)table); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } catch (Exception e) { + throw new HoodieCatalogException( + String.format("Failed to create table %s", tablePath.getFullName()), e); + } + } + + private void initTableIfNotExists(ObjectPath tablePath, CatalogTable catalogTable) { + if (!iniTable) { + LOG.info("Skip init table."); + return; + } + Configuration flinkConf = Configuration.fromMap(applyOptionsHook(catalogTable.getOptions())); + flinkConf.addAllToProperties(hiveConf.getAllProperties()); + final String avroSchema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType()).toString(); + flinkConf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema); + + // stores two copies of options: + // - partition keys + // - primary keys + // because the HoodieTableMetaClient is a heavy impl, we try to avoid initializing it + // when calling #getTable. + + if (catalogTable.getUnresolvedSchema().getPrimaryKey().isPresent()) { + final String pkColumns = String.join(",", catalogTable.getUnresolvedSchema().getPrimaryKey().get().getColumnNames()); + String recordKey = flinkConf.get(FlinkOptions.RECORD_KEY_FIELD); + if (!Objects.equals(pkColumns, recordKey)) { + throw new HoodieCatalogException(String.format("%s and %s are the different", pkColumns, recordKey)); + } + } + + if (catalogTable.isPartitioned()) { + final String partitions = String.join(",", catalogTable.getPartitionKeys()); + flinkConf.setString(FlinkOptions.PARTITION_PATH_FIELD, partitions); + } + + if (!flinkConf.getOptional(PATH).isPresent()) { + flinkConf.setString(PATH, inferTablePath(tablePath, catalogTable)); + } + + flinkConf.setString(FlinkOptions.TABLE_NAME, tablePath.getObjectName()); + try { + StreamerUtil.initTableIfNotExists(flinkConf); + } catch (IOException e) { + throw new HoodieCatalogException("Initialize table exception.", e); + } + } + + private String inferTablePath(ObjectPath tablePath, CatalogBaseTable table) { + String location = table.getOptions().getOrDefault(PATH.key(), ""); + if (StringUtils.isNullOrEmpty(location)) { + try { + Path dbLocation = new Path(client.getDatabase(tablePath.getDatabaseName()).getLocationUri()); + location = new Path(dbLocation, tablePath.getObjectName()).toString(); + } catch (TException e) { + throw new HoodieCatalogException(String.format("Failed to infer hoodie table path for table %s", tablePath), e); + } + } + return location; + } + + private Map applyOptionsHook(Map options) { + Map properties = new HashMap<>(options); + if (!options.containsKey(FlinkOptions.RECORD_KEY_FIELD.key())) { + properties.put(FlinkOptions.RECORD_KEY_FIELD.key(), FlinkOptions.RECORD_KEY_FIELD.defaultValue()); + } + if (!options.containsKey(FlinkOptions.PRECOMBINE_FIELD.key())) { + properties.put(FlinkOptions.PRECOMBINE_FIELD.key(), FlinkOptions.PRECOMBINE_FIELD.defaultValue()); + } + if (!options.containsKey(FlinkOptions.TABLE_TYPE.key())) { + properties.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue()); + } + return properties; + } + + private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table, String location, boolean useRealTimeInputFormat) throws IOException { + // let Hive set default parameters for us, e.g. serialization.format + Table hiveTable = + org.apache.hadoop.hive.ql.metadata.Table.getEmptyTable( + tablePath.getDatabaseName(), tablePath.getObjectName()); + hiveTable.setOwner(UserGroupInformation.getCurrentUser().getUserName()); + hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000)); + + Map properties = applyOptionsHook(table.getOptions()); + properties.put("EXTERNAL", "TRUE"); + // Table comment + if (table.getComment() != null) { + properties.put(COMMENT, table.getComment()); + } + + //set sd + StorageDescriptor sd = new StorageDescriptor(); + List allColumns = TableOptionProperties.createHiveColumns(table.getSchema()); + + // Table columns and partition keys + if (table instanceof CatalogTable) { + CatalogTable catalogTable = (CatalogTable) table; + + if (catalogTable.isPartitioned()) { + int partitionKeySize = catalogTable.getPartitionKeys().size(); + List regularColumns = + allColumns.subList(0, allColumns.size() - partitionKeySize); + List partitionColumns = + allColumns.subList( + allColumns.size() - partitionKeySize, allColumns.size()); + + sd.setCols(regularColumns); + hiveTable.setPartitionKeys(partitionColumns); + } else { + sd.setCols(allColumns); + hiveTable.setPartitionKeys(new ArrayList<>()); + } + } else { + sd.setCols(allColumns); + } + + HoodieFileFormat baseFileFormat = HoodieFileFormat.PARQUET; + //ignore uber input Format + String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, useRealTimeInputFormat); + String outputFormatClassName = HoodieInputFormatUtils.getOutputFormatClassName(baseFileFormat); + String serDeClassName = HoodieInputFormatUtils.getSerDeClassName(baseFileFormat); + sd.setInputFormat(inputFormatClassName); + sd.setOutputFormat(outputFormatClassName); + Map serdeProperties = new HashMap<>(); + serdeProperties.put("path", location); + serdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(!useRealTimeInputFormat)); + serdeProperties.put("serialization.format", "1"); + sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties)); + + sd.setLocation(location); + + hiveTable.setSd(sd); + + properties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark((CatalogTable)table, hiveConf)); + + //set pk + if (table.getUnresolvedSchema().getPrimaryKey().isPresent()) { + String pkColumns = String.join(",", table.getUnresolvedSchema().getPrimaryKey().get().getColumnNames()); + String recordKey = properties.getOrDefault(FlinkOptions.RECORD_KEY_FIELD.key(), FlinkOptions.RECORD_KEY_FIELD.defaultValue()); + if (!Objects.equals(pkColumns, recordKey)) { + throw new HoodieCatalogException( + String.format("If the table has primaryKey, the primaryKey should be the the same as the recordKey, but pk %s and recordKey %s are the different", + pkColumns, + recordKey)); + } + properties.put(PK_CONSTRAINT_NAME, table.getUnresolvedSchema().getPrimaryKey().get().getConstraintName()); + properties.put(PK_COLUMNS, pkColumns); + } + + if (!properties.containsKey(FlinkOptions.PATH.key())) { + properties.put(FlinkOptions.PATH.key(), location); + } + + hiveTable.setParameters(properties); + return hiveTable; + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + + try { + return client.getAllTables(databaseName); + } catch (UnknownDBException e) { + throw new DatabaseNotExistException(getName(), databaseName); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to list tables in database %s", databaseName), e); + } + } + + @Override + public List listViews(String databaseName) + throws DatabaseNotExistException, CatalogException { + throw new HoodieCatalogException("Hoodie catalog does not support to listViews"); + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + + try { + return client.tableExists(tablePath.getDatabaseName(), tablePath.getObjectName()); + } catch (UnknownDBException e) { + return false; + } catch (TException e) { + throw new CatalogException( + String.format( + "Failed to check whether table %s exists or not.", + tablePath.getFullName()), + e); + } + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + + try { + client.dropTable( + tablePath.getDatabaseName(), + tablePath.getObjectName(), + // Indicate whether associated data should be deleted. + // Set to 'true' for now because Flink tables shouldn't have data in Hive. Can + // be changed later if necessary + true, + ignoreIfNotExists); + } catch (NoSuchObjectException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to drop table %s", tablePath.getFullName()), e); + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkArgument( + !isNullOrWhitespaceOnly(newTableName), "newTableName cannot be null or empty"); + + try { + // alter_table() doesn't throw a clear exception when target table doesn't exist. + // Thus, check the table existence explicitly + if (tableExists(tablePath)) { + ObjectPath newPath = new ObjectPath(tablePath.getDatabaseName(), newTableName); + // alter_table() doesn't throw a clear exception when new table already exists. + // Thus, check the table existence explicitly + if (tableExists(newPath)) { + throw new TableAlreadyExistException(getName(), newPath); + } else { + Table hiveTable = getHiveTable(tablePath); + + //update hoodie + StorageDescriptor sd = hiveTable.getSd(); + String location = sd.getLocation(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(location).setConf(hiveConf).build(); + //Init table with new name + HoodieTableMetaClient.withPropertyBuilder().fromProperties(metaClient.getTableConfig().getProps()) + .setTableName(newTableName) + .initTable(hiveConf, location); + + hiveTable.setTableName(newTableName); + client.alter_table( + tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); + } + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } catch (Exception e) { + throw new HoodieCatalogException( + String.format("Failed to rename table %s", tablePath.getFullName()), e); + } + } + + @Override + public void alterTable( + ObjectPath tablePath, CatalogBaseTable newCatalogTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkNotNull(newCatalogTable, "newCatalogTable cannot be null"); + + if (!newCatalogTable.getOptions().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi")) { + throw new HoodieCatalogException(String.format("The %s is not hoodie table", tablePath.getObjectName())); + } + if (newCatalogTable instanceof CatalogView) { + throw new HoodieCatalogException("Hoodie catalog does not support to ALTER VIEW"); + } + + try { + Table hiveTable = getHiveTable(tablePath); + if (!hiveTable.getParameters().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue()) + .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue())) + || !hiveTable.getParameters().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue()) + .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue()))) { + throw new HoodieCatalogException("Hoodie catalog does not support to alter table type and index type"); + } + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } + return; + } + + try { + boolean isMorTable = newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), + FlinkOptions.TABLE_TYPE.defaultValue()).equalsIgnoreCase(FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + Table hiveTable = instantiateHiveTable(tablePath, newCatalogTable, inferTablePath(tablePath, newCatalogTable), isMorTable); + //alter hive table + client.alter_table(tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); + } catch (Exception e) { + LOG.error("Failed to alter table {}", tablePath.getObjectName(), e); + throw new HoodieCatalogException(String.format("Failed to alter table %s", tablePath.getObjectName()), e); + } + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List expressions) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, PartitionAlreadyExistsException, + CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public List listFunctions(String databaseName) + throws DatabaseNotExistException, CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } + + @Override + public void createFunction( + ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterFunction( + ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException, TablePartitionedException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new HoodieCatalogException("Not supported."); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java index ba6ca4efd726c..4f15c4ef0adc6 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java @@ -19,17 +19,42 @@ package org.apache.hudi.table.catalog; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sync.common.util.Parquet2SparkSchemaUtils; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.avro.Schema; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.parquet.schema.MessageType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -37,8 +62,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.TreeMap; import java.util.stream.Collectors; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME; /** @@ -53,9 +81,26 @@ public class TableOptionProperties { public static final String PK_COLUMNS = "pk.columns"; public static final String COMMENT = "comment"; public static final String PARTITION_COLUMNS = "partition.columns"; + public static final String SPARK_SOURCE_PROVIDER = "spark.sql.sources.provider"; + public static final String SPARK_VERSION = "spark.verison"; + public static final String DEFAULT_SPARK_VERSION = "spark2.4.4"; public static final List NON_OPTION_KEYS = Arrays.asList(PK_CONSTRAINT_NAME, PK_COLUMNS, COMMENT, PARTITION_COLUMNS); + private static final Map VALUE_MAPPING = new TreeMap<>(); + private static final Map KEY_MAPPING = new TreeMap<>(); + + static { + VALUE_MAPPING.put("mor", HoodieTableType.MERGE_ON_READ.name()); + VALUE_MAPPING.put("cow", HoodieTableType.COPY_ON_WRITE.name()); + + KEY_MAPPING.put("type", FlinkOptions.TABLE_TYPE.key()); + KEY_MAPPING.put("primaryKey", FlinkOptions.RECORD_KEY_FIELD.key()); + KEY_MAPPING.put("preCombineField", FlinkOptions.PRECOMBINE_FIELD.key()); + KEY_MAPPING.put("payloadClass", FlinkOptions.PAYLOAD_CLASS_NAME.key()); + KEY_MAPPING.put(SPARK_SOURCE_PROVIDER, CONNECTOR.key()); + } + /** * Initialize the {@link #FILE_NAME} meta file. */ @@ -128,4 +173,179 @@ public static Map getTableOptions(Map options) { NON_OPTION_KEYS.forEach(copied::remove); return copied; } + + public static Map translateFlinkTableProperties2Spark(CatalogTable catalogTable, Configuration hadoopConf) { + Schema schema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType()); + MessageType messageType = TableSchemaResolver.convertAvroSchemaToParquet(schema, hadoopConf); + String sparkVersion = catalogTable.getOptions().getOrDefault(SPARK_VERSION, DEFAULT_SPARK_VERSION); + return Parquet2SparkSchemaUtils.getSparkTableProperties(catalogTable.getPartitionKeys(), sparkVersion, 4000, messageType); + } + + public static Map translateSparkTableProperties2Flink(Map options) { + if (options.containsKey(CONNECTOR.key())) { + return options; + } + return options.entrySet().stream().filter(e -> KEY_MAPPING.containsKey(e.getKey())) + .collect(Collectors.toMap(e -> KEY_MAPPING.get(e.getKey()), + e -> e.getKey().equalsIgnoreCase("type") ? VALUE_MAPPING.get(e.getValue()) : e.getValue())); + } + + public static Map translateSparkTableProperties2Flink(Table hiveTable) { + return translateSparkTableProperties2Flink(hiveTable.getParameters()); + } + + /** Get field names from field schemas. */ + public static List getFieldNames(List fieldSchemas) { + List names = new ArrayList<>(fieldSchemas.size()); + for (FieldSchema fs : fieldSchemas) { + names.add(fs.getName()); + } + return names; + } + + public static org.apache.flink.table.api.Schema convertTableSchema(Table hiveTable) { + List allCols = new ArrayList<>(hiveTable.getSd().getCols()); + allCols.addAll(hiveTable.getPartitionKeys()); + + String pkConstraintName = hiveTable.getParameters().get(PK_CONSTRAINT_NAME); + List primaryColNames = StringUtils.isNullOrEmpty(pkConstraintName) + ? Collections.EMPTY_LIST + : StringUtils.split(hiveTable.getParameters().get(PK_COLUMNS),","); + + String[] colNames = new String[allCols.size()]; + DataType[] colTypes = new DataType[allCols.size()]; + + for (int i = 0; i < allCols.size(); i++) { + FieldSchema fs = allCols.get(i); + + colNames[i] = fs.getName(); + colTypes[i] = + toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType())); + if (primaryColNames.contains(colNames[i])) { + colTypes[i] = colTypes[i].notNull(); + } + } + + org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder().fromFields(colNames, colTypes); + if (!StringUtils.isNullOrEmpty(pkConstraintName)) { + builder.primaryKeyNamed(pkConstraintName, primaryColNames); + } + + return builder.build(); + } + + /** + * Convert Hive data type to a Flink data type. + * + * @param hiveType a Hive data type + * @return the corresponding Flink data type + */ + public static DataType toFlinkType(TypeInfo hiveType) { + checkNotNull(hiveType, "hiveType cannot be null"); + + switch (hiveType.getCategory()) { + case PRIMITIVE: + return toFlinkPrimitiveType((PrimitiveTypeInfo) hiveType); + case LIST: + ListTypeInfo listTypeInfo = (ListTypeInfo) hiveType; + return DataTypes.ARRAY(toFlinkType(listTypeInfo.getListElementTypeInfo())); + case MAP: + MapTypeInfo mapTypeInfo = (MapTypeInfo) hiveType; + return DataTypes.MAP( + toFlinkType(mapTypeInfo.getMapKeyTypeInfo()), + toFlinkType(mapTypeInfo.getMapValueTypeInfo())); + case STRUCT: + StructTypeInfo structTypeInfo = (StructTypeInfo) hiveType; + + List names = structTypeInfo.getAllStructFieldNames(); + List typeInfos = structTypeInfo.getAllStructFieldTypeInfos(); + + DataTypes.Field[] fields = new DataTypes.Field[names.size()]; + + for (int i = 0; i < fields.length; i++) { + fields[i] = DataTypes.FIELD(names.get(i), toFlinkType(typeInfos.get(i))); + } + + return DataTypes.ROW(fields); + default: + throw new UnsupportedOperationException( + String.format("Flink doesn't support Hive data type %s yet.", hiveType)); + } + } + + private static DataType toFlinkPrimitiveType(PrimitiveTypeInfo hiveType) { + checkNotNull(hiveType, "hiveType cannot be null"); + + switch (hiveType.getPrimitiveCategory()) { + case CHAR: + return DataTypes.CHAR(((CharTypeInfo) hiveType).getLength()); + case VARCHAR: + return DataTypes.VARCHAR(((VarcharTypeInfo) hiveType).getLength()); + case STRING: + return DataTypes.STRING(); + case BOOLEAN: + return DataTypes.BOOLEAN(); + case BYTE: + return DataTypes.TINYINT(); + case SHORT: + return DataTypes.SMALLINT(); + case INT: + return DataTypes.INT(); + case LONG: + return DataTypes.BIGINT(); + case FLOAT: + return DataTypes.FLOAT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case DATE: + return DataTypes.DATE(); + case TIMESTAMP: + return DataTypes.TIMESTAMP(9); + case BINARY: + return DataTypes.BYTES(); + case DECIMAL: + DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) hiveType; + return DataTypes.DECIMAL( + decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale()); + default: + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support Hive primitive type %s yet", hiveType)); + } + } + + /** Create Hive columns from Flink TableSchema. */ + public static List createHiveColumns(TableSchema schema) { + String[] fieldNames = schema.getFieldNames(); + DataType[] fieldTypes = schema.getFieldDataTypes(); + + List columns = new ArrayList<>(fieldNames.length); + + for (int i = 0; i < fieldNames.length; i++) { + columns.add( + new FieldSchema( + fieldNames[i], + toHiveTypeInfo(fieldTypes[i], true).getTypeName(), + null)); + } + + return columns; + } + + /** + * Convert Flink DataType to Hive TypeInfo. For types with a precision parameter, e.g. + * timestamp, the supported precisions in Hive and Flink can be different. Therefore the + * conversion will fail for those types if the precision is not supported by Hive and + * checkPrecision is true. + * + * @param dataType a Flink DataType + * @param checkPrecision whether to fail the conversion if the precision of the DataType is not + * supported by Hive + * @return the corresponding Hive data type + */ + public static TypeInfo toHiveTypeInfo(DataType dataType, boolean checkPrecision) { + checkNotNull(dataType, "type cannot be null"); + LogicalType logicalType = dataType.getLogicalType(); + return logicalType.accept(new TypeInfoLogicalTypeVisitor(dataType, checkPrecision)); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java new file mode 100644 index 0000000000000..d6cfe3ed723a7 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TypeInfoLogicalTypeVisitor.java @@ -0,0 +1,236 @@ +/* + * 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.catalog; + +import org.apache.hudi.exception.HoodieCatalogException; + +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Create a TypeInfoLogicalTypeVisitor for hoodie table. + */ +public class TypeInfoLogicalTypeVisitor extends LogicalTypeDefaultVisitor { + private final LogicalType type; + // whether to check type precision + private final boolean checkPrecision; + + TypeInfoLogicalTypeVisitor(DataType dataType, boolean checkPrecision) { + this(dataType.getLogicalType(), checkPrecision); + } + + TypeInfoLogicalTypeVisitor(LogicalType type, boolean checkPrecision) { + this.type = type; + this.checkPrecision = checkPrecision; + } + + @Override + public TypeInfo visit(CharType charType) { + // Flink and Hive have different length limit for CHAR. Promote it to STRING if it + // exceeds the limits of + // Hive and we're told not to check precision. This can be useful when calling Hive UDF + // to process data. + if (charType.getLength() > HiveChar.MAX_CHAR_LENGTH || charType.getLength() < 1) { + if (checkPrecision) { + throw new HoodieCatalogException( + String.format( + "HiveCatalog doesn't support char type with length of '%d'. " + + "The supported length is [%d, %d]", + charType.getLength(), 1, HiveChar.MAX_CHAR_LENGTH)); + } else { + return TypeInfoFactory.stringTypeInfo; + } + } + return TypeInfoFactory.getCharTypeInfo(charType.getLength()); + } + + @Override + public TypeInfo visit(VarCharType varCharType) { + // Flink's StringType is defined as VARCHAR(Integer.MAX_VALUE) + // We don't have more information in LogicalTypeRoot to distinguish StringType and a + // VARCHAR(Integer.MAX_VALUE) instance + // Thus always treat VARCHAR(Integer.MAX_VALUE) as StringType + if (varCharType.getLength() == Integer.MAX_VALUE) { + return TypeInfoFactory.stringTypeInfo; + } + // Flink and Hive have different length limit for VARCHAR. Promote it to STRING if it + // exceeds the limits of + // Hive and we're told not to check precision. This can be useful when calling Hive UDF + // to process data. + if (varCharType.getLength() > HiveVarchar.MAX_VARCHAR_LENGTH + || varCharType.getLength() < 1) { + if (checkPrecision) { + throw new HoodieCatalogException( + String.format( + "HiveCatalog doesn't support varchar type with length of '%d'. " + + "The supported length is [%d, %d]", + varCharType.getLength(), 1, HiveVarchar.MAX_VARCHAR_LENGTH)); + } else { + return TypeInfoFactory.stringTypeInfo; + } + } + return TypeInfoFactory.getVarcharTypeInfo(varCharType.getLength()); + } + + @Override + public TypeInfo visit(BooleanType booleanType) { + return TypeInfoFactory.booleanTypeInfo; + } + + @Override + public TypeInfo visit(VarBinaryType varBinaryType) { + // Flink's BytesType is defined as VARBINARY(Integer.MAX_VALUE) + // We don't have more information in LogicalTypeRoot to distinguish BytesType and a + // VARBINARY(Integer.MAX_VALUE) instance + // Thus always treat VARBINARY(Integer.MAX_VALUE) as BytesType + if (varBinaryType.getLength() == VarBinaryType.MAX_LENGTH) { + return TypeInfoFactory.binaryTypeInfo; + } + return defaultMethod(varBinaryType); + } + + @Override + public TypeInfo visit(DecimalType decimalType) { + // Flink and Hive share the same precision and scale range + // Flink already validates the type so we don't need to validate again here + return TypeInfoFactory.getDecimalTypeInfo( + decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public TypeInfo visit(TinyIntType tinyIntType) { + return TypeInfoFactory.byteTypeInfo; + } + + @Override + public TypeInfo visit(SmallIntType smallIntType) { + return TypeInfoFactory.shortTypeInfo; + } + + @Override + public TypeInfo visit(IntType intType) { + return TypeInfoFactory.intTypeInfo; + } + + @Override + public TypeInfo visit(BigIntType bigIntType) { + return TypeInfoFactory.longTypeInfo; + } + + @Override + public TypeInfo visit(FloatType floatType) { + return TypeInfoFactory.floatTypeInfo; + } + + @Override + public TypeInfo visit(DoubleType doubleType) { + return TypeInfoFactory.doubleTypeInfo; + } + + @Override + public TypeInfo visit(DateType dateType) { + return TypeInfoFactory.dateTypeInfo; + } + + @Override + public TypeInfo visit(TimestampType timestampType) { + if (checkPrecision && timestampType.getPrecision() == 9) { + throw new HoodieCatalogException( + "HoodieCatalog currently does not support timestamp of precision 9"); + } + return TypeInfoFactory.timestampTypeInfo; + } + + @Override + public TypeInfo visit(ArrayType arrayType) { + LogicalType elementType = arrayType.getElementType(); + TypeInfo elementTypeInfo = elementType.accept(this); + if (null != elementTypeInfo) { + return TypeInfoFactory.getListTypeInfo(elementTypeInfo); + } else { + return defaultMethod(arrayType); + } + } + + @Override + public TypeInfo visit(MapType mapType) { + LogicalType keyType = mapType.getKeyType(); + LogicalType valueType = mapType.getValueType(); + TypeInfo keyTypeInfo = keyType.accept(this); + TypeInfo valueTypeInfo = valueType.accept(this); + if (null == keyTypeInfo || null == valueTypeInfo) { + return defaultMethod(mapType); + } else { + return TypeInfoFactory.getMapTypeInfo(keyTypeInfo, valueTypeInfo); + } + } + + @Override + public TypeInfo visit(RowType rowType) { + List names = rowType.getFieldNames(); + List typeInfos = new ArrayList<>(names.size()); + for (String name : names) { + TypeInfo typeInfo = rowType.getTypeAt(rowType.getFieldIndex(name)).accept(this); + if (null != typeInfo) { + typeInfos.add(typeInfo); + } else { + return defaultMethod(rowType); + } + } + return TypeInfoFactory.getStructTypeInfo(names, typeInfos); + } + + @Override + public TypeInfo visit(NullType nullType) { + return TypeInfoFactory.voidTypeInfo; + } + + @Override + protected TypeInfo defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException( + String.format( + "Flink doesn't support converting type %s to Hive type yet.", + type.toString())); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java index d03c5aac272e8..0da1aca0e243c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java @@ -442,4 +442,8 @@ public static String[] extractHivePartitionFields(org.apache.flink.configuration } return conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS).split(","); } + + public static boolean isHiveStylePartitioning(String path) { + return HIVE_PARTITION_NAME_PATTERN.matcher(path).matches(); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index d292b3832ac3b..0db960c32f56d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -38,12 +38,14 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieMemoryConfig; -import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieStorageConfig; -import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; @@ -178,23 +180,25 @@ public static HoodieWriteConfig getHoodieClientConfig( .withClusteringSkipPartitionsFromLatest(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST)) .withAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS)) .build()) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) - .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO)) - .withInlineCompactionTriggerStrategy( - CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT))) - .withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_COMMITS)) - .withMaxDeltaSecondsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS)) - .withAsyncClean(conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) - .retainCommits(conf.getInteger(FlinkOptions.CLEAN_RETAIN_COMMITS)) - .retainFileVersions(conf.getInteger(FlinkOptions.CLEAN_RETAIN_FILE_VERSIONS)) - // override and hardcode to 20, - // actually Flink cleaning is always with parallelism 1 now - .withCleanerParallelism(20) - .archiveCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS)) - .withCleanerPolicy(HoodieCleaningPolicy.valueOf(conf.getString(FlinkOptions.CLEAN_POLICY))) - .build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withAsyncClean(conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) + .retainCommits(conf.getInteger(FlinkOptions.CLEAN_RETAIN_COMMITS)) + .retainFileVersions(conf.getInteger(FlinkOptions.CLEAN_RETAIN_FILE_VERSIONS)) + // override and hardcode to 20, + // actually Flink cleaning is always with parallelism 1 now + .withCleanerParallelism(20) + .withCleanerPolicy(HoodieCleaningPolicy.valueOf(conf.getString(FlinkOptions.CLEAN_POLICY))) + .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS)) + .build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO)) + .withInlineCompactionTriggerStrategy( + CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT))) + .withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_COMMITS)) + .withMaxDeltaSecondsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS)) + .build()) .withMemoryConfig( HoodieMemoryConfig.newBuilder() .withMaxMemoryMaxSize( @@ -214,8 +218,10 @@ public static HoodieWriteConfig getHoodieClientConfig( .withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.METADATA_COMPACTION_DELTA_COMMITS)) .build()) .withPayloadConfig(HoodiePayloadConfig.newBuilder() + .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) .withPayloadOrderingField(conf.getString(FlinkOptions.PRECOMBINE_FIELD)) .withPayloadEventTimeField(conf.getString(FlinkOptions.PRECOMBINE_FIELD)) + .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) .build()) .withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineService) .withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton diff --git a/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 47435c745c461..13668a5590063 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -15,4 +15,4 @@ # limitations under the License. org.apache.hudi.table.HoodieTableFactory -org.apache.hudi.table.catalog.HoodieCatalogFactory +org.apache.hudi.table.catalog.HoodieCatalogFactory \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index 43e4ed511452d..341a157e86c78 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -18,11 +18,6 @@ package org.apache.hudi.sink.compact; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -54,8 +49,9 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -138,17 +134,7 @@ public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception { HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf); - boolean scheduled = false; - // judge whether have operation - // To compute the compaction instant time and do compaction. - Option compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient); - if (compactionInstantTimeOption.isPresent()) { - scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty()); - } - - String compactionInstantTime = compactionInstantTimeOption.get(); - - assertTrue(scheduled, "The compaction plan should be scheduled"); + String compactionInstantTime = scheduleCompactionPlan(metaClient, writeClient); HoodieFlinkTable table = writeClient.getHoodieTable(); // generate compaction plan @@ -160,7 +146,7 @@ public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception { // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); - env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime)) + env.addSource(new CompactionPlanSourceFunction(Collections.singletonList(Pair.of(compactionInstantTime, compactionPlan)))) .name("compaction_source") .uid("uid_compaction_source") .rebalance() @@ -280,29 +266,18 @@ public void testHoodieFlinkCompactorWithPlanSelectStrategy(boolean enableChangel } table.getMetaClient().reloadActiveTimeline(); - DataStream source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans)) + env.addSource(new CompactionPlanSourceFunction(compactionPlans)) .name("compaction_source") - .uid("uid_compaction_source"); - SingleOutputStreamOperator operator = source.rebalance() + .uid("uid_compaction_source") + .rebalance() .transform("compact_task", TypeInformation.of(CompactionCommitEvent.class), new ProcessOperator<>(new CompactFunction(conf))) .setParallelism(1) - .process(new ProcessFunction() { - @Override - public void processElement(CompactionCommitEvent event, ProcessFunction.Context context, Collector out) { - context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event); - } - }) - .name("group_by_compaction_plan") - .uid("uid_group_by_compaction_plan") + .addSink(new CompactionCommitSink(conf)) + .name("compaction_commit") + .uid("uid_compaction_commit") .setParallelism(1); - compactionPlans.forEach(pair -> - operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class))) - .addSink(new CompactionCommitSink(conf)) - .name("clean_commits " + pair.getLeft()) - .uid("uid_clean_commits_" + pair.getLeft()) - .setParallelism(1)); env.execute("flink_hudi_compaction"); writeClient.close(); @@ -311,8 +286,7 @@ public void processElement(CompactionCommitEvent event, ProcessFunction writeClient) { boolean scheduled = false; - // judge whether have operation - // To compute the compaction instant time and do compaction. + // judge whether there are any compaction operations. Option compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient); if (compactionInstantTimeOption.isPresent()) { scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanSelectStrategy.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java similarity index 74% rename from hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanSelectStrategy.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java index 3ac9f6c6663ef..181be26d6dc60 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanSelectStrategy.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java @@ -25,18 +25,16 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.sink.compact.strategy.AllPendingCompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.CompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.InstantCompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.MultiCompactionPlanSelectStrategy; -import org.apache.hudi.sink.compact.strategy.SingleCompactionPlanSelectStrategy; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies; +import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategy; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** - * Test case for every {@link CompactionPlanSelectStrategy} implements + * Test case for every {@link CompactionPlanStrategy} implements */ -public class TestCompactionPlanSelectStrategy { +public class TestCompactionPlanStrategy { private HoodieTimeline timeline; private HoodieTimeline emptyTimeline; private HoodieTimeline allCompleteTimeline; @@ -59,72 +57,75 @@ public void beforeEach() { void testSingleCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); - SingleCompactionPlanSelectStrategy strategy = new SingleCompactionPlanSelectStrategy(); - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline)); - compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig)); + compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_LIFO; + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline)); HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline)); HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline)); } @Test void testMultiCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); - compactionConfig.compactionPlanMaxSelect = 2; + compactionConfig.maxNumCompactionPlans = 2; - MultiCompactionPlanSelectStrategy strategy = new MultiCompactionPlanSelectStrategy(); - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig)); + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline)); - compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig)); + compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_LIFO; + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline)); HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline)); HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline)); } @Test void testAllPendingCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); + compactionConfig.compactionPlanSelectStrategy = CompactionPlanStrategy.ALL; + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); - AllPendingCompactionPlanSelectStrategy strategy = new AllPendingCompactionPlanSelectStrategy(); assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_006}, - strategy.select(pendingCompactionTimeline, compactionConfig)); + strategy.select(pendingCompactionTimeline)); HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline)); HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline(); - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline)); } @Test void testInstantCompactionPlanSelectStrategy() { HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline(); FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig(); + + compactionConfig.compactionPlanSelectStrategy = CompactionPlanStrategy.INSTANTS; + CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig); compactionConfig.compactionPlanInstant = "004"; - InstantCompactionPlanSelectStrategy strategy = new InstantCompactionPlanSelectStrategy(); - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline)); compactionConfig.compactionPlanInstant = "002,003"; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline)); compactionConfig.compactionPlanInstant = "002,005"; - assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline)); compactionConfig.compactionPlanInstant = "005"; - assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline, compactionConfig)); + assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline)); } private void assertHoodieInstantsEquals(HoodieInstant[] expected, List actual) { diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index a8104efb322f9..a976d483fbc33 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.table.catalog.HoodieHiveCatalog; +import org.apache.hudi.table.catalog.TestHoodieCatalogUtils; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestData; @@ -1287,6 +1289,45 @@ void testWriteAndReadWithDataSkipping() { + "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]"); } + @Test + void testBuiltinFunctionWithHMSCatalog() { + TableEnvironment tableEnv = batchTableEnv; + + HoodieHiveCatalog hoodieCatalog = TestHoodieCatalogUtils.createHiveCatalog("hudi_catalog"); + + tableEnv.registerCatalog("hudi_catalog", hoodieCatalog); + tableEnv.executeSql("use catalog " + ("hudi_catalog")); + + String dbName = "hudi"; + tableEnv.executeSql("create database " + dbName); + tableEnv.executeSql("use " + dbName); + + String hoodieTableDDL = sql("t1") + .field("f_int int") + .field("f_date DATE") + .pkField("f_int") + .partitionField("f_int") + .option(FlinkOptions.PATH, tempFile.getAbsolutePath() + "/" + dbName + "/" + "t1") + .option(FlinkOptions.RECORD_KEY_FIELD, "f_int") + .option(FlinkOptions.PRECOMBINE_FIELD, "f_date") + .end(); + tableEnv.executeSql(hoodieTableDDL); + + String insertSql = "insert into t1 values (1, TO_DATE('2022-02-02')), (2, DATE '2022-02-02')"; + execInsertSql(tableEnv, insertSql); + + List result = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1").execute().collect()); + final String expected = "[" + + "+I[1, 2022-02-02], " + + "+I[2, 2022-02-02]]"; + assertRowsEquals(result, expected); + + List partitionResult = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1 where f_int = 1").execute().collect()); + assertRowsEquals(partitionResult, "[+I[1, 2022-02-02]]"); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java new file mode 100644 index 0000000000000..a99df0ddd2e4e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java @@ -0,0 +1,67 @@ +/* + * 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.catalog; + +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.jupiter.api.Test; + +import java.net.URL; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test cases for {@link HoodieCatalogFactory}. + */ +public class TestHoodieCatalogFactory { + private static final URL CONF_DIR = + Thread.currentThread().getContextClassLoader().getResource("test-catalog-factory-conf"); + + @Test + public void testCreateHiveCatalog() { + final String catalogName = "mycatalog"; + + final HoodieHiveCatalog expectedCatalog = TestHoodieCatalogUtils.createHiveCatalog(catalogName); + + final Map options = new HashMap<>(); + options.put(CommonCatalogOptions.CATALOG_TYPE.key(), HoodieCatalogFactory.IDENTIFIER); + options.put(HoodieCatalogFactoryOptions.HIVE_CONF_DIR.key(), CONF_DIR.getPath()); + options.put(HoodieCatalogFactoryOptions.MODE.key(), "hms"); + + final Catalog actualCatalog = + FactoryUtil.createCatalog( + catalogName, options, null, Thread.currentThread().getContextClassLoader()); + + assertEquals( + ((HoodieHiveCatalog) actualCatalog) + .getHiveConf() + .getVar(HiveConf.ConfVars.METASTOREURIS), "dummy-hms"); + checkEquals(expectedCatalog, (HoodieHiveCatalog) actualCatalog); + } + + private static void checkEquals(HoodieHiveCatalog c1, HoodieHiveCatalog c2) { + // Only assert a few selected properties for now + assertEquals(c2.getName(), c1.getName()); + assertEquals(c2.getDefaultDatabase(), c1.getDefaultDatabase()); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java new file mode 100644 index 0000000000000..de433d0a3a667 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogUtils.java @@ -0,0 +1,67 @@ +/* + * 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.catalog; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.hadoop.hive.conf.HiveConf; + +import java.io.IOException; + +/** Test utils for Hoodie catalog. */ +public class TestHoodieCatalogUtils { + private static final String HIVE_WAREHOUSE_URI_FORMAT = + "jdbc:derby:;databaseName=%s;create=true"; + + private static final String TEST_CATALOG_NAME = "test_catalog"; + + private static final org.junit.rules.TemporaryFolder TEMPORARY_FOLDER = new org.junit.rules.TemporaryFolder(); + + /** Create a HiveCatalog with an embedded Hive Metastore. */ + public static HoodieHiveCatalog createHiveCatalog() { + return createHiveCatalog(TEST_CATALOG_NAME); + } + + public static HoodieHiveCatalog createHiveCatalog(String name) { + return new HoodieHiveCatalog( + name, + null, + createHiveConf(), + true, + true); + } + + public static HiveConf createHiveConf() { + ClassLoader classLoader = TestHoodieCatalogUtils.class.getClassLoader(); + try { + TEMPORARY_FOLDER.create(); + String warehouseDir = TEMPORARY_FOLDER.newFolder().getAbsolutePath() + "/metastore_db"; + String warehouseUri = String.format(HIVE_WAREHOUSE_URI_FORMAT, warehouseDir); + + HiveConf.setHiveSiteLocation(classLoader.getResource(HoodieCatalogFactoryOptions.HIVE_SITE_FILE)); + HiveConf hiveConf = new HiveConf(); + hiveConf.setVar( + HiveConf.ConfVars.METASTOREWAREHOUSE, + TEMPORARY_FOLDER.newFolder("hive_warehouse").getAbsolutePath()); + hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, warehouseUri); + return hiveConf; + } catch (IOException e) { + throw new CatalogException("Failed to create test HiveConf to HiveCatalog.", e); + } + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java new file mode 100644 index 0000000000000..ed470c9a96c35 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java @@ -0,0 +1,150 @@ +/* + * 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.catalog; + +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieCatalogException; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.hadoop.hive.metastore.api.Table; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test cases for {@link HoodieHiveCatalog}. + */ +public class TestHoodieHiveCatalog { + TableSchema schema = + TableSchema.builder() + .field("uuid", DataTypes.INT().notNull()) + .field("name", DataTypes.STRING()) + .field("age", DataTypes.INT()) + .field("ts", DataTypes.BIGINT()) + .field("par1", DataTypes.STRING()) + .primaryKey("uuid") + .build(); + List partitions = Collections.singletonList("par1"); + private static HoodieHiveCatalog hoodieCatalog; + private final ObjectPath tablePath = new ObjectPath("default", "test"); + + @BeforeAll + public static void createCatalog() { + hoodieCatalog = TestHoodieCatalogUtils.createHiveCatalog(); + hoodieCatalog.open(); + } + + @AfterEach + public void dropTable() throws TableNotExistException { + hoodieCatalog.dropTable(tablePath, true); + } + + @AfterAll + public static void closeCatalog() { + if (hoodieCatalog != null) { + hoodieCatalog.close(); + } + } + + @ParameterizedTest + @EnumSource(value = HoodieTableType.class) + public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Exception { + Map originOptions = new HashMap<>(); + originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi"); + originOptions.put(FlinkOptions.TABLE_TYPE.key(), tableType.toString()); + + CatalogTable table = + new CatalogTableImpl(schema, partitions, originOptions, "hudi table"); + hoodieCatalog.createTable(tablePath, table, false); + + CatalogBaseTable table1 = hoodieCatalog.getTable(tablePath); + assertEquals(table1.getOptions().get(CONNECTOR.key()), "hudi"); + assertEquals(table1.getOptions().get(FlinkOptions.TABLE_TYPE.key()), tableType.toString()); + assertEquals(table1.getOptions().get(FlinkOptions.RECORD_KEY_FIELD.key()), "uuid"); + assertEquals(table1.getOptions().get(FlinkOptions.PRECOMBINE_FIELD.key()), "ts"); + assertEquals(table1.getUnresolvedSchema().getPrimaryKey().get().getColumnNames(), Collections.singletonList("uuid")); + assertEquals(((CatalogTable)table1).getPartitionKeys(), Collections.singletonList("par1")); + } + + @Test + public void testCreateNonHoodieTable() throws TableAlreadyExistException, DatabaseNotExistException { + CatalogTable table = + new CatalogTableImpl(schema, Collections.emptyMap(), "hudi table"); + try { + hoodieCatalog.createTable(tablePath, table, false); + } catch (HoodieCatalogException e) { + assertEquals(String.format("The %s is not hoodie table", tablePath.getObjectName()), e.getMessage()); + } + } + + @Test + public void testAlterTable() throws Exception { + Map originOptions = new HashMap<>(); + originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi"); + CatalogTable originTable = + new CatalogTableImpl(schema, partitions, originOptions, "hudi table"); + hoodieCatalog.createTable(tablePath, originTable, false); + + Table hiveTable = hoodieCatalog.getHiveTable(tablePath); + Map newOptions = hiveTable.getParameters(); + newOptions.put("k", "v"); + CatalogTable newTable = new CatalogTableImpl(schema, partitions, newOptions, "alter hudi table"); + hoodieCatalog.alterTable(tablePath, newTable, false); + + hiveTable = hoodieCatalog.getHiveTable(tablePath); + assertEquals(hiveTable.getParameters().get(CONNECTOR.key()), "hudi"); + assertEquals(hiveTable.getParameters().get("k"), "v"); + } + + @Test + public void testRenameTable() throws Exception { + Map originOptions = new HashMap<>(); + originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi"); + CatalogTable originTable = + new CatalogTableImpl(schema, partitions, originOptions, "hudi table"); + hoodieCatalog.createTable(tablePath, originTable, false); + + hoodieCatalog.renameTable(tablePath, "test1", false); + + assertEquals(hoodieCatalog.getHiveTable(new ObjectPath("default", "test1")).getTableName(), "test1"); + + hoodieCatalog.renameTable(new ObjectPath("default", "test1"), "test", false); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml b/hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml new file mode 100644 index 0000000000000..b13347f808edd --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/hive-site.xml @@ -0,0 +1,52 @@ + + + + + + + + + hive.metastore.integral.jdo.pushdown + true + + + + hive.metastore.schema.verification + false + + + + hive.metastore.client.capability.check + false + + + + datanucleus.schema.autoCreateTables + true + + + + datanucleus.schema.autoCreateAll + true + + + + common-key + common-val + + diff --git a/hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml b/hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml new file mode 100644 index 0000000000000..97424758d172f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/test-catalog-factory-conf/hive-site.xml @@ -0,0 +1,58 @@ + + + + + + + + + hive.metastore.integral.jdo.pushdown + true + + + + hive.metastore.schema.verification + false + + + + hive.metastore.client.capability.check + false + + + + datanucleus.schema.autoCreateTables + true + + + + datanucleus.schema.autoCreateAll + true + + + + common-key + common-val + + + + + hive.metastore.uris + dummy-hms + + diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index 7a9e122e86c15..dc711818d710d 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -95,8 +94,9 @@ private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteJob.HoodieTestSui HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) .withAutoCommit(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) - .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField) + .withPayloadConfig(HoodiePayloadConfig.newBuilder() + .withPayloadOrderingField(cfg.sourceOrderingField) + .withPayloadClass(cfg.payloadClassName) .build()) .forTable(cfg.targetTableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) diff --git a/hudi-kafka-connect/demo/config-sink-hive.json b/hudi-kafka-connect/demo/config-sink-hive.json index 214fd1891906f..7c49784cbf6a3 100644 --- a/hudi-kafka-connect/demo/config-sink-hive.json +++ b/hudi-kafka-connect/demo/config-sink-hive.json @@ -21,7 +21,6 @@ "hoodie.datasource.hive_sync.table": "huditesttopic", "hoodie.datasource.hive_sync.partition_fields": "date", "hoodie.datasource.hive_sync.partition_extractor_class": "org.apache.hudi.hive.MultiPartKeysValueExtractor", - "hoodie.datasource.hive_sync.use_jdbc": "false", "hoodie.datasource.hive_sync.mode": "hms", "dfs.client.use.datanode.hostname": "true", "hive.metastore.uris": "thrift://hivemetastore:9083", diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java index e4543c692db86..3b51fddfa8db6 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectConfigs.java @@ -172,7 +172,6 @@ public String getHadoopConfHome() { public static final String HIVE_URL = "hoodie.datasource.hive_sync.jdbcurl"; public static final String HIVE_PARTITION_FIELDS = "hoodie.datasource.hive_sync.partition_fields"; public static final String HIVE_PARTITION_EXTRACTOR_CLASS = "hoodie.datasource.hive_sync.partition_extractor_class"; - public static final String HIVE_USE_JDBC = "hoodie.datasource.hive_sync.use_jdbc"; public static final String HIVE_SYNC_MODE = "hoodie.datasource.hive_sync.mode"; public static final String HIVE_AUTO_CREATE_DATABASE = "hoodie.datasource.hive_sync.auto_create_database"; public static final String HIVE_IGNORE_EXCEPTIONS = "hoodie.datasource.hive_sync.ignore_exceptions"; diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java index 6ab0469738dff..1a33560dc858a 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectWriterProvider.java @@ -27,10 +27,12 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.config.HoodieClusteringConfig; -import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.connect.KafkaConnectFileIdPrefixProvider; import org.apache.hudi.connect.utils.KafkaConnectUtils; import org.apache.hudi.exception.HoodieException; @@ -87,13 +89,10 @@ public KafkaConnectWriterProvider( .withAutoCommit(false) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) // participants should not trigger table services, and leave it to the coordinator - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withAutoClean(false) - .withAutoArchive(false) - .withInlineCompaction(false).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder() - .withInlineClustering(false) - .build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().withAutoArchive(false).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().withAutoClean(false).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(false).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClustering(false).build()) .build(); context = new HoodieJavaEngineContext(hadoopConf); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index ca1474fa88abc..80be97ebef3a2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -180,9 +180,10 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base return builder.forTable(tblName) .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key())) .withInlineCompaction(inlineCompact).build()) - .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD().key())) + .withPayloadConfig(HoodiePayloadConfig.newBuilder() + .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key())) + .withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD().key())) .build()) // override above with Hoodie configs specified as options. .withProps(parameters).build(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 8ff82746b0ceb..feeb57212603d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -399,12 +399,7 @@ object DataSourceWriteOptions { .defaultValue(classOf[HiveSyncTool].getName) .withDocumentation("Sync tool class name used to sync to metastore. Defaults to Hive.") - val RECONCILE_SCHEMA: ConfigProperty[Boolean] = ConfigProperty - .key("hoodie.datasource.write.reconcile.schema") - .defaultValue(false) - .withDocumentation("When a new batch of write has records with old schema, but latest table schema got " - + "evolved, this config will upgrade the records to leverage latest table schema(default values will be " - + "injected to missing fields). If not, the write batch would fail.") + val RECONCILE_SCHEMA: ConfigProperty[Boolean] = HoodieCommonConfig.RECONCILE_SCHEMA // HIVE SYNC SPECIFIC CONFIGS // NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes @@ -419,8 +414,6 @@ object DataSourceWriteOptions { @Deprecated val HIVE_DATABASE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_DATABASE_NAME @Deprecated - val hiveTableOptKeyInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.TABLE_NAME_INFERENCE_FUNCTION - @Deprecated val HIVE_TABLE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_TABLE_NAME @Deprecated val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT @@ -444,10 +437,6 @@ object DataSourceWriteOptions { val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION @Deprecated val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT - - /** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */ - @Deprecated - val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_JDBC @Deprecated val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE @Deprecated @@ -507,9 +496,6 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_USE_PRE_APACHE_INPUT_FORMAT} and its methods instead */ @Deprecated val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() - /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ - @Deprecated - val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_JDBC.key() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.key() @@ -700,9 +686,6 @@ object DataSourceWriteOptions { val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.defaultValue() @Deprecated val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" - /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ - @Deprecated - val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfigHolder.HIVE_USE_JDBC.defaultValue() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.defaultValue() 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 510a45899bce5..a90e6b8e8e653 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 @@ -40,6 +40,7 @@ import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.index.SparkHoodieIndexFactory import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.internal.schema.InternalSchema +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{AvroSchemaEvolutionUtils, SerDeHelper} import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} @@ -242,16 +243,29 @@ object HoodieSparkSqlWriter { classOf[org.apache.avro.Schema])) var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) val lastestSchema = getLatestTableSchema(fs, basePath, sparkContext, schema) - val internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext) + var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, sparkContext) + if (reconcileSchema && parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean + && internalSchemaOpt.isEmpty) { + // force apply full schema evolution. + internalSchemaOpt = Some(AvroInternalSchemaConverter.convert(schema)) + } if (reconcileSchema) { schema = lastestSchema } if (internalSchemaOpt.isDefined) { - schema = { - val newSparkSchema = AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema)) - AvroConversionUtils.convertStructTypeToAvroSchema(newSparkSchema, structName, nameSpace) - + // Apply schema evolution. + val mergedSparkSchema = if (!reconcileSchema) { + AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema, lastestSchema)) + } else { + // Auto merge write schema and read schema. + val mergedInternalSchema = AvroSchemaEvolutionUtils.reconcileSchema(schema, internalSchemaOpt.get) + AvroConversionUtils.convertAvroSchemaToStructType(AvroInternalSchemaConverter.convert(mergedInternalSchema, lastestSchema.getName)) } + schema = AvroConversionUtils.convertStructTypeToAvroSchema(mergedSparkSchema, structName, nameSpace) + } + + if (reconcileSchema && internalSchemaOpt.isEmpty) { + schema = lastestSchema } validateSchemaForHoodieIsDeleted(schema) sparkContext.getConf.registerAvroSchemas(schema) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index f9d8a60004a8b..f6473c2b89731 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -78,7 +78,6 @@ object HoodieWriterUtils { hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS) hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING) - hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USE_JDBC) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE) hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala new file mode 100644 index 0000000000000..93cec470ec99c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DeDupeType.scala @@ -0,0 +1,28 @@ +/* + * 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 + +object DeDupeType extends Enumeration { + + type dedupeType = Value + + val INSERT_TYPE = Value("insert_type") + val UPDATE_TYPE = Value("update_type") + val UPSERT_TYPE = Value("upsert_type") +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala new file mode 100644 index 0000000000000..b6f610e7d7098 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/DedupeSparkJob.scala @@ -0,0 +1,245 @@ +/* + * 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 + +import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.{HoodieBaseFile, HoodieRecord} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.exception.HoodieException +import org.apache.log4j.Logger +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +import java.util.stream.Collectors +import scala.collection.JavaConversions._ +import scala.collection.mutable.{Buffer, HashMap, HashSet, ListBuffer} + +/** + * Spark job to de-duplicate data present in a partition path + */ +class DedupeSparkJob(basePath: String, + duplicatedPartitionPath: String, + repairOutputPath: String, + sqlContext: SQLContext, + fs: FileSystem, + dedupeType: DeDupeType.Value) { + + val sparkHelper = new SparkHelper(sqlContext, fs) + val LOG = Logger.getLogger(this.getClass) + + /** + * + * @param tblName + * @return + */ + def getDupeKeyDF(tblName: String): DataFrame = { + val dupeSql = + s""" + select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key, + count(*) as dupe_cnt + from ${tblName} + group by `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` + having dupe_cnt > 1 + """ + sqlContext.sql(dupeSql) + } + + /** + * + * Check a given partition for duplicates and suggest the deletions that need to be done in each file, + * in order to set things right. + * + * @return + */ + private def planDuplicateFix(): HashMap[String, HashSet[String]] = { + val tmpTableName = s"htbl_${System.currentTimeMillis()}" + val dedupeTblName = s"${tmpTableName}_dupeKeys" + + val metadata = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(basePath).build() + + val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath")) + val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) + val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]()) + val filteredStatuses = latestFiles.map(f => f.getPath) + LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}") + + val df = sqlContext.parquetFile(filteredStatuses: _*) + df.registerTempTable(tmpTableName) + val dupeKeyDF = getDupeKeyDF(tmpTableName) + dupeKeyDF.registerTempTable(dedupeTblName) + + // Obtain necessary satellite information for duplicate rows + val dupeDataSql = + s""" + SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time` + FROM $tmpTableName h + JOIN $dedupeTblName d + ON h.`_hoodie_record_key` = d.dupe_key + """ + val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0)) + getDedupePlan(dupeMap) + } + + private def getDedupePlan(dupeMap: Map[String, Buffer[Row]]): HashMap[String, HashSet[String]] = { + val fileToDeleteKeyMap = new HashMap[String, HashSet[String]]() + dupeMap.foreach(rt => { + val (key, rows) = rt + + dedupeType match { + case DeDupeType.UPDATE_TYPE => + /* + This corresponds to the case where all duplicates have been updated at least once. + Once updated, duplicates are bound to have same commit time unless forcefully modified. + */ + rows.init.foreach(r => { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + }) + + case DeDupeType.INSERT_TYPE => + /* + This corresponds to the case where duplicates got created due to INSERT and have never been updated. + */ + var maxCommit = -1L + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c > maxCommit) + maxCommit = c + }) + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c != maxCommit) { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + } + }) + + case DeDupeType.UPSERT_TYPE => + /* + This corresponds to the case where duplicates got created as a result of inserts as well as updates, + i.e few duplicate records have been updated, while others were never updated. + */ + var maxCommit = -1L + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c > maxCommit) + maxCommit = c + }) + val rowsWithMaxCommit = new ListBuffer[Row]() + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c != maxCommit) { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + } else { + rowsWithMaxCommit += r + } + }) + + rowsWithMaxCommit.toList.init.foreach(r => { + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)) { + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + }) + + case _ => throw new IllegalArgumentException("Please provide valid type for deduping!") + } + }) + LOG.debug(s"fileToDeleteKeyMap size: ${fileToDeleteKeyMap.size}, map: $fileToDeleteKeyMap") + fileToDeleteKeyMap + } + + def fixDuplicates(dryRun: Boolean = true) = { + val metadata = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(basePath).build() + + val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath")) + val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) + + val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]()) + + val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap + val dupeFixPlan = planDuplicateFix() + + // 1. Copy all latest files into the temp fix path + fileNameToPathMap.foreach { case (fileName, filePath) => + val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else "" + val dstPath = new Path(s"$repairOutputPath/${filePath.getName}$badSuffix") + LOG.info(s"Copying from $filePath to $dstPath") + FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf) + } + + // 2. Remove duplicates from the bad files + dupeFixPlan.foreach { case (fileName, keysToSkip) => + val instantTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName) + val badFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}.bad") + val newFilePath = new Path(s"$repairOutputPath/${fileNameToPathMap(fileName).getName}") + LOG.info(" Skipping and writing new file for : " + fileName) + SparkHelpers.skipKeysAndWriteNewFile(instantTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName)) + fs.delete(badFilePath, true) + } + + // 3. Check that there are no duplicates anymore. + val df = sqlContext.read.parquet(s"$repairOutputPath/*.parquet") + df.registerTempTable("fixedTbl") + val dupeKeyDF = getDupeKeyDF("fixedTbl") + val dupeCnt = dupeKeyDF.count() + if (dupeCnt != 0) { + dupeKeyDF.show() + throw new HoodieException("Still found some duplicates!!.. Inspect output") + } + + // 4. Additionally ensure no record keys are left behind. + val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList) + val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"$repairOutputPath/${t._2.getName}").toList) + val missedRecordKeysDF = sourceDF.except(fixedDF) + val missedCnt = missedRecordKeysDF.count() + if (missedCnt != 0) { + missedRecordKeysDF.show() + throw new HoodieException("Some records in source are not found in fixed files. Inspect output!!") + } + + println("No duplicates found & counts are in check!!!! ") + // 5. Prepare to copy the fixed files back. + fileNameToPathMap.foreach { case (_, filePath) => + val srcPath = new Path(s"$repairOutputPath/${filePath.getName}") + val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}") + if (dryRun) { + LOG.info(s"[JUST KIDDING!!!] Copying from $srcPath to $dstPath") + } else { + // for real + LOG.info(s"[FOR REAL!!!] Copying from $srcPath to $dstPath") + FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf) + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala new file mode 100644 index 0000000000000..1ed0e5e1a456a --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala @@ -0,0 +1,134 @@ +/* + * 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 + +import org.apache.avro.Schema +import org.apache.avro.generic.IndexedRecord +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hudi.avro.HoodieAvroWriteSupport +import org.apache.hudi.client.SparkTaskContextSupplier +import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} +import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} +import org.apache.hudi.common.util.BaseFileUtils +import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig} +import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig} +import org.apache.parquet.avro.AvroSchemaConverter +import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.collection.JavaConversions._ +import scala.collection.mutable._ + +object SparkHelpers { + @throws[Exception] + def skipKeysAndWriteNewFile(instantTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) { + val sourceRecords = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readAvroRecords(fs.getConf, sourceFile) + val schema: Schema = sourceRecords.get(0).getSchema + val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble, + HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_TYPE.defaultValue); + val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter)) + val parquetConfig: HoodieParquetConfig[HoodieAvroWriteSupport] = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.toDouble) + + // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'. + parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader) + + val writer = new HoodieAvroParquetWriter[IndexedRecord](destinationFile, parquetConfig, instantTime, new SparkTaskContextSupplier(), true) + for (rec <- sourceRecords) { + val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString + if (!keysToSkip.contains(key)) { + + writer.writeAvro(key, rec) + } + } + writer.close + } +} + +/** + * Bunch of Spark Shell/Scala stuff useful for debugging + */ +class SparkHelper(sqlContext: SQLContext, fs: FileSystem) { + + /** + * Print keys from a file + * + * @param file + */ + def printKeysFromFile(file: String) = { + getRowKeyDF(file).collect().foreach(println(_)) + } + + /** + * + * @param file + * @return + */ + def getRowKeyDF(file: String): DataFrame = { + sqlContext.read.parquet(file).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") + } + + /** + * Does the rowKey actually exist in the file. + * + * @param rowKey + * @param file + * @return + */ + def isFileContainsKey(rowKey: String, file: String): Boolean = { + println(s"Checking $file for key $rowKey") + val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '$rowKey'") + if (ff.count() > 0) true else false + } + + /** + * Number of keys in a given file + * + * @param file + * @param sqlContext + */ + def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = { + val keyCount = getRowKeyDF(file).collect().length + println(keyCount) + keyCount + } + + /** + * + * Checks that all the keys in the file, have been added to the bloom filter + * in the footer + * + * @param conf + * @param sqlContext + * @param file + * @return + */ + def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = { + val bf = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readBloomFilterFromMetadata(conf, new Path(file)) + val foundCount = sqlContext.parquetFile(file) + .select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") + .collect().count(r => !bf.mightContain(r.getString(0))) + val totalCount = getKeyCount(file, sqlContext) + println(s"totalCount: $totalCount, foundCount: $foundCount") + totalCount == foundCount + } + + def getDistinctKeyDF(paths: List[String]): DataFrame = { + sqlContext.read.parquet(paths: _*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala new file mode 100644 index 0000000000000..b49875c94c11c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CopyToTableProcedure.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.DataSourceReadOptions +import org.apache.spark.internal.Logging +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} +import org.apache.spark.sql.{Row, SaveMode} + +import java.util.function.Supplier + +class CopyToTableProcedure extends BaseProcedure with ProcedureBuilder with Logging { + + + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "query_type", DataTypes.StringType, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL), + ProcedureParameter.required(2, "new_table", DataTypes.StringType, None), + ProcedureParameter.optional(3, "begin_instance_time", DataTypes.StringType, ""), + ProcedureParameter.optional(4, "end_instance_time", DataTypes.StringType, ""), + ProcedureParameter.optional(5, "as_of_instant", DataTypes.StringType, ""), + ProcedureParameter.optional(6, "save_mode", DataTypes.StringType, "overwrite") + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("status", DataTypes.IntegerType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val queryType = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val newTableName = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[String] + val beginInstance = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[String] + val endInstance = getArgValueOrDefault(args, PARAMETERS(4)).get.asInstanceOf[String] + val asOfInstant = getArgValueOrDefault(args, PARAMETERS(5)).get.asInstanceOf[String] + val saveModeStr = getArgValueOrDefault(args, PARAMETERS(6)).get.asInstanceOf[String] + + assert(saveModeStr.nonEmpty, "save_mode(append,overwrite) can not be null.") + + val saveMode: Any = saveModeStr.toLowerCase match { + case "overwrite" => SaveMode.Overwrite + case "append" => SaveMode.Append + case _ => assert(assertion = false, s"save_mode not support $saveModeStr.") + } + + + val tablePath = getBasePath(tableName) + + val sourceDataFrame = queryType match { + case DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL => if (asOfInstant.nonEmpty) { + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key, asOfInstant) + .load(tablePath) + } else { + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(tablePath) + } + case DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL => + assert(beginInstance.nonEmpty && endInstance.nonEmpty, "when the query_type is incremental, begin_instance_time and end_instance_time can not be null.") + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, beginInstance) + .option(DataSourceReadOptions.END_INSTANTTIME.key, endInstance) + .load(tablePath) + case DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL => + sparkSession.read + .format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) + .load(tablePath) + } + if (sparkSession.catalog.tableExists(newTableName)) { + val schema = sparkSession.read.table(newTableName).schema + val selectColumns = schema.fields.toStream.map(_.name) + sourceDataFrame.selectExpr(selectColumns: _*) + .write + .mode(saveMode.toString) + .saveAsTable(newTableName) + } else { + sourceDataFrame.write + .mode(saveMode.toString) + .saveAsTable(newTableName) + } + + + Seq(Row(0)) + } + + override def build = new CopyToTableProcedure() +} + +object CopyToTableProcedure { + val NAME = "copy_to_table" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new CopyToTableProcedure() + } +} + + + + + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala index 0545c140bb3e5..1eb82d97c5e95 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala @@ -74,6 +74,12 @@ object HoodieProcedures { mapBuilder.put(ShowMetadataStatsProcedure.NAME, ShowMetadataStatsProcedure.builder) mapBuilder.put(ValidateMetadataFilesProcedure.NAME, ValidateMetadataFilesProcedure.builder) mapBuilder.put(ShowFsPathDetailProcedure.NAME, ShowFsPathDetailProcedure.builder) + mapBuilder.put(CopyToTableProcedure.NAME, CopyToTableProcedure.builder) + mapBuilder.put(RepairAddpartitionmetaProcedure.NAME, RepairAddpartitionmetaProcedure.builder) + mapBuilder.put(RepairCorruptedCleanFilesProcedure.NAME, RepairCorruptedCleanFilesProcedure.builder) + mapBuilder.put(RepairDeduplicateProcedure.NAME, RepairDeduplicateProcedure.builder) + mapBuilder.put(RepairMigratePartitionMetaProcedure.NAME, RepairMigratePartitionMetaProcedure.builder) + mapBuilder.put(RepairOverwriteHoodiePropsProcedure.NAME, RepairOverwriteHoodiePropsProcedure.builder) mapBuilder.build } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala new file mode 100644 index 0000000000000..bb65174c4b47b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala @@ -0,0 +1,89 @@ +/* + * 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.command.procedures + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodiePartitionMetadata +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util +import java.util.function.Supplier +import scala.collection.JavaConversions._ + +class RepairAddpartitionmetaProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "dry_run", DataTypes.BooleanType, true) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("partition_path", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("metadata_is_present", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("action", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val dryRun = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[Boolean] + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + val latestCommit: String = metaClient.getActiveTimeline.getCommitTimeline.lastInstant.get.getTimestamp + val partitionPaths: util.List[String] = FSUtils.getAllPartitionFoldersThreeLevelsDown(metaClient.getFs, tablePath); + val basePath: Path = new Path(tablePath) + + val rows = new util.ArrayList[Row](partitionPaths.size) + for (partition <- partitionPaths) { + val partitionPath: Path = FSUtils.getPartitionPath(basePath, partition) + var isPresent = "Yes" + var action = "None" + if (!HoodiePartitionMetadata.hasPartitionMetadata(metaClient.getFs, partitionPath)) { + isPresent = "No" + if (!dryRun) { + val partitionMetadata: HoodiePartitionMetadata = new HoodiePartitionMetadata(metaClient.getFs, latestCommit, basePath, partitionPath, metaClient.getTableConfig.getPartitionMetafileFormat) + partitionMetadata.trySave(0) + action = "Repaired" + } + } + rows.add(Row(partition, isPresent, action)) + } + + rows.stream().toArray().map(r => r.asInstanceOf[Row]).toList + } + + override def build: Procedure = new RepairAddpartitionmetaProcedure() +} + +object RepairAddpartitionmetaProcedure { + val NAME = "repair_add_partition_meta" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairAddpartitionmetaProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala new file mode 100644 index 0000000000000..ff185d1bdfab1 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala @@ -0,0 +1,86 @@ +/* + * 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.command.procedures + +import org.apache.avro.AvroRuntimeException +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} +import org.apache.hudi.common.util.CleanerUtils +import org.apache.hudi.exception.HoodieIOException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.io.IOException +import java.util.function.Supplier +import scala.collection.JavaConverters.asScalaIteratorConverter + +class RepairCorruptedCleanFilesProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("result", DataTypes.BooleanType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + val cleanerTimeline = metaClient.getActiveTimeline.getCleanerTimeline + logInfo("Inspecting pending clean metadata in timeline for corrupted files") + var result = true + cleanerTimeline.filterInflightsAndRequested.getInstants.iterator().asScala.foreach((instant: HoodieInstant) => { + try { + CleanerUtils.getCleanerPlan(metaClient, instant) + } catch { + case e: AvroRuntimeException => + logWarning("Corruption found. Trying to remove corrupted clean instant file: " + instant) + HoodieActiveTimeline.deleteInstantFile(metaClient.getFs, metaClient.getMetaPath, instant) + case ioe: IOException => + if (ioe.getMessage.contains("Not an Avro data file")) { + logWarning("Corruption found. Trying to remove corrupted clean instant file: " + instant) + HoodieActiveTimeline.deleteInstantFile(metaClient.getFs, metaClient.getMetaPath, instant) + } else { + result = false + throw new HoodieIOException(ioe.getMessage, ioe) + } + } + }) + Seq(Row(result)) + } + + override def build: Procedure = new RepairCorruptedCleanFilesProcedure() +} + +object RepairCorruptedCleanFilesProcedure { + val NAME = "repair_corrupted_clean_files" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairCorruptedCleanFilesProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala new file mode 100644 index 0000000000000..8ee5055e1fdb2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairDeduplicateProcedure.scala @@ -0,0 +1,86 @@ +/* + * 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.command.procedures + +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.exception.HoodieException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} +import java.util.function.Supplier + +import org.apache.spark.sql.hudi.{DeDupeType, DedupeSparkJob} + +import scala.util.{Failure, Success, Try} + +class RepairDeduplicateProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "duplicated_partition_path", DataTypes.StringType, None), + ProcedureParameter.required(2, "repaired_output_path", DataTypes.StringType, None), + ProcedureParameter.optional(3, "dry_run", DataTypes.BooleanType, true), + ProcedureParameter.optional(4, "dedupe_type", DataTypes.StringType, "insert_type") + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("result", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val duplicatedPartitionPath = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val repairedOutputPath = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[String] + val dryRun = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[Boolean] + val dedupeType = getArgValueOrDefault(args, PARAMETERS(4)).get.asInstanceOf[String] + + if (!DeDupeType.values.contains(DeDupeType.withName(dedupeType))) { + throw new IllegalArgumentException("Please provide valid dedupe type!") + } + val basePath = getBasePath(tableName) + + Try { + val job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, spark.sqlContext, + FSUtils.getFs(basePath, jsc.hadoopConfiguration), DeDupeType.withName(dedupeType)) + job.fixDuplicates(dryRun) + } match { + case Success(_) => + if (dryRun){ + Seq(Row(s"Deduplicated files placed in: $repairedOutputPath.")) + } else { + Seq(Row(s"Deduplicated files placed in: $duplicatedPartitionPath.")) + } + case Failure(e) => + throw new HoodieException(s"Deduplication failed!", e) + } + } + override def build: Procedure = new RepairDeduplicateProcedure() +} + +object RepairDeduplicateProcedure { + val NAME = "repair_deduplicate" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairDeduplicateProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala new file mode 100644 index 0000000000000..7daacb2f184c8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala @@ -0,0 +1,112 @@ +/* + * 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.command.procedures + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.engine.HoodieLocalEngineContext +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodiePartitionMetadata +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.util.Option +import org.apache.hudi.exception.HoodieIOException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.io.IOException +import java.util +import java.util.Properties +import java.util.function.{Consumer, Supplier} +import scala.collection.JavaConversions._ + +class RepairMigratePartitionMetaProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "dry_run", DataTypes.BooleanType, true) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("partition_path", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("text_metafile_present", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("base_metafile_present", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("action", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val dryRun = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[Boolean] + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + val engineContext: HoodieLocalEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf) + val partitionPaths: util.List[String] = FSUtils.getAllPartitionPaths(engineContext, tablePath, false, false) + val basePath: Path = new Path(tablePath) + + val rows = new util.ArrayList[Row](partitionPaths.size) + for (partitionPath <- partitionPaths) { + val partition: Path = FSUtils.getPartitionPath(tablePath, partitionPath) + val textFormatFile: Option[Path] = HoodiePartitionMetadata.textFormatMetaPathIfExists(metaClient.getFs, partition) + val baseFormatFile: Option[Path] = HoodiePartitionMetadata.baseFormatMetaPathIfExists(metaClient.getFs, partition) + val latestCommit: String = metaClient.getActiveTimeline.getCommitTimeline.lastInstant.get.getTimestamp + var action = if (textFormatFile.isPresent) "MIGRATE" else "NONE" + if (!dryRun) { + if (!baseFormatFile.isPresent) { + val partitionMetadata: HoodiePartitionMetadata = new HoodiePartitionMetadata(metaClient.getFs, latestCommit, + basePath, partition, Option.of(metaClient.getTableConfig.getBaseFileFormat)) + partitionMetadata.trySave(0) + } + // delete it, in case we failed midway last time. + textFormatFile.ifPresent( + new Consumer[Path] { + override def accept(p: Path): Unit = { + try metaClient.getFs.delete(p, false) + catch { + case e: IOException => + throw new HoodieIOException(e.getMessage, e) + } + } + }) + action = "MIGRATED" + } + rows.add(Row(partitionPath, String.valueOf(textFormatFile.isPresent), + String.valueOf(baseFormatFile.isPresent), action)) + } + val props: Properties = new Properties + props.setProperty(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key, "true") + HoodieTableConfig.update(metaClient.getFs, new Path(metaClient.getMetaPath), props) + + rows.stream().toArray().map(r => r.asInstanceOf[Row]).toList + } + + override def build: Procedure = new RepairMigratePartitionMetaProcedure() +} + +object RepairMigratePartitionMetaProcedure { + val NAME = "repair_migrate_partition_meta" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairMigratePartitionMetaProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala new file mode 100644 index 0000000000000..043217cf2df4b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairOverwriteHoodiePropsProcedure.scala @@ -0,0 +1,89 @@ +/* + * 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.command.procedures + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.io.FileInputStream +import java.util +import java.util.Properties +import java.util.function.Supplier +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.asScalaIteratorConverter + +class RepairOverwriteHoodiePropsProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "new_props_file_path", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("property", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("old_value", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("new_value", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val overwriteFilePath = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val tablePath = getBasePath(tableName) + + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(tablePath).build + + var newProps = new Properties + newProps.load(new FileInputStream(overwriteFilePath)) + val oldProps = metaClient.getTableConfig.propsMap + val metaPathDir = new Path(tablePath, METAFOLDER_NAME) + HoodieTableConfig.create(metaClient.getFs, metaPathDir, newProps) + // reload new props as checksum would have been added + newProps = HoodieTableMetaClient.reload(metaClient).getTableConfig.getProps + + val allPropKeys = new util.TreeSet[String] + allPropKeys.addAll(newProps.keySet.stream.iterator().asScala.map(key => key.toString).toList) + allPropKeys.addAll(oldProps.keySet) + + val rows = new util.ArrayList[Row](allPropKeys.size) + for (propKey <- allPropKeys) { + rows.add(Row(propKey, oldProps.getOrDefault(propKey, "null"), + newProps.getOrDefault(propKey, "null").toString)) + } + + rows.stream().toArray().map(r => r.asInstanceOf[Row]).toList + } + + override def build: Procedure = new RepairOverwriteHoodiePropsProcedure() +} + +object RepairOverwriteHoodiePropsProcedure { + val NAME = "repair_overwrite_hoodie_props" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RepairOverwriteHoodiePropsProcedure() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala index 792d26b184479..9eeef164a1db3 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/UpgradeOrDowngradeProcedure.scala @@ -22,7 +22,7 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion import org.apache.hudi.common.table.{HoodieTableMetaClient, HoodieTableVersion} import org.apache.hudi.common.util.Option -import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieWriteConfig} +import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig, HoodieCleanConfig} import org.apache.hudi.index.HoodieIndex import org.apache.hudi.table.upgrade.{SparkUpgradeDowngradeHelper, UpgradeDowngrade} import org.apache.spark.internal.Logging @@ -82,7 +82,7 @@ class UpgradeOrDowngradeProcedure extends BaseProcedure with ProcedureBuilder wi HoodieWriteConfig.newBuilder .withPath(basePath) .withRollbackUsingMarkers(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build) + .withCleanConfig(HoodieCleanConfig.newBuilder.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build) .withIndexConfig(HoodieIndexConfig.newBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build) .build } diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties new file mode 100644 index 0000000000000..d74c0444a572c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties @@ -0,0 +1,21 @@ +### +# 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. +### +hoodie.table.name=test_table +hoodie.table.type=COPY_ON_WRITE +hoodie.archivelog.folder=archive +hoodie.timeline.layout.version=1 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala index e71973f94a164..01a81976cf751 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala @@ -199,9 +199,7 @@ class TestHoodieSparkUtils { fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema") } catch { case e: Exception => - val cause = e.getCause - assertTrue(cause.isInstanceOf[SchemaCompatibilityException]) - assertTrue(e.getMessage.contains("Unable to validate the rewritten record {\"innerKey\": \"innerKey1_2\", \"innerValue\": 2} against schema")) + assertTrue(e.getMessage.contains("null of string in field new_nested_col of test_namespace.test_struct_name.nullableInnerStruct of union")) } spark.stop() } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala index 15fed579bba41..b64d386f1fb4a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala @@ -19,10 +19,13 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path import org.apache.hudi.common.model.HoodieRecord -import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} +import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.common.testutils.RawTripTestPayload +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} +import org.apache.spark.sql.functions.{arrays_zip, col} +import org.apache.spark.sql.{Row, SaveMode, SparkSession} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -460,4 +463,65 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } } } + + test("Test schema auto evolution") { + withTempDir { tmp => + Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType => + val tableName = generateTableName + val tablePath = s"${new Path(tmp.getCanonicalPath, tableName).toUri.toString}" + if (HoodieSparkUtils.gteqSpark3_1) { + + val dataGen = new HoodieTestDataGenerator + val schema = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA + val records1 = RawTripTestPayload.recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 1000, schema)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + // drop tip_history.element.amount, city_to_state, distance_in_meters, drivers + val orgStringDf = inputDF1.drop("city_to_state", "distance_in_meters", "drivers") + .withColumn("tip_history", arrays_zip(col("tip_history.currency"))) + spark.sql("set hoodie.schema.on.read.enable=true") + + val hudiOptions = Map[String,String]( + HoodieWriteConfig.TABLE_NAME -> tableName, + DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType, + DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key", + DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition", + DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp", + "hoodie.schema.on.read.enable" -> "true", + "hoodie.datasource.write.reconcile.schema" -> "true", + DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY -> "true" + ) + + orgStringDf.write + .format("org.apache.hudi") + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .options(hudiOptions) + .mode(SaveMode.Overwrite) + .save(tablePath) + + val oldView = spark.read.format("hudi").load(tablePath) + oldView.show(false) + + val records2 = RawTripTestPayload.recordsToStrings(dataGen.generateUpdatesAsPerSchema("002", 100, schema)).toList + val inputD2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val updatedStringDf = inputD2.drop("fare").drop("height") + val checkRowKey = inputD2.select("_row_key").collectAsList().map(_.getString(0)).get(0) + + updatedStringDf.write + .format("org.apache.hudi") + .options(hudiOptions) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .option("hoodie.datasource.write.reconcile.schema", "true") + .mode(SaveMode.Append) + .save(tablePath) + spark.read.format("hudi").load(tablePath).registerTempTable("newView") + val checkResult = spark.sql(s"select tip_history.amount,city_to_state,distance_in_meters,fare,height from newView where _row_key='$checkRowKey' ") + .collect().map(row => (row.isNullAt(0), row.isNullAt(1), row.isNullAt(2), row.isNullAt(3), row.isNullAt(4))) + assertResult((false, false, false, true, true))(checkResult(0)) + checkAnswer(spark.sql(s"select fare,height from newView where _row_key='$checkRowKey'").collect())( + Seq(null, null) + ) + } + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala new file mode 100644 index 0000000000000..57025ab0b6bcd --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCopyToTableProcedure.scala @@ -0,0 +1,353 @@ +/* + * 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.procedure + +import org.apache.spark.sql.Row +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase + +import java.util + +class TestCopyToTableProcedure extends HoodieSparkSqlTestBase { + + test("Test Call copy_to_table Procedure with default params") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName')")(s"Argument: new_table is required") + + val row = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName')""").collectAsList() + assert(row.size() == 1 && row.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + } + } + + test("Test Call copy_to_table Procedure with snapshot") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + val row = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',query_type=>'snapshot')""").collectAsList() + // check exit code + assert(row.size() == 1 && row.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 5, 'a1', 10, 1000") + // mark max instanceTime.total row is 5 + val instanceTime = spark.sql(s"select max(_hoodie_commit_time) from $tableName").collectAsList().get(0).get(0) + spark.sql(s"insert into $tableName select 6, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 7, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 8, 'a1', 10, 1000") + + // check row count after twice insert + val finalTableCount = spark.sql(s"select * from $tableName").count() + assert(finalTableCount == 8) + + // check snapshot copy with mark instanceTime + val copyTableName2 = generateTableName + val row2 = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName2',query_type=>'snapshot',as_of_instant=>'$instanceTime')""").collectAsList() + // check exit code + assert(row2.size() == 1 && row2.get(0).get(0) == 0) + val df = spark.sql(s"""select * from $copyTableName2""") + assert(df.count() == 5) + + val ids: util.List[Row] = df.selectExpr("id").collectAsList() + assert(ids.containsAll(util.Arrays.asList(Row(1), Row(2), Row(3), Row(4), Row(5)))) + + } + } + + test("Test Call copy_to_table Procedure with incremental") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + + // mark beginTime + val beginTime = spark.sql(s"select max(_hoodie_commit_time) from $tableName").collectAsList().get(0).get(0) + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + val endTime = spark.sql(s"select max(_hoodie_commit_time) from $tableName").collectAsList().get(0).get(0) + + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',query_type=>'incremental')")("begin_instance_time and end_instance_time can not be null") + + //copy from tableName with begin_instance_time、end_instance_time + val copyCmd = spark.sql(s"call copy_to_table" + s"(table=>'$tableName'" + + s",new_table=>'$copyTableName'" + + s",query_type=>'incremental'" + + s",begin_instance_time=>'$beginTime'" + + s",end_instance_time=>'$endTime')").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + + val df = spark.sql(s"select * from $copyTableName") + assert(df.count() == 2) + val ids = df.selectExpr("id").collectAsList() + assert(ids.containsAll(util.Arrays.asList(Row(3), Row(4)))) + } + } + + test("Test Call copy_to_table Procedure with read_optimized") { + withTempDir { tmp => + val tableName = generateTableName + // create mor table with hoodie.compact.inline.max.delta.commits=5 + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | options ( + | type='mor', + | primaryKey = 'id', + | preCombineField = 'ts', + | hoodie.compact.inline.max.delta.commits='5', + | hoodie.compact.inline='true' + | + | ) + """.stripMargin) + + //add 4 delta commit + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"update $tableName set ts=2000 where id = 1") + spark.sql(s"update $tableName set ts=3000 where id = 1") + spark.sql(s"update $tableName set ts=4000 where id = 1") + + val copyTableName = generateTableName + + val copyCmd = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',query_type=>'read_optimized')""").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + val copyDf = spark.sql(s"select * from $copyTableName") + assert(copyDf.count() == 1) + //check ts + assert(copyDf.selectExpr("ts").collectAsList().contains(Row(1000))) + + // trigger compact (delta_commit==5) + spark.sql(s"update $tableName set ts=5000 where id = 1") + + val copyTableName2 = generateTableName + val copyCmd2 = spark.sql(s"""call copy_to_table(table=>'$tableName',new_table=>'$copyTableName2',query_type=>'read_optimized')""").collectAsList() + assert(copyCmd2.size() == 1 && copyCmd2.get(0).get(0) == 0) + val copyDf2 = spark.sql(s"select * from $copyTableName2") + assert(copyDf2.count() == 1) + //check ts + assert(copyDf2.selectExpr("ts").collectAsList().contains(Row(5000))) + } + } + + test("Test Call copy_to_table Procedure with append_mode") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName')")(s"Argument: new_table is required") + + val copyCmd = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName')").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + + // add 2 rows + spark.sql(s"insert into $tableName select 5, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 6, 'a2', 20, 1500") + + val copyCmd2 = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',save_mode=>'append')").collectAsList() + assert(copyCmd2.size() == 1 && copyCmd2.get(0).get(0) == 0) + + val df2 = spark.sql(s"""select * from $copyTableName""") + // total insert 4+6=10 rows + assert(df2.count() == 10) + val ids2 = df2.selectExpr("id").collectAsList() + assert(ids2.containsAll(util.Arrays.asList(Row(1), Row(2), Row(3), Row(4), Row(5), Row(6)))) + + } + } + + test("Test Call copy_to_table Procedure with overwrite_mode") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName')")(s"Argument: new_table is required") + + val copyCmd = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName')").collectAsList() + assert(copyCmd.size() == 1 && copyCmd.get(0).get(0) == 0) + val copyTableCount = spark.sql(s"""select count(1) from $copyTableName""").collectAsList() + assert(copyTableCount.size() == 1 && copyTableCount.get(0).get(0) == 4) + + // add 2 rows + spark.sql(s"insert into $tableName select 5, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 6, 'a2', 20, 1500") + + val copyCmd2 = spark.sql(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',save_mode=>'Overwrite')").collectAsList() + assert(copyCmd2.size() == 1 && copyCmd2.get(0).get(0) == 0) + + val df2 = spark.sql(s"""select * from $copyTableName""") + // total insert 6 rows + assert(df2.count() == 6) + val ids2 = df2.selectExpr("id").collectAsList() + assert(ids2.containsAll(util.Arrays.asList(Row(1), Row(2), Row(3), Row(4), Row(5), Row(6)))) + + } + } + + test("Test Call copy_to_table Procedure with not support mode") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert 4 rows data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + spark.sql(s"insert into $tableName select 4, 'a4', 40, 2500") + + val copyTableName = generateTableName + // Check required fields + checkExceptionContain(s"call copy_to_table(table=>'$tableName',new_table=>'$copyTableName',save_mode=>'append1')")(s"save_mode not support append1") + + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala new file mode 100644 index 0000000000000..587f7a4bdda64 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala @@ -0,0 +1,507 @@ +/* + * 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.procedure + +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieFileFormat +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieTimeline +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, SchemaTestUtil} +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase + +import java.io.IOException +import java.net.URL +import java.nio.file.{Files, Paths} +import scala.collection.JavaConverters.asScalaIteratorConverter + +class TestRepairsProcedure extends HoodieSparkSqlTestBase { + + test("Test Call repair_add_partition_meta Procedure") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // create commit instant + Files.createFile(Paths.get(tablePath, ".hoodie", "100.commit")) + + val metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + // create partition path + val partition1 = Paths.get(tablePath, "2016/03/15").toString + val partition2 = Paths.get(tablePath, "2015/03/16").toString + val partition3 = Paths.get(tablePath, "2015/03/17").toString + assertResult(metaClient.getFs.mkdirs(new Path(partition1))) {true} + assertResult(metaClient.getFs.mkdirs(new Path(partition2))) {true} + assertResult(metaClient.getFs.mkdirs(new Path(partition3))) {true} + + // default is dry run + val dryResult = spark.sql(s"""call repair_add_partition_meta(table => '$tableName')""").collect() + assertResult(3) { + dryResult.length + } + + // real run + val realRunResult = spark.sql(s"""call repair_add_partition_meta(table => '$tableName', dry_run => false)""").collect() + assertResult(3) { + realRunResult.length + } + } + } + + test("Test Call repair_overwrite_hoodie_props Procedure") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // create commit instant + val newProps: URL = this.getClass.getClassLoader.getResource("table-config.properties") + + // overwrite hoodie props + val Result = spark.sql(s"""call repair_overwrite_hoodie_props(table => '$tableName', new_props_file_path => '${newProps.getPath}')""").collect() + assertResult(15) { + Result.length + } + } + } + + test("Test Call repair_corrupted_clean_files Procedure") { + withTempDir { tmp => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + // Create four requested files + for (i <- 100 until 104) { + val timestamp = String.valueOf(i) + // Write corrupted requested Clean File + createEmptyCleanRequestedFile(tablePath, timestamp, metaClient.getHadoopConf) + } + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + // first, there are four instants + assertResult(4) { + metaClient.getActiveTimeline.filterInflightsAndRequested.getInstants.count + } + + checkAnswer(s"""call repair_corrupted_clean_files(table => '$tableName')""")(Seq(true)) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + // after clearing, there should be 0 instant + assertResult(0) { + metaClient.getActiveTimeline.filterInflightsAndRequested.getInstants.count + } + } + } + + private var duplicatedPartitionPath: String = null + private var duplicatedPartitionPathWithUpdates: String = null + private var duplicatedPartitionPathWithUpserts: String = null + private var repairedOutputPath: String = null + private var fileFormat: HoodieFileFormat = null + + test("Test Call repair_deduplicate Procedure with insert") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPath))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 3 files + assertResult(3) { + filteredStatuses.size + } + + // before deduplicate, all files contain 210 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(210){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath')""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 200 records + val fileStatus = metaClient.getFs.listStatus(new Path(repairedOutputPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString) + recordCount = getRecordCount(files) + assertResult(200){recordCount} + } + } + + test("Test Call repair_deduplicate Procedure with update") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPathWithUpdates))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 2 files + assertResult(2) { + filteredStatuses.size + } + + // before deduplicate, all files contain 110 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(110){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath', + | dedupe_type => 'update_type')""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 100 records + val fileStatus = metaClient.getFs.listStatus(new Path(repairedOutputPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString) + recordCount = getRecordCount(files) + assertResult(100){recordCount} + } + } + + test("Test Call repair_deduplicate Procedure with upsert") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPathWithUpserts))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 3 files + assertResult(3) { + filteredStatuses.size + } + + // before deduplicate, all files contain 120 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(120){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath', + | dedupe_type => 'upsert_type')""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 100 records + val fileStatus = metaClient.getFs.listStatus(new Path(repairedOutputPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString) + recordCount = getRecordCount(files) + assertResult(100){recordCount} + } + } + + test("Test Call repair_deduplicate Procedure with real") { + withTempDir { tmp => + val tableName = generateTableName + val bashPath = tmp.getCanonicalPath + val tablePath = s"$bashPath/$tableName" + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | favorite_number int, + | favorite_color string + |) using hudi + | location '$tablePath' + | tblproperties ( + | primaryKey = 'name', + | type = 'cow' + | ) + """.stripMargin) + var metaClient = HoodieTableMetaClient.builder + .setConf(new JavaSparkContext(spark.sparkContext).hadoopConfiguration()) + .setBasePath(tablePath) + .build + + generateRecords(tablePath, bashPath, metaClient) + + // reload meta client + metaClient = HoodieTableMetaClient.reload(metaClient) + + // get fs and check number of latest files + val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitTimeline.filterCompletedInstants, + metaClient.getFs.listStatus(new Path(duplicatedPartitionPath))) + val filteredStatuses = fsView.getLatestBaseFiles.iterator().asScala.map(value => value.getPath).toList + // there should be 3 files + assertResult(3) { + filteredStatuses.size + } + + // before deduplicate, all files contain 210 records + var files = filteredStatuses.toArray + var recordCount = getRecordCount(files) + assertResult(210){recordCount} + + val partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + val result = spark.sql( + s"""call repair_deduplicate(table => '$tableName', + | duplicated_partition_path => '$partitionPath', + | repaired_output_path => '$repairedOutputPath', + | dry_run => false)""".stripMargin).collect() + assertResult(1) { + result.length + } + + // after deduplicate, there are 200 records + val fileStatus = metaClient.getFs.listStatus(new Path(duplicatedPartitionPath)) + files = fileStatus.map((status: FileStatus) => status.getPath.toString).filter(p => p.endsWith(".parquet")) + recordCount = getRecordCount(files) + assertResult(200){recordCount} + } + } + + test("Test Call repair_migrate_partition_meta Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | partitioned by (ts) + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + + // default is dry run + var result = spark.sql(s"""call repair_migrate_partition_meta(table => '$tableName')""").collect() + assertResult(2) { + result.length + } + + // real run + result = spark.sql(s"""call repair_migrate_partition_meta(table => '$tableName', dry_run => false)""").collect() + assertResult(2) { + result.length + } + } + } + + private def generateRecords(tablePath: String, bashpath: String, metaClient: HoodieTableMetaClient): Unit ={ + duplicatedPartitionPath = Paths.get(tablePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).toString + duplicatedPartitionPathWithUpdates = Paths.get(tablePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH).toString + duplicatedPartitionPathWithUpserts = Paths.get(tablePath, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH).toString + repairedOutputPath = Paths.get(bashpath, "tmp").toString + + // generate 200 records + val schema: Schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema) + val testTable: HoodieSparkWriteableTestTable = HoodieSparkWriteableTestTable.of(metaClient, schema) + + val hoodieRecords1 = SchemaTestUtil.generateHoodieTestRecords(0, 100, schema) + val hoodieRecords2 = SchemaTestUtil.generateHoodieTestRecords(100, 100, schema) + testTable.addCommit("20160401010101") + .withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "1", hoodieRecords1) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "2", hoodieRecords2) + testTable.getFileIdWithLogFile(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "4", hoodieRecords1) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "6", hoodieRecords1) + + // read records and get 10 to generate duplicates + val dupRecords = hoodieRecords1.subList(0, 10) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "5", dupRecords) + testTable.addCommit("20160401010202") + .withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "3", dupRecords) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "7", dupRecords) + testTable.withInserts(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "8", dupRecords) + + fileFormat = metaClient.getTableConfig.getBaseFileFormat + } + + private def getRecordCount(files: Array[String]): Long = { + var recordCount: Long = 0 + for (file <- files){ + if (HoodieFileFormat.PARQUET == fileFormat){ + recordCount += spark.sqlContext.read.parquet(file).count() + } else if (HoodieFileFormat.ORC == fileFormat) { + recordCount += spark.sqlContext.read.orc(file).count() + } else { + throw new UnsupportedOperationException(fileFormat.name + " format not supported yet.") + } + } + recordCount + } + + @throws[IOException] + def createEmptyCleanRequestedFile(basePath: String, instantTime: String, configuration: Configuration): Unit = { + val commitFilePath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeRequestedCleanerFileName(instantTime)) + val fs = FSUtils.getFs(basePath, configuration) + val os = fs.create(commitFilePath, true) + os.close() + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index 6f2cc50a0af69..cdb192f9fedd5 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -45,7 +45,6 @@ import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; -import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; @@ -95,9 +94,6 @@ public static class HiveSyncConfigParams { + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to " + "org.apache.hudi input format.") public Boolean usePreApacheInputFormat; - @Deprecated - @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") - public Boolean useJdbc; @Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris") public String metastoreUris; @Parameter(names = {"--sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms,glue,jdbc and hiveql") @@ -142,7 +138,6 @@ public TypedProperties toProps() { props.setPropertyIfNonNull(HIVE_PASS.key(), hivePass); props.setPropertyIfNonNull(HIVE_URL.key(), jdbcUrl); props.setPropertyIfNonNull(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), usePreApacheInputFormat); - props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), useJdbc); props.setPropertyIfNonNull(HIVE_SYNC_MODE.key(), syncMode); props.setPropertyIfNonNull(METASTORE_URIS.key(), metastoreUris); props.setPropertyIfNonNull(HIVE_AUTO_CREATE_DATABASE.key(), autoCreateDatabase); diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java index 3877782c92026..8c14cdfe29beb 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java @@ -52,15 +52,6 @@ public class HiveSyncConfigHolder { .withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. " + "Use this when you are in the process of migrating from " + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format"); - /** - * @deprecated Use {@link #HIVE_SYNC_MODE} instead of this config from 0.9.0 - */ - @Deprecated - public static final ConfigProperty HIVE_USE_JDBC = ConfigProperty - .key("hoodie.datasource.hive_sync.use_jdbc") - .defaultValue("true") - .deprecatedAfter("0.9.0") - .withDocumentation("Use JDBC when hive synchronization is enabled"); public static final ConfigProperty METASTORE_URIS = ConfigProperty .key("hoodie.datasource.hive_sync.metastore.uris") .defaultValue("thrift://localhost:9083") @@ -109,7 +100,7 @@ public class HiveSyncConfigHolder { .withDocumentation("The number of partitions one batch when synchronous partitions to hive."); public static final ConfigProperty HIVE_SYNC_MODE = ConfigProperty .key("hoodie.datasource.hive_sync.mode") - .noDefaultValue() + .defaultValue("jdbc") .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql."); public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC = ConfigProperty .key("hoodie.datasource.hive_sync.bucket_sync") diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java index d5a85adcbacc2..26ba4ae8e1b50 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.ddl.DDLExecutor; import org.apache.hudi.hive.ddl.HMSDDLExecutor; @@ -49,7 +48,6 @@ import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; -import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; import static org.apache.hudi.sync.common.util.TableUtils.tableId; @@ -72,23 +70,19 @@ public HoodieHiveSyncClient(HiveSyncConfig config) { // Support JDBC, HiveQL and metastore based implementations for backwards compatibility. Future users should // disable jdbc and depend on metastore client for all hive registrations try { - if (!StringUtils.isNullOrEmpty(config.getString(HIVE_SYNC_MODE))) { - HiveSyncMode syncMode = HiveSyncMode.of(config.getString(HIVE_SYNC_MODE)); - switch (syncMode) { - case HMS: - ddlExecutor = new HMSDDLExecutor(config); - break; - case HIVEQL: - ddlExecutor = new HiveQueryDDLExecutor(config); - break; - case JDBC: - ddlExecutor = new JDBCExecutor(config); - break; - default: - throw new HoodieHiveSyncException("Invalid sync mode given " + config.getString(HIVE_SYNC_MODE)); - } - } else { - ddlExecutor = config.getBoolean(HIVE_USE_JDBC) ? new JDBCExecutor(config) : new HiveQueryDDLExecutor(config); + HiveSyncMode syncMode = HiveSyncMode.of(config.getStringOrDefault(HIVE_SYNC_MODE)); + switch (syncMode) { + case HMS: + ddlExecutor = new HMSDDLExecutor(config); + break; + case HIVEQL: + ddlExecutor = new HiveQueryDDLExecutor(config); + break; + case JDBC: + ddlExecutor = new JDBCExecutor(config); + break; + default: + throw new HoodieHiveSyncException("Invalid sync mode given " + config.getString(HIVE_SYNC_MODE)); } this.client = Hive.get(config.getHiveConf()).getMSC(); } catch (Exception e) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java index 90efd2701c793..619a417c331f3 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveQueryDDLExecutor.java @@ -47,7 +47,7 @@ import static org.apache.hudi.sync.common.util.TableUtils.tableId; /** - * This class offers DDL executor backed by the hive.ql Driver This class preserves the old useJDBC = false way of doing things. + * This class offers DDL executor backed by the HiveQL Driver. */ public class HiveQueryDDLExecutor extends QueryBasedDDLExecutor { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java index 2673e46a9f745..1fc8f082d8c0f 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java @@ -43,7 +43,7 @@ import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; /** - * This class offers DDL executor backed by the jdbc This class preserves the old useJDBC = true way of doing things. + * This class offers DDL executor backed by the jdbc. */ public class JDBCExecutor extends QueryBasedDDLExecutor { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java index 58188f578e1bf..b8b2de73e7062 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java @@ -32,6 +32,7 @@ import java.io.InputStream; import java.util.Properties; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; @@ -92,6 +93,7 @@ Properties mkGlobalHiveSyncProps(boolean forRemote) { String jdbcUrl = forRemote ? loadedProps.getProperty(REMOTE_HIVE_SERVER_JDBC_URLS) : loadedProps.getProperty(LOCAL_HIVE_SERVER_JDBC_URLS, loadedProps.getProperty(HIVE_URL.key())); props.setPropertyIfNonNull(HIVE_URL.key(), jdbcUrl); + props.setProperty(HIVE_SYNC_MODE.key(), "jdbc"); LOG.info("building hivesync config forRemote: " + forRemote + " " + jdbcUrl + " " + basePath); return props; diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 072feeb663160..7abeed480c639 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -120,7 +120,7 @@ private static Iterable syncModeAndSchemaFromCommitMetadataAndManagedT return opts; } - // (useJdbc, useSchemaFromCommitMetadata, syncAsDataSource) + // (useSchemaFromCommitMetadata, syncAsDataSource, syncMode) private static Iterable syncDataSourceTableParams() { List opts = new ArrayList<>(); for (Object mode : SYNC_MODES) { diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java index ba763ddc14bb7..429bb93aa3284 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -37,6 +36,10 @@ import java.util.Properties; import java.util.function.Function; +import static org.apache.hudi.common.table.HoodieTableConfig.DATABASE_NAME; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_NAME_KEY; +import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY; + /** * Configs needed to sync data into external meta stores, catalogs, etc. */ @@ -56,22 +59,14 @@ public class HoodieSyncConfig extends HoodieConfig { public static final ConfigProperty META_SYNC_DATABASE_NAME = ConfigProperty .key("hoodie.datasource.hive_sync.database") .defaultValue("default") + .withInferFunction(cfg -> Option.ofNullable(cfg.getString(DATABASE_NAME))) .withDocumentation("The name of the destination database that we should sync the hudi table to."); - // If the table name for the metastore destination is not provided, pick it up from write or table configs. - public static final Function> TABLE_NAME_INFERENCE_FUNCTION = cfg -> { - if (cfg.contains(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)) { - return Option.of(cfg.getString(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)); - } else if (cfg.contains(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)) { - return Option.of(cfg.getString(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)); - } else { - return Option.empty(); - } - }; public static final ConfigProperty META_SYNC_TABLE_NAME = ConfigProperty .key("hoodie.datasource.hive_sync.table") .defaultValue("unknown") - .withInferFunction(TABLE_NAME_INFERENCE_FUNCTION) + .withInferFunction(cfg -> Option.ofNullable(cfg.getString(HOODIE_WRITE_TABLE_NAME_KEY)) + .or(() -> Option.ofNullable(cfg.getString(HOODIE_TABLE_NAME_KEY)))) .withDocumentation("The name of the destination table that we should sync the hudi table to."); public static final ConfigProperty META_SYNC_BASE_FILE_FORMAT = ConfigProperty @@ -148,6 +143,7 @@ public HoodieSyncConfig(Properties props) { public HoodieSyncConfig(Properties props, Configuration hadoopConf) { super(props); + setDefaults(getClass().getName()); this.hadoopConf = hadoopConf; } @@ -173,9 +169,9 @@ public String toString() { } public static class HoodieSyncConfigParams { - @Parameter(names = {"--database"}, description = "name of the target database in meta store", required = true) + @Parameter(names = {"--database"}, description = "name of the target database in meta store") public String databaseName; - @Parameter(names = {"--table"}, description = "name of the target table in meta store", required = true) + @Parameter(names = {"--table"}, description = "name of the target table in meta store") public String tableName; @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) public String basePath; diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java index c5b98c17eb4a1..3edec8314b1f9 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java @@ -18,12 +18,21 @@ package org.apache.hudi.sync.common.util; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.parquet.schema.OriginalType.UTF8; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; /** @@ -33,6 +42,68 @@ */ public class Parquet2SparkSchemaUtils { + /** + * Get Spark Sql related table properties. This is used for spark datasource table. + * @param schema The schema to write to the table. + * @return A new parameters added the spark's table properties. + */ + public static Map getSparkTableProperties(List partitionNames, String sparkVersion, + int schemaLengthThreshold, MessageType schema) { + // Convert the schema and partition info used by spark sql to hive table properties. + // The following code refers to the spark code in + // https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala + GroupType originGroupType = schema.asGroupType(); + List partitionCols = new ArrayList<>(); + List dataCols = new ArrayList<>(); + Map column2Field = new HashMap<>(); + + for (Type field : originGroupType.getFields()) { + column2Field.put(field.getName(), field); + } + // Get partition columns and data columns. + for (String partitionName : partitionNames) { + // Default the unknown partition fields to be String. + // Keep the same logical with HiveSchemaUtil#getPartitionKeyType. + partitionCols.add(column2Field.getOrDefault(partitionName, + new PrimitiveType(Type.Repetition.REQUIRED, BINARY, partitionName, UTF8))); + } + + for (Type field : originGroupType.getFields()) { + if (!partitionNames.contains(field.getName())) { + dataCols.add(field); + } + } + + List reOrderedFields = new ArrayList<>(); + reOrderedFields.addAll(dataCols); + reOrderedFields.addAll(partitionCols); + GroupType reOrderedType = new GroupType(originGroupType.getRepetition(), originGroupType.getName(), reOrderedFields); + + Map sparkProperties = new HashMap<>(); + sparkProperties.put("spark.sql.sources.provider", "hudi"); + if (!StringUtils.isNullOrEmpty(sparkVersion)) { + sparkProperties.put("spark.sql.create.version", sparkVersion); + } + // Split the schema string to multi-parts according the schemaLengthThreshold size. + String schemaString = Parquet2SparkSchemaUtils.convertToSparkSchemaJson(reOrderedType); + int numSchemaPart = (schemaString.length() + schemaLengthThreshold - 1) / schemaLengthThreshold; + sparkProperties.put("spark.sql.sources.schema.numParts", String.valueOf(numSchemaPart)); + // Add each part of schema string to sparkProperties + for (int i = 0; i < numSchemaPart; i++) { + int start = i * schemaLengthThreshold; + int end = Math.min(start + schemaLengthThreshold, schemaString.length()); + sparkProperties.put("spark.sql.sources.schema.part." + i, schemaString.substring(start, end)); + } + // Add partition columns + if (!partitionNames.isEmpty()) { + sparkProperties.put("spark.sql.sources.schema.numPartCols", String.valueOf(partitionNames.size())); + for (int i = 0; i < partitionNames.size(); i++) { + sparkProperties.put("spark.sql.sources.schema.partCol." + i, partitionNames.get(i)); + } + } + return sparkProperties; + } + public static String convertToSparkSchemaJson(GroupType parquetSchema) { String fieldsJsonString = parquetSchema.getFields().stream().map(field -> { switch (field.getRepetition()) { diff --git a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java new file mode 100644 index 0000000000000..1f6c05cd12423 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/TestHoodieSyncConfig.java @@ -0,0 +1,55 @@ +/* + * 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.sync.common; + +import org.apache.hudi.common.table.HoodieTableConfig; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; + +import java.util.Properties; + +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; +import static org.junit.jupiter.api.Assertions.assertEquals; + +class TestHoodieSyncConfig { + + @Test + void testInferDatabaseAndTableNames() { + Properties props1 = new Properties(); + props1.setProperty(HoodieTableConfig.DATABASE_NAME.key(), "db1"); + props1.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_KEY, "tbl1"); + HoodieSyncConfig config1 = new HoodieSyncConfig(props1, new Configuration()); + assertEquals("db1", config1.getString(META_SYNC_DATABASE_NAME)); + assertEquals("tbl1", config1.getString(META_SYNC_TABLE_NAME)); + + Properties props2 = new Properties(); + props2.setProperty(HoodieTableConfig.DATABASE_NAME.key(), "db2"); + props2.setProperty(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY, "tbl2"); + HoodieSyncConfig config2 = new HoodieSyncConfig(props2, new Configuration()); + assertEquals("db2", config2.getString(META_SYNC_DATABASE_NAME)); + assertEquals("tbl2", config2.getString(META_SYNC_TABLE_NAME)); + + HoodieSyncConfig config3 = new HoodieSyncConfig(new Properties(), new Configuration()); + assertEquals("default", config3.getString(META_SYNC_DATABASE_NAME)); + assertEquals("unknown", config3.getString(META_SYNC_TABLE_NAME)); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java index 95e84e413cd57..fc16bcaa1bc69 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java @@ -174,8 +174,6 @@ public static class Config implements Serializable { public String hiveURL = "jdbc:hive2://localhost:10000"; @Parameter(names = {"--hive-partition-field"}, description = "Comma separated list of field in the hive table to use for determining hive partition columns.", required = false) public String hivePartitionsField = ""; - @Parameter(names = {"--hive-sync-use-jdbc"}, description = "Use JDBC when hive synchronization.", required = false) - public boolean hiveUseJdbc = true; @Parameter(names = {"--hive-metastore-uris"}, description = "hive meta store uris to use.", required = false) public String hiveHMSUris = null; @Parameter(names = {"--hive-sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.", required = false) @@ -215,7 +213,7 @@ public String toString() { + " --hive-pass-word " + "Masked" + ", \n" + " --hive-jdbc-url " + hiveURL + ", \n" + " --hive-partition-field " + hivePartitionsField + ", \n" - + " --hive-sync-use-jdbc " + hiveUseJdbc + ", \n" + + " --hive-sync-mode " + hiveSyncMode + ", \n" + " --hive-metastore-uris " + hiveHMSUris + ", \n" + " --hive-sync-ignore-exception " + hiveSyncIgnoreException + ", \n" + " --hive-partition-value-extractor-class " + partitionValueExtractorClass + ", \n" @@ -247,7 +245,7 @@ public boolean equals(Object o) { && Objects.equals(hivePassWord, config.hivePassWord) && Objects.equals(hiveURL, config.hiveURL) && Objects.equals(hivePartitionsField, config.hivePartitionsField) - && Objects.equals(hiveUseJdbc, config.hiveUseJdbc) + && Objects.equals(hiveSyncMode, config.hiveSyncMode) && Objects.equals(hiveHMSUris, config.hiveHMSUris) && Objects.equals(partitionValueExtractorClass, config.partitionValueExtractorClass) && Objects.equals(sparkMaster, config.sparkMaster) @@ -261,7 +259,7 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(basePath, runningMode, tableName, partitions, instantTime, syncToHive, hiveDataBase, hiveTableName, hiveUserName, hivePassWord, hiveURL, - hivePartitionsField, hiveUseJdbc, hiveHMSUris, partitionValueExtractorClass, + hivePartitionsField, hiveSyncMode, hiveHMSUris, partitionValueExtractorClass, sparkMaster, sparkMemory, propsFilePath, configs, hiveSyncIgnoreException, help); } } @@ -350,7 +348,6 @@ private HiveSyncConfig buildHiveSyncProps() { props.put(DataSourceWriteOptions.HIVE_PASS().key(), cfg.hivePassWord); props.put(DataSourceWriteOptions.HIVE_URL().key(), cfg.hiveURL); props.put(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), cfg.hivePartitionsField); - props.put(DataSourceWriteOptions.HIVE_USE_JDBC().key(), cfg.hiveUseJdbc); props.put(DataSourceWriteOptions.HIVE_SYNC_MODE().key(), cfg.hiveSyncMode); props.put(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().key(), cfg.hiveSyncIgnoreException); props.put(DataSourceWriteOptions.HIVE_PASS().key(), cfg.hivePassWord); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 5940ab1d46f25..badd3ab6275c7 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -783,12 +783,12 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { .combineInput(cfg.filterDupes, combineBeforeUpsert) .withCompactionConfig( HoodieCompactionConfig.newBuilder() - .withPayloadClass(cfg.payloadClassName) .withInlineCompaction(cfg.isInlineCompactionEnabled()) .build() ) .withPayloadConfig( HoodiePayloadConfig.newBuilder() + .withPayloadClass(cfg.payloadClassName) .withPayloadOrderingField(cfg.sourceOrderingField) .build()) .forTable(cfg.targetTableName) diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 45d8a427c0436..dde0e5f73fc4d 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -46,6 +46,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -766,7 +768,7 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir } cfg.tableType = tableType.name(); cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfg.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); deltaStreamerTestRunner(ds, cfg, (r) -> { if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { @@ -946,16 +948,16 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws // Step 4 : Insert 1 record and trigger sync/async cleaner and archive. List configs = getAsyncServicesConfigs(1, "true", "true", "2", "", ""); - configs.add(String.format("%s=%s", HoodieCompactionConfig.CLEANER_POLICY.key(), "KEEP_LATEST_COMMITS")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.CLEANER_COMMITS_RETAINED.key(), "1")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "3")); - configs.add(String.format("%s=%s", HoodieCompactionConfig.ASYNC_CLEAN.key(), asyncClean)); + configs.add(String.format("%s=%s", HoodieCleanConfig.CLEANER_POLICY.key(), "KEEP_LATEST_COMMITS")); + configs.add(String.format("%s=%s", HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1")); + configs.add(String.format("%s=%s", HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2")); + configs.add(String.format("%s=%s", HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "3")); + configs.add(String.format("%s=%s", HoodieCleanConfig.ASYNC_CLEAN.key(), asyncClean)); configs.add(String.format("%s=%s", HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key(), "1")); if (asyncClean) { configs.add(String.format("%s=%s", HoodieWriteConfig.WRITE_CONCURRENCY_MODE.key(), WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())); - configs.add(String.format("%s=%s", HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + configs.add(String.format("%s=%s", HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name())); configs.add(String.format("%s=%s", HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), InProcessLockProvider.class.getName())); @@ -987,7 +989,7 @@ private List getAsyncServicesConfigs(int totalRecords, String autoClean, List configs = new ArrayList<>(); configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); if (!StringUtils.isNullOrEmpty(autoClean)) { - configs.add(String.format("%s=%s", HoodieCompactionConfig.AUTO_CLEAN.key(), autoClean)); + configs.add(String.format("%s=%s", HoodieCleanConfig.AUTO_CLEAN.key(), autoClean)); } if (!StringUtils.isNullOrEmpty(inlineCluster)) { configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING.key(), inlineCluster)); @@ -1462,7 +1464,7 @@ public void testFilterDupes() throws Exception { cfg2.filterDupes = false; cfg2.sourceLimit = 2000; cfg2.operation = WriteOperationType.UPSERT; - cfg2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfg2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg2, jsc); ds2.sync(); mClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java index 13f5ad97cfc92..abe1994a3dd87 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamerWithMultiWriter.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; @@ -95,7 +95,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); prepJobConfig.continuousMode = true; prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + prepJobConfig.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc()); // Prepare base dataset with some commits @@ -115,7 +115,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); cfgIngestionJob.continuousMode = true; cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgIngestionJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); // create a backfill job HoodieDeltaStreamer.Config cfgBackfillJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, @@ -127,7 +127,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc()); // re-init ingestion job to start sync service @@ -157,7 +157,7 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); prepJobConfig.continuousMode = true; prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + prepJobConfig.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc()); // Prepare base dataset with some commits @@ -188,13 +188,13 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob2.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); cfgBackfillJob2.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer.Config cfgIngestionJob2 = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT, propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TestIdentityTransformer.class.getName())); cfgIngestionJob2.continuousMode = true; cfgIngestionJob2.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgIngestionJob2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgIngestionJob2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); // re-init ingestion job HoodieDeltaStreamer ingestionJob3 = new HoodieDeltaStreamer(cfgIngestionJob2, jsc()); // re-init backfill job @@ -225,7 +225,7 @@ void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName())); prepJobConfig.continuousMode = true; prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + prepJobConfig.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc()); // Prepare base dataset with some commits @@ -263,7 +263,7 @@ void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) // Set checkpoint to the last successful position cfgBackfillJob.checkpoint = commitMetadataForLastInstant.getMetadata(CHECKPOINT_KEY); cfgBackfillJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); - cfgBackfillJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); + cfgBackfillJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); HoodieDeltaStreamer backfillJob = new HoodieDeltaStreamer(cfgBackfillJob, jsc()); backfillJob.sync(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java index fa5cba446f928..57270bdf812d3 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java @@ -28,7 +28,8 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -64,8 +65,8 @@ public void setUp() throws IOException { @Test public void testHoodieIncrSource() throws IOException { HoodieWriteConfig writeConfig = getConfigBuilder(basePath(), metaClient) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .archiveCommitsWith(2, 3).retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1).build()) .build(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 8e7bce944f91a..67a002c3bac79 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -124,11 +124,14 @@ public static void setLogLevel() { } public static void initTestServices(boolean needsHive, boolean needsZookeeper) throws Exception { - hdfsTestService = new HdfsTestService(); - dfsCluster = hdfsTestService.start(true); - dfs = dfsCluster.getFileSystem(); - dfsBasePath = dfs.getWorkingDirectory().toString(); - dfs.mkdirs(new Path(dfsBasePath)); + + if (hdfsTestService == null) { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + dfs = dfsCluster.getFileSystem(); + dfsBasePath = dfs.getWorkingDirectory().toString(); + dfs.mkdirs(new Path(dfsBasePath)); + } if (needsHive) { hiveTestService = new HiveTestService(hdfsTestService.getHadoopConf()); hiveServer = hiveTestService.start(); diff --git a/pom.xml b/pom.xml index 7ff6add7b189c..9c5fe24aabe0c 100644 --- a/pom.xml +++ b/pom.xml @@ -728,7 +728,7 @@ xalan xalan - 2.7.1 + 2.7.2 diff --git a/rfc/README.md b/rfc/README.md index 3d33b82c29925..f567d2beb16cd 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -66,12 +66,12 @@ The list of all RFCs can be found here. | 28 | [Support Z-order curve](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=181307144) | `COMPLETED` | | 29 | [Hash Index](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+29%3A+Hash+Index) | `ONGOING` | | 30 | [Batch operation](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+30%3A+Batch+operation) | `UNDER REVIEW` | -| 31 | [Hive integration Improvement](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+31%3A+Hive+integration+Improvment)| `UNDER REVIEW` | +| 31 | [Hive integration Improvement](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+31%3A+Hive+integration+Improvment)| `ONGOING` | | 32 | [Kafka Connect Sink for Hudi](https://cwiki.apache.org/confluence/display/HUDI/RFC-32+Kafka+Connect+Sink+for+Hudi)| `ONGOING` | | 33 | [Hudi supports more comprehensive Schema Evolution](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+33++Hudi+supports+more+comprehensive+Schema+Evolution)| `ONGOING` | | 34 | [Hudi BigQuery Integration](./rfc-34/rfc-34.md) | `COMPLETED` | | 35 | [Make Flink MOR table writing streaming friendly](https://cwiki.apache.org/confluence/display/HUDI/RFC-35%3A+Make+Flink+MOR+table+writing+streaming+friendly)| `UNDER REVIEW` | -| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server)| `UNDER REVIEW` | +| 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server)| `IN PROGRESS` | | 37 | [Hudi Metadata based Bloom Index](rfc-37/rfc-37.md) | `ONGOING` | | 38 | [Spark Datasource V2 Integration](./rfc-38/rfc-38.md) | `IN PROGRESS` | | 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `ONGOING` | @@ -79,16 +79,16 @@ The list of all RFCs can be found here. | 41 | [Hudi Snowflake Integration] | `UNDER REVIEW`| | 42 | [Consistent Hashing Index](./rfc-42/rfc-42.md) | `IN PROGRESS` | | 43 | [Compaction / Clustering Service](./rfc-43/rfc-43.md) | `UNDER REVIEW` | -| 44 | [Hudi Connector for Presto](./rfc-44/rfc-44.md) | `IN PROGRESS` | +| 44 | [Hudi Connector for Presto](./rfc-44/rfc-44.md) | `ONGOING` | | 45 | [Asynchronous Metadata Indexing](./rfc-45/rfc-45.md) | `ONGOING` | -| 46 | [Optimizing Record Payload Handling](./rfc-46/rfc-46.md) | `UNDER REVIEW` | -| 47 | [Add Call Produce Command for Spark SQL](./rfc-47/rfc-47.md) | `UNDER REVIEW` | +| 46 | [Optimizing Record Payload Handling](./rfc-46/rfc-46.md) | `IN PROGRESS` | +| 47 | [Add Call Produce Command for Spark SQL](./rfc-47/rfc-47.md) | `ONGOING` | | 48 | [LogCompaction for MOR tables](./rfc-48/rfc-48.md) | `UNDER REVIEW` | | 49 | [Support sync with DataHub](./rfc-49/rfc-49.md) | `ONGOING` | -| 50 | [Improve Timeline Server](./rfc-50/rfc-50.md) | `UNDER REVIEW` | +| 50 | [Improve Timeline Server](./rfc-50/rfc-50.md) | `IN PROGRESS` | | 51 | [Change Data Capture](./rfc-51/rfc-51.md) | `UNDER REVIEW` | | 52 | [Introduce Secondary Index to Improve HUDI Query Performance](./rfc-52/rfc-52.md) | `UNDER REVIEW` | -| 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `UNDER REVIEW` | +| 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `IN PROGRESS` | | 54 | [New Table APIs and Streamline Hudi Configs](./rfc-54/rfc-54.md) | `UNDER REVIEW` | -| 55 | [Improve Hive/Meta sync class design and hierachies](./rfc-55/rfc-55.md) | `UNDER REVIEW` | +| 55 | [Improve Hive/Meta sync class design and hierachies](./rfc-55/rfc-55.md) | `ONGOING` | | 56 | [Early Conflict Detection For Multi-Writer](./rfc-56/rfc-56.md) | `UNDER REVIEW` | diff --git a/rfc/rfc-55/hudi-sync-class-diagram.png b/rfc/rfc-55/hudi-sync-class-diagram.png new file mode 100644 index 0000000000000000000000000000000000000000..11acf4bffcbc619a684b6052a2ab2ee29b839bb1 GIT binary patch literal 418052 zcmeFYWmuG5*9MHBf<+^uw9+LdJ+yR4E8POpB{4GyDj}uBFr-pLch@K_4MRu{HFO9= z=Xc@#+|TPk!A;-^y=en04o(1$lAN^8Ta%6HU>K)9nC165k=LPF4CnY0ONOVMKYr1Fzm8`T z>DaKY+KwMh{r!5eRDjZRlJG@&#qW#^?>VKOe3Sbk^)i51kpw8X zAPL$zhvz%=l-aUkwf_BW&2O6Ku>IMIED)W2SGSl$Rcf+Q6iw{I6ASGJ9xmUfo%pXI z{vJ?RVB<;o>+lcWhc$$1xO_MRD!ieuR^{@hY?%%nW)Lq_rZWFy5Ma2G$u33$GE-sh zk&F4=*YaC8flzNom3Rv?I#SQi{+~0_QqFCQv(cK4K!#_@Ld95HrWxq*a5!aEs~(aS zAG}xlujy+kPnUO@Cy<8Q1zX-P?3||m^HzR1wP4(R(jHzBhsaX#__phgxvn`q=8tl-%V#X4l1i+Is-m6mYMfM$eBzdkg>9h-_O9z>2?{EG@6w z{pwMDZ)P7P{oZ7ENC3vT$R|X-{=M4_QZgpzWGeVzX6*m8-UCZztShSxC~NiShah(B zNZ^l>7nf7NHG9o8={jy<5(pP=0lLl)p7hD?mket*RRTPvFz6|N5J6j}fSA7il%;P~ zlcQ1ZbEf~c8yf{hhsL~1kC9R60(ZHm8FR43e2ua{LL{E3IQWMdH(kriqc8lYkFRW$ zJ1)V@Y()~z2D8mm8aEPDR~xewyj0dV#TKvLF>PQ3&xiA5#%PGJr0n+FZd#t{h#oKY z>%jhBnEL|=-c+p!g#QZJsw~}^!3h$P{KEL+?a44pzly2I<@)Mceo_CogdrsxdpAvQ z7}ZT{1KO+I=gG2Eg&Rb?+Lsr$$tG-*0}OiE#6YvJ@2nC9+0#a$&!_!{UMSGSsEh9` ztkx{3Pimx?j3~tY4|uPB8_rSJ=xDmU%#KkP7Nj|iVnGlkzqJ=VPE*%_p6ffSJ3~;s zs{^vM@JP|!d#g)H7mf^^DB@|WUy{Ha=to!-!-!QS;`RK?%iTSNnePL0siR$33uOTd zOPq@ED4L#6P}p_!{iII#Y50KKv_bxV`P^R)u>k5g_6_IZf$>H*wo<|tPBz{sQ=07U z?@cXVIGudQzx44)bMt2oI|t-lzzhc}J0UR6&frM|WuLx?WP?bbgU_ksI%@l@@6(VH zov=vSE4ESYg~eerkC*=odxZtcvrLh-0?s@gri|p+AJ7vQ-*)~qm{UC%Yy$OtkZyhn z=;^PpgDxf^`yKMUYSU<^reg&=ZoLT1Ncfu~BPzDBLqXg>>RWS> zxaaH*t?kWyVf}M4{zsSGg#B{f@U+Xb1CLgceiiSj(%dc4#|-@OfS2rPi4l-ZTcoc| zDjz0@Q*j9Cd#*4&aW%E9v$mx+1+z7-3TNX1q~)_K9ueb(=f!l~!KP2^szohH^UCEn{Wx_o8Ae5J44&+--yWHHXhWdG!Jd;EY*a6v$Su&|AJT;YeS}%~3 zM2oHa@xTya4X46OXVXK+850@a9i#YqFff%M+ZTCR6VSXvOJ!kZ`=W`g6G>+uebM!v zeoPTxsdD4%XT5|6I+h!sdbtRHE@nuTk^tEJii%&OC%ZM@Mq`z2yX-DybC?=yvP-G3 zGzws~MTr~-=lgAe7FH%!vL6XG5_#{f!g5!LKt{GAi3|xgN7>gc=BvsI;qart@4I-y zz;@tkY*fcSq;0)5(l-99GPI}LWst+{#rLbqNPH zdcaP_GdlE^=DEWBQ$Y#-L?`f8gUZkgsHX>E=kPLBJ+WLEDNMX_5A+sjJ?~4Jd)Q`fP z<)b3+v>gD*4d^q>3sUtLbGL~Fhg!LECQ@bYEmb$L82j%~v|wNL=~_aWm4THv7JE3^ zebiy>8ER@O4p1=^+Zeh#FISyd*3d``{Q2CpJV@2pHQA;DN(P6`m8NnOAF2CJJD|%} z1XMhQvuCOkqs+;p2c>V~tbE84?QAW0Jr~vKit3s2->Ny>dkhge-&l44Y|ib(%J%ST znXgl4azvhMW>eX)Awc%8)N`_~J9Ryn3_XqOV6WCtS0D9$P*gsJLkJB1BgT?DBQY(C zhjKVH!55ilV(n!EDm1KTI&XItXQ7*f)5KxLP&`jn%`LZ~nen zHo&0rh(`EYb_Zx{)viCaHd(lc<;6wFT^yVO)pA+YTg|Qe2#HO*)x{VM+8fr&F3hp6 zA``}Tsi#xR*<|X%45py!{G#+w>)G*^pK z!R{$q8|B5QzMK&k)MllzS0-NGiGuq3E{maSU#Dc%4aZ873sqMSufh`(C&yuB>iiGL zt1Iceb2ssDa5z~zoDcN(AqpE1@ulCkSrLzH*9>}Bj{05VK#*H19@A9!m`h7jad0_W(yk)OjrMGkwL62JYW)7Q#z&oEzRmHqE9&jO# z=0BmVVaD;bO;1!KX(qXs%V%rVaq1M*cbjR)Sau)CfHci7BdOC^9MWiM|_A=-SbHspo57;sd z4iUCh;PLIdrnS(WR#dnyya5%JAN}fkUHPrXG(V_ks9q(75zr120-|k0&G$YrgE9#D z$Vj4nEG&0qNP$AnY-(6Qk9#UMd9Nf`g0u0cjYplxr&D$am3mMyxGQ;%l2qe?9(Q9g z1hH}@i>n$?U*E~enRw?<>$WMLdQE6cT5Q5zAY}u+#K~ZkdN8f;Yu|pqnV29h1v;>Y z?`V0d;45gZtD76&`GNDLm+w4TT7t<=n0_Yuq!6u~b~23k1NyyR9U!m4It5xED-&8m zv5k_w4O>YuiFuMJMd)vQXf)&)1{IOiTeWdN-K<7Djud8;M4~Ux(fvPCgqn6Dj++ty_A=ahh-HE9vaH4x`(x6bcuBs7I&m`8 z^@^{(i0WG3n5yMR-=DUi{T+ax>D{<)e0-8LvzNWsV+(RGj^x64sLkb@p40?n!K<966JZLraD2(5 zAFGjKTb)sdAB7qif}u^9=i5>0coa0&7L%uIHURcZs@?o;&=i;4Bhogj0je8xZ4kX- zt?qgyz`cBP)bGM{T${`X&um@s;VhMuzz3!{80oPSV@+Nj=c9(sYDC}_S(GgZ3#X<( z#!6mdsUf#8Z%30Ta z){8zz7Q`c=K4!77lCNu+Dg@N;PRt18kum`@u6{|M-vHXre=%_*6Ebtssr} zmTSd|hZy1LDN&~$;KrYo35f!_8)wRok=}jEwF9%oPVz7`@+q*T&6G{i zcx=!;1t<`MzrezAIZD;v8z|$$SMHP17&_Hr;u{JIC{v7Qj{Cfqx=EaT+WAvoNSa|D z(;X{{0>0{&0y$aDm~?K~Y2WLBo^|;6A&--gaL8^<+Tx^!=xXMf_(H7W;NX&Hv7a%d zU`TrDXd$W}wqTo(kkHwGne5)DRPVK^98B$|cySz`9s~pf7w0hJ@VcNLRq%CIx{FXA zRh|*1Y>(*N7JdMqQnrN!K|L{(pMWU5ML{ASilZD7pp*<*p@i;Gy8mWsUHzPTw#X7S z8l}x6AHXEq`CfScg~uBaY!5$npI~IlrR832x5~@q?aPa8RC(%t!Qd`Iwwz#n>wCBJ zdi1#yXj6NNC%$bpZJxJlH7(si&GnagFlO1bsuX;B;IEwR-Y+3!dRb|DTDg}_+4#C* zlVJS4Ti2CewQXJK$6Hn}@&-#jcAOGOsV*6iw#hw@vcZQp5|VlT(U$`<`0zOoM34V- z{q-!eKYMccji6HwKc4gf;DbcI13^Q}pu<}K$s3eL(X>b3@t2%u z-(m=o&rXUjeYe=j4r|x)@`td1l%>}4Ijl#gRQv1=DlBVra*dQ`C9kajB&mW(c2hn9 z2q{tQr+`KZ`Qd4~rwyZFUu$zN1nR&R?9T-!T9lUic9mY&AQluIaSu6-i_JzGjTSzGkjy$W;f`k!?eb|=JRWW90Pb8_Cuit;G5cuEn69rW1QwRbM0!#9CVLro1-ACLoKZDfAS! z-&K|PE$%JhWR+t#LN82RU{h0re?^-cwkeRNacf6l8;R@_1$Lf;qW;mKG&3$=>GBJF zE4=~fY<&E{Kk3**{uDP?gZ)e;{HriIGe{U$Ay)tXK(zO(%Bk;IzkX|)eG*h>FH=m91%%vPWy8>G1FEAiL{z~{h zeun7@Gq-;Iiy7G8#n@(+tvp{zEc)p3mb=xx*3&l^snigzW%7EJdj02YnKYi^*PD3Im!eh z4erAxza_b)c?Bmjatj3s^;)ZVrr!dv>UYCBaSV1FL(XKSO)JD%d8PlSN$M#@J2%q) zB&(L$+T4WssRFBia44qSnMozb_9NP6InHJ_>viypE>Y`8DmPRorp=qR=HLguy^g_N z#E8OU>cm6mOL6(rlO`tsz%^nPhX5`8$>2>MJ~!!lkKo-2mA=|g;1w|P7?)bsX_U;MEg@W=x&>R(H*B;}3x+oZrB3hs+`hX1kl-=CJ!U!mtu z{j-pN{w+{?_{RwU7~vmx_=gj4{{Kph6(iWpxrQ8(U|ivZSKA6RiF!$+>c_uFun%!v z`Z>LWtwq*1`Xw4AU7Ug7Ee;g=R+xZ#3yvQ1l7hP(Uz>OF(mIb>+xuLJw1;YmAF+kC zM~{a|vD{g)ubl3o73L*aFR%GM^*6o6+tcdEI||)PEgDUtxe)P(fSzyBDYup#*AidG z62-#BNAcP3gDehCE~SAY@qgCw)R=S zICg!MkAt85cVrT@5iO)i{ws`0q0ESbla^6^KRiJ1;Vg>$BV6UY4$7aHRrV|LE{apl z_7{s)zk|)@B%Y*`y~uEpC)rlm(Gw6$YvC_drzZU*`LVzE9y&v~vujyu!JWZ09QvGR zwxsLF-85>PDdSQWQvN_vlb=FdMFsvh?*$1Q^XCVMu1`+ZB{gF>p!>Q!cR9cBC$z5cbE39Pi;JfFSxpVsuC=+{Hq5E{HK*z^5%Ti=R zX9n8Ekh2RSyJt02ax>`1Z*Wc>+W-3LNBlP7*Iny1u)#<#@QQ_w^~fBzpHJhTE{D;@ zgTaJ4JR2iA@a=n&hWA0H2aP6;l}sSya|<7x@pY|?}RxNyPdA#mM z-RN{i1N2e!w8npftvp(u9%=)e$i+}5w)D?K5_}sF( z@xlrDr}5EEx89#3|2b|jSOYB8aB;dVgQy!^M!7V)%{H7LSJoub8GS#0_n(N8Ncsy= z-xXEgXWJ@3HM;QAm)9kEH*@T@b;Yzjz9`x0XUu!BlBB{Pm_&KTYS}YUY}4$}YI*0s zdh2Z0RQ!g9`_p0Omk6q@U%Ens=o(?gE=2pxHFz6O_wQFP2LQPY<^kGb_C|u7ugEW3 z@m1w4=SzHOvR--=x5w>VgZrkmH@*u}#fR_wQRW|^b`INvck!=W@Sz2xz*+N%&coE9 zCnNaT{gb84DW_>d+2zi)BH)WAd)QyIx#Tf}M8|?EnPcoNn^ClW7U}Td3X3%byzC%_E=(=Knf{&S?w{tt zT)f?37%XBcHXbF-oaY%pBb?(AO{H;FtjXm?*=>#t$1qj^abtm#M#GbmO#NNLgPuT+TkxW-)BI}C#@l`!(0w-E zMD%R2!%_1H$5nHeBY!DaxrsJSe#CZ&l^#XV!MP7d9SKZ8;~v~|fqwlrW>eA*XnS4$ zAeMny2IJgmH@Dq32oys&xYZ>Ufprbj7gF_P*Js-fLHl|$@5S(s?87U3EZC<-#ZsQB z-OY0~1N!0|)VA2u&cq)&uOtO^8*Lc3)GcBj%AKF=2e| z{k*%(+8|b{jwkbiR9j(0ONe~f>+F;3@EnkZjr(smH>aY(+&9sF-NS=$U&-mKer`J^ z>Qj`bLPy!JT(&fBUiT2s`Er)*JFe>+UsJ_<6cBI&?`NR6kCRFH`ET66S@`(Gw&Tuz z18)6aER65C893*Ku7RJA#rtwI zeT*U-;YS;xRC2oRVtJnFRU_-mO&%CS+|%_Zs5N@ai);9Cf4lO%OR2G$3@XAMHM}dB z-M@=;7CfaR_Q^i9tEO1r?4z&;S9X%RNDs`Ms_HS`SF7SdH&Wo`zF?o z*(BNDKmEkktL&k3{`~?aJ1j;QaQFJAl%Z_8%!)0;5d&ki#|zKXI5x+I%!kF#t`14K z9xr4&XupEWdmewZZ>V^Pj2Ze^tXplWZG7=u&;+(?+f!$6e((`~EKTZ?s0h)g7F(|H>;hhp`zaV){F_S#CI@ zO=ENjgA=F3KK$Ir53Ux5mG$t?Y2i&`KFqv_5<{J=3a=E#YIPUC-Toy|D~61ElboDm z&*%O5rRm7T9NV<|ai8^S_F4*?jNi_Mz!_vk3%a|6^9c<;-bmn}V?LJ8^Qy8P%Mx>{ zdkWrb!gW%tt_hI#aznttgMnbTcGv7BWA{qo8{ zU$v9H9fc*)<)bmZG$!cFk$m^I-Ezz0P=#!bwq3GF;`Ri}?cSdtE5o<~$y?O;{>C;(6CZzjEhv%%-uBTFp_XcsnpE$v7hGg+LE z$9D%I4p(-_e>Dh|Ya<&Q8iZ{Pxy4Ormj6hUbw9C3O?M$>b*(rB z4gpMo5CE%q8dlbhuF}$czipl0#o1e^BbI9 ziF*;_$GHLtQ9-e}MQ1l=ga>LwjQ1nTQTzJ-%x%}Yg>&EipMZ$sxJN%cM_fiDno@uCN zMUV}OFcZGg$n0$intYU9y-Y~_h&dl~Jj{9(!p?X_UtkVDuhQ6QLrlkw8_^rN+!~o` zBYu5_LAf4N_t2_Bgkqfn=#2Y~&aRrR(!tXERJ?TP37Q*LdC=@b9oV+AK6pQ7HQND{ zHlK8EXH#eL%zykNOZkHtw!ea8pz9zSoN?N1B+)* zaB01JzjWtAV(ta})6?axSR0{C4jxgCY5(4~k$w9pTVi9YY;Aw`-VV_SZoY5(IfWeG z-~EqJ>{M|lvJw8PmK6{`Xp{>YpwfR#k0sJK{@kQ*oG&J}q2z#A7*-~U&Ly%M!>>8K zn@`xk=Q#Bsg`0W3w6Ot1F(Nl1E*3Z1aqSZRbFjwRM$|F+SIRRD1>4lMs_|8M%PHpt zkIGHBIV^&S*4Z?wez>N{x8Q-LpTpsX>g3YR5%+Op<46~oMw6+gTHiBIGFQX%Bg_Q` z!rR_YR=u3&(7#n77{8gfWjI}BXU`X7S zTcMIZ6l1yCp`T{Th1S!ZVo^WQM_Uhvrtd6cPJa7M7dGQRpf>uu2>4$mS5Ce}(OyCa z5WbcA*{JpEiG4UKSj;5RYW>jeHX_nsPzg#3tqkCQ0 z3*qVKm0elymD!7mv1^fguj7^78?@@NyhXEmkTSUzAVrAUQ1{lY-o)-4?9^`g{~%SQ_U93Th^`89sT6kX4%nSW@pcMDPB5V z)8OglyT`w^?>|}D>h3f;x+c=T(1HE+Ofm_|;wsYqwrI!wO;HhSSqB2kKwFA$Kabmt zM`P-y#jrJv6Z|##6YcS9A_~(YrTm3SGckBl>)u`nZaBOQH?9Ve@GijSvC{8Yh6(gL=ANg07slk0lQ5171La<`Dy zzgC&s4J09bH4^Pvsm^{=rfu8Iv>a82+_11T7SWy7?eeW*?O?rUy0FJUiX5MX%h@}| z6>Oe|mYrh3e}6DMX(Yt|xte0kT_1i7Z)mJ(@LE1Ndwc2&9^TF0q9;r7Ups`K2u`Gk zb;?QE0R7)1GVgPcwZhhg`5qsquLQ8Nb`@V4$iNUI%;g@T0BU*C&wb7DrgXPvyL+>e`4L#~}DgXFF}%nRJQ4$=@;U`1RcW&(5o4&v*fZ( z+_Gl-(YYKV@xemX8MbHG*ww<2q$EcJs^c9qt}n%W$(=WGAK^Tq9UQP*HeEjy|I@rW z!QR=tr6}e2gr>e~d9!W(K0`(g^lWM~O}?N=3v_Mw{wEh^ zyy$&Mc-0T62wwtvQd84m-{nuMw3y^bVUG#OzJ=!qd-i{ns9#$rAx`*}Qq|wP^t&zc zSo&49g970|MRheYZ25&~Ml5d?rSzA#1;0#;qM{XyqB_EP16fKT;t)=bewM(;StvCl zp7rTcu*24SGV?Tobh)=r)NL!Qi?+&0Va}7Kg5K74_&O8jaR*q(C~tXh*p#`I2H}s^ zh4^KXeja>1ivrqw4>ZtDv@29LLl|QY$tjS_yY7ocPyM`>wb{?#U{9zk#$TL6X^kz@ zGgQTw)dpWM)0V%gD&EY;zzF&BCs(Wu_Iy|9@xEWlb@$+XLY_Wh)=J7wzUrp~u#>w6%0fmM|cOj=;4NstH$FRNa^{u=X)|NijEHSm* zg9A!WBfIx1%54Xaua^NdLC4aLK+bv(4Q~5sn0R%ufKp9-7ag!=IV+snsrt7D7EyeH zRo)n)rZkYtiNcU%>h3p>Q5Z71g^7eYRGU=(I|D8srpk*on*0fF08(FtS-1afkcd#`oaU zh3$zm(oOLM_NnR1#fEOND&0lvWj6!D|0KIcSjI*Sc9$n)ilAXxyEQn+;TD##UFVIQXU6)+`z@rM?D}!Yh z8|@VCEr>a*^!v|Xhg?~ptifQ56fUwmhioa12m2>EJ6S0s^X+=HtOTrVS*`gbUlVn(mwQ(Xd-Jt z)mXH%!0A&Ye>nI5?TsXtHuwI>-dE4vfr9kbr!a2ztI)6ENRHs~ox40~*+ZHh483@2 z>98+0RpVe=H=Bf3ygdI#q@;g2`;odSWS}SU-a0C{|47dFoYz@x`y4@(+3j2d!k1)x zPcr$5xZ6``IwA6MS$haggRSz|?Eoc0+}0EQFMoO6nl(Dxbk2ebU)1bX9X1MqmzR=; zGA=vFhLUD>4yfa67k8ZzC`nH}lVWj%N^v7|@g;QTyvSqD8WtHZf&IXvh&)cVUGBD? zJU-U%-_IZM0==D37&_UUT*v6j(=W>+(&Fpq{Wrt9Q5_(#*U1joB|L4rK2nVPJg1I& zJ-+W*X=Bpm0vfwJhTLB{D@WJ4%K9!$M*W=ob~N4UyMv7jP!`$Onu1JbN36z1K{*@e z9Dx;zthRa_A2x~k7L{J)+NOSnRuvnAtwC+W9sLq4g_BiW#qRqNl8W0!TiEZigh~pH zs}ZY5D2-l*w5SwUo0FwD{iQKp8NT{#1Zu45eAA;xvWcg0;n(3ZdfCnv%m>CM=C3cm zD2!8NxUA{!G`SMi^ud%PE5XWcF3LlwB$Ys zJ19F?M^_AQH=ZGDikF9GK>6Q)Gob*sXPC+Mj6S~HXbesw*FCv7$8SKc*E>A1jb@eZ zu`qBa;pwbCsrE;-K|~j91r5zlcO!Q_hezyXtkc{p!?b6_^P|vT19tY=RDCgv!5h5|LOJg z#QVmgKTRB8{fz0C%ugo{aV(iGkoB`0%*~*NE>R6NSmLj`x=uHFa-6I|RrtyNUp7Yavc&fm$=JQ#t@q3n z8vZQIc;5B#Ong)FI$zu8kRI@{^*B=P5ghsYbF8kuijn|34{|?#W2B+hf4_yuZL?QM zWo)J~Ygroqrz;|vk*3}oc0U`b0y#2S#>IufW>OKBKN9+CUzb#DOevXww~a;4j*z)I zouad5wXDMPzp^5q7ITai#uc5PtWldqitTND14DPmYkdj3nnjie79vM{HE52!B_O{C z@jNa}FIVtHjuJ9Ff%>>-Cbjn{75kgDp?p|4{3!#mu!Jx;QNVdjeZqU2xg*&(tYXzS zU^G}n`f~PFMl)qdC!ha);?eVENzbHXQ$XnmI1bJ5Uw`Bg{qw%jj}Q*Nl62Bc^J_O; zJ`g)KsTgp+8+HnAqxfmX=ScTe)j)IJss7weo`hKQjmB59t>VHv&D4~BsfIAa>ZB;_ zwAaR4zLC;K5qG;B^gUzz+0>wC(+7@p(@izaj{_j}p5xe-c`ENMY39K&M>Or7YDY=a z?-E2l=F;)K_^j(D(AfhDw#eF!pR|$C+!cCRMX!KjThY!pbhoVl;6xX+Uf!^xdz*|; z_rxQb0RR`lEfR5#KUk9H%W$I5l~Q&No|V_{9C? z8NSD@R_!RTdLQijZZ0wuRDWnD8|A+iwlk0#(aohM|w`D{VwcX*Jh zZpzBTBi8NZ?wDe(#O_}2ncVq!2j@n4La{9awt|A#h-05u7j!~KbevosZIxrn!L-wr zO}!s(_}-OC?Q8V?`ItX7wk(f5SM|vgM2@29;Rf4f)8;caKF=R^)&oqi&Rd8SX3WTl z8>+;77j}LKtL16T#EdzQGoDu2Gm+6k6nELXM%c1d(q`7Q%1h}@<)0E)nmtOV4soYH zlBT^bm_Qaj9%kJ~=NnqFHuSQ|q;kj|{j(od&i&wtW1Yt!M8V!cM<7VVu+c7qJ1%qn zTa}}g$da%LSN^jz_EP}|Zs&%>)HLfF2qF+ zsg6~!HhgEPyb~-GLvkFI{wAJsG%v*UW6^6Ce5AUWt}A3N_$t=A+%zTAr^E{V9^IG>$TY7&acx)}!O)G)!p+ zZ467Azt)UDwpU*Th0t0TYW`g~ZfyG?fm zVyQEtCAb?@`0D3bn`sXJomPIB(Z0rX6(NT~7k&{79ruY5Ozi}@)Vm1RH@!uzMgLah z@(TkHZ>8(rpw_pM2#fgI;)Zk!7@xL2_C2#H4sA1Njd9ZSi(c##P zpZb^>{;p|;5kc)=#=DjbcCW=*1#-+$aZ zeL#dePgS#Gro>91MKAM?{$_4{EV+@<$An{<i4@4nx$JZ2$)%pSOnS6+U3 zZCsd&o#v4Qj9~aPnbLFs3mGqad0;mvW8$;U<+=J}e+^>hOa%;p zS#i%ONt{1zv(7Va@_EE?)1m^0r(O1ZW~fes^t-z>Rt;Z+R8h>E60#-I`0zLLzybO% z@mz+=t^SC5nOnkndruD&{kCcqKHw8O8(a&dkgjQtC<$q%sH&O2sZWfnsepu>SI|5) zZ@wV<{FgMQhrd7e{*oVb<5wxt-ToRK^sF?pSijh$Zd)|}ms=45-Xw5P zG9EYrFW`9BY1(}7cGuz7wBZE&^v=*T#|cI3Qg2pzy3BeB7E@(;gWjp>lV^qNq+80X zjoR*bF4VCDx~Y-+?d&^I^`4H}CavZ8dsVd^ ziW$yp`8+zRN<(<=x({&BG*Eola?*X$v-oqHF@$4Ep(jFu1yyTB#9UK6ApJ4=4UQ?= zBYGS}{wLzT486S53HmKe1X2WW3=kI85bj*7`3htj z3zW{S^fsqXg~cwTAe*tpEnGMg8jMK4^9JqO|G?Ah#pm$QLrO@VMO-!LL{nan80ueX z@gdBd9_JmlM_1^=kj7LKGV0!McU%lTdodXh&GbcarA+$qcB4K_rf#ds&{OGKVAyS+ zc{pOkcxmGfx8@jJ8b%oRsi;9lUP#h;B4JC z@U4&Aj=Z?Z_|>Mj_bup5(=DHU zmX|3I7ky@+bGrx1a_^Sa3jMVJYFs zyShsW|Aak}vl6NAfn~(EQ^0dK>@4DUh{V5k48+r*e)D?gyDYZ}p>@pfo&<7U%?7xKg@t*nG7XFP9c)NtUi;PxTwuzGKDcchGI&Y*l!=3hB5%XPd{MJf%`FALL_*61w*ZdoSgHflNf9TeVDY6xPPx4#P-7C$i7&7}3xGMo=Zf;H}k-8bl=<@m9 zcU)X50tM~|qB`uoXVO;hFrTBNm_E)Dl)J2IQ8ip z?hH7&Inj?V4}?7BYqxhg_8J~JA&upk`y0;6!W640D3oBS5?R~k!9S;9CSQcmej4iH z=N{w5x*QtxmL~!3gd!)eJ)@yvv)9o z-Q2nz)CA^)5(f`fM~jQa&lbtnp7788z_$j(cz1lWjOS;&!P1~9!2=VYj-HW6bPdHt zT-6b@S=~(2{61v@4rp~J8W97RI#kovwQ4CbG8=~V`g?6NG@jBGbn7)9U!01j`BlIEHkO-L zq*p8$Kis^Jo*`B7h~5DNtXk46U#2mUXs6X3VmY%=rVzz~(=bo8)I_C&KE-Do0ycV` z0Lxob2K1uu`0q@a0BggVNN$R5aIs z{x)!7$#5~`ab$ET8H>H&{Sdv9yF-IZGoeza#EoAYC!OlK4479v+@S%)1h{eU@bJ6l zg6X({Lu-<{13@As?l)Zt+OJ=WqImd{B|(L!QKw@;6!8^VRs2A5>6$u60)NP>NRg!k zDSABDfp5XI4|~{_=7CFMBbGUbT^z;3jvTC|o3sAm*P6-pENf~z%#B_jc0P_8_MND5 z%x`{jr&aXz)qR_|g(yjrln?Za*M#X7o>RMZGVB6Rb4zYu137=4(c_PcMn>=iF~7P*(&8MB~psw4|=#hVu98 zxZGv@A2*sRTA?oov;Il;%1D=$I-E}CBm@SfKVDT5oVaE7WWWV~_e_xlm_ue%72|PM5FK1DKQA^j9hb6^5@(kQb%vKPtD9^~o zRGJXWM9SxU;Xj_Q%ki?#;zMlP!WYvQ4AoJzT&IGY!+4%AiSALuIlGR1%QD zy%q21DUl^ww!&som1?ESYj_p@xvSmFrckb6TEBwPu>!E4$ic8SkZkrVe_Or${!P{z z{me1bCzx;umq(bVMk-aHLs<>V`;n30g(^@-czH9~zl=Odl((t*wT@E*R(;ib_{FO# z#<(wMUCJWpVeh4{(psNY?I+RaXXVLnmhc@n9GB+lwIM$b(d)v`D9Agr#*7{p)_V*t zJaE6vh>E{OB*}?cI-A;ctMyz|v>VTj>3do3wTYSMD2nYpzM)0^XiKRh(|m$^@juG*0+1aDSo(i$X!cz;q!MQZiBqXmb3jv-=ek>e-X|Sy+dt9Pn-9J zOYoMD)wl&kr`(>{ZRFB#d>6FwJIO^aF}rQ5z&3`5(3JE|BH9Aa7S|i^7FcbK&FISl zLYi;0(rdnxYN=~9J7(OwVL?uPIU7V&VG7(VgVRd(Psv{H7ejMfZpl5lL&-htJ9xiU z+(o7KaK^8a2q%y%M8K(L?xU2I`?Tkn9qpRJ_lWD*S=$S;e#)=1!IUK{axIZN9{Yn zCvP%72yq!8ij_6p6mFKfi}!-0X1%E13VD7IV~5IK1-%#0HwAl^#Ok=Zmi7%s2+cg; z41Vv^CjOPPPr=W{<=V=fcu#gVwm~qC^kQvrZLeF*LvjTFx9q#bEh7@|3nC)@SBdZa zKrll{h&JxKyQwhxXsY9^WqNJWVjXY%{vBMEbvZTKRz4R9zx`7Ydvhbz)rUFBh? zH)<*@q+qJ#MOAF9k`Eu#jTi1Td3n8p0X59r@*}R}nwqQ*RTyP;pAWIjT1y5pSQUov zycoBxE^9~i)HfVlsI&RCgr$ts=DdWT*n>@iqO4iaS|mnc`>zCpfuWz(Lf zZDQB{#i5JbzNycW+C6Sy!BC|egHTJVgl0i54wh;)yU#A_eqB)LTZS(g2xPOhzqDwt zb(_A)_H=A1qn}cxylT^WBH4DRi=plKfpa$b{^UV(8tjeIoJ<@nTm>|ht7WZ~P_PFg za}BZSu`PVY76XGaaP)XMv=n6fBoz$=2z#T)`X+usIhj$5!r`t%FhUB@< z0U8SR!S)ITEf9A}=>#%Caj2!$h3DRXB_v>-Ro?}G-z7^Vk*UtV0`q((a(RZ;U3qsf zM~^#rQ5b zXdl&kAtx%%xa-yAJtzu&qiY&JhH9M=&-*@yN!|Iur0#37P^qGoIO7ttcaeB1i*Vs4CZP{U7NDX`~h)-3z2Uq`Rd{M5F~I7D%UbH%NCVEh)JO zMS9WQDP6yZ{l0sD_c`Z1?>WEokHg_$3|Y^*?>Xl+=QTeMHoOXWi)=?a+{etLptV(4 zq%Vn9Iq!+(>-X{O@kBH^A8Vk9zLeHD{n0g#`I3n&pv03j^f5#M@$w?S+%k z0b(T=?m$9sY5w!q6qX>)4h*~Ak?=T6je2b@IWtUq9{FLGV1o$xSX~{>&o-iS2DcW| z4eR>?9_p;0)+snX)y;AO|Y>|wvF-nm_{)&M$_f);8Lw2ORsx^OmMxcOadIp?AR;Wbv!dSa{ZVudfo zCdSab%AZOFUK$TLcwf-*%2^@_^~Fu$-0?#dLB`ckybAp~dZ3U(LNx%je#Esp{C0tz zWLb;N>FDMsHB^-CQ@Vl@D)?A*FL|3Hff;c+Q4s0i>{%Q|1a^u)YdON3p(FA;GIeT* zZmk0P1RqsNO$?FBv3h%|DjQu7^u_NZf;r|m4g(&lWi=1}PU|gDPv-j(C`y}^oNigv%r!L6GjKKu zhKLaraO;+4rv;EI4KS0UuWuF~vgqY%@BC=7HsEPMj(rsFTf*ie6;eaVCO#!r8>87( zP?p?L$F)v!3sDex7y0_eMyWWmqR11L*M5vh=Agzl@zy|rAlpxb1o~!iNRU4A6=76w z$4Vpnfhde@{7BLjib#xX41*%1eIr0xHDjTWv33UvbdM=sVycn<`up)%=|Ks3lh|w2tg<6(UpoA6^JTFT*$p4a4_$vIZd0cjB+?-y) z2Juuc9p6*5L9_O1UizStx5J~V(l4lP&iQvF!aYw`V!vE-XIjI0La#> zfPEm__hr3SqP)*sx>!09{kN-zvs-qohGaXNt{Y0f=}LZevo;Inf>v~FsY`qL{*AN* zrK=6aEA=J2b%RYK#=N zIyDL+9_0GB`cLHQ%^Z6e3V;8!6$HcxbV5|XxTiVFAGu6Z>4B6Chaia#lhq}#gc_Kk zM49O`pnB<48kRt&>+RA09B20il&MS&8V`Cc9^&+^Wh!p**H>F?H%$Ay|2|BxkS zY4a1`IO@ikvKx9b>(4nD?)9Tr-#W7ZsPzLyaK)aW7Ju9Y1=vPdrNW(a>Ed$lw;Z+! zTEi*YFLS?!!i8FZcxtmz)YQ~+vhlx!v^DniQ>r+cJnGyTxVFLVe_Ofhqe+_IRxP;4 z=rfw6zjv{d49C;fNC~we*w#>-Xm1jGPHozu=+HY7+R7iuq@89TgerwZH<32GI{k9m zj=W(Lq&AY&SsyN+{vI^8^XIuCBADC4wSz71tHsFS4hHH#{rZdN&o-yo0I zHPo$uVTEj~GDFJUTBjFv_CHOXKZR}8QH1O(Ha@tKZML|OMtO^!fa28sB8iusLszuE zCX9c#bn%GwzTO9W{7%(BxovS6#7UpBF(^0-%h@;lRYd-4RQPwFO;IjtL!!ts+k}p3 zn=bm0b#DJq@@8#3&mxEXjFtgyH2a;T56_}};>x#-u|@lXL*ZHj`4-GhWD3h#BviDe zVEAK{K+?)1q&^T{O&1|59e@PkC#+J+7B$i-*UP2BL}gtXgzrQQ#KHBCB;k@w)=Crc zsz)c<0}K9qqh(~XB<=44A`mzm*E}PKVJGR;`lG_3JVm8nQiH!p1L{KU%BGDpaW&^& zQtSS+DHAGGnx$9%*X3`JE&Ss#ul?*I5zZ`_K&%#s4XGuXMQ<0`fCi8!$kOivA~p+d zMTeEuM5(qQ%$OiQ!2zP-4-fIrZM6x?BT0NKi6|*4#}Ck5Y>$ftQ(@YS>V`QmgUjz3I{kz zeF*gY;hvB#;8OVeU|tBfxT6VqipZ}ZUkB)=o%+KyPKHpp>HToX$E7fnFFV_vf_}02 z^7zIxcycop4l+=r?4fNQ7ZL zT-^P#RH1&nmyUrjJ|nFds5o6T9es+&iD63aDz8>BP^wKbA`AD4dszy`4XO5sCCbe; z_L4iSddN9Kweu7EM8|4hv8obGK)fgiXRc#s&oM04n-A*)+|;r6O~tB8wFYg=hJCc0 znIrNEV1V3w@rpHQ%euCVf4ovc}q}TH~n?FY^lUViO2bMZH!dc~XL5&_V*5 zR6uCm8PB{jFl$pCBB$5TFr{HgjKgyM3ii0t(5Jq1&%JbW_Srl(5!`!<*k1I@M-NzV zO8)>fFkDG#L)qZIBk6d<*Z8@{JOR|!I2{B;oailXL@)_) z5{C3NXY z555KU7mq>sdbkWG4`{3HKJyU%682JQ%yeu7fOC_iqQTeU)jV<8I^zuA?$h=)Sc4Ki z%H=OJ21lTdV4aDzI`jnyyW4ZJXE<#3pJTx6XTaYuKN-DC_UzBn6yontAyNU^Jw&`5i{J<}4D4C;AGk6;yCf?qsJAfZmEDH6ULSw|`zWwaMzsBXxv zS^(X9wIWy{ou=52LL;|6!LaH2t|;pIDv&U?JC!76=v$B^TFWL}i*&pQG735CswH&V zJ0n`uZl;zF_gNDJuFiT?M;#YY*i21VZ``c!%_f%J^fAnBUO@KUT%-EUR#ZHcemQUL%rU?K3Nk3B7xW8Wt;D3qI>C}p4y#=*8C{h~}P`xAAdRLLV~R z)Yev%$5R}A{=S_*HLi=~JGIliZ1~AR8{4Cz*pU(#Pc9L;@b=2uM>+om)lOZN|Aigp zrRYG-R-rsNDHpWXlXs6Nc)52gI$_n4as5tg@54`Hb(+*k2O&x!4!V+O%77?g>V@h) z+#X%tzX`u)-r4!m*7`;|Z=*fYd~IS}K6WwsWOmkc;m2Yt&>H(b$1tpihZ=cxVMD!< z#=6Lqh$S+ESNTnhN;Y_^aNm2Y`Kb1cTO}7}{&`GYby%LTt(hKsA!ZFt` z0JERx;ONqBai0rDbOO%A0i)@4xj5{8IkiBGMmh`b(X4#aZ{E;-$DkbgmrS(#w=%JF z6+bHu_srjS(R;B=bD#zzLP?s5ZTZJMV{ZW75R< z+wePJo+*9{zpNmsxi(Wgp$%#^HBpLuP0jtigwg=%2>)z7Cv5NrAXs2h6Vq zym6Kn&%GYTsegZ!abvNWqMil@@x06EL|x{=N*?wed}6GQ)zjX({I%a)jJRIv$@$+p ziW#cQ>#;6p9fliroxL)A8qT&Fu4Opa*5Vo*^i0pK*mkONtlihw2pg>to=7O_D5ZNp zXmbefz~QoSID{XChJtU@4&iLN+0ZP!Wj{Q+Yl?|AK?0aq!q3%rL+Uq!`Y79wm$_PC z%YyRw9qo~iLrS5JsedntihHQykE8>Mi9dv?F>sbpeZAx+>?e56qAdNqh@SaMo|ORs zg_M;|ldXrVvHwFCYT-;WpzuC^R_OiWcbew;O!`CXGVr(5`y|Qd+5}Z`t!ZE;C7ll= zlJp#WgvrWZnVJsEdTKQu8Z*HU(aBx+7~1cGjQ!(`RFAv__d{L)_k2d3Rj*oH^lA;? zXcDziGRw%1u!Pm5%@D|xzlHqpsarDtJY;h||E?pB1(5@*%AMi%r4(1iqQ?u^bjmWv z%;*s(wRWfF!28!|PRx+a*|C^(aBl{55AONz0I^<~MX_;#&fr`W}V!w(!({&)9$QmyKBYHb|;} z#8Nj*s|(88ysTg;Wbt`2P&VUE@=$OeHUOjzU(iUm2O( zoD^mEJIC9XtI8WID>=VijWfVc&a+ffSao8GC{Ripzw6Q>q$Q)(x2k;bB`#idKRf7D z#|=OFaQ}wq>+xZKhr4Xp6*iOn^Y>Jzso4s0#>{k0^t@?@d3{OjN!ddexIhCuuEwDp zd^?c7hx|{E9<`D?Sh?p#{Yrr=n2BzJiK#+GK_=gLu=H&{ zXI4y?bcPLYp&sQXP9aiQVA$eV=~+;@Z6$f;I1v}J?!)$*<6Z3n)B=$ za$*ZHoK&0>(!&I*XU2{~lFx;dv*opnX!1^)N`s1CuNJ@yNthT*id zG@hEFgDo6t(*MnamqR_Q>dgAo{-KI1Ldo!MUiU=HwW(Dnli`%630zmHW*koV=32ou zcc`2#QMW97ecQ66m6FXIsH7mw+RRnjFS3G1VTs#h^qDa6w31sY`R>(sj9A&p+9jk9 zL(S%=5c4+Oorb}(8oQHsdDGsD1ZlQvphGR9!K1^Ze(>?VQSrU~>BBumzGcJT9zG&& zfgVU@CK1Zq)VQxfBK5ED>wkLucuOK6o3}{d<7Q97eoTYQFiev+=>z@Snn;7&w#Zl4 zxpQClXz_V7IcSsEW7HLi`nNbXxk+re-pa3XJu@3zm=_P+og;l}?;`t^g3?y%sVGM} z{3=z=`?d?n$%Gf!dB)56A`WMLp@(YVyA0J}BZp|xPZM8i#o-I})CX5WZ)^QnOy8k> zoz`UR`)E5aww?Hm>x4`A)Z}5G+=0je2V!>qh~(&)mmPKb(@j>o;aYBewJL$P)VE}4 z_p@XlS`qn3M@8r_VO0y+=vD(T)iJ;;MGr)&zpnhT?>VAa@a+-6HvGe@9C$<(x0Wn#`4)y&wCt?p{yr_iUwY2FYRPlet6+A|;(k@b{Cq0yFdq%to^BxW0B_hLqyen~a>M(Gog>V(Gy8GuSL<-C8V_jdRW z$@ln9St!CfNboFCu0#zP#j$-bzUTDyqYvyqe764_&c;wFGfkDFijx)U#H^N!YUlFP zZSge{LKP%x6%b%DPVv1K8liGp+$6rt;kd_d3A?8|d|p}dM<2K2BcXYRd>?W>Z!6O7j(2_Aeepu>_+Az_T z=YquvG`R``h{ZOZ0Kw1c&!MpF%3*Le@L~U$2umN8uiZg?_9Q_-#zwU~%6rC9xeP+^ zi#+Hn8qI6^ zzyJfjbFqQn7Q|mnAsgc`_W9H_`hCdf?I*OkfFYl*IrpJ&@}-Nh*^8|q*x*1O^}pxG z|8(EtULI{pqmcGiV(o&Y`cn3Qc_)t%=UpQn>NkP~9Q$~6wzyduv9{Bkp~lts9(IFt zCv*8MYfWLg1E;H%3A*%#bR5xqunS|ebB_4UKAe9%@X^jDB^M94B|b^{J&rx%68Z6b z`ECK>@@T`RruMf6Sf!k9s<%)3NcbZgpNfvWdG%;h!YFL-eSl=1!&>TJJ#RAh`LJnr zwU-LBTKBpbsCCaya9H%}0;uNM3)xI83z@Ww?RSCRXUnWt=K;v*rrlxzga@N z+6WM!&t!*+dP*kw@@pqS`1{EDE|T4gpi>TFG$tG$6xw+Ek~uyYw?S^qu8z-y)<2i% z0zQHN`3h`-x~ESA;8Pze9$lnJy4_x8p(Yb)%QALK`SiMNkP9dM7RO+@TZU69@ANQu zm+~Lv+_B%?Vh6ZkXPA%o@HVVNB!6T<@ni^#aa1f=*d@CJI&~8}4r>3tXW{kLo6jM< z{OW}8F%2sNE|b4>{G(Cm(bX;n9NNb(Ig8j#LW#d?om80xI1=wWCypv*?XJ&?Rnq`Z z+{pwA_j+FsNBSN49fh$8{#p2V(QlI-MY}Pvpg+%CiEP)LH7M)T6+0(jGAH1pJ91wM z)D_id$9>11dcpsAbMpRvbN+>$0x%{;ig0xKr#J>exzip=nM?sw*o~C9{g)hcJVI7J zABdr2I2-+?bXDQ^7(QVrRg7=%E~0MPR8Xnc+UYMTRt3^#yc9+3dChy+&O8r)-e@ZO zv3ghs-`=jLXNdE?PmffXRuJb0-*rt8mWhAT?PdFH5qi}QG6_V5(1 z!9Uw67wG)N>1C0?EbSiylyYwm+^-!HALCBI@g}XBK`{CAoY2nn_iRfHuJW4i;#na# zXh0emlg|BHRrOznf1RI!^01Z*ga>p>VOFjs;^Y=V49h1%f`f14<4h>joiJzJd)WJ> zNC_JQ5y8_hvk1q5nLJxn)lXqm)9O#B`+9rR@9rw$LTbDoy-372H?zKHTZYm3Gw0-% zKKT9LzrlYf%U#c)+WyC3n>kttXd&}yyAo?mpJ~5+Yb=@1Ym(F^lKEd_Nq66WHl59o zREi_h;D;)J*d@gE@YvgFKcm{XYAI-Dd10RXsC0!m!Q74P4E-Wlf@=;q>&XqB%2_^D z;Pf@f4EjzH_Z|SxgU6_Kaq51z|I2PHXDZBc8>P;Q5E1O!7JzB2HF;I?zybM29NSp= z=jD}$+}@~G+y|pQXI){t3u%AiHNuf*LdDmRrChtVzYWN<{4jvt*_$i=7{CV%K%7z3vqFcSLvAFX?30>{2Ur_1HyD62{ug)Gn0Jp3mzPQBL0C( zD{wGJR817O&Q?IpJ7yN%>}A2>Iw;>v%?w?NqqlW*^v@5#dEQ>RYdhzVn#Dl5wY{Yw z>@UAvA=E6M-ed3;8W8Ke$%c3UnUw^y*Z0}Nga1u9{eoE7cgO&=+83#J*X5AhmAecY z&#~wiFhusRj`M%eH-DI{m3WcA(>j{^IU>U2x!cf9eYLVi3}m-*j1JQQaQG@#P13`s zg3{a`vFNHi2*=W`v)%`#zcd;sVJ1S@k)4Fy9rDP1u?`g0j8QY(zyW!nMVvxt?>mME z%O?38CEQ;2kkPjf-M1`sIMGutw4Z&gq9A=dD3`0t4|h|(q8%pQ0kR0b9(4TB558!< zK40a15;+!A6@bT^#`r$QhWyXXnt!>?0Q~YvWVLxDja7QGMC7n-Mb@6Of*fWV!mmtB zfMs0nB@$N+GvSCS6%4nh@kt*kmbE-xsM+9-hQvecO3JXi47k7VeuM~)>lqOEZgH0G z4<=2far+4dYJnIs$YFeHEPh?Z(Ym;{V2c^(jwU5G_m5vbdZj`6_UqK~hvQR`qS_9L zLebaul1(S;Lcbo<5Iz8D{dZ{yI4#2FncCHh+c~0Ovz^bK_am67mf@v&&rUadSx4KB zn}VMm^ItIor>$l0oo)k`aKz=-7Ksyi{ zv5;_VKi1f#SY2m0yZ^Gb&+uP=eN{oXUpR-kK;Zo&k=h_es{miiTR*yREla#8Oe-?< zP=#@2I-&`>PlCS>=xFJcV{S`B*dIa2)D_$6+EuJhzNnSGNDy<()fJQv_=*B~ME*mhuj~6)I|y0NCP|TOfQ6z4*ODA* zr9pfrrcf;f`SMrE%>JRzi)sxiwBPBA@DbS0(>ZD!C-c2s{FXi^;t@A#K=SF$H=PXl z+%Q)c3DNo*0u8_H5Q25B(3PTB4r68Q;|asjItu#A8bKV4ns0ZQxU)aUj|doykU7`X zH#x6XIZp*TolH%#y$BM23A;!(8?#C-EgSbh-pu3p%%Bv*4)Ii!kO>O6R~R~u`sVpo z^gPxiOX-d1Y_SsEnKuB5NE)mt)#7_L#Ik|qad6A!rUC>P8+=3f3(s+jumW=5umW9a z3d=cg^IDMm3p60lKcGJ>n9JOEextuUt!~-FBe&%Jqd)-`SpL!<|Efj@`aKN6gngz0 zpitfz$$r0&_~_rMqleR&a74L`PFPs9r1K6q22zr6cF=-`Wko!GDuj(sl{)6zE{cMk z=lHMQ58#n{JwTkR!c>#k)y=|<4p3K;8iZ2gXHKfDhVC@i zMvg7kzl{1fQe`e4%?L8^CRW0j{Q;^oBfAXfWm$Hh$1-Z>sMcJbd_+%r5>diL_&-cm8 z76(a45ya4We1^eD>&?OX74xvk$^PC};ikxgHbWbso zRBF!xgSdD>n0}uz=CnZde#L8znyT701IN}G?0`v{mQBb#u7SvVuc;%&*5|t8wBSbK zZY{3to(bjmHk(}wB}jmdZfC=JzoMc4al5r6H@W{~yh;K$3bs~5(&IkDX3hdP3Rc1f z#cWwIU7C^tu;hhqIns6n&%}H8@yKd;l5O^2iKR-dAq7|{Q%qh!8Egtbfz}jd(L6L9 zww>JCR+VrMDZ9^6i`-l!e>3=+(4Og*HplPXcwPYYLmA{^$S$|be1EQAn@tMM?W3gi zDEwBXQy$Y^|Bm`G)&YTAkQf(3-z?HK0d17nq7N8Q|8uqP!;@5ey#7o6_F<1JplH3F zLcAcn4R$h)LGh(@Gmeo0X=0$H{%_OyUf8l=Kjo_6i$nX-Z`q` zGeSDyV@0)Ss^#m6PsmZ+fbeYZkl5fDP8(<_^VP6U9+n9jD^;`>*!ZnEfrMg@%q1TVhz?J2uvY(zItyrdw*;b9F`U zSeWDT1{FN47KCSttQ0+Nx^W7|GTbLS{G7YCi=ffAll`X-t?YlSLg^VlEJ5sc)3sUF zr|e+A4adF4fI)gsvI+ZoO-i`gLF)zO_Rj|p8-_0F?{At_S!1)mA^VeTX?%CvMV8r< z66jC0@<>=D(f@kF|Fp6K1<;HP@LYkSGY$(l?ft zC$BaY(?>F`zKdM+@0j=|%F$2<5fY9~u3-V&XsMSS2+oIV1D@X5L~LuFeN?){srag5 zM!SAcj6BGN{mwH4GmO0IRzo76s{$op>!#^8TUWlS?JobK6QxWXRq@5AEFN{il(ude zhK5$9U*{stP^&56WyO(tZPyH$MGV=QWfnoxC@%Tzg04)N7%kuNdbnEXfz0v+ja_Q? z3;^o`vDHZWf6>2vY{6Z0-eEYDr3IZ(|OHDIBa-ZA9 z_a*sd_KgyhX@c6b?|1V3iTOLrVXgZ25SZOqvxO{S&ydG~z|1Q(5F4x^*(Zd8o)~aP zVm7UyyZVxPZx?_s3X;c>HomxwCdQ8nDz@p}e=65&d5O-<^1g(Qzq2D1**s&z4mOw* z!Jbru&oCgciIdP4LRo6{qpyWQH4aEprJTJl8Mu>{FCx`ran65Ud^2c>o8f2!c@VDe zglZC8Z2S^D6K6%p#oS0!WZl1g)Vu%OntUv_TmBkk{m;SjnNF);M?DtnCq#yXblEvN zzBvKIlh29F*W#kmRG#c318P-Ru7zP*#lTt`>tpm|$b=*{*Kaw?6yA%mf&3@ zm@3Q1Kdvf$segF_g|7eh+bnaznyvFELVb^}+?26mPPiPI4wg}1p~d^euokGPl4y)V z@qB$CFx=>+3P}~-L0t6P=?KE2v}-_ApdvoRJPjf;!b zDUVzUUcNHldz@(fqW`Bi45-lDo_0bWYa2_1|FL);$|Ava5U*agr|khNTD_2G3fCTB z`z$+n=8mb^DQm+_-LuobEcAJ)74xk5>@#I!w^n}KSpDvk=I=kJZw}%|+wT=B+pg#d z6NQ14%2XaTNJiz$__%pPnJFMOZ;Wi>mbwQkO~fR7p}>rf)^tBR>{AwJP|YA{EU zJ$6LIOF_b;iE^>Z)waoc?D>`31as=m3DkLJLbnVT#XkvuR8$d2+uvPkxmG_;_pyIU zFi~eW^14VmMaD^@(y2;f0@eX;8s1hYvJ%SBGRLK!b)D9#rca(+qZ~qOsdNz)Sb=$Hoh~sv}yh|A%47`8g|9aSdjY}q@01s{> zj%G$rYe`^GTnWKFRZlC3@)Me+7Qu?p=vJ+?h}U3HvI#v3rE}=suja(P#kR~juyf#= z46|0Nzw@REuU4A-9_Z$)Eenukj68pf$=SDiePMSWf7p6b>t2>L1~4QZH!s#IqQx{A zo`q^wHXghZv0mh@D1ZSOyfoeZcc8#mCV~6by7g8?j##@`XQty!BTmsTXitWK3aYqe z_~C^EeF%knxuNIT#Mn)>l=c_RB!&L8j0)$iuPEuKT$`QevVt0u-yET+3KtpD^63V6rKoG ze@O|U2JJh;u8YfKEFyw*mz0`@=7jS|Eh5%Pzc7 z17=$d4a;+C`WkrRQ2p}hmp7}M7BdX4)+Nd|q8au;Jrt*@f1l2XRLM6T%-Hkv90W=V zOHrjC(QF_#sa{1l`QaY=-kEvO-#gSg#@gsLD8(t-On8zD`A)5mVi6+SC*wDy^4RmU z^M}T`0dnZ$Q9E_G- z1m*5r?jbH9SZduVW@rr4-gQ=-4!MaSY!=VRgHm-WBiqrNGwU!_C?iL`KxK`$L}K7a z@xc8)?M>Jx&B8MwNd4nDIY2U8bbUp*aXH0>dPRbQAH|L^-L7%QbdR|W2;b8(NWz7_ zj=z|ARy#zaa#`!4Z{s|@Z9PgT`x>1;w})Bm)oN|w%;t%|-UJv$X}tf3N~Vxxr^CEU zumjUh3f5mfHg^+GMT)+!8N_1NQ^zrtRjEIXAcW6_ z5UyWtN|GQrVMud#^51zAc3hfMcu`7AIL6eRhJp`xk-$P?|M%MB7XgAlZvJFvPq;8h zd?ri$C52^mcDTF>D_-Ll5ETunQ5@r|5_5dQ#Y+`-w{7ainkD{oYQq@~efk2&Ui03m?0%vX_KG8ZhRA!e^X$N{VrYUQCA=6T>@tV}`%=zv-kNW-p^qn*d$8zBQBp6`~>qPiGg*9TfxJQz^oD4ME!M;K| zK9pY~*-fv9@149{7By|6%Lo|{Hyi|(`U>q(%@oWRoK*o*mrddjWSvm|r29g{e)i>J z#VVA~);Eciqa7F3K+AhI!U%WTJxIu&>fPeJi(>`oS(mFza;PM_#8ZILN#V$;$~=3# z*+GcR^~g{BE9BMDmQ++EdbVO+<4EXdoMx+Ou7(Z2^MI)sWnRRQ-nhYrWjz`qhoN5_ zVAa#7bGO00TlMp}^^pb~JGDCht{!>tZ)7Dabjs3Z9=*V{*sXI}^5Na%G6

7gJ>Wlp*rffHGIzSyCe*h)YTk(5u;3^b7A;2c58IF~=JbW=eMEE|lLsC+unI zNbcci)#(QGr=YNcPaxsc%W2x?Ry;(op0RwiggAmVvF6Jlu=dlEa`}M?zInzFa+fdY zS}~jFPxe7y?Q&J!S_Sl)(axqVUbo`@GW7uoc22ue#%0B0%?5GOSLGQ3sqW-*Pydhd zW_KyTO3^GgfdT&&!;a@|YLR7CeT9{gPi4JzUpY1(+9(8>sW(9Q_Y;VMOO+ok<1$(R zXM@`|JxgT`JlUQB%Cib$6`>8o@y}2l?Iffru=h8T?Y}b!gu<&oz;R4+hvCE!6Y&O) z+C}QIp0^~QvFqI865%x7RPs}^YB>fmSK@Y_!w6Gy%2_0-Zk5mowiuMa?&&v6FNU#i5fT`u0XKdacDGZ7`4yr zplDGn1TTA=9~A-H9)Xr9m*@aE{`}AR!>x(A`$cL$3{5GxIl)Nex^v0(QOD(2?ht01zkjhzT$HlCM->f z>&(EyzOqLpgSPr(gY3)hf^rWKFLR>#3s;Cx9fqz5OY#YH(|yu^&l)(y-pf+ZAte%e zV(vHHeN;m{EBlOYXj4rd-)X8u5;t9q2o-Q@l$`0LG-dE!*vzyTk?AKVf4-~V&l6=q)ZqH$+QsE66&P@AA{=7%u>wlaRgWd?(tGlBgXsxM!rM&U&^P6_QLcq%JOE(#ue$WL>~fnS zP6T?nX%``BP9)^%gRF7hDpLE6^8#UUqdyVxP1i!b_04q52kH!m(LMotm|2Y1ATX!U3WINcr@!7NPSP3rz zFHe&3jXES9c{fe<^khomwq=>^xuQ*3T+ENN0e^U4nz$2Rc$%>uSQ7^}?E8GkdgyO! zp|}?gs!CQYta)LVO0v4fYc&*um5|?=HqbB%~H=#N#0K5f*Rg0 z2Ob_Hl&w(7%aGF7Gb)^S&uN!@wYfiV*A)RCKa?LdrdRzgEHZ;f;-zvHqoU8P#O&D~ zd&4KviazSwOV2ZAyVHXkdT6b-81ez@X!})ghDRr8`$t{KoS0o{EB_%IVZl2lwj`j3 z$Vy8el`OkC6!*fQK@ROZGrfKDDz?B@D6QpueZ|QV{;%~7b=tMmDU-86p07-H4i@Tu zLpPRfa{@l)Q7#Z4dmsg8jwbn5KuFE5Z%3@NJ)r^;f#xH3w=f6G-wE>IcSG|hSy$vq zX+A`{GV23PqknWHDS*xa13>%n-DEqp+#rC=l_|5>5wFm$u^BZ<(u>>|&z*KaPI@jD zk+&DWdUe5Xl0k}?p#*5_*L+WNEk)r^MWbF!1u z*5jx~Mapb$1Aw*ml_C`Ky8Sk9$Y-670^vK?L)Z-ta2u5~zg+yW730QZ_CzA3H_9P; zKhfQY#c3_o0852Ae=(xyRd=lKr$1sMblgZN_wt)`LLv4qrgy$(hij)}TzKx%(Lx!zWR6tcEp> zF7@B;;aHN;Hu>nWySQKtN4G86sgCXQTu&}eca1qB3@*+q(NIkqssmIDEr8IqUiZwm z^l_R45b9+AP6hk#2zB4<<(eAa#~fC|_u-*x+EcvHs3h}g5$x}9`kvGPpXw-*3J;$O zcEQ>klX~P_#gB+8&XWI-dF8gl*bqfd9h;Uw*;nHy#Su%Z%5Mx6z1b;azW+$3G}Q*N zY_GTbJ?D!T-85~?I6grG~y+1p!$U#bj zpduZ@${IMI^Ds$Gy?gsWRa}Nwe4&a*ySHxOcbtmHRovOvAN_ zYby7K`p&AurbtfZi)0TMuCV?T=w&D%$3l?^#@nRfaa*YSXntyTN3mD`VZBJQHxte z>)E8iEJNUY7N0|w_faQGdzsiAHb1b5y7p6+YQj#J#0>U~%RDf{sh9K3<$Iq^=t}61;}fY*bIn@FRW)rq^8f%jzXo}qBRiee*j;cZ z0;Nj_VK(3ph>adM&ZdnP)}ZuQi_OVkA%C(8fQB?x1Y{bV*rS^68eC^RoJV_7x@j^T z+do!t3%|TLwTTYdq!T{mrSb+|t6TeaqV$WX+1&C~?ev`z#A&qVLAmi1>-A6y^^}pj z5l~l*X1R2sG4!tu#0NBkhh=r}|DxW>{r!CZ%Q^ATNGO<|A*_&8`))6295TLzY?$_X$PyCEY|A^Vr; zR*1pZINgomUltWZO%JX2o=kpv{a=tI6oYI~4%jMiw(=y&ShE8yh#yuJwtswXIp6oQ z(bT>~h#l@KP_%O3?yV`r0mz`jXI&Vz?h!<_1v%m?XJ>*NU(@+)Z%8(|7&4zrRkmtu z!mYRb-l3bn{vG?t!(;aRM_J|AsRKE$(utzti?pvcwD+Y^VNnYeU@ zr>7PxJ-R0GcLohfQwzSTNMOf){q$r2!XxoKz}6;t$owB^_@5G^j<$y5a-R6~0P5u~ z6b~}>YV3M?QE)$5VXzKx7~?mF(-mb;A&{IVu`is9(E-R<=z}wE6^)2jm2jX8f2F(n zGdteJDOKU)KzjqfK!e_J#~+)i+|~*#E~+-L`id`Hi@(^0zrit`nMLaC!W_hm31$fyw zn*=3lMED$Uv-=@?JiW4Wl&79wAHcU@L_o%rYgqurH!{vqUAV+=pPIv}Et^GC9XF@x zgtPJKm438^Y)*_&pk7SvGn_kBR^Z&&w|7mUXLrUAoyr%s!FZ3aNJTu~|4=D(vV74A z>G2|^Tto6bDgXuw4~k2+3IBE{2XRyZSQq@M9vG@!;(|%8k>%}H`Fn5C#FN6g`pfp1 zOq%N(j!f(w+RU4aMA@t?HY-;q5eHkV)Nqz(2Igp8ksegupvPfVm87?ZHr-E44H=e9 zuir5Shs}yWvwQf#ds_ow#8YWZfUS3`w`J<+IcHiMf+$J#q)slHj%MhZr;*QN|0U zELLJ;pVV*{9fvtk^w0+lBXdxU#KR9NRM#ZP!?&Ri*EX)+Gb!<(S4(*yuNp1GB3mzv z30L3m&Kq2(aaoA|*sMgMWYoRgH4$9Mdn5f?^rHOy;d2lMzz2Ned+#>Lip0xiIN}&fM;CGWi}R2&3*QmcEZp1Hf_0A8Gn1fIJqx)JiTAd z*tIWP=z{=jKgzIN0iXsN)Z?{Q&@pL7xJ}DuqAfl{^B-*nRd?ibR#_rTKP=Fuo*o!9 z&R2_U za#`>ko=DKDv)Irb%I3WFQ?_inpeqBXbHHCFV%%xHXUZhmzZSgh`~7LN z(oeCf78S}YUS%XMS^)7mx(JNdrWW4EZ+zVWIP;>Xk>_3;b4|?70=7RugzJqp&Ub=F z%;@l>{3GH+z#uG1q$^JL*Xv=8pCcQs!W0j7?(=lRu=+#^so?%^@$}m%)aa7gTnu_N z=>F8Gc4|`?4sFyoU+n2@dF?bpNE=r&Y&UF|dWfNux9~GnGKvCQKf2de6glEMlQ9z& z*;@?RoA|>@LnvYGB%j^pcDE=QV8ip&-g57 zJmT|$Q1qNpq0gU?`vO?MiA>}?LRJ|b-}_r}){jYi<=VwL9`=)8)Evdo}#hy z3*W``BSPl%2a%y2rRP>|j%b4e3CDMCilIA)^Szm7R)UTU$zp%cAJ(C^#-7`BM)hdk z0D1QW*m_j}SV&6T7uu{Yh?^9qn@IZrwuOt@=l;stWB=>a`w|Az9tOq`TW^2?au49i z-7^Sf&MzJw4$p)at=o}@p1O2oOa)e&FBow#jK46|8469|+}wM{S)}bg{Dcc+F0M6% zI)ob{&nbYDVbVVOt*FA_H)f~0+2Pe*S$#_xONOBKd_&0fr9i2@3&1Db_5{m-yhHLv z0r&^2*u$;ME0jgM7;l^(yk`Pnlno*87jJ&#ToG6f#xFv8(PKlMg589#*5XfQth2Wb zIY$Gb=-tLY8KfyC&ADEW3}+8@*Nuv@6bv}3vUj%y13SNyg?bSlRL{7LM%K7RVO7t!>A|Ii@M@l_dMkEdCUUuXU$Gd9( zxEH_xARm^{!PtNN8~M{Q2RhxD?QKK}jBUHSmd~ze7zxr-*rWN5 z7tZUtZ}%2v0pt{DRa=_($%JlcCD`fuz5eUOMC~WYk^?2y*=#Ho-;%t=Y&1$=1!!PM zXR7vo0pfRn<$w3&Ecg|a`hI!xY zYcv?YIYMrhy_5ufjm7I>a_zNc%2kH(pC;!p)#w<9LS3!?wB3rL?@ zvyz%zUStcH`nX2HT&NRKv(V#N`dR1YigHaw4J8VTaN|%VtEqiGPDEu& zx+^7;YJ2j`0YE_vz`AS(acC{V{gzFkknAq6>3#M{)=bU1RT0x32EAJimfybiy89Yw(A7e`~ycVC3)!afdF z1JI-*JIS;q(LqLlS*YCtN`MO``C`!_mJo1s+J(W9f5i(tKN1e<=hu|NAF5{q{jpcA zGh%3tm@GarN~B+gff;Yru+4FE3qTfnH`C)~x*TkMm!P^ISxhk`vBBp@5B4qlg?4*w zHwU#-#$HTJ&ofwNb427s;1!2&-$ZY^Hq2`sMIElSGIEmf%9ml}Fso*-0)8KM5!)4^BsNKAH zLd_-u7^{@^5?N>~ImWzx1dR`=6RiuI*FN4`sCu${GsZhqxE!HhWJA|mmJ$MosxmE2 zbs6r)2YXk?lTX-Y4zVD!6FupteU*J>Axgs6w(fyq^bJ4!+4f;LOFut(51g@YCFqw( z258lh|No=xt)rq0*RFAzp@))&K{_O5040Wy?hfhhl5QAEi6KNv5J9?2kq+q+=@4mY z92$w=!#VGF&U@DS-u1n|e`hTgFwb*e_jT>Puf1sr=4pbpyI&#$)qA>;7G%Ms8HVRSC*#G));&~ys%cJt~u%S#z*LU7ChHm5gLk>k7 z^7ZMm9fPi=)Spkz=gx+1KU6o5*Ar8DJMK-ZTF&$gJrabhQ!AUaBCt17+025ZrnT?R z7Rr(H!t0w?`ewl;- zw*Jo$DzJ>qWmLECuu=gxm*@TQDmCuVlvNerc-p*k7igoW z-8MC{hoB;8#M41RZr}^MkUO9 z1)NvnVV&;RJH^Gpd7eWYN_{vz1k0YoPwK{=?G>KkvrhqzyKHUIzuW<@&reNKi10an zkG0+gz&b`{LZUtD3yguBoLosqgz4%)V1# zgX!0G#1$Tn`>GLdKw$@06>zS|(EV(Iy#OCX!@!a&5>j>vnD%9`oGi zU>ojRbguMf^t}6sEF-UVcyyMC9`vk!B7I3U}tC_m>n zDcXpV;^N$*@BM8WBGPPV&(UUz!pdfG*+=&i<*wzdjJo6}iI(-QP|i4;-F;4$ri^Q4 z#kGC{(!Y1($l<{$pl)>M?=R4}F}KaUlxb4z6Z_Q=Uf1;~TiM1c^5;?84y&i(8k=sQ z;?{Piu<6_T`bk{mx8A%$s6$IdyB?v)8sGf=u}ov$j!^w(iMHD0-F*X6sjh3rc&Kzc zkFB11SBJ%Z*yN5o@Z!EjrMg=4EMxw?%gN}w;;VmOb8}%pUy&Q(^19wb-H$Gf1-cUC zwX6{H0kstE#p`~C(3ith92OSPi5>yxnIG8Mf8fjf@N6WbC`t;3MKVfoJK&6|)DR{j zZ9q>#b36Rxa}EjS@4X-DlG6w=AhV&4gi<4mUke9=s?Rd40woU35(|TDA<{1_0L17R z50dp`yD^`Y<6c$x@>rVuz0ZQf0z(=141+a2=-Fzx|qf@>p-mM!ZH% z!y6T>0?9WJVUA@f!88fj`4ljkp-C5!Ho3G^aZqcjVb@hX!nXJR%BBh*?5NU+2_M&? zgvQ_1<^{srKhlAOFT=ef-CQI_?&azI3pszH+ziudh4jV}?^0;@=c^i1u>V3ardX{v zKK*xBc&jD3SK8O)la=@C{4j2S_kOUjUEra+5Q*+SNj2_|j-L|JSlihF{G2fbm0(!QFI za-ct$vu&m0z~(?niPR9-c9~}DPsmj8_>#%p}5F~eZ7ad#< z-@+Udh~+4I0ITCq)|3m8@zT!`0wMx>dlYq;))bMOGOjR!ZNPC&-xBKphD4bo8bs^H zv_|h!pkH-`uuow1Vt1>05Ne_$iCp-qLT}a?9Pzv~Oiio&5$-5(GFS^1Nc!)==e%v-+5ioPng!5f%)-_R+2xo0oxyog<_u>gN&1> zf^93+JI*q`lr$eBaz#cm+M4tE_IxB+XZt*1)}Wv;?t=_5*ws78Oa9z0G&tM3q?6Zc*!IpXnT_WHET&8g2*GXH&_}q z>Xw=xOH08}tC*)WDR?JXfloC-i8b6Ao!HveBw=P%VuJ|8ET_lcjek*_A7W~k)t?*$ zdSY8WZ|yUQW%E1IbLmG&3fu^P)Bar@l#nL!tgCU;qI%xZ=R2#5L7t<&eEArWlEChd zjT|wS&&+W*qKd{N3?XZGrwgLZq610a`qV{{98QHa>dQbIY4|l+O{Od8ON5F#qs~q2 zNi-o{8Kduhdoud|T{qZTT0m8SJ3J#|_k{we=3J!|T?@At6Se}zfG~pTvEkl~rXL=o z@%JGaSDhQD^#TZSP}a~nxuSvLLg9eVhBlJe2p@2PYb4tla zFm}RP!7Ybh%cEa(%f(W>{6Uc)rU3ys2KffpeTwUy#m{b86hDQRJo@RBmLP?soxXDW zwaqdd@IRFH*uaZR7BmnEAbq4?Ro_N~b6QBCRy-`vv-xRGg&P6e&v~kR_ioNb!^d zF+K%v8wuyBOn(Ehm(6widnSRU5%Ry~g&auDUS^Tz8?#z^xqvGx1Tv4+uBhD?TRCJ+zppBNKC(-u8RDpXbn(zS#X zcL?+M&@~D!n(=qV-4Cs^@IP8-;8-bA=J$-DW^b@7y#%5ePKd3#k$C6|Xclx+&zqKg zj9FA`IEXYi0cd3EPU8wiE}&*Yat7;6!-0`(s!%a?<9yNW-SeYX?IY3@oEIs8b0DV$ z$_wz2aR-7Cf(Ev=?S|tR#WX=0#V(e-kjuU+gW?xU1+uANFqcX~PbVd$^8;Z7-5knl zw~hu1Gm!!>!niMyu6*^a#xwqiasTc2?@}~tFtyOna8jT8BduM2I6p*1q#>-}cni;e zg`qd>17Rs=pzoMk@MNxmp8f}ex270%hm|QyXR(q-g#6-2c+?|-4`mTbgc*W%iP0;|>6BQvkPqO&LzA!I);;{)9Ec0~xaLnjRyocor6xuUH^sYp+4E z1AuwK>?KysiMbRoR=n2S~+XA4*12 zpF!4OuQKP;w0#v7MJq*-{W<=0qU34Js^!&+2L-Lke)dUU1$P~_FewORrZnDZn}z43 zoLU^{ux!(o3K!I>k-BjD1FEBl;M0ChMb<%&02!CEib%J3ceL6U{b`>KW$0MsSynNw zXiEj{Tw9uC{x-<8#773kE59{I+Kkc($|H@-{6LD}tA zOQ-a~;z*nKkb3ujT;hd4H24^R)29s@cyMXtmQZMMW+PKTaQxkU0H{+6%6Cz~WsjQp`G8+A#zC+s zn~R#7^ae2y){sdQ5GH~F^j0?T_WDfQGVUt!D_mV*lWHF-22fUxOtQ2KQ3$EflHv&l z3_Jx$$(+Gou&A>bm?9p6{=oR^;&w^&p?IL$kf3y7PGYV4pcK+75IR*hrl+%{k>E0= z2{w`_K)|*T%Q-(rLzpz~Hn{FJaq^XP)SVj+Gd9QiIJer^*en#6Zot`}JehX4T#Av!XlB>>Y@y~) zKcZhCpEPr|PGxCQpkQ@}YAd4v;`*QGyg5-Ok^s;HGAT%T7 z0+kbM(DIqp&38({VqkIGaToPC{w$-=9ql;YgHeIXe5kzK;P&G8T}$J{oC)UX15u$z z+NP_tp7ESD4V&FBq*OFhaAp;szclVhR~PsjpJ8b;{j>E}X+PY3a^~Y}Ua*yOP`;zG zf+_!FT$dSUv&#lMt8jeTmaR~x6Uw+~Put?MU+&dH^6g!Od*cIwa(HK$mdft_ae&t8X88)3gt zDwYu#U{bUaWq$=wKHZ#rSxJe)9vn`9EHO4U&CN9y-Ti<|nl5bJXmhpkJ+~`}EQJH6 zsHhl_^o*Yh$AEtKG1#NqLz@sY9Vn#Qdy8$myN;Pq7o&y5s7stZJw3fKiZyJwO}s2d zcQfj$!M;M^oMPL!#9x)cM-SxgPrTu%;HlZ!y&S3$Uw-1A-8tg7ncRN~R;fGP8mOwr z{Rk_?ubb*NMl<*eJ_LP{6B(!0w+~g}ky^#{R0febL7qxk^4FZl2iZ6Y%G%3+@#zkV zl^g-^aoG33tCa7v^VC~q^gW$_XwU6YbmrdGn4CLN;8{03s0 ziD6v`v;LvOD}<2VbLF~^D%D))Ld%S!2y<1n$e#vA^XtY`5YBN}6UR?OK~ zTg=DxQG?fwt97sWfe~pk;T1LRRu8##9ilOup1%GtCi@QZ^OrDuwt>{G#5bzu_L6di zh}d<)Zh3sjnAYGqe{$d9{4GKw*?VO-=AickOrl!xQr|22MJH?&+Z&wb+1Z^!8Gl1U z%DX0tk0S?SM=!oBRS44b~) z#V-;a#13jPt_9StRJS6PbkD=ZPK=py*4=~hRr&q<5JEmZA6wD!EocmnsGNA2W#(`F zTq6A1L$o(WUja=_fkz#*Y!ID#h zcQ93(djt_>KZ%ebmg*az7R=gvOQ|hhZBn!y$5@i9v!Sb==AO!yOBshLJ+gnUN8k_7 z!rM-d8G6If3=_kd224rdaH@Ug3KE~JHXm7s7k}Mcr{Lnu(xOrs`3ytS>6v=Z*(k=? z74xMD`Vk|R@}Or$?pC`3WiM0pU$2(PGWNx);IfZqc)k=mDP@;s}~-h(TTc7@aOU?7o6q~ze6wB%8L2DO#xiInesK@J&stZ#kgHNh2f#w}&C^yLD%*CZY*jRFvIv@YamN)-B$Z85e z@d=WrT72CIsL0Te^6DeWXN%Lpw^mR>Dr~trF6X>dVA+MqI=BqJ9W8G$F!(?%O)HfOi3i-d>5^kuFxDH>O7x*y~ds`G?(pkp7Lodfh6nv6!k07Un~N5tng4_Js-yA5Kl)=<=H##(?$emQ^iRl~6NT=msx zq7QHT-7-RW_P-t$`+qzv;-H;52Z>FItOr`|AH!(@%1-ga{h+9*tCV7m%fp@n`(0D0 z#8pz&>#Z`+tp2mcU7>OC9q%kcCJm0f_4Qm|_H_`|(MdZ+^=HY0l*7#g=d%H*0_RTBN2URqV*{ARQ1(hrd>YP z{o>FnQM-6st|orMv%p*f((&P-kg+dE4y*tDo@2$x{>GRLky9|`>!n$0T4iwkZRe71 z*SJF0eQrs!-jsb+9sgBI0Wt8Hft#=nx>;XkWgMQsq%#2Or<_TOBj!hiNPNEH`$!sl z`?)k`?m7Daw?(#O+X@Z#X-b5L8M9<0l&@ZIwMoWB2y!}3X4LXs@iyp|b ziafgrWPDiwF&#I&8==;J>1F}G4wYIjx^=JO=IvW!pTFVU81?vPrQ;g-k4m@k5Qvcb zt0yM%f$8~2;~}fi@%E*Zts+oNo^1Lv>c^7adF5SpZ5E!wpTh{t4}^))58^Ybrq8>pL5b1MsLds2^JnO;i0B-LtqJHppzfF%%Ps07Nm=ocek7ah<^|3Sr0hi{DgExhKdZE9s+&1l85KVHZdMz?>maHyt||2r1WP%DKa zEhFMh6#durRqV~N7ntz4*!W z?q%>>vFjVJvO6@6sC6m|Q;6}MKO0LQ%&$}#x$@UK9>3Wq1Hb*+(ch&Pl({^|4%{Q2 zKi>Ne;g(nPynE>pRqei*o~d6fsuaD(+jX)k;HaK6FU2%3WLmOm@` z@J|l6B@w=xT?th;4cJo^-JVq5xa()o4(wCEF?4M{0xaJuM_X-Mr0`gKnwJVk8FZR`wHc zB(WB7_B`lf2+;5zV?^D(q=awEEJ}ByZZp$3+3PD%);hsVDVqk6Rt$I4+#}Zs!O7&& zL;E=WG=HHy4Ato7A#uahYjzyDZY^Vv=Wk|W|JHk3SXnsd-`0lWFX7E9`F>Cl^R39N zXj(sKr@4OqvZaA|{IA>VR)>EuN7#OJ|4Zfw$jQ-*V3fiT-CKLJvHylgE2ZC=Oef20 zBLkCTK&84|Q$W?lu0>~RdXU5JJAO+iInzY(P3!p& z>I{a#%icCd{>@X&4CBQ_)XYw9t;qnPPgpmd=R3Kcr)Q)&*7^(0;rG7A*}L%V_gvX7 z9q*rxW^>ikP;cC=_Sk$9nn6_hENR{?eXhO%sOBNMi{;`A*NzKQL;hyr#*flX%ia|8 z`A3IZP`ktw>qEM`Snie~PJzlYt;H8Ewr6LvsgX$%N_VZ>xyTN#YGxwwtHSI2_5u?< zV>gp|+wJ!@kx!uS-k;CO96a0YCADc;_L)XbeM|gq)jyQeC-*?DPTK8N+BAC4BnRK! z2^(6B3Kg!b+?B2S`Hu8c|J*^+RF6k_4P`K>i@yUZ;*Q)i0?dXJ34=c%?s>q8(vTY_F4*ZMqu$3TU=(=m3N; z_-eM}j)k}5rmppLQrYNYTHkEir(HRO;*l)(9KgqJwr>dI8)WlySOIu+Dxg*pZ|kV3 zxtrw^YX$JoxniV829lXJ0sVJ6AaQ^_ z1e>^HS)dWC} zlfwNOIfOjL<7F@17&#f&mmaP$cKzON$6}MnAfge+ZFYAV*@SM0C4kS#p9%=`gl3J^ z)kMy8O5I#^wMHb>o~zCng$9=@Uw1rchWG@X#%QVje3zeeN|?6!?LDjLVq8vQoo*H^ z#KFP!qg~?m+9{LIrR-a_($R`^uLvv|p|Up0z~h{ek_7c4nuvj?E#NHWEahTfiK~*iob~8S z$L)a|5_oz=XOFNfMAOW2{nl^*2TG-d`-gx|fOIh{lS7+F`55g2uWxP#LTZO9KmRP7 zUBwDf?_Bu$5XJ}S*JeDb+OK!(ERla}x3Zjus0WWGp^MJi8Z+R(2_6-VKy6uYEH52b zc;aji8n^Y+dd3bJh3+8`w_4ZT)T3~)Mdn0wE^@p( z%FS5?0eI}C*Ls5Ix5n8a zi+YBW-m}&XE-!y9~@U(X4^+!KtpM&QbXW#F&fW&VR91yuM5L|T$qQ9t%f=~+!kMdc+W zr~h2?DWL4w!@2_acrA1J5^D5hj5@iQOM)q(7Ab?i{m&9o3O6_>J%4kIoA**y&=iRoZa86gS+h&ckoF9=-3_N5lNoq$h}WSuQO%H(D@}3E_1mHHei7 zM6nT86Wpoy@LN;gEPve%Z4w)|m9-%9ft@YJpfcZ!4)o~o$c}cZYL*LT7_%{84f-kb zoRgFD^u5ObV@%;j{^X>VZe#6rg|E}%#Pr{}1;RMN)yA&AM+rLTii#CG8%@rZ~-lZ%Wpet|FXJ-|?ts-CgoJ}hs!mZI1Z6`;lsCVrv z6QF6SVpprN2+Fy)&A1o z_%^TS5e?i|Oeh!W>pPfj6SV8#aEFTDxHoTYWed@gFusNCz-AbGCpTwJvKP)SJ9713 zWc-m-*NCFrzu&Wnakv*OFaI>_SU#Bkd>x3HQPx2}QD8ey-ks##rzxhge)50&_HEg~ z1BKFNdvd#;l-hVXt*;ak#*75gt%^5mF-dxX^yR08Bs4SdZ;Z_9Ukr)zSAp|bSdIeweCBQ5mF7_uFfNVU17896gMh7z+KP~|^cUW>`O8cE z32a%^Se3cNI_yG3O)!j#$s0uhlq80Xed#&Kf*cVXSJ2kh4&XKIsc4o6zW#lKs$&Ll z7)fX=7TDWxxMh3B(L+}BVe>XUi)j#$==3aPh6kEREu@6=bR5vxJiD0dG3u+X(A}C< zpxIK^anLxh2r&gR*Xgza7Nz_`MgaYQxCr+nb|K5mHy|OdyuDDh8=F6A+`Y=g~r0seJB){!AtAkNMY2)|CvcKxb2YbU~c zn(%dGf4=^VIvbdZ#{EqHu$Vg^b=A25yMTK8OM0b`tio1eGh@K&Zl?2KmFy0`wL00 ztyJJHSt1@ECGc7!A@kpD=c>io&Q0t5gVz&;pXPawJfi{hMH1-X(+r}YsHcN1DUIFK z)~bRr6~!6seSQX^4Vqe7ADz<997Q(Mv|qV=W8AguxI?z2EaI!5h)f4QOe{CT;j`8M zKdwH?0qRG#J0r?GlYGi>)XB6B;JPa+YkM)2wQHm270LNggUAwRh_6j}LBVms=C#>2tx|wgn{8jtJoVPjCh?Jd z;NNpegLFmb@CHF(Vo`j~-j-pB&zKJV=~zCM=qr1ze3Gog55D}F7MO+bk4yR+Q#xk)C2 zITjKhUi*PZn&JMkU$TLA9angEUc-(XYZtgxc6D^kAC#S&)uFfLV>Ab_S2*Er5W8O& zBKu&8f`o_o^x$4G>7V(8Rx3``ZBL1ixcRv?bXrlwtdyjMxK*{-X0TJ(C-l6D_O}Fz z_+T%!XFJTQVL`D5U~>pHm1ckE0#Hg!UfpE~&k*ECL4<3^n$$wx05r2Wu*lRHk7|<+ z*rylmWkT)oCXN8vW}%Hi0erD*w! j#1}645Lrvk5EYM_UXii@VmxgF%nn&(AuI5 zuB{BQpwT7myjtmyeAR_w^&mxmomZ4Zuh|;jtNJLYi)Cach1+ z9xK~xw_dHb2bN20^3$dNxE3Xj0N}H0Nyi(#O%UJV6 z?RhCLt=7G`;%plG6?;G=SIP*LFgLAi;<4X>w4g3kYT$SHl z)#kD~Qodd_>3<2P9*vbgNw$Jrw*Vsrp*(+>e))eKWl;O8{ik`J466SUBx#17{&?ut z(yW#E3bV$1IGjp;oYZ5RJ+?Cms7(=h%B>}zoj$IGsJ#Y@Lk3}p)etLz4zMMC@VOyN zXY1t_4f6XJ-i_(xqoMnXFA>qg0Y|C)Wn~f10OcF6{^KD-3KRAR)R40heE9nsSkONE zJoobadQVD8CA`FQ*MEEH(end#8zC?2t(^Hf8_)JZsByUfp7mQUw^k|F);%?rqNUqo znKC4rXi-&xNgU^#H&WDY_}(P{-3>>xzL93t!Rb$HtEvI;j%y zV}6fmpH(|F^Eu#iv9g`K@TzWqd|y0^L>_D?I#Lv7bkKWzp?!5}QTlik*Qtu}omuGn z_}jbZ{+F4>n-kWph-9U5&W#NK3ymF@NZw7Hx!Ni+PZU6kKQRKDq)#-MfAMz|0Sq1m zDhr6VaD)+oXka1;+6PNZngw=v&1yUn>y}D00xXNdf{%>Gr}{`U;VdHan3B&tj09ys zMk*Zopt0~Sal)kzBiI6>uq9aP4cJK+U&%i~7eJvoN%ut#vK6nY&6yY7GKAh}3Mb7Q z*jmx+k};(s7&s$f7!Wd2bD1mxS29Nw7tb+PAzUh~SbT&t%0V5@SYI3J;`~O_gbX-r z&Ruw#ASeB?9XC;_@UznlKOk@g~9=hKNoeTr-tKNCU|i zF#-s$iuW*{qu?@MQs~kfci0lmBEmILYsH)+mb$MC>2$QZpOuhFnTD0v+{);M$3{5D z5X)xtnD{OWRbuByT*1yTV3grv7G6HU68K(@^vwq~fB$9@a)|OoOl9@N#RyJ2pGKm$ ztl^j@X+9BCl$30j582xa@fqQ}a084Q)Nk}CgVFv^NwDFBTU%`xKY#44_IPpfBQ!PO z`7YXdQp>%7Fs6gw7m6KvHEgD3N#YdYqZ;>ZE523=hIWaq6%)RlYZoCDt*&9TYKXTz zZ*EOx>|3Ee3#~AU{US24GCgr#x`5f zoe+X%na7_yEjWKP=^LWPTVxs^crxE^8>b(bhWXw`6YI?Jl{m;&;5r2EtM~9S(<{6 zziroz9<~hK&pL@3+Mem!B%zw6G!@-!)q60d=vTJ#aP`I^5Mq5F*NuxDX@7{|rL<|wjcfEf%h8wwGb{UEeRgm_FO z5~Ke{mqwx4$8w5!2B^IPP&vX?0|ta2dG;?ziAIXXcZrnB*fYk05wX9MLNXbkJE-ec z`|N?gLeZHCnee30A~L#{h^s&(I3ZsE0sL9})x-d91NxsHWI_o{geJ}KOXvuHzdgM~ zcuCJ3#F7#+Q1d80{)I7xsh>p|TJI&J$}hIgFYzGwYurskwu82UPe@vUti5Bpq?n zHfEY!)Lfz&ygTNNl&%@&3=|Bit*1e`@%z0Q3C`c@hgy$S!0AKItMo^NxuxK=RjPir zN4QVkz^$5KT2{10=hmP&btqPDqy`R-aYmrDmc?^f`rH?Rb{a&REHfw&3<{r2N1_9b z^b3{~ThJme7ngPl(Rt>=EHCE4!_wlzH46}Q{|zkqpRd_o>s{WQ0;~dg5+(k!)?_Y< zvN8Li?QzShXDLu48n2DW^dL~a7~^Jn9I2;lQV|x*vPx-Lnxc*1+n|WYc(1w0&%CK$ z$2~2t(hJGrS7p^78H)0JJff}j8BI5PBg<=&cFHBCbzU)Bay*NJMVEX5+nQ2=6DWY4 znTBTNHul(SNsR@&B8vV)(kW67wo6HpC93c^T$#t3c>yzW-jn_DU#Z0{u@92nB$zxH zBEEmtR>Iv?udYtcvmMnC_`W}uBdwl;8}!~|I4*4y@hDaY zG1}baCov0)xhXabyw|jP>7xPA5(2o!N7W&z5{v!IFnBlC#k_T&9vQ#Nd(6Kg(pV5O zJm?2b^!#X#n(sv2qxM6Bi2zKar-;Y6Y6-}0F=gpNc(%p!V_Ra`rIec)#Jw5$DSzX66X6A0eQ=q<+n~8743EHv zKW1V^nc{>r)XWoktN|dBW(t}$mH94TXP>+R(~ToD{f!b;ni?hPF6{`NHH~o(Fl<{& z(V#@2tVfB#jBQitIEJb7M}Zx8`=7!qK?PQ}1amKc2@gnif(Q_{08;vV6i`+g{H7n8 zxTqWEDG!M@?_P+(K&uFi)dDMqCCkSf*ff3Tfvf}#8r7cl=kQCW$rB9Ckk>R@j%E)L zx7)SVN@U1%TT0c595npU$`h!b69E>Jg6Zx1W1qxlf+g8E_*ju&uodIIF&*%`se!^V zz{fKKzZ%-Y&MWw7wZZw5d~T4kmHfXQ}M$H!fe zNz_lcjObn-@_Euh7j9rGR@}gbV#0O*fDB_Z>wc=xo7ec`#hc$YXm*d)_!}vQtqu>C zy90U{ntV_!YLD??CU|+OIU@XL*tg%S#GSAXPb4B_dyTe(dyk~U8Gig9b*R|gf6}P; z6o}ws3f5{Y>q^~OtRg^PE4jcCF7_ z&tJ|#6j=N};<|9eF{@{+f;6!r@RfMwK7aQ)sYU~=3dt?G zRg{ot>D1sUeM5(^5=m#fprkp_V6en%f^DpVGNUCJN6k|?`!@#pm8c^y(rEu+A5c0( z`Apd<-hSOULb=Wl9W&G}MFJ_TNhvqcnVF|%dH%$w^ z4+hx^b~; zcF}=!5M&(og$8fXk{KgRix8K_d4h%t{g?LE$WYdklFuImESk`Q2t^~nlB-3}fCUDy zMGLzTJUqbkaNU?%B zIFybNS;W=QNRyTcFaeeze>cAMl6KNgLJuZbefX1RBsLx|_yhf5quEWu$|%o5lm=O} zhJXQ0$wo#5zCXYsF-2gYW&}Fhy`SJ=#wF9tpg_NXglfb{B=ChV&so2gZ?csUJMKjP z+Q&^x;SJF{#urGJyxmd1u7IUjs4@1>Il5K(WA09MQRwG`dAE!4*p{+9`N8Z$d9 z9ih7`V3W4$iYEZXceE(l%N0ZtA*4u`D`k8yrhF+8#2QXq=$BW^s<9?LI3WEZ)1~NC zx`X}nm*7vDm-XrXMI27R6fp8r`7~VoJY-!wfvBg6@#!kl!mnUOhMpLr*}xGYYu;RO zCXM8u0eeq$%Pig>Jl!e6)= zlFB4G;yC;Xm&{rtKm(Hs`k4_QrgVI2TsTknMC<3v>TTUivp9_Az?XZ?fC;+$AWMs$ zP-3Bqp>mtfxXare0wJyU4aFB(C$ek`OazTu)3`fKrSc&XqpZB46%H{ClkC$YzJfzM zssi=&@;o#$a9_*wNRxnF2IKKyHa3e*-lQ?|lscr*$QK!)IN z_1}*Nea>3tI|B&pk7XeRjuArXmwC z>4huM36-9Qy!gZHL>d63S`V@}!1MqJ#P=yJjw8r;y!c;Y_P3n4LA);=6sU~Sd ztTVE$e(6wRwKspHq2Wu`y`)TjD9Gdf#OH@4vnZhg>NAqP#C;aH5^ytVizv}=NnmG6 zVGO*5ggDADZz-yM>x2$qZ$as+#a({_P4Rj(S=WgXG$;j=6X1p+*ERG9hP^y}(I#>T{+g;|elKSj|La;K zO0K>LmkP<;xCiTjJa6aq%*U7{)x7CFLHK4TVl6h{C4#(Ny9wgOd2C zPoJ{-TPeFnuchXry~l){_?R6m&`(s*F5#lTmiq#e8l^$}*Er3Vn3m!H)VeRr>K9xYOA|Mgf+L|E81HL@PPwdEM^&@ang z_*B=dy27jd?AgY3t<}cfhWJ!bXg_A?_9q&%L`jmJarv$z1pvF4+v1TQKq=&_E=ECJ z?L;S-Y~CP;q}1cs{Y^mKE2s4_H!2cM1`aB*@r zNI<3GHSNvO4KTy*z6mUA!an8Y2Vwc303*&ox8^ryl!dO9x#&D9bO5c`sBE7O$3UfG zsYp=@(b(p&-Nc)-}SkhAbu)njp)bkvd65FHfs-4Prz!g3cexSn&@wh z*8q0WL~_*JAs>U8W!|!&WQ@&v!0Jo>5Um`d5_tygVznn31@5DB-w!ntkNKLK@YCv@ zoZtXT32oc&tBqw$JXQj;@@a-w^wV>E|mKQjh!cwg}#CiYEH3sx3-EH$gp$l{7Uiu_r@Fh%`;HtKSR45^`4 ztXbt4nZuF+>g`}MYa=93Tk2^F%e)hHnr0M%2TUow0Ip-Ba2-qvIW1tII5)^&eVcOe zhEsdTjg~l8h8Ji>swo^2@07eHa2hng`aC-d-;w88W z6FkEvMpVXM^u_8bUwqZyo_BloC46&juZEt3Sq6-Ztx-T(Bd}=GaPj{Yj=fbM*-KeJ z1Ka0qNxWTIrta1MBT^AG!+$79tH0!9?=_R-?M+YtG~G{1)F6&pG2c?|Yv4myW?0=vr&edB=5q?qO2#&AzL}WY+jShAf-rIN`75vO{yk z1=V$LOLeUZOAB>gXE?zT9eSOv5B~a0w@5*+K1q9&tXpDhE2kFi&0Cmna7fB2*GG5s ziVIDEYe4P1A$tw;&3CQ>I^iNp*wxV=*SFJ@^Kj9;+1-1JybJFn{e#1KN2?gd_CC}I zj5PR~PDGK3^BI1XHZqy3l9P02wZ#;>t>jqgbDg3UK|1){I*ER_^I=U$N=T$;@HT3T zinbpb;Ja_w%2Os>)4Pv}o$vy5QGFUv5*utoEFmGSX+Yv6BiV;Wj@UKKMfsY#ix!=fuhMuC(jw9GxZ#UG$e5i`nO#RwQPa5K zK4RiK)m+PsSUsh7j}`(!`&MqFN8LG;)*RWSAdCHIe z6`_7M(crae&lNBD;EgVGlEPO=>}Bi&TE+cBK=aBn>rK^D%2Lw&d-wGIlEs=1JHBU; z2E-Ea8}5It`v$U7J)UGlncszNV3a67`VDNmnOVn665_r<;lP5H>U}M2E_};dr5eYe zeCYAl+oDC{&?u$!_5wM@^{?#c>fGLN(w_Ygg#{@MPRSTm?t@;MKg0xd${c z!SxJewk%h;fVY1xu};zC7NULyFf3zpd~X>R_D5%h4sF$i=C7l9GrLFE-Gj(`e0rx% zyj)ppsphSD=KyJK$NL+evN^IcJF9WyAqEpe{I0va4%5_fSG3`tt@lpa{aCYB1G|_r zsa@4mJdeQf)na~ARUXQ>bc;0iagJNNE%hDloxrfIU>{XwnWDsDu6N*P^sTp_K@NAF z_hfVFbzf?Rh?-whToQ?*zHLX|ninHy_}YFW(rUh5WbfN;CD>;*TG4&UG`^3z#WNhH zlZ&3QR0cpZ&vb+?;i>AWDk!3ZoJf=#F;;m@gU7=R=1wA6+f~#?GVI#ZUeosa8q%*v z{@oXL0o3>Ye-p3!z>ZaGo>5*VQJ}}TH^t@mT z7ur1c>fms(E0Cv*Y;HS>wSwW9M14XyUn?Ti_cJ!TReVNjUim_dpe ztnrAXLf_;Z{?N6WTOAvh$*{>I+cL@#Odrxsy*YL;Q|3}**0I33PWTMZP)Gq{3&FTq z>(nUX%0s21ASRxN1at#TeLwSXe1m$n=7c4Br{f}4b_ut7^XcKe$I8s=7PU~!7B#7s z5MnZBC6#X|K0+RP-mG$J4mYjwIHnnMZ>PD=W$0e+*UOq&MzP0BR=@uBf6*go{d<)0 zpQLyuPy54e*iWKbyYWvH_a|O*vEqYyt#vx@>l+5j>Wvb~OVj6ohdilEmsEc7e9=5K zhWEcMv7l1Wt4VB_lzS83FBozxXI5UWHv5N#*IPul{b$F@Y{Jv{({$bqZ_*RlRX=rB zSRnEaKfo#BdE3&`P#Hdl#G7^?zyjI+Ah~qft+-nm#l|G>MzVo$+B|z4eN=_KFiGo| zDxG!gtKC23HW_UY){ZPoT}u{|b(e?ob{-$;;l)c@oeNdE>~|8kq1SpjLMZ}c-CXwqg*w{u|!bWiJ0qubw2zEGbqpcv1F1#)#U=+ zTG2hIxDUjtXtO6ZDK1bkqt71J4nPWq#B7bsqD?!rMt3>p8A;%-Zzvl)v->cPuUr0& zg7m*#u^|IKOb@4jxRjAOraPXZy^~4h1q7j*R*@stRzKO~nM+GQ7Ii*+9)Vj0r?mJC-_*)acVw-R=Z{wN!r z=zTMjZ1XXUq{fxpHr1`}Wi5jyg{8U8g$jY|uX@ z9(h?h4?Sm-lkRIS#N=!D?zW~KIypLWf}t~hR2J03ybc{a*%kY6nzY%XlG;3mIR-#c zmFtif^~4D$n)y87rj(I}ELP*2Ac?6c3?s!Yl|DgL8>N*^&q1h98jRMy71kY}R3^b} z(r~5Z-=nLOlqkGscEj^!0QcW619s8tu+8IN5u@9YqNEH)uli4uT^Gu-e=f3^*)Xe3 zPQ;=HO>il{&JcVDe%VlRAT4_PGbYl}I%)Td63y%e5=WHU&-t|Zz1Zq0C5D4kMqc0B z2=AgJn#Y?K&WAPTCkk)DN85}(ZR4dS1I;#VBo#?d3v35DS+C1#I zt+q;-!d#V&0Zn5%!1rtY{h@Lc(1~ce&e=4a8}18+L8{{ZSpYK!04x8+0s5#H(e`od zz5{VwX<~zVF*YV)EqHx1u_=qg!>~vp-jyM1nn``LPR24`OMtg&`VWKpUuQBqpjay8 zG_#dHHMIQM)3iymv$g)8&Vvq^EtDiAN6^^-c=JUn-XSLy8>c3h{K;B4I%5irDCT-SAj~y@hvo9_zJb3w| zlQ3@NX3e0EqzSm>R8fOfI*STJQL+=o1tSFiI3>pu*5(G6%qi%OQX0;Vk}wnYSmOyM z%m?q@TRZyhG{z)QKN-+J-3wo(kTG4Fqh48)`GF=niclXbUEzuPGJ;mrZrG7B@%SuPs zsT}!VQq#CVOe&FRU4cSsS=T|=%_h5!mjp2nDiseF!Sm-UdK}j0dXAH+yc?y;hKdtb zj=gk819*_PLz5ISiB@m_76%u8O>mw}#U5-~XKq^kHN8Hx-sU*kZ#Q^B(U#Eaw3xgn zO>-skpdzBYD9s)JglO+2yYVEc`UE+QdGmMKe>4pu7W=b$|K|Lc@&*ux#6EYC0bTwF zVO|(SxaXit7QpCZ6Y*H50+wH@a^2|uKcJbNo!Vlm%%mBH6GVqt2XR7UoM*#a-YNIp9%?*s*S&JD4q-GUE z97cW*E^$5^^7EDY` zT>Ahh8+3eLyxwK2Sm_QcTE~l4R8%|#s8Pd!(CwQ?R8*9wFmwRRW(#nPPk2*LJqN(g zNZP`Jg40vSrla<|-^!n|PHX_PiSE?Ow(38Td)GUz|Aqtu_lhdq{EHEC;3*~H(%|>} zBCS*WS|LPw7-y0b`fdm6;ienW%Ii|y$_2(^d-B>EE-AL-|Z;3ZQ zfn0O#Gnr8azlKP*kj7$~;Y$i7;3w1h#YLEyldyluC0xl{p8BS}?(X{(|Lx*UuOHE3 zT6^)#>_x}gqx;{E!~+x#;O)h7^DMeU{s7I##2;M_z?6g(005*FK{M|MFd35u9D(-v z03bi5VJxqWZv}+6mJjH>^D39~PRcqqb(2s)1E0`O?xg@o%P;|_+y)dlE|LM2yOFN0 z?)ozj-@x_H8)RVks(rv^3*gfDFhdjm(l`j6MM3^fV1zG-!^rNv^}O)R%!n?3caV*Rtx5x1pXS zHnnkoA)F|T_(_FB|Dj#+2OPwN|3qN@Za;RpbjsO6%xdXe4b&bnBS||4gH<)M za$B?|HOwC0oCgwLJqX!V%yZmI-tc%}a-T-0U!gMh!9$bL{?1Wb)8;VVrNt6ZQD^5) z{8MAjm4<1b3oug{lKA#k%ZfYM5N*z zKxlde1{4DU?>LM*K{UNTM4v%DqyT^F1LH^L841A1Wn}ak5C^;=8eqcGvPifDurBBC z&=KDd?_r*z^$@l20%V|3-a^7*J6qdiU@nL9gUJxbH3**6KWj$J2etfnho2t}(d6{B z715Y#^2#Crmn~j<&A{-J-B@rsID^AIoJKl^>K`W@keZ-|{u7GuAFRT-ri-ofqKFOS z8DArgXmFtjKIhTi>7+$2wL`z7c^D3?lNhp?)us&TIOj47%FMDytoIUklbc_Cd)^*E zDdu2iiV1HtOKobFg;QV`N(|KEF4uj2RUBnSoC&OlD!(0cyZ!QEi<#!~%Fyfd%Mg5s z(|w%GQ#Q+W_bgyJm&c&Aux{S@;s}s`3q7!;RJ{HGJ~;JiB&rljz}|BCr(`w)+nUlZ zcciN>PVU)jrUAX#&W|}KHXY0r4_Y7hdvx-Q@*Jkku>L%a5Kas9&THynE!Wo*l&8F{ z2)HxMl67cl*2Kp6flpseV(@o~x<(s(5|MomwluxRR_WJ@`08qoQP@Pf9&8kDOsC)@R>5f)@UQ#`=_Xf;3Mdt-?<~`1THCw#I`Yy@(FNNshJ{Qv9|1U zpd-RXYyu-=5+yPCr6On$kT_!vKJ|3Pu}RQ3c>_dQOMWB{!Gl_LPsN;I9h3-$SQn(K z;0cshxH9~mT`zsESHema&JJ6SK8OSpqZ+{7z?l*4ipS=8+t`%ryRU=@;3t|RJwcyk zL}t&(Jty_hh$Ov6+9(oUB_Ck$48;IoQOx_m8mp6@o}L)ntxaz(;!g-be;F-2KyhJJ z#uUCyz@9+u^BJT`u}GygE+7rdm;PrD)E+(!{=WiR4sHv3XyDSQD17L1==w{O5g6>O z;;X$^n?EN-9VF|2u!)zeNST0mI6{R`1^O-?^`cR^$CCS?i)nW^F=-nqpCLkzOLv5G zH8TC5Zo*^IE&?IeX>#x)ag2wE=aTbAz(?mK zW}be~`If>#9_e3$A8IEt4*u=jAhj-p8~V0bB-UUFlpezFXVGM+9ETd}I;K@jD~0d1 zcm!qsTkKC9h~cjRN|}?KD{$d!0~_I#a)gipAu0V?>I_i(rVYRLBmkx#n)qBX$U3ZQf4UMAQ7p49Fx zfY|K3G)pu}R|B5DVDBZ9T(!qwoQb2Q>K0wSxvKiSp%z2MmHaB%MA<9iMP~j9bhsP@ z^cv?mI*MVfL9b$}r{wCZch`rG_YQvttn^i_gW{2QX=8Pez~?}8GXzk(zpexT^nWT# z3yU8=B#QeLGWk@Klr@cfE)s$EKM@p;SY1`T3Em7Z(ZUzl41rkkU#3ohi?d{QI2@b0 z)@Fct#x$5fhQ7F09}vBNAlPM%RRylpnOeKwb}F&#lS%36#0OVETr+7U!&Zx_8p0-1 zJfQsqh_@4-tvKW608}52JkhRFRMWvk{}WlD;&c29WWbgp5F7XCV}s{w6IIF{`;0G|C(op(AIOC?z=jB7-hD@IAx6MrFSK1Qk$ z9`MQuka<{coTMqy5tg`k&Q( zC46{L(>JLjxUr_D5?%2!634a6xG$ia8AnU(y{^o16Fk;rESsL64s?FwHZY^4>83ve&nOFcqsa#HujemNajab z@%O%_-K=m4OD`{+MIF47t2b}1g!)`rlsG;QeSV>d@A6J@0JD%QCo=J4!q2YHHTst2gk+oLHxBR@XDeZ4QI@?MQtNJzwp5Bw=xDI zOJzQb{3iAI`W}cax^FooVzR<2_l|wHQ+`Cp+T%vYoK9)&YlW?8{ZAS={@>ET=ab8F zM=e%IA3pMCcEIku9~q`IFg|(I!p%|+b+12FdqTY@sX|Tp3a0^}KwQ01JW1D1EURZ{YcrHQMv|qc;)k;*Pmucb13Ug5n!`=y zL8`OqMRoWq6B5^s!a|x+SOaYNGIaHlG&YmKPsH4>+bAw6VwSLQY)zBWX6VlGUbs|y zV@=nbKrfE$nRxyu^I!k3nLmIfpP0y}#CdJhbA?aWi$YpUl0a0grhM@vT^Mo#QJ~*> z-Ftzb*j!4`KvL9rzxt)J?Pk-@B!*HWUyx?U1BmNHunXNYL`eksRxxM9c>XvhSuZ;%fYvJ z@kKK7U4$%`Nm`|n%tXWyk@cr4gFI!@XmZJa>^T1%?0+kM{qH>DnO5{GzMQiAy?6TQ zdZ1ChaaAr?j@nx-hWx_`;L^7o9O{7s-kW&FLP6z1xc&F!rd(U&wivQ_1b^@dti9J~v<*Y?b7&(HcQ->bb3edF9rb^y&+XF9FYZL38<=A(FkqQtm+rq^(JHF3%wT!esEa z-WQNb$!Z9G#8dSA%`qq({-P%Sth#_{dOqLqO?%X-H;Kl3JCUX6TKT2--|8NoYOb<> zYQEE%yL>wGSn_SgquI*jET*Joi+}eSPL+)=9tt-o`vx9&P_{=(MMdltvdkF&miG(( z?fWTix6$gcL)aON3$k;aD?;v4*N{3O6}1_w-`n*f=TG^(^}@HQtY7vph2D!q^y*L5 zLN~wF@N89T`KZMr-xt42{f0J!n#-oG^GSIVLC;aLg#KSF->lcKf`@mioT_fB?{zS^ zr+YRk;#fxwV)uHCz76ChzLIV9R^18_K9~{NO*P#MaBvV=8KU8j&CK2Ea%i*)ThPWc zMeeg|QbFpw5O3|@=@wQNJjcn7S&wv(b~5qUY976_OnKTC2$$f;M|H425R?17>F>_X zdr02rHxrf9u1(h*YF%Ny$bw)+6FyFfH!2B@&iAPFuG;!ze;AmFg~KkrH9z&O?}bj} z6we<2&kY4ISq0+3%xz5c(&nErrza``>Cbw@h~uw9sq+ixUiNn>-M4#YBcp$1$7I*stwJKn=Y&sG6uCrNrVp#ee-bKA ze;maf?X5da$A>}>-k4B0w-Kt;##qeLx0i3%U1#>^G^|~>{~~UvvvZ<|5*BH&WGJs& z#VewuSrqRZX7ma7Lud;cx!3>`u|BVF&LsnYEgY?WCWiKiat#;|_PmnhR+Ni=njb!b zK>i?EDe-q5ed3`h8wf9EhR^8jvHkq+5v+5UyF;9A)r`Iqf-nqA*Mz{%-y{8&G2A*N zxIiE+*1gApG-#VIu}Zhkrob{x{}dvYSl2f#wE(Qki6s!~A?Na&oT{yb8Wbt}r(Qw@ zCpHH0c)zyNoSh}pRqZ^a&H!U|i&8fpsF~=!YA+6RcY2XL;*A zY!vteH|8z;_kmk$R!#- zkQX59V*iu@>9eB5U3*7J+xQma@>M2U2SM|DgGDq?4y2tcFh|ca8Pry@&~qG%{P}y+ zoAy$D()yTkbCUbU+k?vR&}gsFN66cFTyBk45ok!bczE~ic^jriYafOTZfpi{1 z^qTl5i!vVamVW=p;zoQ?H+o_#T?kvRD8xsb)K&Pgsd%3pma8IZ*j=W3lju!OM40sF zJ<8iZj=|xAxBLRqKPTR~F3yJM6kR)gx#qjc)HG!Fb2X}Ejub5y>g^|wTDtvsNzn0r zZJOt!(N#VEXOB5qY|CRkMDVgU@iE z=Pzw<^XsvV?#5hkA{Wq!Ui4!y>Pf9q3j&?b_!T2?C2t~>_pgov3gizE^xj> zgGbTGIu&iza9A@6-EL%OzYb0{@Y@_QU5?;ivCndNXtfMFr{IxV^fk{_$Q04Y!)&qb zL|&7>x%hyTNN<57M(Ik?iIs~U8_wToztCs%P(>$lo3^oe9rTlWqa6IOz80}&HX3ci zj+ng6KaBJ6b1|UjSGw{o`%FY88AgSb!q%AF&?KYdy~0Ot&LhO8j$6p-I8)lj*Q58c z!YMga5b6F+SpVS0F$AZzY_rVTh+om=DaAD(`4>Mu-gEw`v??lFXjc?T;}aLj;?elx z%_%u`v{+*G3S})8bJg6BtMP3If;LgBnH9Ew$PI60@8*I(!PpxzdnQa(H*#1r>je_5 zz6bwTJS<|V{TW>GD>+JNYac5}ocdG;810AHw_ZJvgTU|)IFMSq5C{8Mq_Dd9}0(g}B$2sg$LUurnGLuU`5ct`Hs}Md@iW?}DXVui=`uTBnoOnbUuFcn(lCzrzbJ;EG$K)SK z#{hHV3;}HYQAgI5Bg(`I8ML)^BJ7&(hv0y|5jn6oo|jz8$NPz@ZsW)Yi&G2^!@sW* z+(+kID^e&L-*O>;f2)}JII44yVntoa9Llm32&HoRg~-;}cI0E&o)A?XiD)6~6CQ4p zv-nY(fLr~1vS88u@!7-=onWuAMk7BIhsOH&mn!I#Arch+Xj634;MXiYB46`SZNLt2 zKP2{S4GLA7IA&Q{7&!wm%lhtOn9`to z-yf=~r49{wK*2!~MUwU*?YM?bxQv2kwgQ^85l+A_u-E$OD6j?jfp*0**Jw!Y{keWV zowEhX_)g4`fc165`T*@khG@PAOkW;Zy~QD38PcsLeYgWW|EiD;9Uyp+SBi&g;5qcI zOc&DJl1*!~$g^H-kv{ZZ_TAyB?ASESakzpV$P}P}%?{U~!rl}Cm4G-wTo}z(ji;Yd zg=8J^nVIx150b>YL*@GBo7OM&`8SMoYGgKBJ=U}P$>S)r?(a#Ey7ZgaLmZ_0a?gMR zq6txfqo2a*&}7`jX?0~L)5|@X9TZM~jt|O14Z(cu@WLmQ@ ztW9Wz?eOOjs`cg6$`G25x$1g!5`f zdQ&B1ns6A3H^zQuS9skiA*?l=d|+Ts32G}+4qb9HCJG9zXz)Nx%g`%X7` zsorKzu23$iYg|=_Np(BeHNXW0m!7genk)er)g}wrx4A1`(8yVoky42I>K~c6KC{Ni z`?nBuDB8TPG*#%xv&GLTFpFm{YL}yGd(IFsNPo;r&%Hf~O^%oc1XyeuPbXM%aWCI+ zx#D*CHcj7tJPJBXKC;LlmClJA4~qKPXLHb(6H}3I;S~-seZRPtmvzlweYxs1>@($i z58S2$(Rb%N)ZA863LA!!bFNuuoo@}Aosq%ct}jw6@E#3Lp7+m=`2E=y8MJ~G^Y2$n zp*%dmb`o`eDe67rf%vyb2#AIFLOPNji4loFh=z%JfAGet`qS3lV&LQjDS$kW@YVjL;M;!@IqnWbF6?$KlXdBHyupryacB~){+*Kq} zHTR$2mc?-8?5B5Pc(_A3*&9RZf5Z1>*-h5f;4)}w#j|+9XzG6_# z{_s=(dvp$O*|ZbT0RG*GNk_ZwG_O3OFL*Yr;nSDBC3G;$ekX}sM#Q+55DpJw1&04?9w8G&Wa zhYZ-LKpL<$L`lrmaC#_&(Ean-@>m?*GU@H~#qKIJ%fYCHe1XO&VwIgYV?A{a^W+%q z?wY@*PVPS4ZdM8T7G;WNjCEbkz{p>7fXV@M0D?A|Ym*SuS&3K}ZAEZzn!?)t^uQD& z?Hf-AwX>&2cx<-v_XXP%UA6d*RMGFUKnjcgzK2%03equ;D9_9pZ#m`-+n=&x72Z9Q z)+P$&_fC#HWQ`i_O+r*w#RH1xLhF9DdAY%}D)4?-s(|GNan*j9d>u@3q6{jFx}JB+ezceVo`hhIcjy7M18KA>^z$-C;jdbDo;zv6 zFslqY)lfn$+>haiysP~o`<~}Skq5f=3ir2FJOi*dVkc{yKcjm!THPJs1y!80KeLng z<7(6%)GntKmo01)SB;<8k6a|;tKitoK|KVVHb22A@;yr7MN#`2=**Q-3r17KFqEH2 zjnAF^yyz5IX(1pB(#oYw`bY4O|9=erfq=iyFqH(C^Ee>}+Ieh3#fRh9VY90VPJPjs zcifeea-umDMsTZPHJzA_zRQSfs`RQ8lXL5dp2!iJ(j@PTk{AC#r#uk*Grz#BD0*UV z!EL=iCxw9$&pnX}6RO!3T!r)4g;2rHxAWSa(>#Qy9g-9@#AL8A=DAIJD^}wM?q^g2)fG zQdd{ui3Hx)dh=VoN0J2bl+9M_AfVl-^q(wr!JnR7J5)PdvFu!ZzdP0JXs80%m7UnwV?r*jO-zKRbdZNCWK=+E}|gFMGyq_h$lYBG>1Z zF8Yr=zI>RW@Hyu+*hI6;i1m4dI#ZM5;4m;LzC|)rW+DWYVFVor}^cvKJ~y6K40%ISXHe;Tys);G`JZ1}dRPIb6^ zQs_v=I_h%QD=O3s2;>`>X(@~Iu&X^7zs#Zm8w6NE!v#&-3kLp~AS`>2Kdu;2(SdLLjd-g$_CW#_wjA zy-ML8AH4?K$ro>gh8^DVwJqD$cwT&X7GB`~_|xj)G-C%`et5C$NHFDk4N7w`wmP*I zsx$c03`wxsCT78h?RT~%n&s)_uu4+m;dbpeZ~HTN?F^YaxPXE;*>Es24s>HEsoMR- z)P+Mj$dy|QU{Bxxra`iOv;O1-JhE!*KO9+zeezGLKO&mIx_P4Qh{?)5*P3P8 zRDcaG|2d|nRZM>s5fR~dzO9<&m;I;CN~`n;okbLXe5MN0ov9s-iqBMRlM6iFu#KwK z?`G>E1WQQ&%Y z&Tc)qOb5C%%Z&nCwp3;CF@=@2VPk;hO7yaLokdOCc05gzOvayBDN{C16_~Q!lo^5| zH%UARgp3^Wan)wfgU!)+$DQ~x{SL!Z0!G?DKr|I*H5vcudCvwG%Dpa&bKIzgjspMo z%fs^iuMO5yR{SXVvAZsNGGFGK3Uo(_iK(XpXgQ$cUqC2qAHTDUF}kGl#NE@#2is#B zOVy+p#d9Pv?)b1a*ibh`;;Zr%4RvG6vhZf>8R$i7mjAw+ZaUFyydrO+(bH_hcn8DR zRkLwa@=}@0w%`F&8mq-P2Z`nv%4u+CA55fc5IT3aSul4c@kjNbHD5nI3?olexH*-n z>$hS^RBN6v5NO|7@?&*dbc&fz=KlJw_0G2GDNg!#)vFvUoZMhvf^r=ikB@jsBm7Qs zll|nsE(f*pI12qT_CV1bVd-zj>tc(H+^Kk`x5frrB*S)vD1=qzG`~pt$NNNSfc`n_?7S3^i0jh2Ba+(@ZFkd$zPm4JNbY2XFc8_+geZK~wLQPeIjf?e&ASM3ZtNdY3N=&P)BPno0dnubA3oFgM&sQG1 zcxBp8DPqGxtp}lp$96QTfZ9+4oYss(6jf zR;LZqZ5Zdae=iowZ(d^K8N>Y;T%zhk(SJjaD#cGNN3=q!_CW29yDY5F)RekwWsg<_w* zpn9+lS}gl9j6GN;Up4>jvp&mer;vVK%oJ3?9@Y+yDb@J8>+>m&FLuGa zYPq$$RrfF;1?vY|p%19KQHca3h=esDr;japINwAg9Zgzo{5_CODZ{H4(vM-lTV^n8 zrYL;)h_8#*GmvT#=4!bIct3wxg*Wv?6$1oXfs}Rg-QOi9Mr921UfSPk6ByKKvruP^ z+uf6Srpt8luV9B`jZC4;9MQmp#FzSl{@f!AUjWD&ss9jwRV3`gfzQtN*d$^&dK{x3 zX11xS$2vyc3njFgt|<3_VqES2$+xMzj9BQ=CZgzkVHGsb#!DXC6x!9n_*ZNt9Pa1& ze~hg}H!1Tsv?kn7$ZS3B%Jxy`tg-|Kh7Xp|z^S#_+*gEiAbKyVj1)SbukczU@?MV-C=mj!v!Y(v*ZJeI?7z;W zBDNCt>(HsI$!T_R?%|fQ$n0$D{7|sFEc=y8eU&yiJS8T}Dp~dc=Ng z5S&;zk?4p<3c;m6PGcW*EJOMh%~zR<7)0csik(1PS~D>9aL0uyyhK!RJjnU2wQPEe z^@*QpuI6qV+ob>uwVZCt$X4%9a~j#=cX_*#>I5R}sm(l9KD?1EQXb4#7NYWq#EtWj zj@flTZ6bw*JK%Ga$wQj|@R-CMSvJ?M=+thxrF5h9Md7sf6Hz+J5EEF-`QVpcKT-iK zQnlwVS{u6z64*)vm9Xyuy7hGa`7^Ly=Y)J>TVXs7v3Z|xjEApK31qx@S&UPPfN|Ca zejpvRP~Aq;Cf`M$^-Abyj`9c5NtH`7(Jxob1~qT1H-_oZb9~;4cFdP1%_lQwoKpyC zLjkR@u^tkix2 zy@m;o!@Vigv;MYGLvK}-WO}NidG@XAY*44Rp(~k4fiW5jFW)JM^n`3#|KYvzp1EVf zS9_|gB0Lga3&BK$VWZ}P*46f3&Au!KKaRd8_oB!#8`zy;aPT&$wV5MfK3C}&U_sU~ zoJg$GYv0p4Box!RR4a{(?yg8z8o^Sg7#xkppwZUTAx2_M5)~NMOGPm-AOQaX5yQ)c zqeMOIErDYLPh-NVz@YgzK6=gaf~!GUQU*dG?~xK%a3oLz6S_xJ0CRzv#C;tas2UI? zfGYLcp&kEK$1X5bU>=763uU|-{C@e?1m^RyuQB3jI7G>Lk&yL)N^tx-;60D^3|5zD zV>D=BMp%sYJhBC|63&(Z#Qq63g#d(>8&z}IfZUZ8Lg*ijwp@7u?dNMFbT^x}cgph- zp%8GExnnb@xTSMzaXXsw8sLP0w<~%4l?kb*a{#bi$cM%ls}W%=R5A2v9rY3J6C7#` zmY!Dgaq>2bP6R~aAj#BxKT4}Rt8WBi{X%wL)QHJ;){m`iEEbf3M49*or9tKb{X2ePH7e+!HJ{;y$?s9P*JzKY=dgtZ;rd8Jp_Ovk>e zlMEIMT>07kPxWs6>B=?x$^)O$vQWU@C)=&@ZGKV?j4!V~AQR*wGu}u<6n?x~__`5G zuW|NshJ6=Ba*(_;p%?gc?a=3zUMxY_&akQFgT@Z0 zMQk8;BBYC^aB2DbEDt5>9X~35aW{$*HF^fOx;ug~I4mYbw_#!}E`j#-dKESO&F^Xm zqRTB`c6?b*#LnS|$@Rg$C@xoht89z&cYGL5(_o~$ta^i%ZSG~FQ=fC|A-m-`!Tx(L z-%A~MK7Ubvi9Ut&g3jR>03hixVpLnq<@4y_6UyJ0245V;p?==raYR14GrcrAalb+d zoBOlQTsb816p!I$R2mLJOOo`5+f`Jx0tx;sBRR4XCp4=_4vE9F0}n?AuC7!DJh*pK z6eAa+oaPfgk<&&AJ|J2!A>2E)eBU;0WNNo)H6;)^PW$ysXUvz^JgD2c;dO; zZ2$P}rwVidgIdhT5*7HJhUP<1&1TDVb}4_)kNS-p`V=TXfhTeuaGJOww0~NiipE59 zQB~q3WNfqP6V5f4MB;1gsSaYU1ME-;7-$!~E|LpswqAwl6}gEX`=4(w87;c%g*CM7 zT`xXZq>2RLU@uWjO^iLB7SyBTxu$L`ZCAu7sZP&y8#oBS|C z=}ez4;ud{Rt^PKR>tgSi^eSWt?51D~g6?uzl4fwaMWg>fB_Pf=$p$)jufxV(J&e7O z2fo$!ntbRbo85u~_kUJt1TpYHNMfK?rhXgvVo-SRGPe4?{QVxoqmH1telns`Tlp5pC+o;QPINKJocY+l#nf%nty^j~NLS=8}-S+Q-GBB30 z#%EJkJM@wvo7g|#RVz>jHZt#Q_`EKyL(Eb!t%BThJa-wQ(yG)}OMo)5C%Q4x7O|W? zFnDsntig7n>#>ZPHRArCrBcrAe=n83@!gp*sqFRT`j<-SI}|^Ghb;__s)Ji4HogvP z{z~O94{XxyX;(p9K=JYeuLb4OSQaCv0fm*TwYm&!1F~VUo({gC+wCq*h}pF_AB`-p zK8HW}xep)^0LsuFq%@pS&f(M(kEKXqPU=Q3+DINrD}-Df>b^!^26saGtlg9^_fI!Q z?96@_CH!^>jaO*B7TuivoTW8euQ~%gs>;{J*f{_{To6~n826GK%*hX;;NgaYTtTrO zU)}8Zolgt8a5osWlIe(f-dz#t^O3t%Dc$eNAHa$QqR)hGwOsBNj(dyxwFe@#EqO?r zgy%;@?Cz?zn81cm%otkSgeI2j%O2HKtAW@-9E~v`jbe3q5I?<2d_2=3T_rBFmL|Dx zYkePPZQe!_i-C#^0_Dc?j`}XsM4`Dd8IJux+HeZD5eWZR`<)V`@1`I!Qs;X@dM|uYF zVSeyH>K!^8E+nhQJ+3)x*;``CmF!2pSV{cp`-X<|)Dw>Brvdkt`&!SW&4c-G9Tv9! za`xD(^YYAka!qnc&iiEqr)=gc5-wkF9Q)9SL|n_rr`=CwEoTZ!xUA=;cDXF*qcVi4 z=PvhELE$eHK`cGuLXB-Ia>#5U+V*GP<7n)}QxdY<){jqc9KDju)S6A-Sf20KsD1VI zu{>7bG~k7ZWR%_faXJrg{h8MW%}0x$qxX2@8=WbOu@%#9Xfvldy%Dhv4QU{RHO5?m z4eF#n_$?A1<1RwXtGN})VIWX+NJ`UWXm$0)7jc{7IKn~o?I*})RK2dtZt)W47+3)t zQ$-K(eOcN7$IPZ?=@r^afLrVhr&%Qm>um~?&Sy4c`!4URhCnqjM$NJxcl#k8Li?xE zQhu#Hh_%m*Ep>6a#3k^UIA02g9}tmCizgH~2+fqFMk4{Q$WNC!ePE)rGdjsM`;e!} zJF%n76>!erwk5L5(8Vy9UpfXf=CrO6chAEH#0t(sQGV?7FA0qfyDvp}o74MvJ4secc->;0|gcx6s{Y3Q9 z8*76J@zZJz`jn#OiS-2Mg_yQC4Ey(X=8oETVBjWZ_5db@>vR6}x5cQmVpk}-Bp#{= zCK$TT6#d#MIjlr>K_hA$)(-9ezJ5;>0>KUB&misMu%HSGd&~a658KHiviWVLqe6Ov2@L@{)|2h3s3)J$8mb zXCn`vQJbNG3mo{vfY#^6KE+Ib$ytiyz&U|)cRg_=Clsob#2(}<5B()EnxwCq8b2h# zBn?GqSTsT538!!UUjRjscl}s>FrB9zI-J9Z`O>d{RJ-2s48~Ho0LAVvfw&{wAnYGV zXZ|>{<(?O&N<0DGiYF)=g|Wj$-z7f{uhLeWUcHNJK*adL9!E_IwU6R=--Ytg3H3k9 z$i??1x7myT)k@8BZ8xe_0aIpr{}vCeUud_c;hn}5DL#M^`FR-qaCdQ5)gu)m0V+-Y zX7tUTz{-4kAPyx5ln3H3)c+I}!ausP*#VTBiFtk4@Ew(H`HQv|eE4>pJzB!*)qiZ@ z(vgfgWjR#-@q8r>QrElMk1ORamCu_fJin75;&dFlcUU^JYpuYLoVz?kH%Mr{IQYuW zs@I`3*Vk6!bYv*tX1yMAvkO%3O13tc^BUt-&U+LsSa zVRUnUWBE`YepAg46-Y`2Df++vY_kB=ZSpdtabv+UF#be>uGXa-O6xs+olc-Pt_zM< zH}IaIkq9n%HWV}_{5j3W-xKvjNuB3eSVpOF5*x75(TRQBJy;v^kwMk|s_%zFAMWZS7UI_O2&v>X&1pnkw%leR_DQ%m)XqjB9Y&o3a#4G%#JLWQn~9;Z&{r zFj?DHEQv))8qLYKm+HSvh-#nCa7!k4h4NJ44s=+*ol`y%U0`rbryv4~hvzj>__D)! zpKZoeH2wLA_}kKY9U_|b=#5Ckvy5-JoYU%VXq4UH`!%w4rdi{9m4Z*w=yVSCtdlu( ziw_|=Pd_r*aCN?k(}bUr6#7AUk}X&Y189}X`waEYCv%1|X;uRCN&rN-j_in=48VG14FVB*hHB#_M zjr}>=v#x?*UWn1~b*UT{Yu`Ac_>;=Mb^)h3LZtlbQ&BdDaz|&dlx>F%03gq^+|GDek{>G{>jNVeHiXF>jwE;C-w5t?AK)?fN{u+&D6E0ker1ZkLL zgF#oSxyhVxsRpRTfUw3yhNsHn2$tL>#VK>CH>DQ*S$gKR(CDfBMh$&s&!YBF9aMEA z<`WQy5i2nru%#E{dkdYG7Coa+oG{nSg@QscesZ7c-npj7ARF}!v8<3vweUSID+Q^e zi;YXxS?%y`W71_ix@jfH;m{5Yp_4#=gi*-lxGkz02WvANz)|K$zvg?<b-aIYU>i|h-xE{q9rHeAA zoo&WR4n{Z*`AAnCRa)jQ7+p2dwca2qE1*j4XRE!Q%+-eq=2me4#4wUc*SPuj~PVkj{pBS zd#k9p+O%D_3Qh1J!7V|9LvRlScL;953GN|S1wwH5pur)y7w#@Wi{S3=&YpbTe}BE! z-n~Yz%>jcNoYb7JJ#t_7`wR&d1N2!20M5c>y6hvR(%08A6J1CkI72uQe--|^I8YL! zL9=ct2aV83D!&F8^#n4Gnh_bq<6s8P7ePv6rMv)y+I^~AH8Ut$9w2fhJNs1-b^{W1yncN`xG3fqCKm=pG`g1%qJ65$y_vHxuH35&W zA6kY?6)p_$#V;A=?EIdLvHMj>D+Ca1eb7`u0}9PjwoT{if0iDaorF*27<*4%B3fbD z<2kS&x}s|a18KFula*++!BexzW%v$jv5nG?;9KaS_Tr>-@zG)M zUm`@zm2{E+3lU=3G8Gu1L-tOPHoULISaM?D7ylHg%|3#+%RYQ)c=;VXg%~}BQv@4w zl6Dx;CuuU55_=E*E5&qfjR)FQn+l7eLV}FQ=xjp_2rBWCMAoKuFVM$+cV#$KQ|#z- z-h&(57tK;?+sY0?7Qu&5V+!i)h@u=U>r)WQz`ejR2J?Xwz#`K=C*R;mlTOyA%!B8Z zMfEMlWsE!jWUG%$IC?D1IeL_&%xHw&oC9f(AuFrN-lpnXbzN07Z1BzJocm;zTp3fr zT!RgWz=UU^Xwh0Y@!@VNA6;2Ar3OY{ig4-K{gKzui}L74bA6BZPlBmOZYK#Sr@1S= zmmBr=X`7)x3B*C$08kvD+g zrJa5d^Yl_AY^MS!+k^#?FzPqS^e^aX>1lvGAhO>T2}->!K0oS~`LOT0YH@ex#Fy)1(wFUYVl+E7L({5Q(C@Zg+3CD!9MP|_ zI20^O^K}3lL$>Pq9OB;jSqO_%cM30FeA|6sg+`Saqq_|5__ZqSYYIF!+tr2n6Om$~ zdNWO&qGYg`Q0P}{TcJc`5RqXKwwu~SIebvLdR?~!vvAs5(N*cI^^=x+9junWL|*?A zs01f0EB9cxg7p{jau!j$=cJH|jw+hfi?va(LEHkzoj54()bf=mzroq}XW(L9f|8Zu z&H7{MQC`?%wt}$W%20D$pYA|HB(LKp`^4*gR~Mhxm&#uo-wsv2?d?YEeK#9;1r_>o zX0e~X3Bd(Cu6g}+_Xu@m?eV}g%Va4+5G=HcRw$wdEfw51S9GI#L-kt7Hm z#C>od*$^r4TMbx~0EZ%Y{QyX!mZw4PO1O0FfjP!K;Xi+&!`*?A${m)<;Of53qGW!< ziA01T8zNqw5W)zPbz;K8pp>-&egl|rG?cWR$c*9#XpwE`fr<3!odgcV?LTR>ACO`A z{j8(pOBf&&Xa?rV1Rz$kCkq-OF;R-XC01-%3-N&XKnw!|#-y0EtDb=gyD{IsWEw@F zfG?{Aswww!!$8@i;J4^TXAsWGN(jp*fg(wrke9-&XtokOe<-H#uSMf4ukI7s%1@;B zvNo@qPv)2T#eHJc;ZoRvc;*d41{O1Wk^GF&10>R(s9o#XXi0#{tAC4Uu7SYvzlvu7 zNg0=ZtW{7p1OBdW%H-rp?3K7Fd6Uw3Z8g~W800l1bUE2Jq)woFIDWM2y z!#ZpGY!Q=Khp7pQTx%_eL6M-F_ESYRj0_25+voRENmU2b0PtGUlxbaY@XtQpw!yPq z#-Nsmd81#HjeUs6RY))%v-%c|^!$DFr?(@50oip`^sOyu8wp_VY!Rnej0cD8x# zx#wD3JC3;>dF)I$4tDi$GNG?;q&{fsbQ(I9LS63zoYpSZe8U{z2e8P~9om*3H2AAx zR@~GlbZz*ZMY#Q<=%~BWI6KpzZGU=ZC|HVAnK*DE#`WHQnxDT5p)Zx6n-bgX3A*dv z3T)l1@J>~Qg%JR0Hu1nFHAS`4O{(-AXS~z!@>|uXez$=~I9Ze)c>0@cX#7VT%~LkrGfXNWd)_C9@x2X!1CRT3BB&-oqp5{n}HzGxsygtSJB8+{W8$)amkU+6XWJ zhmn%^goYXK!k2ZOgs*JIT=m)A&56TI-!4>|kG#2a6zqbh;4-^F#w5F|<+QuR(h=an z))TwnebbHalC&(8kqE4|9f4PAV7b?>0~&8PM@NoV%||K&T_eZ@3wvk|YRf5aD!BTr zoSS*W?G|NU{hjEC{X#M|1 z#UkE+s?2NOiE9Uk?5g{QaEk5KOBmbZ?aH6p?nXJhaAU2pFABZgp>}VUXP{T##<(x~ zJ=N9P_w1-!5EIl%CK9;WXBvH5iiX%cce(>vDT-&7t)YOIuVw%!<%I;06ia_gGQ%)8{lF`-~l({8&v>8pr@9coY+pcE`4iMv-C zmTY&3P#94O$*3=f8n3dJh3v+(Brz)r%~R=ep!A79kV;wouer3k0M^&fz?I3*?QW|M zn&Q#O0t1jTgPY50N4B&LMXG!nxQnfnpS?zT)(H4?LjR{#y>GU{8$ffTBq{HScM_66AYw< zwb*Wcx!5ZRXTIPi>#d(kTy-h-0NE(r@rAzOt5wdYe-L8%|A9yWGq@X+v}>dOs?QL= z#*aRzp4&p=BgBAt*B#!ts1w`9epu!jgiNsX69NzPI4=DX?yalv{B4WUO)hs8!SJv& zIBeZ4bcp{c8*3Yhpo346hMS_LCzz^mm;|N2^FNDvo# zFk57{XoxwrY}=hmwvoF_QaJ(c^s`pKd>xdMS16?5GeG)ofN%*mwI9;;urAuD5^E8` zfnY(n0w9*YHT&Mg3_4q5IH7;KZ8PNFl7AWT*9>REzwk^xu(N!gtJ@X)q>8VaQ0>_K zVb$1a>cvi$qBi&<&?(ktrpiw7O}_ti4Nsz#jvLBHgk6XgWumC{o?oMm0z^W)%97sn z;1{-^0&}MfSM}aK;QSeQtfGKi zc{9@7T06Y|2@g*-20(#w%9^b!kAnL>J%KB;YiU~?Q)U8yTJHU6@>Jq}^OnO|+V$xs zLxxdsz}rYc;tyv|o+PNy<4sisNqXxIoJ`nt=zg-R>Hvv@l1yfYP-ZdoRr&ui*nU_vX z&4Q{`=rvW2*yNYhGcpM%DmWf5$kzOwNOm~A`KRTd%B8zz_>UxC{@a4;28D4>B$@yl zR-3LtCd-BU^)OOvyS|}Hdr7*84Y4)#e#eGYL5YeiT3KJApF;NhIE0?BU^RFt*s4) zuJwgddBr}yxY5`oIe}-4j=myJ!CvD)z0_FGWl?hpUXC)USCYv{~=%UUcPqbZWC4enB>?R$P1a^r96Q(M;4wpFx zFs(Y)!B;_b=VWd^Z(R}f=Wc-TuTxZLOQQv+%{3ahgyMKyqO8Vx0bL}hyT_fxdxP;_ zy7|sq{aGQwoDuN-FG#Bnnacm%9c#hFKv+r1m`>G31Gz{1Qn$;OY%3L=O-1HS-ph+^ ziL430Y|Q zWv$l)XKf+pez7J{N9mu>TsW4ygf5R>34P`zk9RG{KaNT`PvM@17`LK^pK}<3!YSKO zeP%Y-8{%u8->`FI&cL^~Zgz_{C#p|An)~bcO?=BH53rzZ#ZKmSIhdmRDI@&uv+bb_CuM@&*cS#n6I`? zkk9PK80~<}F()V3_uR;;v)h!nynIGrhL_0{uZWi4{5Wagxv_cJC#@1gK{g>B+Ka}p zD9s{IY^dQukGELIMk-zHeisjtotbY(yM<oOrByCJ&7;-(7RHQfaG$;Ec?mJ@xnkh*qkd2=5IQi=3fX5-Rwgffi@ z>5VVU?Ek*I9G~3qBb;F$x*CJzJhd`DyDN3?w)`z(T?5!R#=mis_gIfxFZcef&Y9A_ zDC09I+MNOK&klM1Imr7uGk$OMgsl;TdC^D*JCWj%M^Fj|(<42b4sqfi;q_z0^>i3H zaTMC2g3o9T`Fyin?+f43sjHf~Pn>N?@g2-NMSFRgG_*u?u)JWQi{!s`e?S8I#_O+1 zhD#SHg3xJ+QQ`<)OAh?WHQNHD$@Ybe;l9t%ON5%8;B%V!7eoJg2~#%B$Z=xi1#v z(s|8w$hQfR1BYftMp?z2NFl-dpORP${91Yr?J%XTz<@52m=t;WeY5tSctNA7NJ1_r zz(jx0;xO@*4x+(hxA?*dX^fM6Fly*mF?=$)&gKR0w>;mAOlHYZ7i0Sv-}eV3iw+De zz+gNy;3^^eb_8$lQS_`KB8f^S%QoJ3Y3blLP}{&w*`3VNmyay{=%DAzVIUNx42mlfmnzo$ zuQhn=&t=;7_*|*JnO4QhfvCi}oyyoOe5bP_edB)oG!TN%B6!=rKmix_eCsucnNgoq z2P**%pBcGR;+oiyOlFVj@eAh$84l2&82J{?ENac=>ZicN)!PY*{Jq}+@mtdJ7*1oO zCA**$=u9E(#Sy^?l_j-(MP2!p<$wr-Qu)g0cG>Sn;Mi%t8!7PTQm22Sjv}tw=pBMn z@ns!JB&rMd^Ai+Y;~anc2sYO5PU`*rC&s3s)+x(Bmc2I^*trJcHs}cO%k%0Uv?K$f zv7N&Q>EFYMtU`6M-Av>*4Hp2E{*#WMB?aa48l7Q#~O)*bj6T#w%@$6izmOZMjEgf?iIEgNF5inL0n|J znfExUyXeWyi5>2l`r#EpP{W)p9krSy;mZCSbY$P z8sUJj6-7Om9%b~17iC<$rttOrFbbl9nRXEu2&I@7e;T#ItIn- z?Gx&2$EiWhl1lTq@FtJ`6&EY#B9sRN5eeKCwt!^1*q<b!p|^+FI*&*u?*|OuQ&NF>I{&docV9@^i- z!n|sW&f>fxt`>fids*p#i*PpBdj0>QB>1Ob2{^1tgW*^=aAV>jrkU#rjg1e#aQ_7p z0>?o2ztIV63E`wjI(w2af5QnfIY)Wpz`Sa^UOw^vEf3F)v4rEr=FN=MS2SIvOjdlF zJr~XN>q%XT5=YPUgiOtZrK6B?@0s$l`t#a|i()ar#BPz&e+M*wYxyi1PD9(*6d{y( z?^$dA)q1`KaCzT*fr#{(%}#*>=`jBkk(mrk0OI*sCJV}*P7CANS_I+vznNPwMC&d! zwdZLj@B;;g*(?^|usaW0XhGd$d=Xoz^={6iJbnrPKrvF1Oa;j(xkZ9hST}L}>>zTu zgLNbhVDJ^_Wv1PU3;jh(yd}z20gn7Lcm{~Zq_HSe8m*y9N*y-G>ZyN}nh@sWFosJ+6Krcn{J;h4ti zl7uH%MgA5I+7)Q|J4R@)#z_#JEEW^+bhzjVI_guKOv*2OF2ykWb?$GvDX6CKACVAD zi>mGoZHJ`ap!9Y&^aAz_|6+E5i2czYWQg`XNSOTbTAMX20VbdKuSNs_d2`4ATVFKL z#~JnirHa^~fVZ%J1L1<;ph+wvKBO5-=1Bcvk1qYpmc(`I!5g2Fy_sM)mk~~^?Mn}# z#h{}77Eah`f{eh1oZuCdXM{lj7RhAUfd9OFcI{Mg`Wpa0H{{rt2Uu{)KoTD@j|_*I zlno(*B}_=0i6Kaa;c+VL86ExDgd)ZsPEu?z%l&_HT{Tmy2hMvFwhz|-`B1B{zrV608MWg#T@|jnWm(odn z^h9CPNP&H{e4Jc?g7SC!G#mvW%69SP?#Mc_g=JrTH6Lb8{JSN8cg&5LMgsBd_2N0sZgYn}4*>9o z7x_^z2GtQRHn#`g7((LIni8$ZRDc(&zWPOg0>|Ij==90z!|b013ZV(GMM{x`fdrO; zI0o=r0$v}%XEbDq$0zj#e`C)7?at&B$tD860^_5><9z=5u%V+zZCsL4VX=m*{V`_*f-X%e{C&lO>#7#IpAlF1|#M z$?kHOq1JTZ4NHxu9-uMJr#cx-R6gm5?kX8^ir%}?Y;kHpyRitY+8I;s_PK@*y1*-KLaLv6-%vMXE1%*%_CuAr~Ux^TUE)@`FZcDrnfvzViEy9v7d=*AIZJ zRq`j*o1JXehG$Zi0IMB8oYdk7-5`L#TeLe0T8+Y4E}q)#Cs#Uw{*=r#0k8f3^XI9qxLS2iE7@DGcXjxG6~#l6p_W+wi%JJ-CtbWAmjGM3!4j%KU|{ern)e zZ(1fF06K`y3gJ|=^b}Ry#s1!6!m-?ZZm}b|iZL~*{IWqO>+2JQ_=+;YXyn`q8;ttd zY*Yqj`+Jd?65+oufX*0803GSZNwvv7QSCv6bm>?4b7*G>{#yxMb%+$5nE3T;Cln){ z8i%GcH!t|eM&VE22Cps4-!YMTT>KVr)c+WLT7IG+A{t9$HC`QZ=4l@J)Or)GjK+H- zb^XSzrW!2s2a5>GO?u+TO)+00wN#Tb^IhZM|*=sE(2&d{T z(Y_~_alOEw?DDyt4M_Fd$sy!TEk%J2Hg|(skGv-v7JaA1#lFSiNIO4HVnh>Xo80Gr z?Xg<_^hNl`9&40&B#RpfopNU>P1gSxQ}|z|V@M(rl!W@O@ShM^Ho%De>LuC36dS$Z zY^-{1-9SoqAa=HofuK8>P8wF2!A6;Lb8$$Uz} z`KBkJjzm!=c6)IsHu%FE+3r29oWWGQoSiJp0ZW+VSEKjYR;6Uop35nW2DK6=xNnwb zhG;NhMa)T#rU~z;gQb_*b8wkK-9H6X=kt=gjRa#}nKSD}udENq?qZPQ?zuzdt>~2s zv*SnU)k;G7as@jH1ys&)<%kTRsj8YAZd0W-ZO1o|TL1K6(k5qaJ}&1|*v^-S zoT=h$xtxZobX8{B*WIC$<(s`R8e|0@WNin2dfua6G;wcmQG4p(1 zvR!T5GezMM)uQJx&#By)*H_*r@ma;2*n2U#8n55cu69eu>q4nC+I&w8&vzU>F7{M@ zubbV6?-LnRc$l4a27O687T%ibFHYi+9c4FlPNlBrQn#DS50?mS*fEZel27Tx{b;TJJIrcmUL2E(hc|m8KC9U++#-1_(W#{@~>6_A%{a z#Dtzl)7K`azw$r#Fp5;f4Q->OIe${S{C(&EvMktQ_J532{*!_$(GTr_jrKLQ6Nnr_ zgF%}zh0WS1)r0pf`W=sz1r%SGnz>6?dxy*B)pf|rN&NlBSYgkT>1lB?EOP7>Fm0iOtke3RL^!K?SoMAOBpY!My>#hq6&){9Igc1d8!0h5mXq`0_JXJb z5JdMsg^z&j%h=R=xF?0LIbGPG4ebCd1u{z%a4C$R)2cG^hhwkAX9`_kI-zzi{~(K~ z1VX*sbK(YqF_my030GJ*)Ru36-XEUNFbo7DCdXc#Y8^`p%#s)=B?%~F2 z=mjg1(~m4ENEe#=W^<%?g;5DfJ=;tdS|AQN*S$nWhY*k>JCE>U<&4#GYbvRe>`vnNc z3fYj(?&DOuguidi!5lSWFm7BXwQL_$Hep>{B^uYBtr6feHd^#oyNU3xGe#6!Ze)y8 z**t3C-mbwzwjSr=MGrjIUL>_xCk>w~C;S4?!n~XoKsG1#83U4Cn#>O(+<@cxUjBj@ z+MS9_7jQmtx1PWW3yEie?WW8gc5{07@-175K( zY^z{h)|#GO80<tbA)L4U}kTjH2vc1 zDNxDnU&k>t8(0vG2DmwT&3d&#rv5qOl7ByE?pltU{vlU{EoS`3b@3k}_>O^VRSlMc z*K+TcP~fiMsKnfJxb*lK&;Y|iram~$l>_*#B_aJKZi^W;gNId`_K>>>$q4m;Q1=l0 zU3XpPH1}mhUZ3+qU%1Ajkwn3U&@>)VY~2=OSL--nIn#}sk3<{yJ$Ibyied60+n%aT zND+$JyXj>ZO0%PsR7{WMNMNQDA75F=&(VaA5s96!XsngjyWmDkLmjnJ<~D+D!bA$cxo%UID{>Sx71pc@G0`zE_-z7Y2Mnl15tWEDL{caR#O&18A-y!sz)22 z9V;5ga>Mv~K}4QHUiEjc#gLNNT?)f@KO1H81F)pns}767CpOX|Anj-XgxDd+t139D{PX$Y6)Qm3M7|Zjr>~Ht2Ez zQQf0o?c{7FTzYPZ0ZM8ht26ix+?GKePV&=ZCG+gXHGE_en_IXqv8wJcD6tnFgE^j| zo7-hWP7i;TMsbR*GJYCjWUoSrU&IanDbYC1>S0V)C13oxpxfBciyl3Xi`Z3ZX%vs8 z*m2`l(Dux)gn0DV2Jiu%1l^(~fyA^_8wO5zJTKz%HAR2YZFpZ@IiP6m12|I;`seYp z>rEGuHOHZip)X!l9%;J}S&&I*8{g~4D*olu5l)sTL8(X>XX~ZnfHjm;^tzlwqm<6G zL$eQTEchcbgAMC-PU;p8esZ23TBpoy-Lc7VoU9NLzXVLIrOnT!5ycpKd(C_*XP!V_ z0#8+bprs0WI|tj#!OsDt7zLmtAC}^$(kLV5PzBiHoYv_&?~Y57GF*lFJ!2;5=n0QAfh~MbodtXvl1%zT3l2D_ZKJ7tf-+Hh zNqD@~*iHJVnvgFvcs5KNGwXXB7iy|;nf2I_lCr6!hw_M|ioc`d!WqJQXS`M14JWN0 zfz4K{PJ_!3>fnEyr-!7UNJB$#`HS3iJMG%nUTM_0g`zM+`BxRLZf7;x+j zvpL&L?|U>Grjmt~jMkt<^Lp;Y*3T~8f!FLOssnBZimrvC{swHWsU) z=luFg6+K6yss0D~01id(8E@sJe_**M;JE@PBV4SH_l|zgz~@xnr3w5%aJrqhQ}j-n zrhAxVTcwt1NaVdBch|Bmv<+O*O{$iU6@V5BvQ3xx}+P6=;9@NUu z(1%@1$;86bDq7fn^#ksH?{Mkvmw1SvqUp_yV{jfuLMJXPd%hm00ABzliSgKodq_GA zpKq{Y9tI5McDHvA%`6)X2Jh5i&F#!hU;3d2KBGMLc(0H^gxynz_ifQkK0-|rgPkIC z^Ec-oej{5Hcq}&musWc{jD67|`u&`cuUA~xLTVSkW&oV@H+JdWJ!D)OupEQN3$JFS4b|jS$Zk<;E2cYiX7QfK`eV^fy5z4;{MBw5Eh=-*k!Ho@c12*F7%xlQzA?4Wwf+=yew=ix zr0N<%+y_Vf#_U-m-o)QHSUp}n)7tQ@gXU6bsAR^!$x`gr%TSa4<8T+~eQCcjTD`q8 zUJiBXOeD*HNNg7g4dMJfS}gBaZ`Q+=3g3MR*iR9`^HL$LkaM|)rtRJ@H2iQgGPC88 z)SX8@gH^wW77`sayuSspuhjtnfl8e)X#Ukf zjsyg0I%}+IiQxROGQTZ2pD_ zGUfbl_$PL*9Ry8(pI+f?N}f z4NK$250m~VF_eTiIg#Dc_`BaM4F(>~h8UlqtPPe&EdVSoI+C3&(W?rB@mD3R;4$1G zxz^e2%>9{RS$DNRZzWxO$^CMU6XNbl#z#l!v>DCK=Ng$lRis|F%t`W5R{<@$?)2d{ z+dgTG9k9Bpu#oqgxI_I<5)wkoDB;Cg+ zG+M|_p(r+f*J-|l)KTXkg|CIaMgM~&83f&F#@+l%lHFvE0FlZb6E5`CCW?IBF6LB8 z&TmYR1nvURl93x?DQCCs?wV=k^>-?6aGt8>FdFN`(k-w*6jUjC=f3^8j9f7tb{)q( zyQ58b*|mw12qIylh0p_TKx(x(8%ds)^sKez3%%gu-^id%n{Iz2gRzKn2{bvLAUj9G!YJIOHYYhRH$N!n; zE4UBpH$Y^ft-pOo$aL;BxF6#K&1C8LY67BiB@?0=GVu6N@p7;ftE;8LJTCadVFKA< z%dZTCgIBQ|%S0@992-MSfFvBJR2^C zj5nSCd9lfn^|-1V>Uc_1pkG?@jj=%P}+-i)%mTe1vv&n$jQa!=Gq@czO0(0ujQ4yfF65IVLv&wGXEf z^b{g*RuH3qhJA{FMH8X#J|d}rn4tG(?oLl?``-3RhV#>4rsTDswB)eQwtZ3Zwj(jJ z063Lt1*w)$;69jV!W2^0VPEV2X@zc{+)UnFdYx5TjC^TDETD*SQ2LNqqL@MC66C-; ztk;e1P_P6mm}q41yGZ*uTWyMRE7gYbysc(-)IA35)( z2(Adr6jGT?&w^kFrw7GyZ1;Iz9?m%l)cQ}3Dv$b&d1L(R6GyvqNov1=o<}ziq(1e0 zNd56(%A@hKsv}Ygt{IAiQ2)btUY;oOW}Pa@^g-@LYC~ejer*$Z?T5O6J7@B@BD0H5 zYk>l~zvAOf`x80x)QXk$h^j40N+gjc#3>2G?tKAF*-rUiZAQT7`u{Mc|D*hXzZcX8 z5?Vq7BebE~_cswZe>6lrYCpi~F~WEiYitBx@jN3YZnmbkd7KvS%uqE0j;?clM54jd zY{UulDrHkY{Lh)^5&I$tG2>e-LxL&Le5rVxW|QXeP=7L&;oO;=ybVv2jY7a@VLK3X z(7k}7w|1on4MMwH(Ora}8*R=ltoArpVhaesXBe*A+K;s*`R&%(lH>?P5F>-&L|)jO z-hCHrnDFvH-B81+FFqnH1KF>?L$m!|js&g6*3jjMM7m5E^0@rA?lV~M&1%=}Ft#aA z;Tvz2XGLw}>}Cfu%|PWoG~oW>n<|PNNQD6ReR$?+TMC(!j8eYTD%wR6KC$e|R#p;IlfDa>FE%%VIrasPji?m=QS5%f(>5Yz7|rJGTnXfT6WP(4y6yI=W7Xt zohYVlN0i$OJ?I;Cp?|M7E@A!nyfA~M`<-+Spx*EBOm%b;A?HgoT(AgnI_vP9Zgh%R z4=0(uI&TosVuUSTz~7pd4*10vz0r{Xy1ym_Vge*ME>VCv?Z2#d{vsg%Po=j1&yvCt zMONgXoh??4Kg-A}ms5CPyk?O1!}vv%5SV;}`C3}`%Uru1rJ)Gga})OjKHcdYVMaEbhGA(59Uwpn&;*F=XI#RWuRS2?LJu99ADns z+`e@@Az$y&>uEyD4{C~wGs3)U{|+xd8vH2iVMk^zseI`3_;{y3;^cF=gNM(B_+GRG zRUdyYa(~BV)i?N+t@RsyZV`25wRkF%*S(*_r^*70Dten)-bq2q{EIlG&Di@)E|MyJ z(jU}Z&d;`aV`9FU(8PGREGO@jlpKt2ru^91DH*w*@R`Akcr;yUjeaS@DRA&T7k${h z%A(Ha1OI&mHy)WD5i}mjBm4>W5tubr&ZLighk}2Y>RhJ}2*GqoKz99e2at_uai?_YtUN z%-ef;z3VXf)wajc_9pMFdoK3*>E44yMz7b);;hPEzhk9;c$=I0L?rJ>Z`Xga6u&NJ zt4NpRB4w)5dxhoiW%TfdpFd$E zQ_%CthEXH20aLNktnHMvMB$Mp#mn0BI{`P0We!Vcq3x$xZh#jAE0k>)0i+SS1~%5hY? zp3mlkLBxP+zMMwR+OD=PdB+KH0r_sR&(ln?KjaTy(5Akgjm`>RNMEQG@-y4?^Be5U zwN0VaY86ESfw}|N>H+9cgsfFGm-m?o@ckU6WmeL|d4K@|K2V5_Y0TUKUsg=2>gTq1 ztPE9_0IiwQL+eHo9Nlhflzi=aq^g{qniW;FmMT%j2lyuD$<1Hu*4H*{YNcoTvzc2( zxbv4}V@i)|3cEKc@0v96Vc2_J^Uv&E>Zt=URT1k`Od6AW1befKBz79yd z0p97W|Lb>pL1OaG;%lbiFjV~%yIVbENkv+2$q|W zqaxd%ORn4(;hzG zXA+qN=rg79O1W=M20W&qZ*_#kT`7bcyi|>4Nx6yt!1^R=?s6rtyrj@x#QR~<>u`}K z-wJdywBJs6V5EPY8gGwMLar_nh?|{LRWfZ^v^E)FpbB-*8V{9TTb)ZjW?+`-f?9ecX1pl);$lOKK}z>=Uy_{T0)raW=SQgQ|YE@R9*Q1gj(I>F(d6)v{; z6}(`YW7UdjR7y7>FVP-A;r(m?fsLJSjZclJ?1C~#oC+gqS1!RtU9D6ip0-=)ZbHJC z9DCgv?6d@C=QQ zIr3`2LH2<98@w#Lj-vcSsg|X%W-$h)TAu~)ASO?faN^K2RmKmxytHGF(R{EW~1=riG#BsFLLt=$FYhPAUURP!BhI$&_ElSok1eK^L7mn#Emk;Ml zg$RC`&AAxN!tN~!J3-D;n_d0&!`6R?47X<(7feP4AIk7=YvA0u-H4=l{M*yn25Mzh zR?OcaXS@aDr&A**Rg%~JNvo`2D))84MXj;5=;aGQDsFmfXk#i*&tYg4EJPhR`l>D% zAs}g3rVSciw6*KV%<$(#+}lY0BgH3;?_%x@Z~UfI2rKZ`eVM{%(wxX!#`UYB&kJoF zM_RIw3RmV!`i~9SLh*0*?pa*te`ga5Ph5_ZcYX3$PGi{Xx-AK9u8dQTBD+kr+!}j- zCOB27Jn_tzfC+B$)1(ab{krCSjkUXPuk7b#SpsZWMqOvpd-D49f)7~-@6OzC;+S}& zk`P!&s|#v|_0q}e*Ht{$|1@Max;Qj0`5T{qISV>i}kEwoJ&ij6=_@WAZrtn>xIwoFGL#bgZyx6dE zZZVIA6>@-e7y^g?PdX`J+35Z?q;(8%4#J0rG`9@sV-xSpyH$%n@Su>97OfZ z$Ru_xSov2(8B>!S78G^TC*-v@i=+?3x(Z2wRR|x3j7t;!nZ#^;D&5A zd39oE7#pnih%afv$;b`DQz*?03MK!01u5s{bImPFyLjH&p#NiY)_TLq3%F=Gp2wN+ zc<*%7&NwM6S~>u(B-E2!~+0o#twf4u+V@H_s! z%O?k@Vd)2y8i7Jn%At7LA&r~DYv-S1g?1V*KgiZp71b-(^hG>&@b=c#NORjB@26$I zyy^+%zIk`Y;>vj)f}{BfO9d|JQ?_Iv$m`5(WsfhVbI<$k7FAB5;9*~-$kampFsY}syyJi zQEZ*t^(zn0p*Ia4eO4DzY6)c8)pKOKnwFSFnmbdK>p`AfVxPojc46d}r~5-$zqI%7 zdJbGN3DG~sbzq9Gy2?bfLlcML5BKQJ*ydqvJR&Ak zZMI_l`72=uAKV+_&%fh`8y4e-LLT%pTk!nj69#=+$#~9 zwv4L_c!bQmOyYWMZ+m2-CLnP8fBYI+@~J+QIyJ;TRSba9tN+e0uz~Ot_~8nk_SN;u zWR?Gplx1h-GjFSTk~z=rV~fo#C=A-!EM>Hegx<1bjCXXgE2ZzEHDyuKhj?_c zk5lOUT6OQ>*Q4R3uJF)7i?@oJcg4KNEzfCsMk{Gp0l>(BNHWV_mS--1zNR^#`q`O+$q_n&VuKAn< z(EL3HJmU(DYo4`@uZDI0WROI)C=p}NL``nY{`kFcmgrV7##M~DWm!i=S8%nmxi~iv zCj8-kN524_|7PM%mqiYD{{$}UD0iVHQ@^ssC)8Rpuif>0PCBNDDV=A&ng+3)@3mqv zm?{{C`A3J8u`V^PEvB}e+ID6I|gO$Gm-c> zMVORB;{#T(WvUm_wPCjJRmPOZk~W8{DszVjqDrfNX9!q&t9U&gEDV01#mOnrpYY^e zmo7_~tJ|hs;Jn`wjp%^J5F2EUv}~BNEu&zMK-OVr1D-JuvsN#I8!oRxJAs+fDe?qT z3TkRnCUhTKWj>dixBNccTIx}&?`weZJy5wP-8$^Zd#J}LVel&-NGah5#`jI$u3~zN z_v9Pt$5#r`1QI>u;V+(vQY;`Eh4Vg>LhSK(! ztejv(-WUTEF}!nXup!kX>!z%Dj!HRjx53O>Urd%QkqvpPy*VXmmK`DUPoGt*gplRA zC`qFKhq1Q~i?UnWhfz>LloXNfE-7h{ZV>54Lg{9ZmQcF8r8^`A1f--v8fFOTm>H0c zf#G-als^00`*^>9JQSJxp0(B$=Q^)yvgm#%gc8B@OS}MMpVaE-#Q4`$OVLw@+gFEF zOu@Om`9WrZ?KSf*hzRDP+qh~&9uB27rUbdkHHZ}FzVbWCC6fT6N=b-N(Ki@+^()>_Xk>mq% z-<)r%3JqW#BKUrNCqU%K`M7zr|GJ9PExb~Y&S;@BZ#G1W?VhxnupMaHjOx?YFl(fy z&Q_BJnKis3x}> z@3x(=Jz;w==_0$va;#n%F~l&es>id`7Nh}>bSK@nBAfFPwqN^3<`eC9L$Tmrsk8Lp zF1(w$KbBls_oQIVnGIQA#iFF3kfvNVH7h}{$B`T+4~!69k6o9SKTP~!6GAOBCogrr zE12*4S-BbI&+K_j1>!X3c+TOx(^kkfG*XgaHg{0yKyxPn1F7;>C{GR0iX> z;2=(_qgku7eI9Z1)p;Gt_+jm0Ba6cj@AriA@}&5kjeKqPJhrJjdxs|TKTl^w<05cz z?GkZha#Lt&Q%cV4?DuemaM5sO>xn-c+B|dFc#zJsP1S84uOs*BGkq(iP6g^6Af?579<_$BA|?b>lA#x*w?)NJ8|NYCp*Hjq!Eh z6~4qfjjSOtokuNvTCA&l+;S=NavXR2sAK{L3?C;0RD0tths;L&24h7~N7==RRH%Lg z;rIymw)Yd`nk99uJ-oKum=AkZF?#gqxlRdX1cZ8(Qg&;T!1)YJep7#pC)Fl#eSwID zHN28o>NHO2Bws$EZwaK3!;S@KF&EJu$)cEe|He6NjL;GMeQ|jH^)!0N&u{xeX8YmX zNE|+IJ`KAh&O?dYsU_;B{z}hE>tj8D$Gd%<3IwM5QX08StH-k6UX@*#NxZTrF@HZ8 z_JMO8pA{GF6Y_g)xl55seaS?jz$Yb;koX!~=Xl@c^Q*^K>1~_+yD#=A@g4d|g-HW_ z(Z8Pg|03bep3%AB)BG)uacT(3JbRScr-jPlWah*_00;W4nmEXmM2EED{XAwud zT&yk&XtS=*W{x)_cd=2@vHlnTgYs1Ck8_!Q+sRwAQ&@KdMYjO1m7~8b7up{(h4QZA z&L8BML&o+*QM9MX9pFCk8{yE;X!`OWnh+&|;1BZVS*ByfWW8%0lnLM@e&cj7{sfaK z`j~(I8rdIDCjX^?=8rf_erD4D2@O!b-T(8=q=h=SQQmzC{AXkS!VV}6kN;pRZH1SK zbt`rRl8v!F48(4sP)7e+1#Mo4{QJL2uV0HEpOFjV1XQLOYlU*+!5^<_`i}*tf0#E5 z2pr0nkDupJq@n;e$Kn5atZV0_0{817$J<2Rz{6tza^`Q>29R4fXy@zn?LS-{--pWN z<}8&gVSyG5;K6+V*MoV+?KzjzLT?_j2Jm|S<^WItSarh*m`pssq4u6W_;u!7VJRB_ z_#PCqJAc0Xt~x7lD}3t@{37%(-sn%`k78nT>xM`AT9{{3VEQeB=$X%j>&;vaxWmi%hGbay*Qn}&fwIc%ww}Eer{Xf4k$s3^U z#xr32X5-K?K;Hb{k7rP-%C|~%ZA}2a(hbc0{SyFV2mAzOR_~9~sDl69(<6Rd0891w&BFePnow-n{?hF6nZ6&yfnGO4L-b2%{_94Q zM0Y<8`$5TV_U%Esut$x)O3MK79oO#s`q-aN@gG@R#hB@f+0hfU7(1|VPy1_~PgwrcQR4NQyH~=%$ff$j5us?Sq<;(5f9M{GDo(GZV1Z++6e%Fl9(R9v z5h7_N=D+PJPo9`A@1s1$y(#68!v?@wrQZJkZdIv6uzA7y^5~U`^ z4hG&f`?%v5uPuWcfB$c;0mY>JPbu*2)KH)f(U&kvv9_8n*9DCTY|^d=$jJoN^q6w? z1A8z#{XZpN1`mzhDo1e|m;QU78a+;B!M2U{MGA0yko&(R`oG#~8N5n=@ke<)N-YUl z%uK9`M|eqrW)E_o0Lp;>>!bgnzH;OCPv_QUW_kgC00FwY?*u*7?T8qcK>#{ri}`;@ z?~_pAjj%XSEH0I?X=)4~pi}P31mfss}Xs0}MfHget` z1Y6}$eD2@L)Mt%NdwfyVhi4vf6_r5g=D0n^U9-_x^w0aR@^>DNQ$|J&w2jXUCf z%35XA3Km7oje>zeIyU!69pCxvx@6a)VhevifjEdk{I0K{5*D2Yq6nB-W@cu;hT_cz zQ~$4BdTtf~RZN>u5gL9ynw9zaE91kNL0b>Ima~;`cw_sOT^Ic3<9NS5{$J0BI1g&_ z`SIiX@6A1e$v}>Sk!T!gADBMEh4#MGDh7U}_fY~Q*$(4>Dyi#uKytM&Fe45U%8-?^ zX0?+~`3~bzxU@DfzrIpliZc?8_y3oe#67N+Ly0G7^xojk{*-LwS;EdC0w0G~wcFn# zfBVrTt3Qrl0(j8xr~e-c9T|%m5%ZiLhtz`H3&tt579$zw9m{^2SmoW5<$Gb(nDKbT z(e4hgnBTnHe>GGz7`aJ~Y_>eod687yS7{F*4*Ut*`v+=hnUcw<$Z1_vRMeUvaS(3~>5TUOt+(eL zVs1a%%}?52Pk)U_TFD`4%<|Q^bxxPkgJuV4L~;+{4gr9U!-Jhm0dn8E1Rc8qWRAkS^W9}-1^nKfPF`gX0W|)Lk z1*vq8gNvIi@so<=$t7pDID)C4lev#e#-VzwDDOh={&JlEju-w3o!;3c57pi~jlOF| z>mQn{^~pJ*zPuM!Ka$3#KjBbO=2c$r)2F9JP+rg~xAsh7dR!|iZ*6APuEOp;;|E(S zo9v2+oq@21TP?7I_PNvOvs9ncd)GF8&9DngFJq19V#)K!;)l8md!}Fn zR!3|77S`KsgBP%?_bA+7C5ZVXtj2B7h>T9fQ%D`JRrfl2vUUo|YMY+3rC^f$~-AHwRymrE)=~Is8|A4<&*O z02d|@DY&+9u)L14tgun7<*PAY?@odScIIb~q+c-(u*+E0Rd6uRm`NGs3E^+srJ?jD zs*#17X6}Taf3Mk5KY1%9n#Y307VMe#!f?Cd;>+}{S`G^CVQ}8M<(M*>EcOEw`u}C* zY!7+fB7cn6%pI%;o&@4^INlDCm@3!VuSvU=I;z79a;6-%Va|~de0XjI(mb)Mu=O0b zn2OUhAPYYvA&urywVEp^#Y>xH8#p6O;huE84$T zuM^SrZ(#BDnd-%yBy*J>kN)(ynV5iEQUyB#Bb7ETHeLxPw2Fz>9TMV4XdKF{R{!XV zVk#?*S%pbuwc{%;0{0xL*kH_}HeOF+Aw+`mo$BW)^}hNg7IOI0Anv8mc1) zTGJ7KVb|qEZe!V*EXwaF<_)DiFk9Jnd^yNFoX2g-&`0LfD%U7*(Tk0aXecp=(#x=P zvTL7vC6-~XW@@1wb#f1dp8l72`Wph2%Y_>ZSX^VC@&HpH8Ue0pdi~tE2jL@$f}+Ga zX)k}kHNQJ067XGYtdLIe9(*gF!~HhVL#5zX+xZW~gV5W59MJeeqhW1QNNRoT$wq|> zbv3>4=`y5jfFbH@W3d$psNzn_UhpjxC7$2#CA7>c2XGMZrUy`=9mU^M(_~pK~?~kJrI>}eW17fo& z&z?(coKM}ow-zts_B=95inEJ|T~PpfcL+Uc5gr{Vq&a06R@VcSAT^0eD@-Zw zBp@;^xf& zpYx@8JGs*tr=Q0;qQ4TDXl7U(`BJ~|CpDUY&iQ zl!<;inH*^=KrOa>9k27PQLq$uaZEwcNNQc5W?g~uu%70Pt~SuR!*8G{`3?#`?k|7U z9hdz5CTx^&mM;7E7h(_xrW1UK4IBcXA|90zV|w8jk20_fKI=`|C?ZgAsY^^KpG{an zo@(2;P@g4C=6v4_vmKjh`$Ap1n2X9zBZ?>6N|@EwD1gDewA$yX0;U=_k7cGLuh3et z?)2)j^3Z`(1OSkgp8iW>f_nIWT@!}ice8ci(!=p+Xp=B*6X6DNi{r#U?upo-MMbGT zeknG-a?kV&U!ctZPtW*`pwI$QL-A3%@lhs}XYYpc*h;Y{IWG;k`StGPuIo{Ojq2Pa zHHD6f<)`&Av$R`NYfw<|QGeTo#t#p(NdMwfQA`eNEyt+6;Po@GL~SD{PCul#7l6p% zLHH`wz*Lk4BhPI*!U4bi9zw&^q@>3zi3&79+_$;`ucgVok{UQqvPPZvlp{u0&_p(S z1)x~Fc+!q&d1EAT=4Nqf5I9W_=%UilIO|_25)m%G%JKRNgD$m?duA>tg0pj~2~oZV z{bt4h{}Grj^XJK=O%(e*%BmOQT43wrz^kjPub;cYwus%9D#cR(@e)l zq%UA8fThAu+p>~cc>d7#F;Av3IFxmbX&jC7bsu(=MV|5=VMFjX>V!J#Yjt?GuO04* zB83ns4Lm8S@Wl%je(#8*w(Zk+U%$wpg6$d0IuZD6l1MVlciW8DQ`H}~nTb@Ax^U4- z6TSYq5adHlMp46q(`3numE1k(93!x`r8}Kz>tJw=g3!FzVB3@- zh6ohnKP;_BijPLvlMAh&^}u`0gVz%xvw};Nqo5TNS1`s|vyxngL+ewy4X1iID9(x( z74K`8h(Vor981s|I-XJ%6)vTo0fO83fPzLGl0sNG{wyr_LQ_I)#N^b^uc zg;xR#Bkp~=>GqaOIm8s#dKO@>I=-(1(Eabwe7CCn1mAn7UfI9co?}{{oX=(n!uB&& zipOXJ55p$QpR5a}N}imnSD25KX7c!zkh~^R;Xn))oRs1FEl*?ih$feA1i^d8-zi3m zEpLLRrjn&K%V*C%=L^WIqn7oHqyw{X>DsCkmzH!D5_%&^Z6DIQeV>_l!R8c0L^3WB z&XDarE=?TDDkvxj1Ew6-iHo3dagbsJMAwlh&`^b-TLf52L#-O)9kJeIx+PuB_+4q& zT)Vd&lC_+GZDd2-r2N{2&xJcUWe{vS)nD1$&P-c=c4ERE{1EyJZQCX26Zt)il9G~c zkk0dh9;Q4D!z^GzSqrRY@P-1f2QFEQdiwf?NUNeuz!SK>c!>mt*SbU8GU`y3wBE)e zy(g+tA7kDj#xSi}LNy=TEEwfG6nMxd!R{oOqBj6o^_%oR{6~iDjxoAoe~?km@}))i zk5~WUS#P)bJvt43=~D=dNTMcs_;T`Si;4U*C^vNom*>0k8pmX9<@U@9@#hJS)xndw zQQrZem%2qG`N812M9s~J?zvm{VCm%uGn}S~qlFY3X9?&kg=KIsgk6kW_+eAOD9Wn* z%X`qu40}Mvm|1`xFWZ8}KtBKGQbQ?f3LWmR{&NZWpe9xfBWIc7F2As-?Q7 z(IL0sqUu;IZUBC5hyV}8kwH< z<*s|X%gSTigWAqS81ycd$k7Y0nU+P1s@9V<{XICm9zYe`POTW538KNQR2&(Q3-nDo zhIDMdv#ygIY@a@I@1Gq~A+^Bdd9Srt+4(n)J0(TnUNIoWm7<|)?`&BZ5Z3YRlBD&N z+Ab`iSzyYAQKLwL|8dUfJs{@LN6{?&<6k!Lw?t&LN$hdQOt0!kn6EB5mL%_4CxkR2 zkOZ+aj>ilUqYtz(g?2*BU?c5IBCK(tECcYy#aCj&7rpH52IUvb(?32-wRKM2emJeU zGU+++=3?uqS}>l`D;((3nzGlgan9|n+Ms6S@YGv|IYQg`Fm6X7tJQd9*O z91gc`0GBnNDTy=YeZx78`XLWP zKhv>c0YC{DhpJsuX;<~-AS^wsq^fGPvU!IYu_<=FT_`7bIK;hhBzqwYq;{_J9>(sd8ed<`j8tsACp3Dp%!SVn47G*|f_ z(@g>V?mN4ChSW~K?8@aw9v}55?%sb#tRgPNd!GIv72-El(4NH-K)qw}@CgZDvYxWe2P7ayf^_4n|5*Ie)>}e0wtY2BN=s@R%WMdwbi-aWl%fct*CIinX- zXn?M0+>)&OT|jg@-0wMUW{gXr>gyZlc(y3wTQ6BJ#4D`pU#k~p*lcF{URPg;`R;HWB#iKc%?p5W-%>GSIqO73a7kDs8G zEzRhr1Q^^Ph#en8X9_S-65m74ky?BRaB90J@vHq*!K5oEBu}0=jyweeNq=DaoRE2Q z+UEinTX~u9R=t)GSu7M$W7E?c4T^o#v^ZnXY`1A$`>W!-DCf=G)ooH9pExu@ygf9A zBtVUdRG#O~Buyy&00>+c3kvU_;*FJNu(}DA6TtM0!OKoB0J8?ssxpE%3@%HnE69JL zjuJW10r3?&?mq-7hFrF&*l+9Y91e|RY1wC$R513fk3B~-0eSw-_wK-qH*zBib z-|)2RNv9LrY?4oPjQu)1J@45j8!n}NouCj@X61R(<&O|aV z78ziuw@l@F-+l2aL(RhVOhM3~6@WLG0VxtIVO2hGn;XAAt$aws_oP7E$fMxuq_N#i zomD;}=fK%3_PaD1?uP;S;+&E|+JhLcp^C;uzz1(W7O3dsm63ZuGZr32II^5#7WFCG z2WDn(h>-j2JR)H9h3|hopVI&3#kdR_bx9=OXAi_b7zv#Wj)futoB+@ z8?vu2H!Cv=6~<3^ZKglZ#Qofm6}wP#j<6U^5)h9m)EftdYG8?7{%jsquq0-}a6tYH z7NdnoIXYl$7FeYoUEIQW8xR;FC6Pw1uxA5=>g)&TL!FM=T$K_NpUYbM^vBWd&f9-p zzMI@=(sBD#Ko}V4W>2Oz^zGx`gFltU{NJGa-wublNmh0vn?{cg;j=NxqXvFB zx7d2<-3kk2m2koRhZnf#52qqW^w>%ma)9}$qkIeIvZQ`r$dD7+U|#n{t26`oSdqK2 zqc1XJaa7$TOrKyJOpX*I8I5x+>|7)#)dDq-&02t*ul_Ey>+i45_E7=D7FoD;4xB>f zunHOnp4o$k4+TC&pfQU-qUIc=i56ZtSod}w=AGeR?hEH=OCgo)e5lMzdz!GgXvA<3 z!zhK;5!y{f0qj%!@sB!cZQ@V|o~am8ZQq}P6#RtgG*SGW_D$1mBp)b&HBh>MxHv`Z13)Z_ULR!sh7hGLk92))#G$F00iOM znf*?R0}r|75~Clnn7lX|D@(q;n4Lt&K8|#bPUW%)gQ#YQcZ*(R&>i4js2nW_pUj%| z?;n#Dh+#f?=pz$q-YO~-^bl(Dhby^bNd22;uzJWZJ^J=56%3s$*T2TAL4z3~z7t$N zN)EHH^%Byar6FU1?Jcmtko7FEDpVF&Wu+DZY7(yK0CQ+;nW>;2esvMe*jmxvwX54Q zj-0L^-`%rrOmEhfc6*(;Fb>%rzdl^QvTfmj+4xZy*j($>cJ4meab4Fy?pn5vImsW! zE)q=n3%q~aQrpsFzo+{OyV9y2;^`xF811HK{0dw4SQ=len(L0ENHoHqtn9USO#4`Qct4#oxefMgaHe|enR3;>6!bk{g8 zOL%RhzR4)2ReJSkKu)RV8i^3v56Msm(AW2FLzzB9ncJ!FGW3DG{XQv)*gz~nMuC|+ zn}~x>r%oHI9s10}a^2ebXUg)lW8oQ2u36Z^Ki{zw`2^xHkaCBZR<@q#HC8gkh+dzM zwpiejng$iVd#S5@o*-Jp(@Wv^#pfVCTWLfDPD$ewRak_#JdeuEL72%sD$Lyv-Pv)j z;(IefdfLVVPn_;T#J;1C5g|7p`=fSj4cjp4w^RG~#bR7z;S2v`Y<`xm7Qc+sJ~3F$ z`_VbKs*%R_Hm;?D8P*I3ZPql0WR9%Ol`!eBbI>|;6yGCqW@GMH(YT?^DBZ#3ajT{F zmxBz$!wXE%ap$ICRs+-ZLGotv7aKy3>cPjtW`2gqQiyx^e#m_Hx*Ei3F3#g5xtOi# z^|Ej}{#0@q;RR^pLfa?fdJ^>{j=(Ve(bD&a_ihtCafoByq7Zx_<_nbgl*iym<}f9# zQN};62%(JeIG)lr6llus%UqvSS9N3OOg3;1#c#AErlz_TT4)_ND;<#VCfjFvgn#wH zrnfTyCz`o#px^>0?Yr2Y8-(p=@A0bvAKV&M`0BnotJO8-+c6DQXPvPGC2=(BTZ4B9 z4Sp2rH1D+`=8qKcqJ)np)$LoPSe7m~`0RSg-0M`mSMb?6U@TNoXrjwLgBCJXbp8<| zlBSO&Cn79=-*FpRg>iT_=0~qHN!0uSk3q{th4VWnrFz z_{GK~Hzf>ET^j&~7`(?^RhQ>!N8~Zh*9!Aa>5|E*2#J-&3RbsVFwCU|zoXaFI2DPc zlTqIWtm5whXZiooMj(6Zg@AAQ+B97p6EJy!JL}8fhKxv&WGSyDy4K|@cybDdEg zU45L}qcAn!F~P8_hVo}pX%nu}IlW1`9a;nkTMrEn@E~eK@w@}s6qUXKQT6&2)YeL) zvSQMKO{*sHy^&3%wc+6NWFTaueexD4FVN(W@Tvh7TMm}d=@ZxUR>>acRp>NGs|jw0 z`ptGa`W1appz&HyvuRYAy6|^=$H~Bpo|m{Ri^Ja+LC`U`=7-$$%P3P6RF$L5D|xpu zoW6Mw-jj=8E@w;a6dc4ROW{+8AC8v9s%vU|wWFPhzM<^k8ISKICx;(IVrOTkQce9S zc)-%L=*CyMVQ~TFTyF?5#G?dv7*F-mUT)=4TonUn<0P-EoStuHl|G^f#rwQvfXcVz zUCTnlsr0;e8j400_5G0|-l3(R-AsaXQ6?zOrPTwtV!+UNJvnY=Lavc}*UhV2hc8lr zdZpeQrr^{g(|LetL|n#F1l>8Pn;)#bdAtYwnu;yWCqdNqkXarH@$r1dpfb-Z#+$RzLRbMrhZKy7ILuDZqpzVFlET-~dc ze7D@~BV=0RgMpE($M^RKl9nw8S`UxuV@1egi=KplD|j>w$3d&&h!RG4~Fc)Ae)p9;x}aFiJi7a8ueg^mE-$(u@qB zz5#=nY+tlz5#I@w1?ZT0XU9^Et5Fr>zM+BM!~f?wW8zPSFTN>ZOd@pgCyL3~y_)5E^~@d> z)gqD86Of555eusV8K27hWRA9+6C=DfLo&_^{-o0y=iN=LD_(N31-XO9BK$9s8;(zPwVPnmLb+Jy%{3Vz#H7hQgxhJw$Z`_m z2yg-wl>EDAN78iI^vlql7*dPeBkaSy+RgGG4&ZrpN0H0Qx#tPmuI?a!60pYkW~|ml zgAQzKK;5iFz)79axA{U^4i$hONI&7MeE)nmv&>t~H3eJmIHOM>DDYfsiGxDbv&CO4XaQ0Lflo zF8f1ZbC!rW>kjaJ$%dLo@9#DQ-?gjgeu#=XEx>PLlY2bXO&=^e>Pic_pC=V)79nju z$}NR>-j3fVcbTiPLWp6}S>vTz%5cZneH_8)*1PoXG}&hXH=B@iSz>94IJ78ti}WXy zB^Ol48#W~=+3>}nnkjn8%mv6dq-*vn+q(6(C-lNG#@~sdR8u{&JU z=;odm!);EJ=Fq4l7NEU6kUc_=)~xg!O>*{1{i5*c?%5wQ{)PCmx2RUgmkZ0mo9XNjADL@LsJW2z8?<1ZbyiZIy+a*W3g%i~k zJ_&u~W~{km9v^rzCJOT-sS)5|BQgD?6dl07X1t)c))xG5VMz8usP>s&^7D=IH(3oH zCEpa_2`!e>pD#MJo*Q6vp(t&QY>QI5P9(8H>+@7o?L_R|7%RK(&*Y)%aGWlBWdm0) z)MrnB6j7FH&^q=%3dmTwlh@UK`W}z+PKTPe(A>luP+u&euIlF{(h?{TM8zrg02rPB zaua?x9PMD-X}0TXlXokFkq_P6wOuR9ery2wc(`6{`Bsuwi1Zp)LibdUaf*i5(cqa$ zdc|#DCXWZDtlG<+PAH2gP6P+2o?&Sued(89uTGQ_*vlT3=Y4+z@9`znn|=;jSsXL@ zo<3W>^`hG}n)2*LR)fWR8Yq5@?Kb*ms_0DJb2_gm2ilsdOrI;Dlok)41#GtYF)F}8 zUrV+`A|N%%x#P9)`Dgq-{(D3-g6AOS1<Ok7Fl?h|@1iYJXyp2!~woNbGYEqu)(hx@;x7 zmB=7ZZtteS`Jbq^_kr!e;zDZg5%daGZj$MSaSr!Y$L`z59}X9gk4*aCfWl=xtxYr_+vdqf!K#6T&SQ6e!jyWi7A z+L_U1xzp1rEB%{jrNB1x=1>y9A8p{oEB^%7t_f>R_avG_`NM>Z0^84i3$7*3K@q!* zW;qc6^gi1nnqIvx4Ixp2-$8kb`&%klX;FNW`q3ZB#wLfWkmS!IPGb9qd}heI!+g)1 z4b>kIlxZ<~bSPaS-pCgAQ#(H@W#R(6Hy-h37s;mK^-a>ogi@F=w*yD;s(Ha3&>Izl z_l_|{2sxoX+DgX#hVrMcZEyl2cGmD3>?F*M&CJaHTqmiyi9RKyQL)(yuaA|eE<59l zfgy$|l993|6^)a0%DP8dr^;$)ioeDp^ioPkLBuU2Tei4ziWgaTExL@08o^)K2ZHMx4a~oIDzT_f0;AjB}X`u6!d@u9&OrJk(}2cU!k@cGp8Mu zY5`sGDGSX6vOl{29lPAv9s48u;~ksicv3eOS2ONmqYctOkvz;@3VLy;E(oXxHJ=H+ zeDB*21g%LFFgJqr6e^Kt>7Lp}E0|gt@y+wgl&I9b#q3@|ln4{+Au~;2w zcSf!w0q^mKhQ+|@@!XYcILvphFNxt9ko1j;pZ*jDzngPeY)0RO?-pL)VIXG*mA}(s5di}eQRGTZCY}WKYPj-CN#rZPXZyFBV#dYESqMl6KYAC zBQUI*i)1ls5HJjfHB5!exuh(PHT01iL97QW*2r}{lT&mkX%vqWp3`5xd}{FZb5zG` zpp(TiCAZNhs@t(I?A6J18q8^w8bmDHi98_wq~GXDv!H=zNF%B)n>-Dh+T2e;8>?vFXsu`ltO zfPRim%)4N42#`4g>5=%qRjUIfwyEh-f2O@Q;OuuXl$Cbux6>q&gpiB4$vAl-sD-Z2 zW#JbE9CO1gkHJ%l#?=BWMgR`TY9s+0eX`MiJnJUKzWQMu6P8;19`$A3TqPti5 zZ(o=X;+}`UMR_aLR%6DQm=SFLcw)ifBrB@^?qHJ4U`l1pK#l-Yijbq#-y6Hk> zp6U(1!R8N+eOpX#^`UO}z}ol4WtuH2;HDiCNmNes_`yIlr~Mf)CXqH!T5HmWQPT1>d(T{8jm zA06|PHsr;$W}Z#=^D5wWNxNcTdKZxyBAw%gp_6bTT_LW6bsN9pRAX>@aS1v-vvYxm zX`O*WuZN8|nr1dq!ly6}AgK__QD!Z*N`5O9kgz4(!i=1!6^*5&%Wx^{W?T=zFJq+T zQj^>I$b+u;$WRTFFhf_*vt^!>Ru5yd@K9^3J|l#InQ+8RSwoBZ3B!D-F@yI975@BH zg<|}5ciC+J5luvS$YOG3*Ycsk_}H!}4Vxk(&8^6&veN`~ThEq?GA$We?z?~`uq8Rm zYNrTMQ&q|PPNywvY&`!;Ow=l=rE);`r5Kn=^9tKt!Xtn(BruZ4k9%h#DoSU@}d5gP503B;seJ&Sk>CY{3Yyz&B|-Wb2jZMu?N?X zogNT$T)W)mqq)sR`1FEywOgPxl=w+?ePGuFf6NAlp%ziipjNAOW2K(3fx`->hB6k{ zfD>|-hT^?5Zs4%QxOS)bYXb?1_yBi{*a`U*!R_Uy(OC;%<}0Rl9eAhB2+B^78Dg%6 zU~XAUD}zM^Q`8MQeCZ(fzF^YVep17WdR3|9I9r-GK$Rc8g8DvypoYe*1dDA3Z! zzLklBSKrv=&J0eVS7OO(-B-HRjT$i_x_ayBO9~K{Mpq z+kdKa;?tau6(1KDXRkoqNhkiilmTCQqPvxtbowOA7sLFuUN*SA#=Q}M(;>fd67zR= ztJ%Ga`o%(S8Yetsu$J8+HD5v0P`ZBSnAtOE>O(fb2yB=VZHJ$C2;QTOVd8g+KoEEZ ziqlY2e5w;PpK17F%h$*T%0+WGyg$PZ;Jy3ncen;r!p-BLq+dy>DkTBxMz!yiv^^tU zYB>yk62)}Ys`BxmO6W^h6!LsD14ZWc<51nU(|8I9)*fUy%XeEE@x4gQs>Nq3u))=> zr9JDx5yW`!tfkc|7;blPUOO*Jft+N3i%?v?V}t6zgE*O87VH|q*FFe{?L`k^)3IVz z+rVNQl>HPOaY$=pq zZrxQ$V6Z$Sim0&c{s>PP>{P}!3uvH3&y(t_qoZ7tiv`K03p{DfHZ(CDE4^D3&b!HG zCaD%&nQsB8sd9)jz3CFOdsXrG8XKEj#f|1$wUSY3u=k$43wk?(dYEhg5c)Q$ zq*b|ES@a<%8@}6H>vMT8gt5GsGmj<;%K;g+M?a$qQF~UG`Ta!Q^#*5F{CBdWcBn`~GP#c{Lma?pHa#+AA-6YOV-@ zz-Y2B65xmLNA+cmt18I08rzrrvh5lVRX{5aerC{IvpP4oxm_=xSy7iC{`QE@>BnBZ zl`XT{*dmij?&Tneu~BokeaoH^L-QDPC!jR< ziUn_YOITAy<*Bpt*rz(f&cyV+mR=0)wx7q6w!1b}XEkbV^G;bFR{r>Rw@Dx2om%>6 zx@%M49T%|FJ@2^F+f;LW!p}#RIOeO)@#U*e6dnfCXe&c&Bma?rWqWn|$XIBAX-*^< z`p=QFm%ekhela>!l%v-if@G56yET}U?z8UYS9^t>YJbmf)fNHf^E10gCftSG{LT&k z+EqA5h>Z|oVq)5sDKHj1S_x1{;dC>Jr~m*@Wk|@>sGc`mpm*=HKF{_M`G2;$^D>E88hn@M&b|Q0Apy z{cL%nNpI2Hh(ul_#*=u@cG4O$0Dy)@3+ssfHW7x>{E;{ia)Q>c+X&juT+dFlIrep^ z9PNJ^g_~txb#C4?bYqy0nO%j*l35Auo{*&?Sah(2hx!xH@gx4sm4{ zWWX>_at z0(BC#J54aM8Ps=T*sYNs#`=hw4Nimj#TCF;(tFAQ1Q-Tc-o9;_A}$}UU~BQV(=f5Q zTuqjEJHP0iZ8~j3D-KrAg~H2yZ=tJ#(tF}fle)h8WP*Wqq_{F7#IX$iu9T&SXRPiYq^dg^{+4 z3J1a1UXf@Kz0Ihax9bX!?ben#@ukbcR=D9_-Kw9D9(6=)Ux5&Ft;s&M*9>_JB)RQ` zmmx}x$j*3S!A@lC^eiXZZyp43QAHe1Y0tnaW&7;ByRr)g5(%)*29n{!LbeVR*Yl#gq-#T5n}zA~Hq1ywOrTw|$Vs(6 z(y38Iz#*&Eqn8Y}QdZ@vPW+xJj`o{;%Qu6c$%C+pF~s=(dPLr8`(77?a{y)LW^}aJ z2q+OL)xTc^mgg=SiVjjuLbwUKONZyJ}9+bkI}_w|LIfj7Id z=OLH@xu<@+<+>+Ph9D;uK`+_fm=sII=@*DEXs16A zhq!(j;}+pWYl9#;&UjTZa9u~^>JXQ4qr+{TbQP+)zI+jmoN$PT7V6$lj1DLIf*V%} zbckEUwHdghKel9+nV6eC&Y<*(HYX(LDP9=q+wa<$FV3BJd9Qq) z!8ZB+v29xcuhAKF;cGJkS z{{%U|2{T^0(1M&|8doF8eI}rv@pfHW`WuF z2W;2|7obBI69p>W)QxEtaJUN2VUHx16E#_;Dm0C^!FoXuF<3tXZkOKM_AknE-P4WY zO1of0H>1UhQ;+E8Fcvr>vA-zF+B$rDQgCO4ZW988(W6ce<5g?;q^lSRiCXt&wCkE@ z#7?6(^}Sq?%-rjjx+)*@My2);ML>F%&yQ6KmtD@P#=6;-MTC*i)^|xe%{N7ZEo%cT zu>C4*Br@qK>w6)&b}+sh3|q)K>=+s>{Bn4UYk>C=3O(hoa)eM> z49Q@IsXwb^Y?-|Eb5CU3txpdgaE*y z8yg*Cxl@xE4o}h?jDsdoT$a((Fm%nZJbW;q`7Qj(3d3y_hluuL$$h?~DZPAm+ld+*LrX(u(D-`S9agb7KZhYp?t8IqzC`&E z$m)yfYKIT>$*<=WWf3outG>xk@#ybSVXaxMYyrattExxZL=jU=Mc~SoJ^QkjW?L)$ zbUQ+_)c_V`9J(y+yYoeCzZ6nci6@spBEM2e@^hM*qX{Td^ojkV&3t_739hXl#HBq+9m*0P=5Ekp)1j}$Fd zWJ-#bU?20|=&D-g&))@7?twhI|JEc_|L#)Kmhly98Ir20Ji@`w*mS1xi#6Li6+?^@ z>R?YmzsNUC^P}+XSX9~W1`51BgGTS1g=Xhe_ZX++Z%achE!Stn8_<;Y^D04H$xD=X zfG(@8TmX$f&!iwGM{_b`noLb4A9#jifH;`*QIev{R*qGm{pYJ^n|LnMM^>RfrK6bB z?#f3oIg894=3QH5i~dAmkQoDoN^6|d*ld30@@u|ehOE+f%`u!VjfN}pJbd};yU^#p zM0!2rCQt?Zdg>I?s7W4HeNP?`Xr5v{HyxBaMwbwW? zavA6)B0ZRfg-C9+2S9ogI5R#eTlc5wiaGuWe48-Keecb;9zr_PY|0unx%^zSOEHHF zzD=3DFO-hwOU9WdkajQa#w89_ksjroN8Q}uy&nz~uZ=IkRH4Ni8L8YEfe|jlp%Mtc zpZFAwyIXp~h|)3`8^pn3luhSHp~1_nlUNsJoIbQzrD^F8Z&&}I;o=0hysUHDlR)-5 zHFcxD&f`71zF$XeC3^0>;|l}F&T>+vi+Hc5Y?(CQ8*1}C5>P4Lto(RW5f~8gPp6Ey zw+T;wrDW>gUrH*z8mmgX;vn+YSU`2!q?~vOjDWsL5ndAkQmD|lx@O9CP~w})&t=or zXT|A(jBNJ%)nl~BSoc!*(VKe9iMQOSZhb3VwCB(Z71b9gx*P;S=CZ4rz6s=e@5zeQ zHNpX;wL7Jqu{t<7m?Y>wkc@rx;W$C@*=ABY+<5UsD;+fU9_cx;v^nqvdgV3S5$p$1 zhU9ZO%y}@RdoJIx_p;G0!tb-xRTq?ZJ!p}Sw{0NRn)Rb_jqy26F|%#w z@Bs=PFP$oFdrjJfR9lfm>hvv#Qv&s%N;h?K!yErv8B9^bxA!?P(`8YqQ5mr<>}Jn0 z?FKf=Mkc_Kv90qO3M!8tQP6zenm?S)8~L;`v^|l*C<`H=m`C8BY+nn0XS-h0W&kqG z54X+z3YN~0r!d8(GtUDer8Kv!)?>px*IunfRaL<;++RY$MAh|z7vurj&N zg#-MbKV>aX$Fd_onA|{o<8&E~D7fq9UvvTSQHHNsMR=NSK4wXj?}Z^|DE`f^XNbU@ zv?6Ng!s$-xbZzGaF_V!&_RyPxM|_h^L}mkD1jg;chu34wqDD$fFNJLin~?%XpUn~< z+{Q(_J&aC zu59TGR23-+dNPpdvoC$H)6!LcS&7)CX$8&c-UY_`cwbBA4Y@Ujg%y|q08;d177~s?W$6pYEEPa>%kFxg+ zhqGPVh9!DQ2qLH_xJQcL`RtWfV!?RURlT$g9)t}No@gVnuvvZLBhdd{gF6CLXv)mwY(65$-i${!fQ*feSs4#M8Yz58@=eZk6$ zm11C&g-HAEJZV(cRH~Zb;RhkumN?s2DmuT5yXHfGQ*?FeEvRc8%DTklS$y@abOgoS z)$1ik3-;ca*6SMa+~i#OxfB`gEEf@X`5uzYv%)O=YchMqTf}+vFfzsFv46UY@@hPp zr36>gg2)updJC~`kzP8?zKl1&EJVeZD_ral^LmQ|PIvu11O>c1-vNdSB#825sCOMAN zBF@!}Q89tFtQ=OO6hQO}zgGDp?{G{xjB}t|N$n~_L(WS4cr=ne z^Ge`)cc*jD1N(utU(>>Apl;AQ^J5OjUcO&@a$g9~LgN|Joe$&12g$5;4NQ!;yf$Wq zA=x0~A7TsU3T++troA_OJs>6<1NwrVi(cvwZJJm15)CfV%4xiYOB51zG783=++gz+ zT-M?uZm7t_sEpofV%>{W{Fvq&K8K(fr?>Y4{rZq5Q8z)LB|J#+#O(*ni#GIMSNPs< z2^is9A$_tf`af@`>Hh9VAhx(|5_nIr>FxqYD@4s1_>SA;#txIB=HVY#B~rE7{}7?y z20|6QSC+9*j8Ry`ixFxnD~t0equSsLIIJwl)W(aw#IGcPVzq5Q(2lmkQ`9dkhuu}S zFm0MMAZIImt!qq=$!~$gU(ihNeOaD}lx=-4Mt;ILKtp*p_y35Jshg@NK2MMahjRVb0D8v!AqW7 zCc<&bJ|O3b_o%A)YB`(UMA2mIA$nFnzHM_RR{`ieJY7s-%vo(D`Ls7~o(QqHwpRV- z_D%{!Y*8u`S}mLz#{%cI@b)v#q}*R`Bx?QocH3})dkpy6&q^4~G!1>aZm;bz0p~je zkv}2!s`m?@d7li}l|D`K5pd+Cre{b^$~T&=*asiLj&D^(P84N*d1A;X+MyWgFXgQr zwaqZ4A`GSTd%ad=t(vZkhOT43l&xcYJ+V&(PqW`wAoDAxAc? z|B6eH4ZAI19;PqWJ+ z!jaNOP5nsi#dB>YD^CC4esB6_vt%GIx7PN5KT+)s+ihf&==)F^ju*+=_1VdLLe>G3 zLM}zt>0Tao_3z7a1#cfeWlzthpoj>Mk)i4eL^m;i+hRx9r=^e)g1SyU9qc0(&KtSM zI&|Bv?R$ruI@e=&Hmc9J(>7hiYi(c4;|AI36BLIH_V~QLZWG`P@3Qs$Wcgx%UAB>I z(Or5neJn6<`tE`QefY$SBnfy6C;o=>A0xQ_dc1)L`TqZ8BbGrO?MW+OR7p{C7$bdm z1_Qc+xtD8UQDecrG85f0x#pM9O$*Ds_QWDQ#_kujiGJk%7t9X$D?PlNw|DrT&*qDS zA~cT`g3Az?;07^o(EK$}?%Gv^pc}YcPvpo(Cuk!p6AxwIhZ-N3ck{$RSrHfP4<-}b+b+&j)sSy=_;*u+H7zSSi< zR#PnrAJzy_0QnqC`$d5nv?s4NBsCdGq{Rez?c)hARL_#sNCs?O=!){&kM7bBwOhyz zf}2?HDu%2MCpx<2R1OtT`jg3+TB943u%q}0nSme* zJnxiiD15* zVmFo=+44&4sDM*bc*AD`DUCZ5ybAO|D>Y)W+OH#oUl%>6_tZ5umFnCGVx3%mY}20e zG3oR{tfZ7K^-=J8-XzI{^@M;E2Q9bRhYx};o_#=aXr^3VQO7!JV@qk=uW*var15_` z@|!jQPqcl-4XgOk=as+G3E$H3znMCYiilHk<>BC8X@?n*;$12`Q2-v&;LfdEDyIMB z1q9b9LHf=lJKOh+yu5k2IFMRZKS2Hak!B380z@{5H^o?w>7Yj`x1wxY!dtTw*HD&% zqM)<^`3C==F-i4}lhKl5Y|ImVB{tNg(kU|FK@8dS-9@j?6W&7lfQXs>0JEh=PG2PT zop7}WB(mXKccg~9idr|`yl*$zTR6`s2d^`5kQ_gNDC-7y_$Mj9G})SCZN|!zM`sxs z^$74p-}5L_32^Lkz_pKHPA&g^7W6Fn_OT}xw5(&r>Hn1pSW->CG=%243f;y=Ab)ZS z?K5co9o5?RE%@u73|I;LLuu0c*53c8=-8@vz)Y5y2DSgLBO*p{a58m%#(>JkYZkG? zggZhW6TR$PiCKA)g~Jsu@2{r$jDk_{vU;xCJe;4eGmLj!sC;JQHthUsb;PnS$A`|$V+k*x)|ux*gE5Zpg8TyebU6oGhIAx@kg7ElSnnR#Z28DLG>>iWf!zNH-z zc{=xErR4it>b=RfJ-xaLLK7+z?+7^YfNB9Ba0cZBi^4eNrO!8&3YbV@5UtIW3thwd zWS?Wzl|!(yZi=Z?zI%&YiGIo-%f=e(-|&~kqcSm);}}SGacz1n>PQiHW0WBgTtIwOQtx{))BdRL^*(Cw$kJXKerq9iKJQh9eIALh^`{y^eD}Y$h-SNn{WP%qA zZnnkc+3l-~7O|SE^SZe@G20j|)VnJWgvhBH34pFLTy8AUWG>EC`_s?rDpLa=r<*Zb zHe$HB1z^|jpKq4;8rU{5-dvxw@LvLdkS{&{`OA|O=u zf+8z6H!!2`*HHEs3H(P~eVupMd&MC6ZNY>CQ|V*A6{WT zSjV>We6#0D2z}(B$D#k3*G}Eqp*X|-XDzx!zw>q5&j|@OQh0xI8NjZM{x9tM+69xs z2r?BmCKP@kU{)yw<;wLN8;YER@#N`7zjpCup2X)7#T)P zr=Ghdr*9c-8DsM6W^)u2UVQiZx$r)?vSXc-rp&5=-JoBZaHK+(492O@FXI@aW6X5s z_^_K+Gkr_Y5yCxXYtARo^Od`q$P6N}GrDg&)!={F~y^ z<(-Dz@y&*vdZk-#pV>JvTWb4K7)(u)JlMSmpx9`Ka+^qS5^SQi1>K9a`BAUnDy-@} z<+gqT@GSOHPLcA&krG?B%Ah+rU2izrx2t-&hVxbFBK-U80M@y=g2uuV%u!ow9^1o@ z{L&~75}U@oPW!cw&ewU~3yppNhfyDkfE?-Afxua_?&YlMi*auKT+8)#tFlY6E0V^s zf6?`u?vi2syCK(m59%GOULleMtUtuSIr^vXc}ujeDKzVSGPf`maok{d>V7=RFW+!W z=vY0QXt~bKVfZ>pDD)GJ(JnBg3s>9eiek}y$TOg7NFOX>it(ph8W*UbsrOSp_}|?3 zn?e`YlLXoie`PxaK(>?K&2@myj{0@ee&0xh(^PnWttyFpgcQnDCNX0kDe@y>;5LjG z>6+i$J0j-ZefFMY^I*t(rI#T!rlcl0%o1@PBa)mjSsrF-*10H>JT#dh@TpS8oN;sV z`}gnG9VJHQ{CrIA^8T0gIU0FNGQ{Id?#fNmaR|A3+u61Kj|fr(Bt3N9D}L-3Q0Ea& zF(z>-8>CH*C3`fN75OP&-j%>8VngD+fB#;~a|tPZUysSt^=sHkQqqIwaOVEmUUEI~ zPB4%rh*%6WtPf`Thto-jE5Sj=@+uW>9G3uBJ+8v8hczEn-f~@J4THffXdKr1VN!Q< z!T=ox!5O+2o;_nx@RJ}RFMxcj*m(w6m-mx?xM`3_<@Na{w=}ZyEC7Y>+a>8CZ^hg0 ziF43C5Lk!5^i5#KD!Yg&f!Rq9qxHBX%H&c-bejQSwXWEvl?RTRx|a5T7o3NO@_}1l z-9@#1!n?2x>88Ks|HNnrOd&`-SK9+kg#lU?|Hm~#`!S0OFKKc=HIjZi{u=`>^xjvF zFm3IgNYR~w9C94tg!|q2xlB9d&F60zI|8=5HiY-4L(?n4kx>)rdNxF=bP0xu-Xkp{VneQRQrsW`nMw40y(aQti0fX(}soe*5N?Rr_Ry+lvBH`&;H>eu80lg?uA*# z#>2S3D2!&cU!NV3pE2MvE9NREu$?KtSuSpuRr(T?F8|RZIu10Ds31e@_A4`(oib8c zAe}qL^Hok9lS$pq6OtIiUzYc%mJ>pk#>S{d)vo+z#7| zBlD@2A#m{p0CghPw%n)yUVx{}JnwzuSuNh%l|&)Y!$HIbz?f{m9(*&@2*JcBLuE&a zvrQN}e~iQ&4vj`wd7aEUsMY}@EmdY)Q`dk9ik_fOts$NQn8@c>R{$=_XYP`3zzL~z zy=f=ZGM9=WS~9!zJU@(HOZWDoI7$W>dQihhnJ?R@bqz-O0S#S70~$ayLlKEP!MV8$ zaQTLg?RsT|0d{Ko4GIti$iYZ9X$$Jcrt`=!eVL}Wd3XkdPc8f8FPK9(es~8q2hR;n z+)M7Q_AUYnDiqu388+i|1IKwOHu{{K+^Gj<(3_;SUxeRsQ;=9;PbU6yKx7q98~(ck z8c_uLK3U7eB&HweQAvT@ofbB|e=2&|B;}h?y4K!?eT1PDvTh;vrsqq#H^)i1n<`eM zhbZoRbc3v^ZSO|~m#$yVxx2qkNG?=g($&}1)s-fBo@dVbMtG^7dmZP^?oCnEx6D3y zQ+s%Wa3OBycJ|_zHEfDUI2CMRXL;`kk7FPvOzK$@xd!?+lMvlyT$mK@U{xU?G&3fr z0@|;Nls@e~#UGWpo3y&o0QC)mH>aMzW3B&JP1XHn!B&#r*KcdEcjB(_e~-c}ni5aSj9uzfiI8zW{O`*o@4NfM)db_hI1G zS0m2_VR!MC@#FM;jvmF}xxS@AtQ6q;VR;Gxm+MQ*=Qe7+b3pn9bSu|79$~;7XY5j4 zfJQ;iZjh(d#K`kItyVCb$Kr4_8eS4^;>158}p`}aExHC$! zgd&Y_Ghz)_7!z*S@vx|@&13k+z@|=SNpB|n1kf}}so!tSC4JA2W>04MM5`>b^zFT> zRUjX$*=nl0d`!4pW$63^ws!Yn4$RP7)*P?$-PHqgQWB{gDm5ot2yOVEiuenrm46v% zMC`vb!Mz*(!l!y-Oo5dm?H^XalO@l|qC~5OykFloHDDDLy4nhpgSw@bgrrPPZLhLd z4j583VN4?%J_-_6BZPoi2@whoXJiO;WE+=7<~rH^|94!@G6 z-S8LkG6L-wKZ1%!N3u!ah{UrfiR;>CdAUlA_^2ecVQQ%fVI(_<^AZr07^^**_cq3* zL?YBW&>9beS){30Y$YRWe`SSL9YNpzT#;B&Cs{C)cte5ahxVRV`%w-xT8I`LnIdBR6Xbo6ng<|M@*!9_%L5Aat-iB> z5p#F>-hMU)Fqd%_8RtSUz*+RtInsoKpGl>IJteijRnL35Tcg_(As&BC0!K++XK;1i zbqzdg1~jW2Ie@IJnpnfUFDcU+8MP-_&BD$;!fWR_ZIRCk4i*8CKwEh*lZ(sh6>l@g zCM>|)H6I7=qcMUyMeUbSR-jM3Geiv zAhS=fI{kYH2F_C*`iER05WLN>=1+59?Ye049eCXiU+dX69aRm4yCpXObZ7ms*RM$i z1y|7@hhM2=_@OU!jN`G%tj2WjE2j#nr_-4F4OaDwZ2oE-luOHf`eXygJ8|p7JV}b_ zBKg73Z~kAr(zrLHUUfY&96NeMQdMh|aNFnxn*3M3)`NeG)&HSJzg&8V7{>_11@}r5 z=%sMbI$POY4+1G!WnPAzf?<{am{QxK+XAd7qtX=NLENva!41enK*}p+Ia``Vrdo<} zK~rt}o3R%`J?(6>z)Ks2w^5u^QEh~0xe@!6@vO0=Tshn`S7lXjl+j%^n?T_>b2rAb zKx!B;J5%o5u=B3;?D%LypFiFBVofNM2h`^4^Ls^`7_kecDD z8Z(}#&pUYr0a-YV*Y>tBn4^OFiW|72J;s;-n>=O^0$P-Mfo`DhKgDqtF%bAY9+!Jv ziUO@L%~QA>-p~FUp{yvagO+XvMn2E$v$YK)+(o=Mf-Jz&#P(_b64MN>aUuqy6!!h3 zFvd=7@cUP7ZcxX-BeVD1wK}b4u61zvK2wkyP4R1WDN_*^|NUn;v0g1fYg(ORC2V!DtOeElu@`k;wz zlN;-x{?OSEup?lScY!{W@!#8Bs{Wr7{QBrjc=wi0IJjD>^~H837cVn!`(OOu>Wg$| zS@@MGbG0`zyI3%bg+g0)KBFOEF3;Jzm8{gC2a=9!Fn}5 z;Rhd-gJ%%~1B>(x8p9nNDx?vJ{U;ahFgC_Ve6Cu_ZW?mL$ID9(7}8hlCHF<)1P&#U zG)OTOP%+SR=0~D~NW@Nm`cHXLB)6YSi!ortLD6<>em#-AgZSY%0dpjb6I09cMf2^s ze;}m?CrO^6?`#tSX-V2-UUIiwW)%sVc7~rtxT(5WU-1IHL1N7d9th@D!0eqN3@NOm zYoISNAp8o!O__ob7{>*%$N3EPP?y+V~j><#CdBnLc69d?iG)CFTFqhbjZ2Y%5uV`Bj$44^zC2p4frDH@?pZ6wGB!{bwb(U`T~-OeKk}%cX%zSx^JoNbHA? z6Z(qKPNPE-uhWI#TeTBQks{Ip`x1iit#4sBPl5b8nGc7INSmM$#eCn9!f5dTLU%+; z6sy;KbM4M3^Ai%xhl4pQU0reMeYI&Ek7klX0>8@80vJItrOYKHk`SNMuhcv?A<%Jw zWn7&5ysht0`~p$67rr5P!05^!_!KUL-i)#|)zlHDLtUcu2{6bs4RIBiKL=9c`Y=2b zC4#i>ho1~e6OLVP1M*w;zrSH~XKUZxyL%Ut%SASly3kP!_!p<%jXHtG2k$#BUSm6w z(7Q}}=(^X*OZ0K(Jcv{0h}@Gr_CYhX3#RjON$`+UdrQVJ$2lt`@3jp|71()onHrDCYj1;Ir$W{)G18?yGpjL=5EU7|imL%cjj>(vD3tvx#(Y88=PT(6dAuhYDQQ zKTU-a>laGzFq2pR7wjQK%Bgq!Z>?8t0Ha`HUB|H)s0IrpotWs!s;6Wv;F!44Go02f z-@`9v=Wk}IY-QN>V~BKL>`pYkH^fxfP^KSovDhi{y#~}RExbQ;|Ab51=74EE?WRcoxu>U|iKORKO^!tXUmUt@M$j+Urn&M~wcDYL*{n(i_A zlsMV|ILeL(&00%gM_Wq7%_fT)k0`jcC{a0x(iyp^1lyRm>h~^4F_PIBQKaKJ021a3 zFyw~$B>03oP$1Kmso@CcA*P2CmhUI_q#@G?%ZpdVYmk%e0Mq$* zSIP~|+G|I|Ba^?pV;s;DdGDuO^4R30;Q4P2S`BnNBKA38XN1{)84J*Ol4Z0Y7h;zr zlPv7)zb}hq<>DH5va9pYMEX_XmLWjC$d<-NxB?jwp4a7`9IV%;#3u_219M->>uXcg z9mqYix1tz(CanRulv&5VDUYWEUYoE{v*bxacr*BP6nH}7K) z%Hc4D<4g8QE(FzO41Df71-G_?tS2i>rqft212FD|kQ7vqzipDfAbJmy8pJX)1{Q0S z;x7^%M0~(~Bf@fmOh3!ASxv1A%8{ag6F~Jd32Z7*rWh0k%z{7O#YSRlbbhuN(JIA^ zk578*TZO+99j@!2^&Hrsp>-#4?Hs7%UmIJb@V?BUS_r*f%<7&MRF~*H=xic6f;>V( zQK}zn-;3N_9(>w{a(s)Az2mGH9EDH~R?eQt#yNm!psl_fZI_f&>-C#uBmA!%aMlCoWhBK{eRx#><*E3I z^wMIuJTBxpI)F&jgJM$PKmZJl#xN5684&_O z>c>^Q3`k%zQq-{!OVBZ5TT!Cq=TO`SChEPj!!KbztOaF*LiAl~HePz6M(nux0x*w? z&zJA3a;>#<&G5nY4CX};#ldbOmwh&knD^0=;f+-|!-~W@so{Zdy@qQra~LNSe|3+_ zh=I@K^IHzRo%XV}HqoAwmvkDP~S6hrqlze=CL=L=(;@qo@FBD6BS^Vv6bEx;f>%)PWc_f+KC8qmv z8aFE9HRiV{?!*|+T(&yO)4eVTFRtT!(eoc1CiS+JWtOK+ejs#Tpg#yS2nkk_S~uJn z+c}C@b^Mgo?PV2z)Z}6$V|PWW`zd1@#Cc9~Ua7qk(tkANn8c-M@Bv6cu>#uhFE0Cg zdt?tn@n z3vL_gF%lPO-#yXuzB+=%V^rY>+}+;+1QHd;eU@KrL)|1FV&*=W2Ggy8{lcjPRSqP@ zeBVK-kdSbqmpQ3`@})css`kkD)c1llx#_Q$tPBL|%nU?0bVR)2IOwf*Ji$mypQrfp44-AZ zlh9EdV(gpvisDFT5KPY-d+plu0m6-gpuX7g+1WpPmF?Dfhs+AQ zAjNLRsXk1g1oK{-;_)k@3Ry6B^;`ttTtdkJ3gib5gSIue91F`SsU~Y5uG?PnIhlIkH&7B40e-=2fXT7Nj}Qiqnp#F>U$ry6L_=7 zIy5%$%A{5&bP7{)<0cc?P#lBvH;947IcY$g^ev_ooTz`vf$HTO=}OYkcqTg}Jobs) zzMkllGv%g}V!Mr(p)E^Qc#uqC>eA4EB8jxsp-c>&laAzxn{zmi*wKwTXYkK{&%Wq+ zoWQJ%jdTRrqZw@k`@HpGr%>|uQPojN=`yu<0m zC4TDFD3c?N&a%P*)XTkT5aEH@0qQAN-N{~O1Ri%BsXZ`CkU_QrwiA=??IUtgGQ6DyU?s5F-NCDKD+!E(J~ zPj>&LmwV(e>%*IFZuWTmvRb%^^|&U}^Bb8^W-6;$>HdbTbVV}tRt3l0RL7w}{o@Pc z{N5VPeAKb~7T2cJ_G5$XNdpm^s>)06k0PkiGY42iClDhLIM3^X&T)Fl1Pd0G$+6F5 z#&jS(d^rX*@Dc2tE>A&)`(mpsXX*Vkxii`Zs8!LOj0w3kH+#h_K>x@K#e5h%*kjPt zt|_>r>8$D7df)ZN{xsA6RDl`g-5H+1daw#uFEvBWtb0@>XGx;Hd5+%4D>^T=Zhpg> zS@`(+xQxi*tnRV+77+!nS4nkwK)Add$2o+aq<881pls9WcSQ@>3#3GEPW3n#3 z3cRWxYa_SliA(^iW_DL9T%M3dDx4%-PRD^9ZfbcIkqfXquX{FBPKYo?%^@Nts*Q+PQnh z8GbhtxWsx>CyDmBvaor|0VRXy#}i*11b>Dpl>`qaa6j!N--1`a zKiTNzt~zp>b)m02X!fmQsgqb8)yx@TqUueIWmZy`3GCBX1KD~VX7+KiYG`0TBZSEp z7HZ^DXj%Ot?0Iq+>R#4Mxxt@95D+`uaj(!`S#}+>759x$g161kV@5RVA$M%o?W>c| zHl%NvRL=Bd!>{1D5{DDh5KNj4P=x!w!*da^qgs%APhnPnAH}`|jVS8LjTfw!*w#^v z?m(W}q+qn|%b4p8NmTFokB088*1+jqoq3wFXIVdvXLL5RI97OiNJmbT)7QhQUm?=N zAFF6vFfZAM5~a}J1Ki`OG?DjITGK$ur+1$G8~i{9E^bf+LT5EMl8)~iAVL6^7Q0Ca z^HO_B+0G4k#=*9SI7d$^1z<=$HJw!5^Qre20&xgssCm_nipLd=2Z-zH1(0nJPVpys zA%lQnwW7F&j?8V!frAXgK>N@$Lfz`Ir{vmj-*~mFh!I@(nBiGF1{yt^?niK8=LXJ_ zRiE~>xx=j%WAS@jraPr>7)uh1hvwZJL&ewAI1`;`2~X>sG}$2wy^iz&aB0gbJW# zTmb!KX-U6hS?1cyPiF$3WFFM=cJOukP~1yY7V&#-QOZxT@j8Ss-Byh5YtLHI{Ek9c z7g;s_;(T;1E&85`zUF3m;eOTyj%Vm<*st&hf1pjeGheI!YpK3(FMQDPvNJ&mnkT}j zlW~8faE%G#I?b&ZH|;SCey^Tax&L%WH(3YHAi90^iqSfwM}d_^}azdA{jdH7I6-;mSn%NN$}oO+EQ z@BK;wkHZBHXpqY^^cz;9@@~}mGg`NER>y;PyXev@oua#gmX5Hv!*8RS&B^DV2waxI zXI=a{y@MvIsPZ#c`=int7Q3~P^5#BbD|1db64uMxN?;n0VZZWSup3Y_FSN<;`$*az z3yzY=bM!c!dR^DNyR8_I88!M1{zE*Y<0 z?C%(U=p`VRHG^uu-MsPLO( z16Kb&P3~;O zpmi_Ovja$SQNJ~~l{j_H=k0~`9{{Gxh00ZW@<}^3Y7C~1?9P2{f)VoD%&SbSHw^v; zbi6PAi1#rQ6(44CTR*idbqFqo$Q7{Hjoc!R-mKnz4ttA z6CcMZfd;=Dp$J3#(oiedUtIp38K64gOV z9G0e&Y0XHbbE`MphjppfOs0^#L_6uAlXr8U^S4q)=N%8e#}(}V)KoQ(H&x4+a-8=Z zH6uOBu1CiWahtmwz^>X9_Z|}8K3r>B{t6Hr_MTSHO4-z1aOov|J{UWh?@myl*>v7y z!@fMi`IycbBTzGC_6p|9`aG?7keN2|W_uF4`siVy2*Ci_3eCpHh61%6?@QkGgv0He zPXjSm_eGW04yJUB)MNVmmW9=BnuBSI0zMcuY(JLhPfM=PC@SxcF&*vDuz#9e*$MQT zy;+cjf_z*mAKC20->B`w&R0BSUe>tiP|{Ub#*#2QMtd+VhzlY>3((ub` z;UntryHz*XSy-bFxSDmxcZX#Jw#Sr$#E+8_zR?ibB)HkMl)~Q~#ZzLlt^|DQk3)~V zEGCxWI-hu`$U$-3ia%X$;jqSLv0Mf{VHGy-Nu8=1QxWaC_I@grxAuL2Z7>55eF(E7 zp@fPNPRV%HTAM%6mFiSvHEU_fxD+QSP@vx%&796aW!)5CG>D9PCojk{p@tPmxoWu| z$12ik>vakhIsOGMZ@DPrU|}EevaSCvpeXP%QdPMVz4GXTRU6U$zkb4~Hj z0`7S&o9LBWJ~}%O-d&MMkBqUrm@QfoyU4j!AKF3T(Hn2U9_!c zRzFFuNqpBAYkqkf)@%gEr(HbuKDmJiU*zlq_H@4ZfgS0rJ2;_Oyz3wab-sxSp8m47 z%Af<{fd{Cuwbf1l6+|+?je++e867e1NX#zP-E@+ zKHX5$bWPj{A5_M?qgrd!+SuOVxgf6704kZ^Fc><*w*L}69vsc>wwe%^tnm*N;oZMg zJh=B|loGD8c^rO&|DcG5GytB+aXOYB5Sox2cYU*^qiBn)_NR1=0q#`{HWn*pJVl&= zS`w5qH{-L`0r@wq3)gYwsdqEhh$b#W!~pv7#Q7seGlk-U@(DL_%BT7dwFI`tpU?2+ z#7QmOnc3Y6a=PGO4LLcIYfYo7o;Cm*%QJ z!bVBOmujF}a3sl>$-T@NU?o*w*p@-=usHQfVP&zCgW}ghetOK!_8>Igo}B&-&)W48 z@%O0x@9nG-y;$8s>F!4`JuM@y`R5 zi9N?1X!-SWeXOB|R7YD;VIRErGq&v=b-{1RExmk$M|D5n_nP%*NtH%;IL&-~f@b%-lX59fLuXJF)u*Z7BWxui8dM*UeKsVI+(k*c%u^kbo&=SI}v{+7i zLPwEa@+D>(4``6b*E#Y-TAKA&1rvHxdtS0W-!Mj7;MSkG%@=x2k((NR1UnR(tvu;r z_$1PWZ=9q{XOE5&6_GRFuWf9oAuTxDxmTqFUPYZYMu(vx z@p-!+Gx6zq%2t5F74U*;x8ZA9$QL=UK0;JPl^(=G?4y|^mC<+u99~YCw6wY{@r4oY zLMN*WYugVjkCxqx_L4rl4=Zom>l^?(#bUOuJ?Os^vMJEMg!Vk*!SWk=hJ)n~!ElGW zR`+~<*L|TGH14F0i29eb)oJ^`O_-P(`V?7Eaf0LQ;N?I;Bb^EHf$Iy{lv_THS-Lc# zdg9ZZg=XxRSY`2VaSspKx!Koq=C7pVK10QO(H8>6xSNjhii&6FZj2cHlJG(FOD{aJ z4=3E-s$zgwdb?v`SIb8y^Fa3av?zdl7Ek?pf3rLmK$ zFXqx5@nz~9MV%9ZgYIei^+b*2D+TZTSc$Kh)2PDt4EXJEVCdYl6^Q@NYq!z%#k6dY z+Pig_PPdcjb_oVm&+N%>>7wQ*w;ZFh8WNZub4!-;fhjWN_0s8xO6<&TogAQD%CP=r zDN^G@9Ov^iPKRhiWYJ~R5?!k1g++xvn677>!$^8NSns@m9oMf=G>KkA&fHqA2T&2 zW2mmE)`%}68(j<{dSvskJGoFBD~b*n56?DN3g==B_Zh8uqlERTZVNFNt}F8A<-+Nu z03y`V|5IQzu6^`9p;xzv;m!!E{0RUXUV3@D6lGV<6dh)DXUodeL|!ozm$6P#U(=IT zhtz zR!-RaR!9gg>(~CbL2s!S&bYVe9NqyU)jh!Gm*SJp?!Yd^USM}YwrlITRI3U(Hn1~{ z4FEY5%mM?M=~u<<&*XV$n>&^I+(=H9M zZJLg>g;$eIbE+U$mbk4az`CsR2|J0_v+#7P?G^h+4<{;D{rAkSal*5|Gh9&-|5e)= zd_CUz+a-C+qF=T3Z>%<;$4am#(Vq%Vv8gGor|{F=YfZ65-PYM(gXt7|8SPY?_N(4a zy3o7+TF>X(*$U&1IjVPzzoqLk#9|_?V(f2TwcT6(mY=3&mhg$IUM*e^mA&H?BSyYb zxfk0E>-f-ob-}KO*@^;pQFG>BZHo#2Mzs_fFv`=$)h%xnBW8X$!JHL05?Dh;*Hb^O z6lkg>#300W!%yX5IyRf$<>SvU#``0j=fF2Jh1reA*Y1?6UK^}FXjf~@m=NYc@30%t>mYi5E3PvEGz~cn6oHUieJl7^YYIM3c! zT*IG5tKI7)BBm=*b`e-~114Q=Sd?Ne6)|o*z)uPTOG7?eDQ{Fjcr($-p^O+7U9|2c zseqKpltc1Luuxka{@v&!*0LEjP0dezo?;Sqa3R?c22p135wz9U0OoL{Qop%2JOVw1 z-!kD4Kokyy&)Bv!vE8>fhgZDtUFFw^2sBSH$C<&ozzY9K+&V43*G`ga+7(&V&6I+w zp0O#u61oLJ4**+B3)|_?_EH=LUJqtioxu+#NjX}MV$ai*t|J4;Nh58)V=d+0s7oZ; zNKyWyz2UU`w=3`;n67`t2~w?u?&L>C89Ct!Ey~|x6eGpX7}iG}NX{(AmHFwB1^Ws? zS|gsloK-m=SRKrJJDaK#q@zT&N4jrC1a2h<;z{p*GmSz&PRwjeLlR0*aoS#;NIO~t z==TI?OTfN7()9b*LGp|QP6cJiY*s37&~}=>7|cZk&qO4B; zR)v7T;};ZW3Vk3K|0?FV-ug=*-20U=W9hO^LUBF*fo!^Z$(aq%w<&O4@;~|1BRXn# zwOwP;VmE+bpq=(X3rH!ncA940yk2jXqfNLcumf)4LM3LM>$56P4~ibNRy7d+?}TpZfPu14LNQ@`+klAy;BdP|&h7_(#otk)jzuG_BX| zk-4^aaBwi7{U?c*nP0TrE?1!&P>u)qbqq0+Ox-Lw>MOSY#EtY%;Sk0}w4)r?NlP z0j+Pl7t6N6JU}Y1s(gww;P$nrV*!BmbBHo}j|6N7JDA&nOLjC>gk^5Wi2(H1_>lII z79D^!u|)~*IVyADZ8Asfw+HSWTUEY)kH@$Q+_w87szlmfZ`lZM#Fpz;fd&Gw@KE0z zz9)jVK#6zpaKIo0K$Xi*v_CTT$UNDbEl@^xB+9ZneuLTR=TX|oH99O3$xrL(g+RB2TkD_RLfsB0=b*Z zrFZjC5y{FXH&&=mHXhfLb-sL(Ri^S@EwQJv><0jpO6a;J?fel@I3c>#2mt7q20BPy zKN)!&o|tdwgkm6I>0_c*We3^*Erb&$E@U7%?4!Hqj*@wSRuC$7`Mb>?;42kWVef>N zcxs8iJ_(|-QL9q2v$LzdoUAlm0aiXD>m1g4fHh%{(XsExogJ*b8c^76xoHLfwZd*( zlmb>s0NDUre-&`;L+>})N41-L5x_0Jy<01#tV%{=8H_ZRInZvlK} zu^$1B0uKvnkZUKOBdvMq8fWYgoc?*5m~bf%S^y=S<_CcC7Rj5GA#rH)KwbkdUpeVf zV*>8x(=6pA9uo>kxAVMb{!+u$@r>hTLJ(W;^~G-a1h@|8H?{cnscA%w&}xwQLNg9O z?S5a1plX`SA}-9?WJekqbv^rTv=~Y=3#veMa(fv5&mfNkp$fAAiYi%CgCRGJiqB^8b72vhZ#^y+xGO}E zW#nhi`ogQU{=DLpOokkl@N)~*87Zh=t+G)FLqYkakb7Rtv)cT0%)_XCh10eY3P`?7 z^lG}Xc9tvXtL`~aOK-YRuwPZ}hx>aCI(b2CJ0fab3&$(t7*;5+L*JcFo`iMUDKDXlrC@+Zr6us= zohIWq$#5uZP$F+0bsIl78mNohB2Ouip-DPM5Vyf`hsjXnU3@={C=k)zx`%G+;a;x` z5|a75cgGi(Pim|5txaH6+=V^T`|hY+#Yo4$lCWD`;vaF1{~h7lyBk#0~zx3F(j&hVJhf_j}HJ_WPdiJKsNhxz=8;<;*cU_{^gH={m7_nHggY%5&DqzVAI$@wR*u|Qr z3qymei{;YbJpxkY4gA*3AM#_o|=3w`rP2@o*01{aZf*lDIc)hTuJ)2em8d7(QnoMK z+)cCZ^D794YRF4W5NwvE@WQI|?+>O5tC_ohcvGY|!0?I4E0b8&MpvPdTx!)YmCLOs zS@*G%xkD|vOv&#>r>exy&t~KvQBgWhf0T>MlRVXZ8fvTG)-1gl#HpS?K&r7Abenzb zNwYzk0*&=cuF;6&h@y7uLx-GUG3s1$gAJcrgTkd+f3_{wrLZ^h9i_@J7CiPRELGrx6$OABNM&Z8r4CiB3~A^i!ls zKBvv>(=Sqm<;lT)n=Dgjc6)hXC+*CjNO?9Ca(>1(ZV^S;YO(X7xP!3G#nt+F;LZ6? z$~pO_%!xUT*y9jF8o_(m9Nrd*tK&fcTq!U{#SXXhsqG@5Za=XA&%xmerrXU?sZ$>^ zi1DKyB2hU1YtiHcI-u~pqk{DQm46CAO%JYqGc=_EeidJJf}h2sw3}*n4?f% zG)CUl;XWn(LJOWAAnaN}`3Rk9QRO=c8E~sO@ae$Ol+6a zN*|sVsKO66yhYEVw$Mt3AL#8aq zHfkHg7(soHip^_mSElKkcgZynGbO!f+GeABQunIutjm8Ys`RU+l2)FimqI_Y*1+wI zD>C_f=8^jklJ9>r>KOKE9&g9Nr0wS|B3H<46&W1MixbIcN`cr4Ou&|)Ikk36)E>@d z`8LZexUh+%51K#0=tooF!9Xaz$JyLyTELhe%%f`i5w1`X^sUFIHlU)Fx+ALgQFReY z@Kqqoe=uS+k1HIyKe%PFykVG=HvPN9VuwylCyJ1P_wsCxB^-w+Q4RxVC?qtToG!iK zFfA10T=iOxD|EFTP1WzZOVuZLu>XCd_REd zPE#mJ;~&`CpOc?+mKJnb<9ZGYczaKrSc%S zv5%JfU->1x6-dQ_;p}8l3%D_h_pg%PQu{so;gI$Xr%eHyer>4H3v^=petm7J2dbj= z_M;FyERt57^*TA?5ns^rIM}u1f)*>{ z$jJ^r?&HNTcYEXugt&bTMx^7+D9F)r5HAvXe6GiaJH=Dj#R)IJ^9pt180oKA7ny5J z3_bma$Dj}J>HZbklGhdyO!0_V`Y&7iLwq2my{ZS;Gy>P!`z9$~J@s`9W$9c+8tHd{ z33-R+G-zbF0-Y$2Od(!So(7jSqNv7tnUZagG&fm@jczAabP@m9*p9AWfaZ{ft#+(n znLI`)S(NRje&<5w9cJ{eILP5}iu<|TixIXrH3sg$%?H)%o#U^A>IeaK%J=BSgze0; zq`IoSc<7B$ZGM8a*VlVFLyNE$!mwfe{os^mtE2eR^)UYA{FjXfJfEYM>v>gPwo`dE zXhS1MuOn81(f0L1(r8y7;w`aZW700hLzB#(C6Ac(*SC9a{Lqx}{&nm1d+%3_pbxS1 zv|YEoyb><{!Va(&nAuVq)e^bxuax5tC~T`*0BW?^doZiw`=6MhMp+=_BcyIYOY8>9 zyk%gaPDNQF2gu-ZV)nAxA!AlM&Y9ug?Hz?tsdJ8D{5|yeFS4}uR35svm#ynXS;_Ng z@%P9Qd0fvU&=z%*LbI)XUH8MEGwpo7{Gu&$narns4i}OKL?0yXbPpvt>&+pX1%Tja zyly@$Eh|HK>v-2njtM;)rk3h>K`kUd05iQ!mB+);&q(U6W0BV$bb02`c8F$puUWj# zd;NW5cpdS!Jhwa>&I;w&V_ek`$M5VX zy9zoS(co8d2}abZW<5ODqz5G13y!ngvY?$ji`X9dLlsts0e zh-2gqh;}v7sr0iQ=qv$|@{d+^R|xP@{@*ODs^%~aHMqkpk+ynP>%%hcG(^F5N%xy5 za&oJhmK8SRusCwZM0lH$k?ygks!DcgG3VDpA+~NE8DEE=f5b=5(?4<8S$M(tgtn|2%OI$ft;CtyAL7d3y)PRc6kWqrT5)BTL}SSNGF%c8_XI zUH1Fw%paU4_z^AY(X84y>$8DxS+AAcR~{G+Rpcuof5nj4Io5SP;;%|RJskM$;2y$V z7UkO8p3>g$9TuOJHC>htY_k94M>;#}!qHH+x*^%bkbq8KYfu2efVE}P`Q1-pK32~Q zM)^i6)0v8QvwmtPL~>`S?uBz6K6&CFyodNQT%m!nA4ck3K`6G}@a=xfN2JVdCx#jv zEje@sqc+@$!a$5pPOCOZGhI+_RJSDY$B9j1^oz6$QssCbveoDd^hwNaf>WJPab!PB z8rO3Kfh~zCdmw1J_?h3PQi5whmZX58>yY*VMGRxnRQGLW1}qvUIx2Ogd^zA5&Gp|A z|9}J6+oGqDb_zM?xNoJC#mtaG)4K~6O#iAs0fsFRrWNSwR?Aw>WL-Q#H$HzpgdhIv znaUfXH;g@}2SCmJ%^y*DJwy!;OcGyFY5T( z$Zc3fKkZWnL4iMw8;w+?<)5=w!8dOM7?1ynK4!lHkcHodw-ZaNWmSaIR!g(z<>)&&b2TvTQAptWv_^0e3jogio7aNO**g(<@%I<&Su_5YTbYG zY;*j<`1HIsqS2p9eD&82TjVm+K?SC9`R|zH<-4UxzC?C3VH%Z*Kv|bKcPW7j6Uj)M z?lAkW`6EiO{0GDLPls}Y9acvAFlizu)c=jCGGjkO^Ugqui*M+A2GhAMFp5!&0G`lK zR57%qG)L1eGHj_R$7m*YOXU7AFrcSApb6a~FPnH{jUEfEdVyI_>Qs-~JXcZ-%QgsQ zC$S>I?+u!1t=uj1Fnm3{Wx9=$l9Dgp?U#3cW>lDGaUc~2@zC6`V5%h{7%ky`%&CH@ zU=_9LbW{Tx)LqbgHB9;%M2{()ehE;xN3C{!qO=Kx^weAOW}iaeQoG80thZ{5C&}*z z6qzbkXp9)CYdqZZhgI~zUaw$ZEjQ zL1I2vx%A<5HR~{5BH+aJK?7Pzr)*XChns3roGpUKk`^0SV3yD*Z6X_3akp1$v1oZ;+E)6$n0kzA8ecDan%TDXXd)oAZ)+Vg4x>)xe|A7#^Sn{wMEy z$@n)p>A#_e);n|e%RUscCRnIQR5YffUX1s##!Lsp0VGXcq*MQGB@Jfj^3_CL|5`~? z1=9>egTx0Op`BqFPQ^NrQD>a*I31MD1mwI`2VIG3vEFH=w|e3C-%aQG0)H9@OE+Ij zPm&n~$Goe(C<|JhS})wAnr9Vcqg z@C{1}r%~tr{wcA&zbX`}|BAW4Ch9eG35-LKf2+hzrYUT#6XUs3Wt%y9d56iBp|nOB zE1|CqyU&1)!%!Ufa}S5n9ncghR0tsQIQV?oR$@bMFfn{T;u4W^_q%v~)7BTNz0?1) zH3v=aUP(aM-8|r5*P2$ADs-7^H-07!$;W4<+(5IMbNFV;JNbdkP3Z+~;kFj=cQ5^- zt9ltz-!yXfw=U=P^imk8P%4Oj z4B--N+mTu9vJ8!0m#x_KW79o>2i8WgUVh0tR`<1YXw4C0wHr5aL+&S3D-TfV6~+mj zdJX)xsB>AFVDHdBEDh7JBD!~*qaWyXD%(&3%YPv)f)oflom`oah0JwT5D{IZkmw$A#YI~i;D=bgMYzMt^4eD;Zu~rQHx)OJB z8ko8VfhB^r;)-;2_VMa)hMxHdzOlDf|4TW>0TJ`^m+kT!H9;Sp!+jVlHE?riY>N2C z0%RqtP;e<#{_$LRw&rQGvkBFm(Sz{11u<0;Gfheg3Qk>4Q;VpYt7fTHhlTFQp)u!^ z4dLDX5s#T5xd-bU`8J5Tn%PO5^JidgQ}*XZl&C<#*Vc zWczjXG~G$W*{j=x*$}%OqvhOVc8M*Ti01hB>}N_0N@I8fusc;g(vMjh#H0WU;~zkR zvU&B3-RHsnT$TdFg690Y-3$58`4QKr8qJnn=f_hninY^#t;+-t27oPNp5-it!QEmT&sAy0 zm8k6l8=H$_6b;_3IBm@4C|8~0uB9~im@l2yc=F&lH~0%J^TaetOr~G4z<9s28Mb!(ufv(mzYS;pZxYr95nuwOCsl(sB=bGh zv5xDtk712~>uZ=j@^;WQ_^wE4SvJTf!&PpZT896?Ro5=sG-(e zVTwh@j`>U{M=0`M#*cIo{eoJ-k$wNuB|-ihC|I}w)fStqzJTFYB7E*^Y>pkGe}?OD+9G}e_AX;_$!Y6s2fg=v&Q_4P6?_zl*sr2fZz?k~vg zi(Agda_sCR)GZB&LGsAgiq?th`Y{h=35DD+BbJlXhsk<_4rlh*Ri(x9j%?igJ~I zL0_=>rQc)wk!orAGud@Y@K3*b$&;;buy+M2LFs$tgnkEPU@R9BaJfXH_t1g^3vQUW zca@pT@r;Cq*Kjr<2{lwpG<=vJCp6lA+HNrb_V)EWZ=znMLOU(}o{5%N9{B52*LdrSgDs z@=WI9O~CELHQrZ6Ay=Yfk7A`spHllLZ3xj=tB<=6o7A7$4C#*iNf7b<(Ma2Q@yu`{ z1Z29~)N74yir)Egtm4>q*6cBMj(p&LyPPZ~eGuqa8(U4Re?Po$#;5pTy z)VZOjt;MYNBvK?aU^cv!4RDX)eZf_jG$VSY+&JgPCs@rrU*mwN!BcLk&NK`4z-z>( zXV>0Z4l}ib!i=(7a4D>2cl~_#quhk7BK)_?tJ@PnWuYq#+T*;X&87ACC1=I>CoVp@ zjlDh(NWEoXil@q+7pAQOV|LN-MB6h352R9UK&m}xc_6!Rm~5w?t8zi<{Uvc)R!Jh9 zqX%W&ep*?0bWJ6ETWa>2? zv%Yob(&zn*v4=B$PU~%7m5}QvdwQ8qnv;UAjGJbj-kr1sFvA~7+NoZ5y%Kv~iYuul zHarI-#ip4c|Mm1Sw=ZPT0}m$5cve&`7y}*CJ%4?D;63dxFd5j<(SP>(?jXF6F*UIa zpR1}0t_hT8M^XpS^*T3_^+p?)Q+K58B$(HHM{Q1+lw4o38I|DxR!idt8FTvXv7c4= zKEE=_9Ab*rL_x#HIc;y!osU=ANT*aM)zq}!IlWZ<$+I%?K-J3Gp{{Bu>Ez6IRMrX` z`7rVi>IgqF0DI(VNlMEh3QKbzxskN5qgh!t_{K=&ZLRsvm0vcw?Z(0H=6MXEK7Tnt zI8y#6;fTw)7{rc8zNF-M^0$=q_joqKxY)Uu_0aJKZU`+fvno@&vEEa8gR4p}R3e&o zrmAvv0Fi#mV;F2>t*U~EJ$G@A@G_pQ zAtoKQp=Wx@l0fpxDQL+r7$kbDAy*)m0Bqo3OMmbGG_FfBDSLK zhA)2(w9}oy$g7pG&s)9`b5~lSsyF9*%Iu1Y)|WCg%&4^@H1>)qjV{60 z*Rve)cmr=ZA1P|2_hx%^XYZHXjuQ`w;iP8tz*PJcU&6_)1b?GKiifS2l!YyUXrVK5 zF)MeZRP=4ld({*Q%xK$HHBM_lLpbs20|(|_+tfoWTOWtlR5RtN1}T`iT#yX4Nc+?W zQN*_yC4F|kz&oEo=YQsB#ywuYlWMgbgwBcq)#uT^5>pGp+fh&?4?16$0#dw-^U?2P zacdSQM6GAd7`7$LgNXv=vXa*E6VEu^-`E>H1w?KsY7zn0>NH!%KBYHnnxW z$ad~N_GlQV@wKhgwox75x_42Uy1c{VL8MyRtF2-UJGMAPo zT%f+!N=>Bf4s^-A<&LNu-7}nwvz3tep%wQQQ)4d1ID^X5w}O7V%qr3bXm&k69CW0n$_OtZ5Om5%bZ@cQehF0A_zqn33_d86+Ig?~9&YgSEL$UI%jdVN6~VcKwADF49hRmWy1VO`UJXuJf79sa09NP>wM^UxBtSdi^0}ZlgnS) znpWt&kCye*+K-69w;cowwqc`zxc!3m>R^9}ATFDpj(K4Sp%w{m$8s$gwa>t*e^r%K z=pn;@ujS+K@O-LkYtX|V3b&z5fmJstFEQdY;{ng$6X49(AnlnYT1 z6K{-)DR}?%cj2lm)m6?o0)WxuqaZ;{ov=HEKEKT-^A!m)#GEt1%9;&?YO&XLnbks_ zUEPDB)VfhQoB`i7bM7@$&sU~CpD1Zb7$xrCTv<~el@7Fx29nHTYG?+?G!&YgrynLr ztrW~>K3|zp7f8I#FJ?IMVaxi|?#x9`N0)p0cT@e$p)AQy^SU;pxH9gQ&985le|*zU zx^`@O;VHdrNppMM-b|wGO)CQY+*J}3T6w#{ZA~mSM9^=~PSRk~KXTvaOQe@mb|ck% zzZZ6`fs%{oIh)K$g+L|lYex5SDWBc>JgZQD9H@rXuB5IJSwI?2o|w-FB$Ep_VYIq5 z)jsINyy)I>0X4B#%blB_MlBYSGNr2RP`~pJ^z?=E-yLZn+5dF}s;wgEJ@$Dkv*ZXG zLVVRQZU>dHAIMQmY1S0Hs5fnZB4$WT7H}1=~ z#@^r<1oX?@Vzz8tFqc6B)brrhWc`F#T5^iVT;TCQEJsdrZ;R*qiN@btHXmci9#%6e zL_G%K%CfRNC2g2Tia5j8xz0Ixn8oa!XK*?qgkWte-7E?7#gpmI{DzINf;&&Yj>apL z<&?X~b3cQ-wU}mQon*TG|AquK)-DITAglI^PQBHZQe;jI z8^j+F5YTUC7f>aGZg!qY%d zsU9umoi^<^!!cwy=4j4)cyF%m-qrzdExJqS1`qnLsxihFyCn4kab!)9;J4dt2BG=_2F0?DSR@U!BzvC z0-{at{lf|gltyY=S_uFpqqKk(uA*@mN>S9XsX`t|HMBAezK_ffj(MFuo0do<0G? zV?fyCFx%F0R5*ZClPLN^bv)}|P4C{%yeFj3ZkQ@OlU$KoCaErBjj)OB7qL&V7-fWP!WQXO5GbJB&K z=eF5SFVBX?m=5Qx4ed*UiRW^2zA@7WsBvmKBlBc+f#!(=i59(PEHh-lcnFx z-PbUr3Y&`_x~EYMnJ7pnUOUce6b)z6pbN@nCXI-jXzlnVp!0~%dAt*Ln$y||E~t*r z6q94H=Wjov#WKMaAPP{pdUx6cd;F^yw)f7$ZU9NX?r8}qE^~px+uHiYDx=CwuljO= zwZ0|EgbaTmIZRkcw_@o?GnX?P2wslZLp)7;y%HBZS5hSwDe@rCKOXgQXl#>WI0|-2 zRG)avZQKn=M~8{?xcPIYMT*Fk@TGhbQA{*6#n9;Yk8#D8Voc5k;3*+Q zmsUaN1d!Ic&v&L(+}vtW2tjwC^yypYeTo}l`9*$8smBRwyZ1cG@_cgq}|J+ESryG<2Lg_-F8r&07cRDTVi`b7q$)LPv zeeI=EqkMJh7zbLCl3)1ywbw{_ZQP&q+pJsxOtAi66gCq@c1xLUQG1{7=W0y2nC1r^ z1*S_pVTS+XmCE`1D|Ke$fK;|Rm{(XPhViGB*HX-?H(!qp7Og@RjP4uc3zQfo{E7_{ z7yt-@yedH)dbj56;1x>z^7&bLi1hl`cQ3oSnnxi0Prg^ZxR6{TG1fCL>(i1rrWfB& zd`_X2(c%N%r0ws>m*aeP&+k;)rd=MrDt~(QtnJ`y{7mFS;+5ClSHAd@giO~-TfRC} zldXX!r@wDb|5@55@IgdpzjqruX02-WgJ%}3gI+}MwF{@sdnxavrBP}9A-3R7rm1G3 zZXrw%UPW4bLNR_6-*|!dPS2|HTses2Mp(F*B&2H@J9qNa_W*2y@KOC*-trQ>9Xf9& zjHCS@HZP@~s?Z0>c#z<=ABXOw-cS1_eN0JTwpvm0s(ndmlU$vrKq*9F`}x=4zLq=q zpORcZ`(`*3!(_3a-{jLyLe14#i=F1K_jSx1zjPXUP%VAuyn< z=kNtgU(d-r01H<{@$PqkkRSWa3lzv1D7sjH+vZC{>I7i9a-!6f%gf9C;9td#LLeS& zP3ki|bhYR@jBw{=@VW#z2IM=a$XIn_Mm|g@ z8GG4O6R!_+2;V<($jDd>iNYiI!Pin%RBTbKk&z!->o{3g*xT?*nnlX>=H*Bm#1TD6 zEsTGdq1Or^6%plz0^1}=yBQsKUeO?3!$0rqB*)Y5J5c=&demdfpT@(J;)AID%o#zV z`6nZ%RbKcIagF^fLvvUzN~b3O;Z2a&w&A*>Npz7ZU9*WoHH=GpJ9(PBYYxbE@!$jw zEmcr=M@0DUY6a|ovXT4n9wh`G<62LTv3uz97N5g zA4Yd$RIl^x5zO`QIDhDSL>z~e7|T-i*T-4wt>Sn4S@UAEv*~?TGwjFQB35EUSdLJ9 z@x5L;N?f!c6^YCWo(RiM%vK?#}CihxH`Oy6o>*6nFd6&{7_0FE9{=}!l;p{C>xb@UKsO!-r!1n?%Lyy`G0PV`K|P-A=NIQDa5;kyrMp78FvT}3>KkTIqV2x8rkZ8Sm9LL_sf(%4UOPQzs=B94?-nthvVXur5IF_o3s7AynKh>&s$o7hb~azbg<$TWLG*ngd8#)sEeEWwSQCIkrDh*eVdGi zD&{NrSk^bcC#ya6GIq^1CX0AzZ1h3?QMD0QVDA;D{^6ZZxgDYK9{-!~cmaYClvqjs zD6x25;X+?N$)`cw@I8Bu&)6Js1)Mni^g;l03%L#fIXc zA1utgkqaMx>{M=0+3+%z->mG=#+W@^0c8FX47Y97d3BW`%&8jaCwaUZZl6Ua9(OF! z-}^vBf9s1y6D#E{sdHK3Bz8@N%>mD0%{~DYe{yBhlv)%aO}?*SRq_ka+t2 zTf#Li-DN@T*J4^#aeq;@@6KCNY2lV|&<=2ES<)j z>qT^!@{a#>!ap$TfX)O&jfviT$}ZC~GfFM4tB|Ws(H>h<(ox(U2LeY&yQj{T!{wE! zSdA-nHAV6_4Wb2VsmAVae)#++a@;H%%X56{{9@SHI4^X&rIANigJehf#n8WC za@@SpiQ^g>%Kovrt44!7pLm&I<`pI{MA$W6PF8eTj)9Lzi<8RC^VMpf(2M~@H^c-u zC`)$Ov5$==JB7d}N(CxoZT=4jdYbzHQtH@#aE)5#Lrt1z1`+pWa3xwdeiX9;|MiaU zX|vpPSK>Wc_IP7Z44C&I^bRnFm=->Elk0k{4u8Ec&cWsU6@o$hr4wc{&Yv2b&(NRn z)%y%%@`o-*YKt`<+u)-o>rPYXB}ozCi4!0;x=hUh25{^e`OfW5V{8&;WUoV1d%=jT z1WI-qAYEFZPo_39@j`#>JE(y84Ub|?8vmB0E4tt%$>6`mVY>AvAiOBtHSfN(|Mm=C zTXU!Q=26nxwd0AtZ>f(n^Xnk9YNrj_*0f-cAMmcXA3 zcNYP9GZ`ERjZtQ3Tmj4Ql~ zRe~m5kE)8?q@LcAWoA)Mh<^!$CYr6D>*T+)!ZqAa86a@;4}R;?+l4>H8ldF@UY%W- zwUVU1Q{o#vNX6z6>g+w>jCC@DSfX;K5#H>2*>i}&<8Q!@Rk4jRCIrF7`&Q zaf4kW=BK;SO}r3nTx=Db=8ubQZIW2`v}7(ly*j}wv8Nu_0O1r{uMtj+4ctf|gm(oN z*^lN3MWA+ceiYRl<%iCE!Q}EmP`p#uejObFt z`XjDp-^E)-&{il@+N!MOS)rO{ zbm6WqOU)_eVQq^Lc#HE174pl0D)L}drvLep^1yAGmx;w3FOu!q1F0ewy7q+RPIO28 z!NRMavj}_P)mfNnPyG$$0fMDqcv|0!S8E&>oFo@wdcEqu98<{Pd%tB2^|^C`7i=52 zGBk-{s(r+k=|Dq$|6QmEAyl=sD1H2U9t$399*!>7g_OnCfaunEjxq&bISXq@@j}oj1;IKw|tn5?~?P zuAk)^wo!>H+!hD{@zY%Sb_5NueRUEJ`P6v}%}g}_Qnqn?C61@+i}cmugvFtlFMxrQ zvgO@83e_j|ddnB0*?>icK_g^O_(*)Gd1-Mo3H@HTbCl3+pO~v#f$mjk8}PLX<9bgf zD!TIU0R-(u(v$06zFv*5UuBrI0z_L;fq81?qhadb?(UvWgX3qW$gklffA#r#^+1Vz zRevf5e#ko_(Wx(_n#1GE2R_l8w5T~6L=wYRx(d+YEoafy1;+=uE8n?n3WG^=E}bhR zkEdTEx0vRJ_B&p@vJcAQ9KDv%J=4s93TVHc|I-6`b5aNmx-#t6?h3R^o;5Vj?rqsa zZY;ZH?zFqTrGzg$%a%5qJT^bL{N*lEx+qSq)&J+>l7owD1gQwC$CKD7d20)pyc>RD zgx!#Pk4w~}y9gjq2#hzWr%qgK93qwN+Kq=VLdYa-n)eb71G_(&OMmPV(-PYfBjY(i zoy2?t`Y*K3ZiE!}V$BnH*Nxw8`w8;G4LCo-oZM8^`;0Q%4e%?L&iyD&w%x7 z_#x@x1irM{;Skjk6xsqT7D-jj`N*rR_!{l*V+F**GVlmL=qwBdiIy+YHquZ92UlK- zGs!QBY5^EY^jvL$avtXSOQ-VoI+~wrz$B=B#@rr#c_W8UjrK(nl)-MkD3E?D1T?5r z77#rM4bZLISPoYwb?(E9vix#mNJ3Y8Lp|7yzBo&n(f8h{zF}{f*XB4rC0g6Od5>7l z47o2X&4_UmwHs`IY!9gEzig_jsU19=QvWbOVlt%NCu%>1&v$|>~z zNJwIx-(*YF$kz^wvC;pE>(ljrI~)J5`I1!~9v|I{2eJT1_dR9gV`8D#WntI+c#AnQ zK^$|~dmk;^HB%fjC?awTVYr2IoFwwju1^a2U&0t=8Y_Jf1gstsM&QP*VH<( zMj6?9maKteTL;2-Kgw}Ud<#N}%3FCKHmHp1GbXTi@`gagdLgRwHS&|4q`bPx5lsC| zCFpN3IFjq?>cn0<3?#C-4-54@rlq%N?xnZotF>!88X&p6^NT9oiv5Q78|3>C9Xs7-Jkt8NjU7@ysT<~kE$3j%RHGgCU)H!YWvRA%@JLWv*atBHmi){nHf{fXcnTtD^U9WlAgy4 zp$Pc`berXUAs!H4jpR^$;!718UM{;KLFQia$FV|a0Soj)g2uUD3^o+})1Sq-6;uJ; zNRo_~+yOKH9c=Yl8L%cqlZ3+d^y!6Ve5~J}TR$l6mgW7gec94qd{+OvEAk|h80Nc4 zjhOH~+59u`;0~0v=C4HfP)?V3pO2fdv?sdBwH4HlKrd z`iaZAZ~Mc`>XmZVi)a#l!zAu28~KyY)PSoA{Vb44il@90CQM3}<5wlMHA@~&a!w_v zvx^Hc2w~#Ygz*uxfNoanYj4b2v6f`wL*_uxC1zCbAvdr{^)4%aoqE$KytQy&cEEeu zo1C`p(zAvANId$S0IIF0n%-+`Nd(b_%`G>-Viyq5j(h&yg!OGjZa>RwnSAZUQ{$^; z^(S8Hfd1oUQ$gsedKBrLbbN8Te>io3P>!bx46WdACPL$xL|Awo-afq5;rX5~u=Tg7 z^w+t6q{ly-|I=ehH)D544u{74ex=>#ytI!yykHU(_0 zNk`3e--0^&K^yq?3R4ObEp*vWSH9O(+>%UdH+=Gh`Ld-Rt~K!^uXL!IN7y?MvHf%l zdFK$mQ~bHHBlW()ufT|!2E>#R<&2YKm<~gL%=nqKV4tSAtpiU|J1D8(C3 zQg*w;gMTve+q=(|{I&}q6)4AC=1H5`*e$$sKEypl5pRsYm1bq`qG4rh*U}zP&7)Q9 zQUT{rOJ8z${3egtOiMwb)qH(_{mU2Rx6|S!&$Fl~&%mo4c-5!{(fXBY-}=G&nXqud z{<*(`qE7U#ba$Yg17I{&;}4J=t$jC8|Gg(Ouzd39)Bb}OoU}_aiy$dl^re=rVFNw> z`+^Td?Qt@Q@`Q)sm{0^dT$P_WBGG|prp5+a8j8W3_(e(2!6&~;qT*$uZ_&PHH`HMs z%Qy@6+NlKLe9wWppz4gx%$8PtV9%{m^Wd*8`PL1K8-FXv|6Q91iC{ldctUXp3x|e8 zlL{2--glEtVe=Y3#z+K&{fibPl6W&}W&j3Yu_}SpAZHk;Ne}2%>%XSVbl6I~@R;mOpD$nW^mr?AUC zX?6Y2Q4*v>(NS2ayV|no8+{wYV{?=k*4Au*ld-Y;NhBmbEQn4VYSZ;5%v0m!u3V@_ zh>C=!KKSr@hDJ=Cw?=|loP+fv&dvm+7}tF~@WuKe{XYWPw~}C9a5Q+A9JXxlj-Gd} zT7Xcisb;YcaTLjufwf?%6=K+CL8~QhXRNT1b>F5Q`BqH;U(?dMT~Okw8zG#P=*bgY zG)N)ZYhn6C&v@0Qp9%J1rW-eW?OS`NVmC0X5{X3zgz55wd0%Sy+DWXc#a+eH-#|pa zmHpEdck;MlLGm9@40uZZ@xk*?>%}GdE7VI!?*I6brxi_l_Z=g%nUYQPA(}0(?UUdr zPa?eatCRN%t2d#*$4%mQ2hT-HP~3Fxf~f-fMDh93?}Tq&WjqCI6Bg?-1a$eA>t_WD zt;0kN;2bro^5ldsV3;6JEwo$0haaYR`>M(_tTu-cB2?Zd6#TT*C>U*EC3Qo*i1Zwg z+!%BTOu_c3Aaa0k;C(2_%dveu(bpB}MCe%SUASMnuWH2pbE4?r>Wn)>n>~%rFfji& zb$Kfq&0mE!n73~H{ltDuN^n+U4tV^R07DE2FfK%!hfqfr`>z>7{%O)SF>Dl#1tvv5n^->v{!xj8f7TR1Jx$1*&1v zPXL4J7{CljX@nock4`}!loJ%iDyAtdqq>TUis6G6hHU#~Wo2sFZr}ONc-=Krn3|yF z;?#e75n0bq2LJvQ{oB;%Z*FQQ(v6ozT1XfeAgow$x$rsy>Zm4_LiR2eauVl+w;2kv@5Y zlG?kO2R|@-@124&zYW2R=@MkFe8fRXw0LIcUV(;*4i^Wmp$W(nSYf65K*O>d7;_BD zTt^WlD4onr4ohLpdzL@du+ zYNkKm-6ThQ&dh(3e0YKPgXq7W+I@SEXK>Yy6YXfvGfxONcoHl+N~1l`5zMdKnVDZt zv_^SKYf>T(AEvdkt(g`bIA!H#R2k>O`8gO6>!P0jD>m>$V!vtK)#Pv6mH#<9Fz^aB zfkXiX*1r|M?7p$i7L8C-U@t(?3MjQydl;K!oTr2%-2wmXLzImgB6hVJABSe}1RO!H zIphV{)kvY`3s(Di$q-$zGPtAq>{zymYhPQ%npvkU*WKQ>fCYk!9_y0d0_}usYEhp5 zwx@w96?4l2NtEMOlBS!tdZUC?K7acz%AE3F)5ZXizjr|oQU(9F0t!S0g`P0*H!@B4 z+MH-{G8FCD^qT9GkYSShPUXVp56zfcR_In|Rm`9BsEPcx6G?GvPabe<67Tih(5uOp zXpd@j)nuG7e4rwynX=3;kb1G9#)aFR7zcY{To$#zW%6VjYJseCj~z(z*y4#GeIkIL zFnwe_xFUfB9*ng_WzruB3z1O171m9)oTlnijs$)7^MRDUC&I+18W@iwV z6=Bjf-m57%9?U-;%?%cQ7+2b~Kw8cvkUeHvzlq~}v$EuC(08`r!Bi(u?;*{x#dOi) zDaVpLx@s4peCOWMlZKdQwIQ`Hj@IjKuZzC9#tTPw9gEf;R=#H8JDnCnT5DANF2y;l zJ(az#ak;ejNRY0%de?5Bbp)$^LccD9(B)rYm7~l`Q>X5EJgaMuFuohx=<@c*>dQk# zWOn1kY6j!W4X4BU);k9OJSl&@j*Yi4ZD-`z>j~jv>jg=kr!3QbZgyZ$Xz=dF$GW?y zz6uHf0|Ocautq2WjVJ^^mAhMLb$}x`8Sstdq=6k>9C+;ufwz6*R*y9JU%>iiAXemm zyi*6pk%Oq}ZiF*g+*j`t9W8r0Gfs}!=~e&GrqtxMxj%;1bUCj zD1KWMF#|WsywwyL5KTl^w9)~5TT^qq(_`;kEV#H9)^%+Mlesd9OR(264&04_gCIH0 z+)wqq9ds=uqNMx|;Hi8jy_XuN$)V4`I8}dO2eJKK>vcTSXwv#di^4gz=tdmI7v##& z1Rmqmfg;^MI)b16V@Gh47{*9@^B?J@zIZ=HW!ZPCJ%UuKPmsti7P?!!=o=-K2K|wc zct}VsHExJErUQi!UV;F|GDOSBykf#NC$N)nM1Y$M9}O3hUbjMv@zj;xGSnl4g`Pzl ziOpRLqil$U#IQm!vMmLBy6Y0DI|ZRpfp=3$X2?Ghs697S5Ysir+|YV*SgZj_25UK; z#(EHFSVGv|M3?eel--ahv_#w~y#D-MZC_l172eO^j=nt_F=jFecGUR5!ujzH0|vZA zW*R>lGtl8ujWd4-B1lI4R!|Ebwe&=Qmp@*PVtc6M`WKHZ6%t#pwjD+twVh}gG)l6X zDaS&;^f2M;oM}%Z<_H~qQiLsGneHwJtqXyK0OV2F*<^NIi&968_8ckQ<4<0u`V?bH zqoD~-!c-MG_>Nztrzp_W@W~auge(U`XsR;wldA)DDI~L~F>O6K#H*?4dc#^|Y2xW1 zlTl$~8>j&M-w*T;gg%dgY%i#osDKpDnfgFSM+dq+)tH1D09xT#rYR7M-($`M_#w?v zG%ffQh>&Th6+fsIq%Y7Ow0dn8oy&4Un0fUhB)qpc0hIzC09cdmAIX$Cq2YIjxQerH zt*oqMj#yG0fUc9;bdxhBVV5sf^bI6{2Ug|u{fs=@0$zUn(;sz;N$i!qXCXGtj7Ghd z$-!6qxk4A=TGFSsC|ck0AW!xEw415vx1?IZMQhl`-HmMO0Di$%5?8<}?QlHPeXF!C zB%7x7T7t_Gb4>pAdPhNcJQ{3(Ed`x98_JKk8$6 zHWqmHDkwwLB?n#LU;xbTZ-MSvK?0Bm1=C{)bcxT}hnK+!^lL)E%{8cv7K<$8C~@}a+d3UNgzU;l%);F7 zT$4t`A-|g&TM%N>Cceyax?66AoDTCvZk+!f6b07ODP>JA9Y3 zfhO#uIi-D{eX*!*tS}G0NN;j%QX$YVGi;ut(JKbnV>zE6OuH=4+a!Sa@hra`4g9PoJY_VgaY-Om?^-PHK`44u>upLM=O3I@moWfNZRnEbptDHmAz{%r2m z#B4A~B3GS^jf|jdjdCAS(;hR#IhBbB8jR&$-JYO5i%P$sLLW?ybf6fFoxgGR<4n{{ zz(Y9hT(mh=tk}|FOTxBF7QY$OBgAduD)wv}rRTw#Fr~aZFDU&HzyIE)nhejAf>A8y zkb+ACI*Co{1B@DsFQ6%CF1DSFEgE!&Ciwh}t&1hg5Zk)&XT&dz-iW2MNdly85cdq0 zB;^x*7sTX+(=e(?(0m9=Ql=k?qt||e;C`i`33Zalk~LQcfR&8+4iK{dl#81&jU0n< zbu_nH5VMLMHzcB=`5p8P9&-D8d0ICWc9bjO2@psz>MCcJ>KAhtgr3J1wgBYOGHPV! zJYxnxl6TF{2I$R#t}mZqJ*sLybF3tB)vAB=*%ow-TS(!K9Lv7@zl%{=KhtQ>j=eaIbLJMcJ@u}dCF8}`LST2 z>Y1`Ikb!1^>uJ78GKN`zd0*lBmMk>kHsb-Gjk6s(AsWLgj(p7!#SqDH>5Z}WK>rLu ztFRsWm+AF?E=QgA4z%9;|EVMXTT)qEN^cuN^JIW2oQ}}wgh$~2Q1(?(Sw`!+KiwgX zlG4&4-67rGozjicAq_trN(xAKcXxM6cXz}2SbOcY*E-{zGsf`_FYv-SzxlpT?#(Dt zKnlC^7zm(TA}02fGTM1RWm&JZmhiqzB2{MQ3*Kw9KT5COsldwp{L`FJ#No z!Vt9*uu9chFKIC}oM;7cfVJ=X6a3|krc=b_y^?-+#T6wEY z+nTn-?Sz`=q$SBV!&7e#J-fpuaNGA|FwTJpGBQMY){w9m13h9o9yt`#yKY|+LHu@R zDmOlHeqv@Fvu%dh@5Bf6;_05T!;Hgoyd#o*wJ!L1h2yGUbUi<6K)Lt)vfSBajAQ3< ziN{@a*L}Q7o{~$jS*%G&eR6Z5BT0Ja{s!$fXX$Gu0?sC z;pe`cA0wDNQE}iAuEp+o+IsBn9dr{p&K<#=jNkdFk?;PsE5Avgflav)2@DyGNrEeg zx`s=Fm^j010c`+D9Z2B;80c6Qtcpr<09y0g8}Y8TR%)aG3z^(8K#tH43MfZL>+}d> z@l!@_?RW>98BpC7j5gPZmv6)k&^y!gFhB{e?txXTPUjv_9UA*qs9QIXwE32;)VN9l z|gmi|`;0r)uw$H`puptU#1v1FzK&9;?aEf5? zfJPP-fH-b`x)@4*nO2rph5L{w-WeK1kp}1|HlBRr@)^-bfMk&z4<+%$y3P_9&5R9_ zu6fN@&4`~UrceuSJa#JaT)7O+_`B`*{U#59uor6-0?Z7s3zkAz~3gO*Y4+QA&$rj zS^?^dP`PfB@doSnM|rU3@wg0|v7q}}@tAdLZC2%oJfD~c&?Yln|29l|Hr-rI#Sa{O zVlkIY_q^I>(p1pMJQHe;P=HPck>ZJF4{?Y|BSEkv2cTqz`V)cZI+%4Qsq1kvA(Dd} zaj>?AP99IG2zibv0{A}^y7WwJ?Fx7&mGwnLEg6q;AUcYo(gEe{t?@x8beuIB+!qu4 zo%H4}y!YbaI~Na+rvtG-Uk|B`L&Z(c1$K>=Ee4U&99CvHe6s=xr6W@1b^`Tj>d#lQ zM`CY{qd_PVr`aA{OFrKyUwgbZKxiJdqU9LCOg_sTbzu9PC~G=9OBSkVN>FU z#cG!*W;4l;cwz3jPv#T{lxNj^)KLrdMwv`1(IE{FLljA-@wXrD=zGv6wDr%FHiy|i za_3!5j*ie&F@L19AjH_Ww0m%>%X(WN*)9L&51|1LLJ{b_eRPK^^dEErqhTRvK&(^( zGGBJzj64{Hyo00*0GWW86fhd=0mTaTs6^)nI-YJv{&-7YWK>31dRsQS^DcIf0-6zw~`rz5Lh_ONl|no*m;;u}vLZ5oR%EdtT8)*V1+SNc1H23PRMc^3Uq6**3J zYYtp1#9xS7Lm3UZ!Az9}!*O zh=xAA9Q&CzyI7nhJ1S+fQ7~X=6iltLBs2E87hCu3q3A(mT0@=Ul?eFIzk+t7gd6v6 z(_6^`%9oVNuHvZLjf(5!Y6^>P&vD++v0qVKQcp1k3+~pX7L9iUegreKTM)jJYSoJ@ zy5-LE$Ur2*xc@gJ?;mLv6vFc~-p9qFi^FEF$=!uN+iz;%B?8^y72}Po)$7zlfL~7c z&dnIs`$b#l+9t32HJ|nj+NQ&%XVJDD`P!#&b;xn3<(3+r?O`{xLK*owE9yNuSf|p+<^zR)GOgvs9=eDNSS3l)LJ_Y;^ZLEu@R}cB)(nq*mwZ6K~sUQzZVP$A0 z`60#G+Livje;@Z(+2b^d1j5~3RQhcZD>})Cn!OQZBj^kS#8@SQ8HW(? zMZ01p;Rv0|!$iQxDOFHv#>7TM9Y|=Ls>~5{OKY&)Np$5hMxZ zu(00NBLHPk!$T5u^GnmM=UfV&{p#cwLHoi=PQdhrqyZE0iZL;XrvikhZVR(AyA( zZbHjQ+`t9;|6kkECP^GJ2*Q`8KHQVbHKKaOcupW!GPn_x;$DD9U@R539qr+is~U{z zID5YPBs-A?U!w+9Sb_JyB)mqOe z0m;jr3wbVI@%hanuo4p_X9`s(k58Jg6YH;AznhM3^dJg=8BLqdJ4kdV_n*gt4;;A# z@$G=Bx2HLF?V;ohs=-J=Da>=y5PVcS#_xEC&?nuL<`!$SSQ|5eNg0SrU-S83xzk~g zb028(``$mNFZm|`RV5R84QMRHsCG;yRC?kW{O1dYb~BuR&j5w#a^86W=QWsp~c%QRZraIW6FX3GgszT9IK9; z`a2G&p~x@ioyM$MG+Fk?H=}0me6-k;h@x+1OrwWmdJ;dp=d=K`eVtQVsyFcn#e!Rn z1WEi!`)+$=%VM{Wu~6tjyANo@zO{CpG!1kBS$@yuP6~6DRptX@ZLgF~&VRLQtrwK! zcn@U=zxw9y!ZjCquzK?xRWuQ}Tu6Rf0eWISmALL@MhWOg6Xlx9rhQg+J*ZBy#=YtY zHBT?NX2A=qb>Mpze!Mww^yPh&^4WLnUcXo!sf}p~U?w?UZsxt8D_L>QxEFrbC%XG_ zv+4cHtHKTEb=^jvIyCkCxPSLlq|GsM`YY3>(R0Q_w<|XWWbE$|v5Cs`&Uo5&F3p9D zHt{N(usYh)Pn$ZtA`{JZ)lPrm()W2gi|R$5axA&@G2UTIlIkG2Qnqb8Z(}WZ!ZwX( z$gq#_j~u6XhGXRolGA4R8j(U$fK1-l%EKF5Vlx@GYw%As0>CAeA%BF}D>n^|wTD`R z1|gMF13rMHw+(LD?-Hq$ZVwQBgl6HCe$Lzk)LZInMt5iHh#4enD2a%|obhX-j$?ec z`M(kD;iM@cngQ!AM>`?s+?v|uMmS@kGLmJ%Q0ChA@Jfl*Txr~qXh$wr@3V-GN)gHc zNmPL+`ygYGLTq0F@$nn*H$C&?M)3zA6YeV|^%>plXR)4V!UuXmgvoR2azpbmr5V=$ zLaIRfIsuyCZm%v!QE|9cV;~bZCH;~2&h`}`0j?I`M5I|Cb-vr%TCTRX%py#f9oz%? zPBg=i)LpTvSAVX#HeoWmG)B{S)K~2D!|5eWec!owuLt2`5g6$dPz{MgY=fYgN8ts7 z`MU~&jj!HqIslvRp$Bvh6mr53qshmg8OZ|wT=zV^8vkEfh?3T%r%;O!0uwV@64tUr zpMlY#4k^t-#-O{o$@RA6S)Z(9(nH7-f*lfNB6&*h-b-bl>mi(k7iPX|k=3?-9Us+m z6)gcc{InreAJl!^b(f%duDnL!4p3saAwn{v4<7yT?fkKP20pFbHPVehTfRI`|GW}= zeY*8TbHmZ!hN>-W94T+hy-Y(n#vG509$YLeP`pNR260md$c&ERlM;Fd3DPeWUu&sC ze$auRi|HZ8o}m-ay7RoTTSMs*{rRK_9hs=!me29Xt0H{PtG1n0 z3M(#y0yyjTm9Nwu8E_N`fVhT7b#q*Fv9`E?xuFx8Gfp9qAx628S8pf8Yh!9uKzAn7~}? z?_=2haYj~LI&dwIi;BC2q0qb=XMuH-?)QfCQl5?NXguRyINZ4O=qpvK^z5~!7zOm| z^*dI%Bx5-~vGu4rShU1AAYkTkBQ2VLvUSmWP8S;)pUYNfPX{y5iJ?JbJukHFxv+fH z-*gD;V>Z7at*_j#-9`uHKYO29cSho(z8{ftthoN{~p69+C;t zzz~EW{ZKxvc=7(ikn5u$n7(dViJ|q*Xaw#K$_;Z8!&kB&7Y^4q#df8Urrj`=)jVo8 z(oqts9fHpI3?_={@whI-c8M8;zbO3Tii{p|s6iBZfr9)IWl3GfA&7ez6sPexO!31j zlW^+Ijs)n_XRknXfNwWVi;z+0Jea zyy%aqQJxxR`+)az?%CfT(!~ivqk{qtqMtlTBm3!%Ga$tQaBzq!@;>vMqXoxFNh(Vp z3{JQwVeNhSL~}@;LA3NXknJAaO~D5nqXFts(o#vr3Q@wzA#t1ZY=M8y-vAYS{68Lm zgC6INe<^Z4HIIo_`i$4GcoqPmzSbUY9mKKfnq<4!;-*cjJae8H2OTK4J3dIeKuzPNW zM}hxGKX4A)Su!_F3Fb>4qZUciaa|dA+FIV*TB}MOxb0#;PlBQV1Bz_bK|6@q7xS7F z>NR+$_mtr@AvW1;qiwm++7dy;A-4`{M|BD&g{=mKfpYMeU5ka$T~W^<;Q@C#8ytam zGJ_%TX8(fh@4P9GSsi*ul}4>yV_AZZ9K{gtGsNdcn=F{l)zE;r00P?+apoS6@<`e` z#Ow%hZp|BGS+74)t$LOWQS-eng81OJg!w^ewcXKNY3Rzh0pCNkPs;;pORTpM`Q(4v zzgu6l^TGSwXWK56XDlA%CFOtKgTK2*76ra!Xi>Qc3&&0dmY~HDhfKS&hI@g@`?Y{- zuME%UgeMXI1!H6L-YiQhgnY&&+z)vm*nAo z;IUhpt}K*J=OCIYA8@Xy@oBHlXT>r6$Z5?_3G2v*_lk8PPnEV|1Bg&J&)~lhL?7jS z^|T=#@?vfkWOU;nS5B(^q^U`ZBpv2FK()i=ln(0dd1|MT$STWByE>hj)1gYcS&>hK z8^hs%-rhK4rA_|L{%8Mt@5)1{>r_(w*oB6#K8Vd6_BmE3>TvLNnVjcziR(w|T`{xk zL(1l5#(GurKd(F>H|GCU@`4%2({C*S!3Qcwj~&7Tm+`K0>-wQyKW~^a_^8-IkzmqI&6LEb@QGR?1~m zR{P=8zooN5jC5fR^7lzVKaez_OB%K>v=NeV>GG?u&7i5GWDGLO;4=X2 z=bTQ-7a$Fd8+s@-4WW zwrrOhMY%XDWa5)p(@&1l)Fjf}EA_%@iNsOSbkr95d9eeks)vLM$ zcGMH+>hloPL|e`?P}ng@1oWRVP`5LKh;)H)5jp-tIv5&_-jHp-zl0Bx$hOPweaBjM zgPt!5>dp%V6>*d_)R5!>m&gW)_UPBWgfQbDpZ?M&T>=fJ3~eXe^wH-Udb%Yi%yfR; zlgjauUMSxTk@xC&PEOwmGYF+u;m}X##-5y93wlO8K$bi)3XYXCUU34w3QHWYp`e_X zm1DJqJP#yIB?aoGY73gQTLD3a`Xd&u(^zS&d~VjgSM^m7?WAHOk3ybrf6IIkZ47nn zw;*&LjJoZELiU;t?OHXHva(+vP~2r}Q7;6}uco$_N#u2`LL3(EZRDc3VV-wSnsR{U z9XsB`tey6N$%H|qR#wy3m=gvNA$@aTdYOK$Z8}M`(XLez+{x2!9QX@Ikhz8&#Z}9O z{oKiaOi6Sxll1LrlQEd)s%KIW2=Gu?GFcmQsdJg4=(uZ!5IIj7qNhhl`o6*Xfk1n1 z%-!q19#r6Vy-XNhXAlGLg$Ol*E=ZL;O=$A$i!Mtt)WYW-k^mO2D00Ap6Hjgt`$K6~ z{p(l#%JcynQ7@Kjzg+3LhSuiUJd>8#ZOTW|6{-Q}ltD6is3--Nl7ytASLa1%?**#G zj620-UTdMwjOc$TLg#iq+P?m80$9X{`k45EDy#$GZ~|o zaHBP^JjzL5>BBM--k*?kVMsS`)#7gN*S8uNAc2t>Si0LWYsn|ne~S;?G@a(5!$;vX zB4dmLSvQ%rQaDvlG9NQc{z?LB;JeM?j*BmUGJMiIaqQ5Dic#N33%vgFhchiw8;sXv zbwT*(i#6zD-rt89z>AXu@dX0xem~-eDALS!h7ttNP=6q$n;k-7SO23qHtMs$3yWl$ z8Ik+dyA`-I&{O@|#n5(goZMutJIJtp=Rvz^0Ki)RY>&eMjU&79PEA%z!N6lOyX)=y z3*LwcefI?JdqLGu=Gcz|hsij!c2tjdd(t{K`Bavivn^v?h$}MC!U#97*wcomzTd_TSDU*IA z9;tQ(-IwSSEk>7RgkDoa%L-wf+p7Z#;5o_h=LLG!5mUy+IaBEvXl7rsT5DCXt_|qE zUmauKenwa6llwlfPL-ayYcoLCj^69Bc%+FKy!O}WiiV1oT=)?rbHj6Iu+b0HAsfE$`Ay-YC7I7;KItUd)s_^*sE@)c9{sngb*l`I#5eLI zKJ#us74^u^B5kzCVF;L%kLn)<)0#8Wh9`{uEAgqc--p+tOAT=33mn4|v1lNno2>?* zPCiV{y|x~+>%K?!{lO)@8nX&i3u8s|c)0o8k-F@pT=kuDGeS&&ii=0uIo|RHj=au< zNRhE3rZy%qbl-=&{;ZM}`IUWu2s{n{Md_~(=j8&=65b4)(6k#v0b6|q!yvRTc}=Nl zne6p;+`f+PH4xqN<)F#cMhZ$a&&0!ePPMmjnm*1+PIqovRn9fE6#%FJn%Zll2)K{` zCZooX=dQ&MI?EdP2TrruZ_Dv@Sv@-d#6nCD0lrt=DF%VpVX$8>(vwCw`|0L5<@J$0 z`fI^emBN41Uz(c(D-4OdN>-ke_($x}>T<4kd-B;};jCCr>!rA@t)bL*owMhQZziSV1_w^%8br(j~EIs9K5Ald4)4rUMn zfeHL{u(2Rg)6tYBk@taQkQULm3j}-~)5_sNUGg_l@a0{6kJ1-9zpI_#fbCkMmE0w- zTPr;f#V?yY9cI9E;JWPPw!fSkj_i5LcfS@VzqN-cAOE^&McZ)r@+eYA1Zj*JL6+I0 zP3%Qo0u;C4VB_Q?p`0NK1}7lk8MT8u1bk)aBu&v*cuU4*)Dw3; zp(TCj9Dv=8Cj!EA6opa`Z5uS#370*6)ooiZgTg= zqVl?Q9oG%qZuafhLoLo%N(U19Gk013c@#k#=y%xMda$kAWcljDA8{wZda)z4p*^S&N@eO(+JbpB;Lz&mwMRaW89A1{9s$`1%N{&7cW z0o3x}?nrjskAZ*p$0^@+g%cZ)M&LeId|$rNNFY5$aN~9 zE2IG&1}xE4OzAa+N_sjhBK-YB2rCA6Mq#a^R{mr)6Jsx*TgNv>>{}ajd?e33wS3@G zTo7O@b3|UktTa_xb$}AO10=TqEngzav0qsRzTEnvLnEMR@akl$)IbS{?U9wl7(&7f zkauhYlyi}SKUXP;uw3Px^v*Buj(ZmSGq{V@CH5&cL6{vumX|q8#1ZgLGcCiMP2x1f z8ngTFxy|T;B4m#>{s30EA)K0naUnVWM{AGs>$z!7^15<`Bs4889l8=AjRTeY$==1H zBMnMCw#e>2+L?PR6Hjn$-V!&x%#V#AsFns8B$&7a4GLzxJp!5D$u7_h! zn_YY&p_@X8=@ROarMFkQm)7M5?+Mk2+q~YRagnRdmOoHRloE~`B)tdihLl7IZ|6m7ttNyQ zp}p+Yqlqe~(!Uo5L@+!=0CWqHG09uBGOOGamt zTS`OGQ&~=;-w5nT>A$1h{?=9Ld_SbuVz}O$Z-`r-C^F|2FF@+EYd%D!PvU2__ zUgN4XqcAN6%Qh4R2HVCF^YU&_fbOLqDdA+xT5%ur_y z+}!mrXCg_9yi(6YNPZRwj>F&1X|dN5r&MK<>1Fc!moD2WlKbqlGrsZiWeo^mNYyiK`amJ1j13wud1N z%u_%FjkG(d%1C(bnZ6WbDFd2tP%s^Gm73~6dyr3mW-|dz?B3X~2{(1lpUACJPu0FPY|LQF`0w@h- zMv|_Ei^&-hG3fqeAT3L94Q~wzg*=_*WJ(wK|Zv?5A!j?)!rz!a5zpx`CV^;pF%^^M?5nelLt2935HKsS?Vs6hgr%FQEA1pADJT z6fe|HZ_&qh3lOMBx%YAa8Q}M!wgBqBDsT?LJgwkz=>eHD4&m`(NUt`F5PInJ_;_nLe+ zrQglX;aiYh!MYdpvfw?oG6T{35O_Z8o$Es#*J|>2ybM&L*f6`=9WTQDFx(s9{V}gK zk@%X7l5nn~tR2&OhZU-q_;rIW@g28tdXBU^{G5x5Ig z1BwtATb1BlXV1GM=to3Mu?(0UR#OoBF~Gj>1IWR-lG)CM@g;(J_RDE2;TYB95%pf& zI_n(xU)sHlK5MPUfZ-pCUg)A;yD}+jQP&W9$fG1Ve$j2&4;PXdL(suFSurw$&blJi zrGkpOF3b{n{JB%)k4rb2T9!8VCP{lA{_#}<>=Ez(U-pR7yYrrmRu3;_Oh7_>?|d$U z&-TiZqOVLQJyc}spjxw~dU+9zevA+ z9vBnG%m)}f)i*zRJ8PlAr4+WI_JjJcii;5-lJ|1?4MH0H&M@>KB2;TgWt8w4s-WjP zYWAO4UX)*|C)(&Q{Aap)h^DwtgD z$&V3S85-Y9^s4)G#EM5|U)9*Rsb${Lo0|G<#+PJ@s}Kj7`ycv!qZ3FZHS);T1rj%s z2#pGU@9GhwcZnW2L-}Gr;B1L$i_5)ewi57ALd#~~EYu3Kh92V$7a&7m9~eFhd)Ff` z%v1K|ZOictK1T}_GB5!`=y%tAexAiP12;?Fm2!xCqWd-{i%+t%@>U8be;h4yP$jI5B#)>U|-->yElEFqmysI$5UPB)b z=h~;%h)-TZL=)Sti0(qmcnADHo7TKRgV7}~mm*i(l(re^HG>M^BuF0ZjvdPg@S);B`eZVs}9Tvc6j|E1Dt|4=rO z--~c#v9WZBVIugy0l=XD1OV?oIz;~4)cIxV{cY+Q+S)pChX41d^6p7%eBMWO8XY}a z4~m3XZB;T4)tO3XE@|pBmTK~E%v%n%J(p_nLmw`El1Lk>)W?3_@z5p979HvL23~%0 zO`d2A{<8X=VNPGs^Yx876T}83!{qi<#^auKuvaw1&WGDO5s;pBrfc}&j*1&VoCQE& zA`*(urLe-^ccVkLf7zPv;NOic7p{~kf?^63hIW;O)+m>MX>^v0m-7qG*U^;PpRC^d=yYl;h+`GJx-+H<55WY{?|V7x8bE*r3KN8*7l0I{ z{@!tsjfYtA5GX{6$Dbd*>R?_+cM1tT?SP|Ax^oZ~1rCldw#A5Mce~krVz;&Mp?MOKb)@-I?WVu* z;a*o{G-)kie`-@t%GOImp`-^723H1*4f>P3sGTDVa;vTB8syL%)5Q zn|G%a5Rlv@EzQ#J$E6nU%cBdl@^z_3ow!TJjDj>7_6xIb)h5#y>&}W4gV%|4T`M@DEl0|M;N)bAczLZtm#z$*9aQzoDWfF!`}Pni#!11{m&L zZI7i=njC|MQBEW-)u;QLQ-L3|g?EbSZoxr@KU4Np}0fq2FlEc9AhL) zwpIAla>qCjw6Cu}?-fkBh(+rFMv~WrT1MS?OvV z0n+p4Kb^LW7sa+mcV{MDIj7fhTneg`-o_%Bo zF(ThMRcI^a%7uZR;tQZkx2N;BDV?%e#EkgK)kLl<8M|q^S#mG#CD&9eUtGc-ibg;ez~0|G2jE^Q>9#(nSSxO{0R{ga3K;w>XYtat7!-LtYH8A?ARv4Q{dH z0eww+>8}Czx)nN`6rX{_wjkfg9~RzP_=5c5QeEED4QhnKU*nDKQ6Im0w&tsk>uF1c zc_PbHddu9XN}dI=mem_X@*DDNwb@6_x04h}JL>!o7&J5XwKn&U+K)GrA&|<9LVH5? z&o9V-imHY6V*fpn>7y-qMUA*Sl~EFi0P5;Y&M=CX`TV;F&#pS~9pUis7_hsT^B1e>#PpqD%wPLnl;}g>_8vST5YX#`7TJ2M zQ`(k&r)l|2U@|<=rI-JKw;jnj$iN+fh`tO8Pf10)L{NHQaPc#N%48I#B)Tu|Nweo} zJp?YSLHRJUuRjAyfUz`~gBg{+aTtB5M>LAzX7cZWMZ0f;r3VHO!1JC7uk*lkXxrkU z&_0Da;9V>#5n_g@pkKp|eI(Z;XjtQsQ@q@@_c(lmd|tT47lTgVz$%Il2N`}{D}bNV z43%vb=cdH2y0oLlmthuXNcu5|xi&XE+krw?(O&tZjof;reM4T@I0OYgtx4S82c%&* zUAg>5mO%FG?eE1tD2SB~WbA?ig2;!gAI40YEcG1}1dJ}EqbTeVE`#qaqWEF)DEiua zd!rWWt1{}EO(M6W`ia*9lN<5#jXEx>Z^i`H4RB`B_KQjCOc@5ExZ#9Wi$DW1${cOglN<+D)@DdekBB>w0J` z{kJJb?G>T(AKwJyu*Z=9lO^SUTfsLtijSPplD zIP%PoEyU@!b}V!Hs!w~>-hd5$T#Y)L*V%%3+&}3SgmzH;4uS=-EI#QIvd|i5IgF*t zeIh*3IPUH{K|#QX*e;=(te4~k(YQ__fG&d$0NQ5T+r91n2}L9z(!nkeqQ4Lfthxy# z$XAim%>IWTl-AtAI70WbTC5x$^O+L9FrLE)uF*;X#SHv7bSu2`SBy(S=Se$FgCOs-{; zQ=53j2J%*;4T)$N0~FXO+J)L$s6+SzN)itJA8s9@L4uIj%brC`r``wBAHvzkO|4K$LuAwrP?G+FuwKQYA&V){L5eC-onlVxQkyd1O9Oyda-Wb z2ABq&1*-oPjgvC|JD~OD9Y-v@Mo6TITIq{}8m4H1P=C@Hno{?+kOn2GU_ckLx&6;` z&POA?hl-NNqGax;Ks4_iS`tj}vIVSgh?G6zev>=500#9gYysMm;D88W7ScTf#%^pd z{3sMR9W{f>oo~{ee$;y!#@6%3Wp5v}w{U$Hj~!&bQuCYYLlDRpBq=JSVjzTdxv`@2 z3E6rrsHw$bF6mK-+lfGNE~-*Vvo%`{*5|x|;M;P=MRX#)^W$(hLp(48NRJ5inyXc>E4=hUs$2A`?`ELY?q#6J5lT)SaI!u>hl zt}kS_`f6Zjz;#00l4~RgC*vHo&Ppt49f(I%bt96!y#oX)d-8`)P&CDKEvWER5=ld` zs-I^rI(9P#1fu!W-fSW#Uiozqscedm`13)o6)LfQxN5aRc-@HNqW6p_y>@#cr(2gX zWE%~!xYNN*Hh&55nfCZnTG-^He`3D#g=0c_{(0nd$Vxl3==VEZNQgcCTe@pf(tvIY z5jY@;?hS#J6OSNM&RoPa+*(X;Z)!KUIyf2|CQ$>B}RVK z*}Dddjk~1EW`gL*87W+ zGb=)*N&S8?GAaJc9oAEuRnOp_4xTkaUZLvxSc7Vgeodzc4(tw6CRSD+`(2*ayO*X- z^w@$(LX*zZ3JYVg`YKqq&3_Im_TU1YV?2>!28!m8J{N!7d!eVPGUe^Nsxg)1>}^y+ zlRqsy18O!}LvLbbL4RR}iSTdBJoPwp_{%i*A4=7YF^WCJ=HVEqJwZRb1KjDzhmjHePx5Vx`rgSG;S4k|f%aovE!%6GQK3TUE*+umG2{vc zv`&AHhQ$OaLYjVvHr-zlkjvnjfZ2jg@^StC3x51!wdFB1eh{&4}v(f zGKzXSL<*(%>+aH0Hu=ZrR$m1Xv*K*_fb$-MVFp7;uP7F$7yBZr0@cTxYR4|vgM~V@ z&&`hv>41DOg`1)o`yfzXFh@|Ivl?yDX@FKnh57PgPuXTa1vkS@f{{TI%Bk7pTB@-@ znCrpj^2-hJ#X75E`8$kQbU_ojK$VZ)BK(U+l?jJbn!ib&uAb@{XDnyq>xAYlZ`z-) z2J+wiT{O(aDdN`?u$Vd;dwJpcq?^mKqL#%wcYex}yBG6kiTkiUP5xJK?ZO`t<`iS| zY%;#pt~QOwJ+z{#`qm`oj$kH+q?j+eM%nXlt8|(&GY50OU*j3XR1ape9P8Bo_#`k$ zR^fV%ky9vo7$%UoHRp?mNXT$IO3GFeImo=|*<0V9|8`csuRKCSPhk>wt9B2$6-tgh zGtK^H@~Ag!FO-MK67~_r4uxCZ>Y^F=0WoOfH0U~DJcIrV7cZVFSdkiw15rh2XHvmG zLO4r-8mk9`VY}x=aSx~$QG~${5`Ca-HBv7m2|!yH zUNnG&=67$}N>*OaQmJwvdvR|w*|sAHqhzN~a3z=rl6$oV0-^7zefZfI6}24(C`lmg zLp~(Qv=g69=}_RrGw(7>e&h<3`z0X$N~0RP^IQ*};boxY93+Vuh}!rOZ@T^N7K&ej zK{zl*;R^|~1T_U|mT$3d^s8`Yj7q$qAcjGwpCy`C6s%!iwHrTo*voudv~7t(x|+@3 zZYhpET6jJQLr(D!ZedFFnfLot7pB?|NWy4w=t_jK0n`Wf?u47HM?<_JEbzgQEKBDL zDX0sbDEH)nm=OU9qB^lqz`syzkoyijX1F=4OgfKGhI`B2_YmLs$D)yM7p z>H5WP(~%y#Z2{!!T)oztoN+0`oa^e-177V9pEm^&Kd_pztSFVzMMuxkVf~Eo$(Hrt z@4Ke&r4q$|LRlD;he`R=PaVeA{CSVJY@2tf;O#Kp@fbdnT$wlh5XCmZLhOlzRRuS& z#gqI74k@BoMc{P~+0RlgG}*$pzj@(b`{pk{M6$`1gNJTnFhAz%{oAw=mwZg--Z^+< ztqC}vWKfv|WcYV6W4Cnq!|oJ0Bm<{j=91g=SqfOFCynX&V&ew7lfqR``kRieQ*zPq_}(G4NiM2tzH&q$rDd80t{qnlIWap zedyT;KhR|rjzJJGeYGbGVi{3*a&?LHB4urlsF4}mK}NEV%5AA)MRnFJrm@rvtdTcq zykCB|CCl(V-sU!hyO-ce?i>-arEQ3Tic6FTk24mE2fxwHyk zN!c9*R;HJElG<$Bq0$qUfFxb=by&#X^m@XNH9<`R@|w^wWvHE+r1C zmq=02s$>?hnpX--MpRh29E^L>YbkI!6`LN;K6Si|0*ausdAtY99u_O_RX&w$9egWi zb0@3wZeuLd#eBU#I4F25$s0k`4J2Sj|?rR84bzFcmBbVb{DGUdWL@X6S6iX4xHxbJb=Oj5?j^;qe8tWeYFQeeB;t0UgvORQuM{GHjYRjM+X7$;& z`;e}8c(W7=&YjP2nI15XKkH>#eYP6UDNqi!{3BJO#b&>!K-qG{c7G#lTMlmBq=QjhQwM_>QUkHiV*VF+ef2ZEU?j#MEYbP53&O z{YdN)DD{(@`0LaUTi0hq0a1LO_XIK{fD7u9*!#(Wvghe^#m(}tRF}6@bJcq@=$n!0 zhdaV8+-lQ@y1C-4Z?dUOp>Qvs^h@54?aFMDQEz9Kovp5*@pT^_yt|uDvwK?c;|#p^ zb-8Lcj#g|N<2P24QQZ^(TiU-6H@RKny=pgN)p$P8e9{~kmM=+ta{CxaUOYq~9I>h* z^hGgnT|0P`;YIppcK>`$;(*&6{kwtu{z|9B@E+@s~x zGfAMXgR4!#>_tWenIXI3z?2N4;u9mLLubLx`HF(?V6AXvz!H)dowv^J>mG$_P3%4v zec~R0N0zTgRWcIo>=)&_Sz4dDJuk>H@hE-nDSY|`+@J8odiy3jF#Gm^1Go1`5k7n_ ziWnW4_x}1&>9r$(da-(L|GqRYGPU}!onKiWl90>AAGp;rH6J)KL$;vR{5%d87-Rqp z52Bj@%x1-5Epd8%GU9coKbG#S9duD_-(~0>ONjkzb+*iW=3P7lDTK+7(?q&;Fxv;W zBLjbAEa7>$eaowd?|dn~kl7vk;|hgZMK~^=wX{JxY{s$@IV*5z7aC2iPb?I9f2NjB z+vHzq6#dOSpFitHgl{?ty5li>^Tsh4RZh|j;ys~Z;|Br?m_O__)>A*^()or&!^iWf zxd}0Zj~kt{3VY|^9eNHvW|3Yw?~ba}N?~(Vj*r_NJ%6b#Yu&iHA?EpF z$jn@Qx~E7f<5}m<{0MJ?JZyQd_1WTqP!=XtvNJR!<}50_h^mzlJ%NGJcnl_0!MBn{l_qZ}DJkhc;T07sD_84rq$2aL zgc;wX9$|bkN9=IvH(Fns9Vsv^)}r%n|4P5Emeh?0L<`6r*4b<}pnh)Ex>!O)eZD?i z{H#X=Pu?}M>_-yN0OEo*7Fk{Unt$mVA8>s&_`$BcjKg+yv@gc3H{$S+*n5p@q1>1! zI1*A45t>HL8e*N6K4DHO30HrsCh#+izJay8+d=np--BqznjLz+KZA~e(g<#q`Q^7R ztsnAzvgyN4z7@Y;HM+KC(gMCjtg&(Va__*L>nC0zQ6E>u^*oZcn7{{H$p#GJ8}i} zh$(09%P&|ZSgxIsVkMxPFXjs~y7(jr|Ncl#N*^T2R@QpXJ!3fue)P5fBJXN3MxFwU z`D$k%&CLB3w<>?+5XjS;lMxfRz+$0(QglA_Sa@YvSXb&M|F=VEG%5H#@6Atg<2 zP9d4VJIZoS$;%M+*Ynj&4Gb}1&>=4{(PrBaR`pzLQ9QH=x%ISMBU zL4zg0-S&j>i1#A!!R{e0MoMwNpn7diKVPP%kN_5L)Av;n+Rsji$*&NhH>6>WNwCVC z83K|NxNCmz;Y09?-DB4RRUs{*471SX_SXvHu2y3y_8)Wq7hP`|7Iph=0T10UG)N5% zqBKa)5K@Aq0)ikVASEdvF?2`_p>zrgigXG{N=qv#4MW#R&H(QZpL3r7Iq&-(KX~!M z3$MB7-uvErt+n@hkPZ&kRn&N%E&h4~>%wM&@Ps$=A@*TA&_?J#juqkW5LjY}5`*Z# z2eCp1pHe`(G$oUt4=vJMUgTLoPuK`W3F~o}yD7TuAj0hwx-Tg$Ed3w6ySw5ga@w@- zGidxk7-5xHh;8w#g(x52BNk)R&q4-Zr>!AA{$bAKDPCE?m#$TxWTryMETw>1$?BIg2jHQ$)e?!mIy zQ}B#R7?wn?NJTw>AoAImlLSVI1|(eDu}=nOoK4g7A|&9gQ1WWqU$`Bj+&0gJ&+)Cp zDTIHGY}oJNTttmLw5Pr;DU_j*S6@BaEh%6+bH9sQi?lPFDt^ z>KW>xuJyMjf6B#YvJ_Kv?tQ@ty_0!X!M8C%K zJMm4Vv{`lH-yY#-6XlYz5A-SKkvAbB6JZYKHLP#^QYSyO+ zYACBE*tlz{n2~<&ZAK{yeVVTLrYstrP$W6*{h{uuj2ICJ{|HA3db~B8I_r;opOCf# zvLfnbfm~!sUaFlH0&RzWLeBnJ8h>($)6Y+4bk^ogUL?t3fhN@%L=+I|=Hn#-!+H~g zd9`9D1VbK}V<`X@;#+Mb#8cfyzyIeds*@wU(b8|E{-YLj)_ZR?%s9kdC*siYu}t&%wL*B7r!l4`t$UCJJ)i_&v&n*7G^XM;#Xb|@w+-j!SQ*u1$vkm?p#>*rN+niovN z%GR@+O-`{Qja`Xdn3GLFAsmgf#ivCsaz${sHs519h6bvrPa?ha-x=6@p0S-zLD`sg zQ$(B@79-= zh9I;UWY@vBh2&qKe(0BQthykMr(l&%LQ#u}PC=#2TTR5wEwp++86_y(5{93S@+gIS z`EB0!TW>6zD4HG6dUEcvNwwF6P#v*5+iK@!{W!}sVRvmwiW5OrbX0GI6#XJogY-ir zoR{jZcqS9UC3Bhu_=8axPwr_)Ld(wssLH1!}QjBK5! zX+|{(=Xt{GVMU)eE2}%TJ5k6;LxL|?@f1#^GRs~hftJZHY%FB*s1oD zKWTs}+sJ+}XN1@&L1B?%XQb@Xh?egemStBSB=mTbzt{M-Xd{s*fEn<8dO0uL5Uk|` zofIHMK|Rog;~%5HHcLi6krRub@nqIw`AS+?x2uqE_m#Bh*Hiq_vR zuPv1~TLBE|74)vHL^O%WXd?~xfnuxv?vpu@EfVU;^~M(decjD(j;=0!f{P(dx_qi6#5+xi?a7aR+L?ZuJilKb7e4usA%)8@-hD;glU|7WMZorwrTo+{VQayc zAXyf60cy1VMX?~_YlHq4phemy4H1MX;)jLoqE8RcQp=w7t$^H?;wa`nM!Z`1eq{nI zD!FV@A-W~zK$S|+2h!yHvEP)i_u^2v9zD~vueAO5b2(vuwC(nd}1VI0qH& z=BdE`j!08L5;OD-?exb!Msfwl0y%@}6+Zd>)pPiLLIp*7^RulTu`>gni5;#!8681#QMs6ld25og~)-$cbQuI8BDqS8W2Tot%R1`=o1ly z!(!@9XZ0&v#{Ckam#<9JMD0 z#xV5@olSkMt!F{+V=E(G{qDQ!3UsKZ%W=AnRpU@(xj`9Zv@@i_II1m*yq#|e`=`Bs zcT;R`&P8u0dyndz5u-N*^d|_Iq}*{&Ez!Z3&8^p~L?@FQ{K?W!YpkTdXEwqDAc=V_ zPMr(7f$ekJXg|T+H;u4Xx7cX2N)hHyq|R1zKba;zt}L-*>IrOF%`-m)$w^kn3x9|- zmG85G#F9AXm78Q9Z0@b7`H~BN(@`ZwDzrsz;9NkxhK#cAd19naWG`YH6?hRC zO$m*D&S0GL1Hm?nD>B-dx<9((8raetE88^xMo$MK$j!RNlyD5Ua7T)$l8t#KdVxmf?m{m11xp#%vJG&>oue1GmU zoPZW>w4SRGDt>b5H*r`*-y#g7b^KX4pG$7#4pJBiwa4LTPbq&c#{TSdl+)9ru}Z6p za+!coG*ngQ1%G+-F;C>@O{kBs^Hg1&U-PBp4P5a;(!Px=j&OX^_W}KQ>p)5Y`%3rN z#cq%;8kuW-G(doJ;lZ;1XH(lN^WFfvH-l-FA9wwmm$JIYaG751=XZ1`LFve@a}UD; z{ri6Pu52^Suj#3o<|8;_52F!Wd;S}WpQel1wbK;hBzLABx(5AWr{;eih$c(e4cb4o z62(o{TV6S14hsp%7h;~dEgNd^XH&}%H~BW~wo>cWRDEhhhKQoC?ZJkgR)p_DERl6s zxndQ(JbV@FsIx~|!CDC)6NS*1vb$(5_htKbWz+{Jyo4+(^eF9gi z#Y_?j!pQVy7-cU>GUQGLc+_F?gk5}RSsK*qXYX}tZO14G86w!k(=+r17vEn%!BLJw@K{%+{2 zsu`kn#;axBXWbR^i#UG>oS$JrdtyI)d3`#vv)}rj-h6+lcpvjEhc)N}%Ma5(oL|t^ zOw^_os(a4oe8aV2w2s#uD8o;hw&b#1tF)=H`nBdyc@jkuyLNJ;8+upXdc`SN-OU?j zAW_$_xIBVJLC|y4oxIhY*nFG1xp7`yIu_#hNupFIH}=+2NdoRu(v>QM)*R)^^i8diGo#I< z8qXiMbw+VjViNGlRulg83}+Xq&DQ0c_NQ1{4N?PU*Fwz~w#2WgG}itxdWth^(H~6u ztYrC9hm0)-syoG8$CWF_cjhfKjqu|o)>R3T3>*F~BY<+;}u6f*uN z;?$2|4JEowJY=l*B2FzKbI)8p1c@cgMwBn1@dkp{j_fJ=MT!8zw?ddv==N~x)B|OM%o>;<_C$13$I)mU*hu|>tjYW6D!!ATWKB%R zBgo~p#XFbJW{IlW*@KQCM=}L@uzw7z^C`^@Fjp?YH%po;o_HBq7oR2D>9y#8wwV7| z=jZ~rB;t`(Y=XegFWNY&0t+AL5CcVmxA?4qWB*^BM9?EPa<3N3-Wy1QsVd*Om7k*CZUqBrK*9 zJdszdX9&XN&bs7RNqnllHC%PPEvH0sg+Fp{0?Wec&e<##9`6}JkZCc6v(>l5dfX~9 z|Crh+3H~c(#xuK-G(EE*U3EhFUv+mQZ2{yJyVm}9%-3%h#GhG-^1FHZUM?5_Xqel3 z&^{a7bKZL|AJZLtW+9f@`Ej^=^mEI{!vKlwv9oO4cV#HOp6T(MXXLI{Mx0QW2{6F^ zc1_qE=F|63?0MOJlZP4EagK?>({XQjUtAxFYj%b6es7e!c|-L6cX)L8jU{UT|2`tH zB&X|T>Mlem>np=LUlfFynv{zv)w2qvKz$oVEV5=< zOpjJ~V6x!p>rPR8$icb+4)@_?s=S=wVpcF2+1 z+8u2(EYvM<6`4EA+8yE_Wb%jhg|M+~0r5(Bp8ITvF<5`oNh<|et}aGkAKJ4r#I&Wv zI@9jo+x$NEj9lT}A!-4;!z%qGK3iV7y_XRF%U%{oAIU8n`!Suc=kOS}GXWb(k ziezYZ16lGUw@%^`^o-30eAM@(3;OsWc89bd7_!^yH6@mikf`loGWYx{BzqnbBIohj zvWm3s$1x9QX7$Ps?HZ4Ik@fR7`LNmI`I}_>|Efro0=ZL;8oplFPpunpV%gzk}zATf6q&r8CY%Z0v6f4=8j z;5Az(%D7?Ei%0Gpjxb#LsN+HS9B+cc0FUv0Vnh=;PIKT($uvkZ>^FKa-^Zr1AT%HZ z&2Fl#t^Gx~`?NFb0ofp19T_vjfi;3HS!JRQ?Gv-4+_(OukP7Hy3haTy_#e%UrTk{^ z2gf1hWPshhX?OBPyjW5&*Zx-jPuTBbi#Gq(fxj0i5+?Dw{*fU3}@-nFhh( z{(MLC?buDrR|tQQce|H{^xqb6N#A_OB6=^Bg6iM;_TUC|^dlCXC-~z(=4g8c5n6h8 zCDo9-e%C|=(1Ylhp6skwXb=V^fA+UDI9&+}?A|4iDMvV1{s_7%8f&%E3K7}HVowGK zB9*L|BmIcx`GuTk%dK6 zJEYvAib2QbW%&Wu83Ojd*X&YFhg|Imma^*hes`{{wtfkC-9WLN47;q<2IB2L6Jb$U zf{<~lG@F$iRq6x8<~-WbND=>f#h`hT4h{fKpB$F-F$-jC-IFY5}?yw>5yPch1kg4uO`pozIZ zibn^T7wl~jOExM-U9>-XE*$+Hg0hVdiA7m)v`*3iyKrBmIvnLzCnR(YT~e;nM`~sC zO>##<#t&=rwGd0Pe37pW0A#s<;D0L`{tXfu;`!>apGJstUIuk0!qZS1F2+@nbgXj? zBVJsDMQ~^Cd@p`cnoso$SW>jQPCla0=i!ot6MA4FD#9zgGUC>oeytVXRKs6v{g!Nj zNnVuB{b$=>sZizhj*WXm-f?2`?ChjFRAF+j@CY{u9zMc=I<5h>(&RBPgaU6usT=Gc;~jU zzkBKO&g{)i!xr_wF^>Nu4Xp6@MlHRUk0RE3EIbPlQqO5EeX>LvMuAXPky!Uvy7Lk8 z5OSQ_N~zr~$i>b;tv2U>7|!BNtf2g@wan33YnccU)JT_{-M-Q_C8DsynXBwvBpW6^ zdqib(!?gWjFK5Xb7WrjLR838jrQ-U>o5p6Wu;%jSqVW+EHO>=CcTaooB@Vsi`Qa4G zoIKQHl@uk@ESW_lnm57+1?V-JN5vt7Mfte@Faa^#z;C@_1c1+{dyw3ZdjSr-Z_;whv+u-lkMsb5vq zJ>3m76H0co8eGQNneC!937t}9?SHxpHoQ7N&TN$iv2}y%5ko(s^Z$-9#`~0YU z0dtJByF!(9NtKdEvAJwOW*1)vu_DofLCt)O-GpqtGH*aw1FMl-U2fmb)cP%%*rUMl zL?x}?1Vmb)D6wl_NcXDf_2EE8+Mp*ZNjE{~*(lxB(81~YeyFpgZ{(wrG{}mXV?_`T z=UX19S@qZaYKq+?e8KfA7s?OaezOZPXHf`CFXUt=NKO&Su;{q{N$)>>q-&kGg`j4> zKd=5bpLef@3g4OS!+zU+oX5?5AiTNnam(6>c3h-i zclR|j{@Xi*%+DLTK4r=BGVGl60XeYbL%4?Jk@A9e^=a!MN-NQwOYtCkBE=HrgJG3Ti3y%Y`6JuloC5&F2N z>N=v6nOu*HWs=jCIK9gXFH;oLk7n3^I~9Id?zn$}ujux>U5MI!-gV8X5jS0nua`^r zRKY{4E6F4Gc)tJI*2bn`shhBTx@x*iQdCTgUqIlELCJy(eGfL)IbQ+_*Zo_e`<=T# z$Q0fV1D_t`Af{{5Hk2y4%G?B%+;J=XxPn%n$6$`IYb{T)&b@PYt%JLtrz+k?&g^PR z!!F(gln9OZtQqIY1+`Y(UBj$stU11XHnmcNc1-Abh(f(-8`2Ru#e6;c8%u4cCoVZbnrr~p(<6iVDi#o@whFQ006`ht7>F<4qr4LS(fiS*` zZu0_Ouba0f>--fj({*wX(fZ4&mf1U!FWJ_bitccoC-Q&357a%oM?V@57dtp2GLv6x zx7Vu14j{9(p2laHg)b8)zH*Lc|ErVyn;DpV)ZKF?TY?Tg}lWP`HPxI4c zWGDgJ#PC9Hl$sNtyqV1deS?8Xl}Gm{?psxh@(Ie08$3EpH4;!*=j)T2Oe8F&u!i(3 z-U=Og@KMab{wQ9;6EVh5m&#+7cP0^vI)QMKuyGzwW-ZHfmFuQXYYZh&*$&A>aM`m-LVddp6oBgDm}4L_6A zb0pr@8?tM+U~LfAj!181NEF1|H27B(Y^#Qd@5=A;TyoVv=yt_+>cmNt)2zbxvmIa` zQ}Q|W)Fo;-J;*;bBeV_XJFzVg=_G=))ZWEI>E+`rweY~#@Gs_fpQgd=Q;1eO-#jj( z`@2w&Oiy$7^xQa1kO}zLEw@;)18A!6ZQp-_N-O(wl(G`cI}p+1qj9(oa=)I?=PSeA zQ|el-{-OPE&gQsxpwB9_xt80%xWso`Ao;@COmmh$T`75-?~e4_nkEjiM@n>-Iy59u5jdlMZiI*2&|Bg;#KvZ80Z(3HsR4(fbf-xHK!5{y^hK< z0t+rp+?i7xSd z*;uH-7~9K2^1#P^k#G(!g3JwJz<+IP6!5i2eSWPD{dTb>;I|TY@92?!qMU{^8@lK2 z)y!8ffxE^PaA#m0oKq0y5csFbS{U@zi7nH&dbEA#d_8h-U%>PnZ?|Bs>}iudYW+u6 z{qrBU=1*~KEdtMNF96Rs9}5ic?o=4ntkH?c>lSsWQxGm5-Y8%RHN0&(T5z9+mUiuJ z{pHSSsy3|Zaz)5)_*K2j?`Hqj@{#(z-%aDqzNlt}+1>ZeL0W+vKZ>4}kB1x)<6rE` zHzJ3;$%s(Ts#pL`hYbwTC!Gg8KldpboIc;&fM2+bc6w4!=In$CAB^`^ZX|H|I|ka) zryEf}0i2B3Qa#I5H%B?A&uuo#@tucReuI1O?P^JnFWUdfr8vXNnAnuPFi}gq^o5`z zDF54#YS9g=7x?#iJI8)_XH*)-Xu92zyBn@Fp8RtH`-!nCak5p1m;sA^Sf6YrjT(LR zm=usmznmEx`^7W{79JgQjhQ{*7_8+MzniTzHZCon{q`Ysec6YWbr~A%A!zZE<0Egn zqV&7QmI}PLX^dFr2XK)^%0t@(r@xy_@`3=7T^gpvt^H_6NlKQVb5bN&5wnWdY84XzEWT#@7(**?)NuRJCl%ZN zlzj6RKi{4r1?Z@fxV^CMXc0KXTv9C;ln!DE6K zlP;!^?H(vw@U!9TqTq{>oN~FU@1%btdCtCV%@D}is+GLGi>d@3_99ha_XS}+@&xW9 zy<2MGuWmc4{H<1X`LB5I(4WH1tY5aYOQ`CXM|toLG(z}kf{6B>YSE?D(q*re{7bz_ z9wZSJGWg3Rs-5FnU;Yxzm*+t``SFAoKG*ee=L!F4Gm;+bDU)c zb+PMb>$68=awDt*;w)D$~z zeP^Soz~X0vXew^wmhZqYQzEBY!(g5BEcQhY&&x9nMg2&NHT()R7TkWU#DI4{t&In! zuApCHN+=Yl{(D)cDYk{|hyZiP`K(XE+av{WfxAa-QOLYr_#joV`L06!Pjg)F1*}6_ zjA@mP#!Q3Tdn3BtvuMu1h4+_$JGrO`58z<8yt3kOeza!z<%xvfVlaVwtzGgXE@-P8 ztDFxaT2fa4dnd40x-){zItc+AdRujpk!pS7ukkHNE4cfP2k-GJl4*AG9LIL`L5|R@ z1lM1kw~P2974Ws;y`Q4CSq-fbyBxAQ*o%E-E^o+?S5wJ7*aocOHjR5U|92zauNVW&Ol>PvIXd}RiYw5XoZI0=d z*Z1O>rIR8^8I`st%TJGjE`haDv2%gw-BOn$z>?HWp_kz%f?cgiAuJPLOjC(&O%aIT z9Qy;FlFWMmPzbsRMZkJV$jLttQS-dC&k4pM5ItW@)=|g@wuB#@XG%I3umS4mVdZOG z{M+2Xk0Ju^@_sA?U1#6erwVdP`JV}E=g6gBobC@hc#4acGV9~fXphXFqmU-PxWwZ5 z`T6sW1zjB0)o-7VkasW;AtDq;JYb#BMQ{HWYKyC$cjR=h8NVH?z# zEPI+glvw=^R1a>5I4v`c_7b8?E+bfJEFvbBw}5 zLk9l7+5Pu*Wd+0Dr3ICyzebAhX4)@zFG&0pU+~=dDj{w)j(_VKEpw~IOH6yJV!rf? zv$)IU417)EF6(BP_O`>+$s=)>Y7g!72BOK7N8-Mnw=?SID}B|x$MifN9SIsd>pN)u z+V_jXKW3Z4l}Kiv-Q8T#j0umCsg4Sd@J`+BZVN0!#jT&8dsM;!1Be(R0*Iyfi;1X& z(3*oE_ZD=gt86!~(EE1ZDZr1O|H$|raIqLlQ5<-AVffA8m;4nj6&{F+s8_gPm6DOM zkC;Pg#AiL-ZUcCzEUgs3JgIGX8itFlYq!!JH+@UesW(3z9FP2v>9VX)<1ikf!2k&t zcb(4~#;PO;2<;&1hYUWnvf>B;Owv3T0#4l+L0oiyYBk^9)?!dR{{YA#EYlS(HaL0E z;`Sr(C$%rWuPtT*W}LoV_+iF=z|#36FiQ^Lxw%*U#V5qAK<-G;tN{i*o6ift{&8S! zdz?16v%_`*w$+!0y#)EIfM?U%$^p?uxW`PL^FSdO5w`aY?Q$~uK&Yk517DwP9q$UR zg9SE1Oy(;S@JvsIk(n4l5@F7GAU8brLgAphrgf_xuqz+Hq|Ikt&0v5MbTO z8mZZV`R&%jBA2TBQSn>xSexC!J^u)V1%#WO&4YM30Wk~P1cA)O!U^%S>1bXm-*G;r~2pYpY~i_I4I zo?}p;wUnKja*(L?-}$m>P=YKK@@3j4Dk)=i z?jBw8N}ZrV{xaU?V`Q^uukYncXBwLwicO{Nwn%S)1FmYg1uDU@$pi|xtEr&F<@-Ef zcpf|e@fb$v1FTPPQD{D2&(tag!M-s7?yL0_c?%F-c+Ja;KMp*Oa8j%S?1_h+apig!OoQ2F95W@tO1iMPoDiw|>sC{vx}$i;y72 zCK?JaAWwOb#~c=b$hCROZlNwLvU#4yDq9?}%N<&d8SH zBPaUwk+c{jIWSryBL=@Z02_=+t^wKB9?)T}dBw)K|Vz;|sh zwkM=}P)ABT_xfBmZsAx~d$ulKb>IAf!K>1bORww4k+Sz3Z(uaK+u1dpg|EFcJW0ra zPG;3Lza5>u!g=EKk$?8yc<=97hp!r~!camarZSu-^GHo}oL_9Vu^p&2$@Wb3(Q5fk zW&e!G`~?r$e9}|U-zn!=7PgHvWfS5s9~<}?dAK^V;X}r{wpu)+rt*i_ zd)que2Xr9KpGeKj$CuwOA0;~#H4GgD*1h*5v)%=rlM{wJHOn1X)C2V+*1y^HHK11Z zCpZV;5+ST%e8KtzCltjc8j6o5uFewH(A7N@WUGI&N{49#^ z82JBmOgEImmp-`oOW6y1ibTh%fz4}DsYHX z5RgdU=N!ewlG%X4io!f$t62FsyTQVtl6vI%d5acA0=P#chQ^CRXOuB_8SeFR@+CBO znW@hzQ3ZDp!BM)oARC_IF2J#TkR+PhgU~%PY>Da`=8Nxvj0|fHxgM45Cbp$VJm6}k z*@DNAuYV_>4Wrq{TLDR{Ogsz2aM?Z|%IZUg7`vl>+!Kpl1uR9@zN#GMJjX}fwKIy&3cOCFGeq%fUQr9lV zVgHOezLuqCY5u1LcU4l$;gKW`xx<5{$eg* zjep9rYxr^DN)tWDe=sZPL0$@)k$70$#}z)z;1da1_>hg{uP-#3^X~f?n_#`GjR29X3pQ4e)x?>VPM#&b^xVjqIeuB7}ym=!zr; z60@syP#*6uM8%>EfHN^&zSQIEv!V$IyJ z7D$Jya1>G^Di=)=CNMfBAFdR6po*sUUbM&(T2bXI0*r$XMPi|Re7&EIX42$eOH!L( zVNg1cFA_c0q$UEuUx`!%N}+rw&$Qt+IPeB86<)0F7@7RrT*p&ui(-hFMKReq${BbF z=@6C%2`Z$LsF7sj=V$ul#IP(Jj(6HlSi$(#b-^nKWbu|?tnT6w5THa}-?dOsRLvO* zn6_G-6UF+AmfRzL8EuG{9->!DB${R+#8uuT)GE35E1HIRyM&;Av_DLjSmg7T%JW;{ zw$DdO1`T7h9_EWNZvF$rY3gK(4`S3wMJJP@Rt>9N$7pz=JR;#k8RFss*$*?ewbfN` zXXkld98ElY!r~Qg(IyKGqKRq6KD}RZ`F;yG`z z165J-jt-w30v&h^p;|F(Z{NivxINeIdsPntYEjjaT9Qs#tUF}qJG`?)4_s4TmOVOq z`br`ipZGg%yJ?8@{%@8hZ(`9Sk*9>9sfCu;_4JZFU_roW6z0Q-FahcH+RrtJ2=sy= zOK=qw;B&ctQIX)x&VsAIY5)7H1T|h*o@H(m&;{^t!x!Nz$f0cO<^DXy&@(AR7)%`$ zRs$*^qmkxfraXd0fcRBra1XJM%Pd-RP7^7r@H&L^S;72yvDmis>}QE6s$f11uPh8@KM|#rT$>`A10nlkO`i%HS2CA44}XvV9iAW zJtxX*?5gLWGy1@g8RC?e!sYHx-LZ^vJrG+gdxIGzi@e2!@vjL|LuAJl4N2<50r3Eokb~D4E0MssYnaC zGp*^k(z)RS@MjOT0O*z9Q$Zw{L*IaZ%<1zDsXnBCe$hUOpqccM4KFar5@l!Pr^(8I zDQdWR#^`TsCkKsvniA8R8DKx=r!gYl*68`RX)7Cz+RuFS-<(|s&b%m|D6{)rdaiNb z{Z)Cpl9DyMO~%Frwe|yN9!@b=liFqLK!^^Zh!Udjyycw-&lNRG2JD!RW|DLsJEhj4 zTz?r4^GcwXu)U2(0#=C=a~EzAt=w-e2wum>#R3z-=s=91LLBy9*g38ZR@k9Q#WSpI zDYuYL9H3dA{)~f5&gXkmPsOvycKbpe!&ROoL%>P|pnNQwXp`#am7=6Hk6;l%2uCsN z3IVgtd2G6Cpy2!JG*fgOWT2z_-(zTfNp+(sLt?`$JEQ@#b1<6Rd4IY(5}%y87pUw& zS#K(deeo-PD#1qJqU1_DuyedF@~};l7C#9#H73GV+zyR5ba#C$wiF3D&gizd6%`fd zcsi_!V4_LFY9LJbH})`U_{)b%R3W*q_U8If_(9K_?NMLq=a;JpIq@udFF>$GxI4uK z8h|5S-e&*IwF92P@5Ap`0ie868eql#5#>Cg7so1C{mDrNPFOQ`0qMH=#7kXrK+>M@ zOiO7dQG)EI9Edh%5QuuHJJoJX8A4(!oKQf=)#m?RYvEgfFAJ9Z>l2&r5cX;q7LlXl zj|_<&k1)8B6hAl$RQv~SgLg!gk`A+1GCj)#TT%T|N#@aR_QnDS1NQG5yP>hTttzUj zejb9>)yT!8V2`zjn*((3nOS3R8sZoxyavy|oC1I-50-1kySXHP(}7RN8y*#Qn%Zld z5r*?{8yG&P`(9Xh*t1hH!9{%U!#`6pf2=gAzNfKK>#b*Wl$=|JTK6WXyY*-FKP*74 z6MIPtq(9Ur)|ReMe%FJGqU}F5F4r}~Iy%=};rTb6xe@i^B%#Wkk8VxC4y>}rvrfq~ zUfkl-9_aY2fx=|xwEHFrUGGf;T-|c=(Y~<%;(9_@?{@*G=5=rV!2Dfwdj#s4&%Q5r zE)8wWXJS|9{)@NVubpXa!gL{9v}Yi;TemVkXRVS5MskIU5`M&aM2(<^Ny3jFobLTL zd32xfwk~;a2!0ekqz-O_P)iEddC6=;1>&MYaDkwTpcUWapGN6U!IG3i_+iXo zbr6_lBnLqXYlfNXQYL!9et{J63yEIOLBytP!<=w{yD6bq%nsIpJr3p|nb01Iud$>S zc}Z?)#3fGYKL&VdE6}rt!!BUA!#E~F5h@2r0JCoK0D)!ZeOAc!*pxVe8*plwK?)^C z)kbecy%pb1e$+N)?A%z2W%&{!DiFTE|2$6X%KxMmEiKNKSWDUZ&og|j3A%cY; zZ*JefT2Bd9?$7Ezsv>%3oKO`3I-|yA57|Bdn-TaOR?XaO&4iN3ym{3IK!qzI%h2bcLJqG%dh)`zRs^%)r%Pb^6%$-$N! z$%^rA`UvZ1Q{9JoDN-^z4xT*niSf4FBSwm{4)=E)zVV#rZqvqebbejv1GH6_djHCP zopAkixP<616c5a#W_>FKpVK}vK+QHpQ#(j0DW&a-`=3?n*DgDgn(0WDW?D;6q?tjN zo43J|hp)-Y+bw;|o)Ukr4XX!bA8(aCamBN4zxquqip~C}R}#X83i?H;OH2jhT{eYKr8Wqc`K?9 zYDb_$tP{$N-UXc|XV7Xo3BBNfWE8U`cF7S$=VZ4s@9DH^jOf*QB@v}P$qG(8QO~a1 z%r2JXbl1aC*V%gzzUX0GDf(3=zTZ>FGndq7NPMr(NXY700*;%ToB!Yfbe19*vP$<_ zd#a}*q4E1%?;&QZCLs3e*_sul@op zJ2JJ?h_~~N;3zV;m(%Nke4T~v(8-tV{<_w{HuDEkg2<^=#^4|_yj!1xockBB;DP9E zA%S4NR^*)|p><@j@7bZ0NF*`1A+S|4>F3Ez7-W9wQw81E!QJ+$wE*#7zGc}d$QDJ4eb#A4gqaOim%Q>N z73XPk%0t0nfyvHLOI+A|zH+2J03FSLW4n@cy3gFEAn4c;6#ulmmF(K=JOSOuQFo1d zPMHXY^WC~zG|{y5BRjsNV7 zS4&;onshNNRGbsCg-X>EXl~!%8Jlb2>Em$>9@&~;2N-^sU$?a0<kiS-D~b?^*mhfP&29FO5K(I25OX>VlH!6P$yvzI`k6QfNEDpLDl_-l^1UDy zqgmn?yt+a1sB8;edAwird09`E?(~#LKc6)-g&{OF#NIv3=hw~DD2Q5>l1WY1^3Y+; zTNAOQ;hnr2zAV)Kk7)jN{=bI^)WcLqOPYg2Hws3MH(B%tPSWLUWpTiP#f6lON}hve z1Cye1L5UV7nGMetI59{fu9}24oh`)I#C<4z1rham?OlYz*-zUrnq_FuK28wy@)X8z zoiGh$O&LhI1AO6EReYT1Cq5`nV2ZUARJU*$fQx-TV-d-6#jk6DnAv^6h?Y~#t0cyp z2Y0WYJ{`rmNH{#h?3uxt#r}@%yS>Lsb$x{3XV63>lF6wrDiUV8!ztGJdf||jo4=B` zsBDe`4<0vF{@@9xiFoGTqII#YV`(&Y4LdG=+PKDt0t7UGs&~kfNEFyz<4$XnWTJ?V zn_h6eqH;J4+Vh@n45In^faE_n3pf;V>z?=`5$o>N{_%xC0C$Xl4UzcF_xa+v0skjq zyAs?PeQS?bh)|;|D7#)+6f0PrPO|W6kbIRZ^@GA6Hq9IPb1H;TN=~o|2RHZk*}7=4 zVkr1fd9mDdkr#B{1F0BR3o0Z(69;2;K`G$58`hjq_qg9|({%y5y4TLvXf)V9s~0uw zmFJ#{J(O7ce9|rfiWtWJl8V7Rs)1MjrJnc#+ltKo(L6Pi&5>nuF9wSKA^#LI2<+x7f5~voi~j}Mfl>jxR|LpFCnviJkN(P< z6H(YXcag`TXlb4|#e4$2G%8$eBG5xDgkzI-PK9N%kkeR*c0)zqtq=^2B`DM1I`V)> zg-=m7c&jhLnEl+F>Ex0Gc=U0%n&Mz8RP2FgD6A$6DHrry5cI9v^d;q@95lKIsv2bK zZLxb^X<5I*t2SNoM;pWEr2*YJPsG5!QT0~B0Oi4vZtX z@J}FdGWjsHbz)1df!F}7BPeE#&v~`*O7z;MY3340HkdF5?ryW7fpj$h5;1dZNTI($ zaBKbtf~(}Vn|C7$b?f9n;>qo=HitC0(bx#$W!eBHUyA^Wgdmn)E^FE=0Qgm(H78s8 zXMnl-M3wD>dAfo(0mBHHq86|1U!zIf5A2sZ!fPx$Bjkwr&f9egD6yjhz6CU!*1L$m z+B0)qVAFT_Rci7FAWKnoXE*`(Ta?bgxE$~)TB1W4N(br2Sj7Ess} z-kbLct-ma!fhKlQIE?2|6hh9V z0RKJ50Bqd|Jv;N^iOZ7q;4%7^n`LQn>iDZvqcf7CK3)bq)+M{&tb95Lq@{w0Tm%Pt zcc%8e`&z#q&<>ZVL{k~ozIX)iS+d;R+=p(PZ?ty^Rbnf#lKN8k4VnYOTwjb9;882U z!@_Vc_L}zjZtyh^0JWX~bT$5uY;=csynDap-mzker^|X=zn#j%{Lk+pl)$mj7T83>(q~> z0EF1&0F3!Spzy2@a^+E0v^LrLpvdU+wG>oTx`+7Fq)wEc6%erQzY>9*Tqa@G{{}$M zo<33kSLp_<;;KU=_*r@9+(_z8R-56-Y(Q*wFCrvXo2#OH;iNK@fI`17iQ56_sC;SE zov0$?03xtLGxk~m(iI8N+G7A7zPpCrJ~+k0O$USrC~tdvJ~AfBSj|+yA0=%;6xbs= zroQ)&Kv|}rAKw*XtB4j*i)YTCb!mzNxcQ{du@h^p3s_ffQwVS8q^N*9Q2i(}2Uozp z*YJ}eKhN=5pX5~P*R{0bPgBkzJE6Wu!-*Eb_qP!4^g`x^0NbhuR3iK$B8dqAPC35* zBeTT=7XRsKlVobD8{XE(ma7VuUIM;-AzrVufj8G8L$I#&0=!ttl>x@4)ZJVE%uw9B z^)lKB05SpbwkysnWjmsVT+)`%PY{m)trQ6-{pbbkM8&d*HN7Ar8yEDI0&O?uvVQxA z^5WEAxSLkN^8lD=9plW}Cjz0rzqO3BmM6=jEYA zr>AFx;E#e=1YHMS{2A9VVK02OKhJEXK4C!4BBNsAe~2d##R3hKGkPlVV&WM$byfn6 zP3&)AH47t{9*`(p$`OkKC^s|f#`^??0@rWn5q;$Dz+SJM0(x_w_H8B_wLE>CG14R0 z7HosNMl9&cKPE2jxEt7nk^sHx4akym<8Yp-iGEyatjHqkZ9|(P)QU+8JYS<1eDe~z z@&EAk)?rm{-TE*o4T~0uMR$vIEkFTjK|leK66ux(DOrScOG|@viy)l>(kTs+N;mxG za_{|~eZF(f=jA`*b%{LBTw{(q#ywarBlpI+p-JCr+yp~?GJ!6sZAJ0GGR{>WDTw9>!zUfk`5d!If_$CK$zNX$B(O?Y?kq1azXsF|!swwrAmV!?ehn(v7!a`mWywFZ3Fz}f$YXWr!Ei?63y%;4 zA9fy00DaXWErAbt5?Xx=Oe`Ats%cYoKQ}7P`h}r~l_uTz`2B$o{2baBX6x)WR+3nb zE{`@x#aUPbnz2n&LBTXb!N>LX^VJBZlAmHe%KC}AvVWuKwFlg#e-YdF>_z@Q57S@U zpe{V2$eVl_7?aWhAKn?sG6^?=AQS7cWEDZ^Cxr5uM^bNpyPK8$LW!f78$b z1((6P6WLf|w@Vr0R9C8vcNLbwRn%6mZo#0!L-Rb1*ZypY2{K{YrJ3=1LYm5t?gf32 zl=Fv}jePTT9i8ixI*_~|PmhbT5)-FO7yWWYq>^Sr^OKmkK$a5M@>2B5w)8dha@O00~gxml5JhhMx3x3 zPHM($JI1xrah5%BCbMkf?)BR$a8u~S1SpZ5eCelaRQq&WehqbfZ^0D zipKj$1AiObJXA&6pYFwKBbe>S95Vc^$1Px`cxq&n0;XbMHShygzuA4xeF%J{Bh;>Q zOYu^kC!85hfbbc;KP2A1OGo;acC^My$4-i!J#-tD0_8N-Ji|L7MZiL)tmQ_OBT=jH zNl*wK5+*k&LpfdVDK)=%94A#H`{U2Z!ZQVJX4Eqzz3VyQ60|R_PG;?#Z?8`e_doA7 zhIJ#2;Pr@|&bwyFk0Fbonk|O0q`UcFzW3mxp^|ss`Dk;7x;89SaDe$8Lyj&E*28*4 zAV>7|7Jpn@3E2;gku=Z`#X1Ie#`;H-$ghAghU1p2gMO_RA8%%;_wQDWruR3Jz|D9B zY?y8?{oa&;Ok?OZN#FN4+fuMYYj-ntKOtNut@hN62iKhD5JuU0LHJbaqV|Flpq7!iR%V>_Fo zNKHQA5J#X0UlAZ-Br?Pf8&oif!Ria3LXp9)jgGuNbl_{^+V%hG=(+Ol$hM_fnURnP zw}<`u9l_cSEwuJ@)#2XBIa;X9m-VaiGvUU|XtJZtpBf||(jU!xHY=&@eV(MN$jKWv z*-g*OPv33x#>e1m(|v6$&NW+}e)F`E@zZ>5{E+N3*7mpc{%?kT604il>F7Gwq2T^ ze|BT3C0vV{@_YlB{t!XO+Zfx5@`46VdqMv}<5}wZ9N{z)DP57!j180Y7!xoU94!x8 z^IBk<7m5J$J@$)VksFUE^elil*%WCdF=j$fPwxiZ;-ZyoI+?Me2|<;^h%%0#k?6Zz zT3VtS#K^gWWXVBmi}`)ElibjhC>SD{DT)#lcf37$=n!~_tcc!09Q8&z9R_D&EkjZd zB4i>ih*+FsQcC#U@OnEz&o&EqUH2WLo*=QtWvYB|qer8NVq>f((NYW4r|T1vHZ^7B zD3|6!&x$}QD0!=X(8p0`KTC^}l>r#D)rw`R9knE0zkbV0K9XMN@ydLeiv;_uZ9N8&;?S=P>a=aCZ+p-dq@UjTn&M~2 ztdi2P(eA3NIV5qKD)i3%aDD#_GKk|Hw*i3^AY6f~$s1rXuxhGs>-mktRh3_b+byG{ zYwg{&-!K1i@v{C0hnUKenwEy4gAglLOJPKIEBmtsB8llNiM03OFWY)rGc~J0Hng%X zaxW7%M*tS>6vi|<8bk!j28SvzK?5OCFQfQHkaGx?=-z2+);{_O&h3Ka3(Sv6)mo|; zxX?#t#X2TW9_v36J*E^53=!K;rf>#{&!fZjx}ge>a|+^2xAtGCS-PDX#JxOdSH6i| z$58mZ(dcpgFtnNT#s+;}s}|QrKn#lfgQfvX|HPZ%Wz3zLF2MfzN}^jzJ+ zuMwBl-LB4?H;wf!RZfA z9=iZ%lo;$dC+lVO9&x;T0*f|LPMLR2h#x_H&WyB8k=+@Yncpc2s+%HDqna`xZ&h#C zY(tnxArCu(V`Qiwk(g$G{P>LNPAC`cCZ-X3Ajw?aceQFdA368ccpk$aIPU{JY=|?( z*O3@0AsWNJNXMYubCRFm*)V$lRR|?D(jBb<9Y_(ECrMelX#P0mz_^l50Ld|>QmA+% zdAW1nj_Twb(*_-dfFxq4T=T0`1}CavF(RE7K;KVbI2FpSbq4=*+_tFx%59`OR;+ zWqRQ@1wo`mq=kgOJSM^CGo6`sTE6=}Sup;^Z6^8qv>SAE`M|>pkkB>^Hsk)R!Jx;w zK$1~TwtVV-`#B?Fgu!N3J$I(&b;ky=JJihF>wd<>b;bK};3u;HTQrs3LU_Arg8h{f z2l0)%pE|=rhm~%*pZVABPRmAPVUd{746CJz$(!UUuZ!k=yG`lkD0yXaq0uSMNWf3f z|6Nh>2qZTOn70x*FODgu9F22&_%0%)e14rpEQ!S~Lc5aWq#C5Zc=1DUMFx`<;=zj_ z?$D=GWpVy%d)m%B*yj<5OCFv6sqnnsnBvR$27`95h#PZKuaNrM{!sq&CB)S4B*9-{ zqjNa_kyrun(2gQ_bx_fJ1J6eQWR0A^IRV*yji~7sUnyeTTHE=p!ZPVJR5ZLmK^Zs| z1bXj*Wk9A(Hzlx=+r}zDNq?HEyyidguAt0QHVOT7N_<|;B-slBg5{1 zQzJ(iPWkoxZe5g`w6GsRW*%zKHNG4{l(Fe&puf`WquWSx5G7C0CDmCg32Gq5cuP+BXu(oSjotU^okP65Fr_2^*$cl z9L_>5LtmvkBI)9^g>9?#bxrx!-o}hHK*@NOVB-CceiIg)0ffX&hudWdEzbe?wQGxNS@|vZL+cP-pJneR@GkA_IDB<3K_9# zBX6iZtTV>&ULCGxt<@r<=7=|hGz&I_Ramm2!9c9{*YB1S#n}l4_sr>Ac>CJzgrQpPyp7IozCnz~>6DSD$W6)poXHH= zMEQ4kRb3+oC@+Mb zzkoSIF(~qMWu^|m$H9tf^kR|M-}CP84-5Hf&}ncok43-!k{sm3e2zOpjlBNenYsHs6Fwa5(E0@f^4_{(EuoR7&EW{kpQOvG zo7#2Ef`ki7To<^~9;ZBpQ4w{_<TSwM4YV+rCf;6*pl+pS48PqKQ%V`Xrhc(?L)TiF-tdpEwfbF?@Axct< zq3b7X;8aU>9-MakZ5T1(GG_=05kKtfXLn21(#_OlSb7v-f~_fNGv{Qr>D{)Eh*gxh zQ_(kYp1XBYIluS0H(uyRl4uL>JmGfTf>UHkOXoU!Le|{2-7g*q*B^=xnGt>=f~ai^a+=KQ$t@l;hGx9-guq9=ajazu2Ki(q2{4QWIZB@ zFS3`(Tv`-IPE~B-b&9qt+n?nogml zYfb5K#d{*X=Ml1zDh8dpo&R-S7gf8ptLU7#oG~c=99F9CO;)c^rf!D`-RCd9TD^9^ z9`>d(wP!QBN2zZJVWbt5zIyu-T__ST2)YKMkRDp=MOWw&mIF|YV?aOgVjd>Lhb2Qy z+#?O#r&`Xk7i^k&J$iOum;k$xwJz@tH@AmjRczz5QKvqHyr&(0`IttISot&Rx;AmB z5MibI9hnr3kB@S>dF#`Yr1Of39;4q^rVA>@ImXQAp_j@-%I0CEbfwI5ktsP`|6o@H zM-?JR}L*Hnw7kXa zreFHCu?*v!iG3(cJa7>M8DG-?*(K*%IoGt5v*PPGm6`o%me1Saca0j{8|}qq%|(Me z%F$-si>0#Hh#?c;du*f$)(cn3H#L#>-nsGfxUah8jk?E&8QMc^gztav0;(|v9#DbV z6MPe!ZVe$SGsxpk6Z|aNA}3i-L}5tX!;sq8qm6}<_e^1vU*zu5j(%@NKGKNav!p@3vi*3Q9ZG@GrAS~C|_vOKTK|@g^ayS>Mv4sG~=Rk?&&D|rMyA0t`{{L zXONfDc3O6{!h80sU*#J9tW|mf!P{ z5|_Ms=6;l%<@ilSsJF2bSi20g)uhD>W`Mu$oXf;?Hkt4Cv|E_Q9Du&lT^QaKtE+g*$Q?C07}3 zBUl+0f(d!I*yERpvVR8!uLHaO9o3sDBgY@^8=FTG%rak-0}BHos`j|}0`Q0R+>jmu zzwq3$l9IT?At}R$576?g_~|ef9|h>rlROBVOmZ5M(20~~4H!nLh!E+a6^1Pu>C&vN ztvP~8xh)6bcqIS3R4O|j@9OU?;UCMh2LASM8hDsnUasCLrVQ(JRzCV3MJMYFbsv63 zJci_w$0-MWi*)CdDo;v=Q%1Zk0w_^AX79RFkj&10FFm)Dlh64$f8?a05uISMd%aUN zCBhHs|L*c13z zQTHJ(m9;O-!(|lA{27Ufr$t~5%1c23;kn)4g1uhzWOF$o%ea|z^Q|Qqt=tvu zk)EuV%GyIbU)9rq3|rTz-bftN5z^9kh=ZKPt6Jq4nBl8ST6s-1@V>BID%KeI5&LHq;C#+WISccD@(tIq}>n-G=M*tW0UDMVqLK zYSd+T-H4F(cpa|@&od`M-mSgBfaZ+SIiylYx~;A{4XPe?YUC|^{j3FP_vsLzYT2LA z+tgCdX*fzLb_r!F#yFN*3LmZUuK!GnM8DuFADJM<%d<@&rJ(Vwp`;3Sz+1JE?&#@Zj_Lw+O@^X`_*_tyZ+SkL$lr0 zJl{!)(-_SmaV1pz2dkHJ#brfr^W*yw@(DpB!sk zxd3Qe$8)h{a6c!2@tfDp6+LaV)D}s7-CO48fyeM|5)g^1`vETN1{9(&?Es5-*a7x*Xh~A*vZ&4f6NlWQ}{`{ zqSybS%9Ez@$3M9id39NPCphk8!WlHX-ITZgBi!q?Yk zJ6>om)BB)*KxUVR%Ei7MhIDl9x?lmw@0Qdlox^Nsuk2;~Rcxb8vjI=T^$*YIH326? zk{4S_X+&Gu#iavBved>E3lW9Eq=<#QC6e_sTsGJ0NfI@>V9(;;70sE-t7oOGZDr%j zZ-!*>DNM{+LvOOJZu0+0gU{a}hrVRzT`$F$=*GaPqt9)O9Ap#gn=0@1EhKU`T$j=q zx$mit0TW_xNC!r^=tfxapwYs)+7~+ZLByg#KiFj`1+&s5`@<@knINlr)6) zYRaX>J9mRytlxk@f>la6oe>r4qtJYQ&O#c}B(dfFr*%3}?K2k5F)C}O55BI6ymDB1 z?|eajAT1{&F7rX>_Bz7rderTkYuIjG?1#N3*-n;3eW#k8+FPG{Lwv!^yRd=)-YzsA zoJ7u%1IX&I>IX^e@BLC>xkU55+mGU>>y(Zo?i&gZH3P79c%KMLsft!!&8<0>$GJ)_1c7=tC^hQu?s(# z=Uv^Cr_~c8A23ja_2-5Ai&5)+dpA%sOl|rXf}qN02$3r4gw}vR zHZpSYEIz1i@wMDSV%U*kJ8Yx}rSN2a~dzEd0)lHxOTHYL(A_Dp%V`ub#c`$z@g;W#j_$*w{! zh8ASjwf4p*8uwV`ma*BT8r|Z1>z@;7g?**}IY8M*pVou}?EQT%!x~-z5$&NxJGVWM zn|3IZcv^7ayIsce<;{Z1V9OPhU5>9iaeB?H^818dua7(xw-bBc)42+JjVk>&*DjUU z8K>%mIHfF@Fne1Vtt{1M^9NxVzQ{Tcy9b^61i6W?f`K^=|~m+*T#sRWEaux=|3EZBD#e zc(>i(HG2~GQFV(W)_iE}!ckc~okm+6JiDrkp2siRg4=cqJHh(ge_rkirqowhY0Hi) zXue~ro8C^oPtbwPcf**;MCr?c@7JIAJr=LEp_fF@ns!X&Z#QDSyV{TRBlZ2^gQ`;r zYovBl!nzsnRvKPc}N!F!`zeAGp0*8>J-TtvCU49Ogl z({8Uaeg3(=&Hu;st=y%-Fx@R6sVLpVS7qHCo+0=k?IhUjc=;jwRT95vLa62)srv$? z?YvAeA^LEW54nvQrumPSbw?&RpK-Pfe&CS07FDUUky08dmnZkcZ14-M1$nXM@{&N)_?UV!8BwfCLJ>l@(?#%;}wZ_y$+ z>`#osu{{c8BlwQXjsNNj?ek5Fr=uFk8vmdOs9B=?9h(fOl_Q>V;c znPH-9#8HI3NWkdgw+>q)tm|B!CEBVTXGtU8$^i1^_G_Gx>;MM@;+h~iJAZ?bd{r;q zrCnd(G}=+LJDdhSQ;q(jRsuo=w5IF+j)l)F6*{TVoL;aMl|HG9Z86v3d-eT$=^4)K(TfvT3l&wMrEU zT_Gdv-4cc6jd5<}{CIk5ciQJmx_U%poGM%MQDasbG4C$S(1kf2`Ztd^<{tnWv}b6} zXkGhhp8JQMpr%cnVwTGdF8$2@a;~xQnSsXj-R1J9L;%w~^R}G0gW`jcyMy>v4LI^M@0u~&J$vd(Gb(M|$afmrORGB%HxU=No z<24+R+FY)>cr(INA@?p5A{^VPi2lWT{p$vHc;{n>>2vt@n6_A(xa_`~7x${A=SHCD zpLaV?;FCQMW0vBMQq-xx*BGE|j`PrE{E5C{>|7kDHX)WV1sW6lzAnD^F6p#!_paGI z&4>Q}U(Fx)>j%ptnPxH7AhD3{Mv?OJ$Tk1YZ};SB4G7yv<(T15AfHSR1#~FIw9)CO z1y)tW>C+o<$aA`N4F>Rn-TI6S4lUZ~a$d}8H`FVx^o(w)Jm(1Gt!sRI8Ib9se8A^7 zkUMF}A@-|Itqc9trtv3??kYweeajKYUJgn-rIYCNV|Pq<$f)V%w~PDEC*O~$OxFM5 zl>Rr2J^<^Il$qwgp#>F9S7ibZ6AdjoJcdW@ZME z`~BicL+?3;&=}7YnV7d-u%^xC=fOJsKsly? zszYH=Ru)~M=^NcTB*{RGZC-$cR)OUoKr|;D^R<#uF?b~u+a$*2Wav}jNN%Xg7KCZ6 zH?`=o%j=4jr|Q|+L1ZkHdNDti*@Btqi4GnFlaopc(B%}33+jp{#DxiWb#SLha>UW4 zUp*_bXD$qm&p&5U+A7q%Z)RX&-*&!OeB1lso+VkMlQH9^|Fq*(9$5KW_Mk3TqCt(OUb~}z z2aUM0eUy@xQCl#GZ0ScQ*Z)mfA)mC~bT}9bj(O?Rxq~JK+#r(uYEfR`%W_{1VbRJt z|K)qDw>=Txk&KRxelVeDXE84yewVJZvvaeVbTdOe32nb7vAN}EcGwI_p85n3AUOeV z;q}47hiBjYFyr=0K2J4m#*%85KC!1WZ#H!3<8P`5ZBBgFI;o{jGSu^bQwrbzq7-sC z^zr6lc~$5x?_uI2bF>fPdNEAK6r}h9b@jzb95}|PBMhs<7Vvxbmmx0t76aJkl@?mc zvGd)6^bpcvQ*?$vH?ar?p8nYhm9dX9=_Q#=%4(oNi%NHiFq+>oJ+9&}{D}Q3LBRYX zNIc1(ZF}7qUwF$gAd^+Ewx)l}%)RI}vtAyyXXUl=F?N5QV@REN+Z)fb>;JP3GoBE# zPX=prJEU?e_2anVag{+EBLkFKnZKT|fkzPsl4tT-_E9nLa2l65>q-#h9UuObJNMyEJChU-yp^7~Q(-e);pj zFmyiWeu83G<&)tcTJAq}H8q0ahQoGt?;bQ0eBt0f_4yTWu!=R4IpLa&73-8~J3#V-YEQmr z?1Xi4klA&ZDC@b-ImnZ0Pzc%@iMH3aOFmex-2=uY?gg*I_XuK|#%&Rq^!=i;aTHlAj!e+yuNIJ!1 zmvWmoKW2XP^Sfgk7E2GPBA>X~ls8Q-Empwif*oWGeQKgLSk6U$|PfByrn{naq5Zzhov%-0vkqm|892TJZBRn9phgx%+2ljR$M5q2AC zF~Ru~m8|bw;NvQ6?9+N$I0h>#jvI1GkmTUt77H)){*4K1h4j7lBHtcM;q8EnPwn z2;q`o*Vj${J}D{b^F6M*x$NuLZB!R2hoO>-;?$zf1$zv)1d?acqe-)bYcW!n76fa< ziF1<>wdW)DG$RSKb}a@&54YxT=qV2Jd1lCZ-iq}=oC&j$klrFG%E7dNR+HB{RQwmN zt!?{;BBcK}N0CO?(U$bUV6mcTmSUr>=-{~g;Qx!H2hTb)e0qSIzc8#0N@4G*B1jd1D zW{X!=();2_gAT_m*^8%!sv`AU1Xk9-;IZB5&OL}5WgcH1M>zD4N#bDp%=}HiN zb~|h*JV|Ss<(upu(rmPIdokHkl=SZCD9d5qWpeXu2h<+XPBL%#4En}wDYoEi^K|#T ziI6wQyRWuBg7U3{h@hfAlsKK!i~uIAh5BmR(Cqg|Y&Q9-Y7)nB>9QSsWbmxJlPi%@ zFCt+%9S!a0t?rMZTH4x{=f0P-3nQMyTYj1ZD3r?(lxUV+qpNhU14R}2&p(yA5}GrA ztS;RAUpiF@9od@_HWUIE0xbW|y756EZ*p{;cbH*qhFi*+sPCwlLeR}H_B9$6 zZcbg%2l8n19i)9Q72a+mMbPaZNnmo?-}OW2a1WW_yhtn%eRD`J?M{{?rf}_?qY@h% z+jjwA)YkEPN3CgS$fNVZJr`RT@?g=Q7&{0uOnX)?EGGbGW2LM>8|JAdSi(^svZO}W$2h3fJ0m7Uaw;)i9v z*Gen)omlhcZ526cKN)5epp}s~MkV%V$Xx+~krekI)e&x%Q;ROsW*S||Lhz!cj#{xyX_@0&jMlq>lB=MH}1pX@7MY~l0Mh;mm8 z2Um027;@7oH6L-@)2AJE|G$M{vdeAH%ip9v)h>t*^B?4&)(z`|2l2y=21U0^oTaj3 zic1?GrwPAZunDAvQxWnnrufMQuLU3>FJULu2e5_SoOMfW2?*sdq;G~o45z9ez*M&x&yf}MP4p` z^?QyHV38>NBafJ0ReWF0H5C>A?z0vU5;zmROH-EkrDj(++;P84lY{A^n`R*tzv1K|1f#KQH+` zxG;pN)&4h{Vad=V7yx|G-m3lG-iiX*$@^!dZ}2p}fu6Fuzup~`$i_xdQZHIe;o~vQ zHQ=Y!M)p&|Gc_jsklrH~NzAOQ&ew;q&L(t{Jix@`q{iou!(;Z6w)cCD9*w+534geA z&yN;d%gyQzm@;GsnKU^SE6M(temFAsh_{oX{Eu=TLj0=m=qb<`#}=d){qOgm?XNue zvgUWzwuMba94ZJ=;!VS+4{5o*{w2DLpB)8U|0+dAMOv(Wzge1o&{4rs(cq84LAavT zmQfB+K;=`@(9qbTJp={Qav`Os=j$T$U9IX{NR0FTmiI3g=s<%ltIf zOJ({K+Qmu&;p49VF>E)6+!~>h*0YaxV1jGC!#0~|gVOa8bPViK$oif?4bjA_qYLxC z&h*4R#&nUfR+e4DZzIiweC9u&zXjN0_I6VRNU zsfB1}Om5S6sFtqEH(THOip4=yg~lcARdaob7DQ+KtAD=y3&Emu7yhRW|5{-6tcJRA zl-g8VM_L&1s(>PWytClGFqbSL14g8oeCt|6=#8Gy-H{Qla8mVwxHNCu{qOO z21<@W!Xg7MLSGau3kdg{S^-{vThidL9&kw&)1&9zw0=i}&`>!I@NH@-UcF<;T&$b- z8B)9t@FHn8?Q5eC{0>pDuP^bL;WQ9ZU2nKa$~`%X4!Xzi?I2!MmiXx^bE{@d8Tfts z7MQ7#=0TEXIM4-gT_El|72QW}5f9@`T)rmXap)#i6ORrJLi*kVqF7Tx=^^X1R4607 zE(HEGc{_f<-&uIpF~YkxtjLlUmzQ7mX2ycWWz*;q2%D3RDpbbDwaxAOfmejHZyVx} zQ1H;8J9TWS*x1-(@j(ZLJpL@iEEJfgKa~PGnNy8L*@=79xs91F?kZ!-=-n0DdPHge z8VklN+dSqcU^Jb8fkG0<`jVbJFRd*;_>j$7YOMvBaGyhKWi`T$HYRC2ccLRGHj`-B zoiDJAaDVd1ai->z$wO(Gv(l}y>9FRfVf}?J5+D8xUHET`oWU)j%i+HnauB)<<26=)lQY6bA%alw42bV0$zq^T z$e}3922^23K)t5AgV5;iKyQY`50#SzZBnr1A;vqkv$hsK>lZ{f5i14$_c3fKmDgmG zS3lq0T;^NYTfqj}j%0F>5BP;BODV>1%%gzwxOX^9_Njo$AVYPCjnl65+*|2)jRa6h z{T0x#@&4(-K`vM@bjw*1E|hc;vO<`!=+FOZ;BGR%S?EbGp*&UAAa{N36##1N@Jt$j zKHjj0N>{_Kl}S4Gfwbu~ga9n#xoCb6O~UAP(50!j;`&p~S^OjivuDg(u!bYHY{m|W zqhkaV0fR^rR~7{4E02jzs<6x?ua7isguDZPclHVS;jiriQ1mIVaD04x=Jtq36R%>Z z1DbHhoU+d>9GtFd>n?aLYMV}9CpD%J*&xLr3zgnZ>PKs*H^_tNT~X2^^%Gto_L$#l zf>Q!NIu6SelCRz@axpA1b@JZNHXI)1(Zv_-_4b^D z;vM$~k8rMs+=I7qy%j0_ZYnoxm3Pom?_&m+ya*>Pl~(o~ZyqK?yiZ^_QfmMhAb%Dp z+kWezpwHw6*|OvJw;Alx*8{n4xFUp-)|QovYkWn8*2JvpeYw8}{bO2!Kzb@AYU>nMIjHu~ z!GYr^)NE)6MYZc-s)!>fJ(eC3ov;nyYE$c9)a`&UuOiDqH&UG=O3!U8k|uig(R<;Xl53 z7Dx5=`L4z5I2WAn@Dav9SQF)&r8C2n(f{r-@A^DI_Mlfl6Ic)W4wK>EyhgYEc{ za9}1D2F=A^GTf4bYM4*#dM!3p6y}$tjU_t~(&=8xxc?)a=mF_Ot&Xh6 zR5et416CaNF&);6r5U~B=6Eo4puLNd995zg&zxK>K&+d={2l}(|1Os?DiY{lUG=+7 zTorniBG6OzB;&ndTnCrR374Wwtn&AA-7coh;BP66AIZpyi{lqE#W+AyF}gKlJ}LjN zI@6Y~>quq(p2<5o%6)VA;3CTjxJ|Ij7Yvpm9JKa)?d8Ivv6u$e0)tP!9(mE ztp0N0R8vl=*&%-Ac4xUqx>CH+_?k-{qfB&inAO?(^q)4pP+WrO;nidUTA4* zG60UUP)R9?hEuYHXE_x7(%!{^y)_zTpvZC3{AdSXU{mA=ne;%-eTF~xgHlZ!FrU%i2ALK9E&oLlq#;;#do|yXU#6( zuG}@rRO9-wCELRHV~HJ!D0=gZ})cKyQk z)+}`{F|gR@;?WO%^&;}R+!pUr`&81vSa>X36{elne-q~$BxzumzT%El-4^B}85IMb zaG>b@>FXcjW18B_Vo9yXc8vtJ5l+a0MZK?(58%GP+Uv`48Az1xI$V^O2>Zg(ao&wf z>3mUni$VoWAl$>DGiVmP%M7-~Yl;)HLYL693G_9SD*G@mrEF0HC;NB}rkbP5uiP?J z%FOmB_Hn-Ry9hiF1Q?(zFQ{*PfzN2Vq}zoqqkhon^RiR>$!xQ* z0j7ht(DwZ>JKq9wmxw-kQOdu59IJa*oCu}2@r!hxB-YR3VGzkM*|3mJ$_-SaE)ac*07-36lhA?XWwJ-e=v<+R^xpn!KlkTIKYh z7b>rfE@hErSxd06d~apHKh}Meq{ft8V4c!3lFB;k!k=9X@)IV)RMD;t#>84*mvZOqhW_h;x%@|6?ntk`pN zDqwk>g^40Ac>E5E0dY*^hla7&%}2RB-VOBqo4u9v&&U{Cq{s1|@+)oSG&XZj+fJNW z^pD+^pxt+0Hwb`hBDSQa?r==D@%z zziAKT`ZbA<7dXzF2RTv}$)^y|+VcC3H#JfNgd)dELm}drgW#AaO@w0fQ4)0JUMdkw zmZxnCNvbZ88M$W(x8>>fF(tvVI%Z^s0*yYJ{!|hhk&v1}DzPFWA zpC$Qh;<5WRN?}29T;EL46Ae8EGZzA=a0#S6tV<*mF{=-7F2FN=`JIo zPu{bScbtj-BjUA|MzTl!sbGJId<$0umx)K`r4TB$gE-0}BN=?2D?cjpJ}1CSY%wT$ zn3^$z?m^VVNVrwJAl=<262V8?P)xf*UC{2{&c zy^*?j*LoX^){7uWI);M*ZQ@H7MrFzfm8PK!HnOwwG^GO4469YrNb^e=#hAo}kV173 z8sauWxVd!yz64nZG~1G6NL3wh+IEralE3v*sB^hINiQKte_-&VxHK;NVnBYk^5PL& z>8JS0t|0!FiY`!ub9r=LKO52IKv%kFHkj?fW>NHbxhSV8>k<1F;eY^h)(gv6@561f zzEsA&$xfF8u=3Z`bZubOmE9fQKbI#q`1q^J_mK(EBvC^DZTRd4wa4u1TjWL>-?K0& zZp6c7^N-}UMp&fufy3?%YZ5#N34V9k#JCm4s*ObaR#!jO^~%$3mKivIjX231AVW95 zF9=_VaoXtZR-W;y%|dY`X%lBDHfM<3q$M>1JM}1zo5HcnMKvF>($q8twT0`tk(6$* zODPlYMN(@SN5kK%OieD;Q&44CamrLY)!QJr26a4$50cRCq1#t0DE zYS9#~ugg}Iz@2d3i>LF_UU+>WT)|)6Sbll;DVY;qBrX|57BxK_(X-@8PljVhL^(bS9r3Z8a z?aQ<5_qp2>;-Yyn*62vW?pCmNz|m>$taX`n&)ST$&Dm|QW^7+fYo8m@ELH4JRW2OB zgESqUD?y1S`1Y!Jf7S0fJcFF7Dd?>QW>dk3Fc|jo>#y{?pDhF(S2W{}25mT&HkLi_ zkwQGW+Nec@gj-_q`w*~l=bI-e6gM(2jK=-6uqRoh5m*7MU`^DzI~ka=su@5E%QYP$z8!g2rlW5Wly!6=yv?Gx4Pvmfdcr- zuEGUzf;++Y1MF-k;T~rf1VlOq#o&n_N|O2#fLrl^)PK6N^PC7rq|KWytDHX3zWqew zM(kG;1MuxN0wIazQ>j@6COw()U8DIGej^Zn;W%xUxaq;|aAzAa!KrWGir!)oW)qxE zE|$RW7$Rg1b~H&P3N3hPJ_^5^r8J{xxWS-vZQQEFYKt}6Y4SSi$3^^EdE_T=qt8wu z4UH^EA%8PrG2me$o?-G)g18e+HE!c0mRr3>g_!MZ@zYUI(2~dJSk3TUC=!F;-C<)A z@Bji^JNCPG%*lo=!slGiar{eeS6aPOWEWj36lj*0(Ufdo=OY> z`{N@sm{7f>$gb)i0`2R+-BI;GR$m1SMxTg^9|BS{|I_o*U5dO4=>dmtvkyT6JA!9Y z7ka;<0}Ptbf_a2KHc_k~QXnry%R{{d=E=kE%NPq0_ALOhH^+P^y>hQ{yAOzHT;vAM z(ONq9Qz(qThpK#+6XCRxnysm+;rBYTPZo2veIAKW{P4wpe|7%Q1mYbYBJBURnk;#2 zJ75wl52=@H9cy-c!dX45zYky^9!l$SKkKIbuF78K4?V9w*d$=P3{Ws-#42IB2(+rR z2Kv=Ug740yCcYn78hjjbDLK%8F|yolb{cf@8R1O={*@a9jWPd%HdcfpA+L??v;HeP zx|rQpU-*3Ox8x`cs-)KipDHZXCl;W7(du*1**m=(jONyhg^GqxY5|Tl=)om3%FEEbaJ=d$Nml4@jKCvW6 z80R_uT}LsNlRzE-Z^T5T(T!g}pfFW_6TU5B-W&5CN*S7TO9 zEy#`o1lCW%^RBk2-dL7?zmj&1qr#z<_X`Dgkt`QKPg$?q>gZ<_ck z8Cfiar)tQ68lTN9>1ZzL=7~7Pp@Ab^9&EO9w3;kT zDo>xxXt8yEgPW3V*(ob(zKhutlvhE zM~$6tcmXzKn1TPtR;`RUeJEJO58UQOSH7U2#KrPrV|lh)mTOKmSG-lKgK&ev-J7zzT4o(Jwi$cCqb?aI?r? z@Y5aAWUrA1iuIIWs|qink*Z4nl<^lf{=8eO$wEF+*06qW+UxM!(|X4#qu@;k=_|om z>yM)tgpnV@4SbcYpFDfUhS2pzKl6GM(UTJ!9_`KRJ-puql)eXLh*6U|P)6jyt|;$S z;6EC5terB(`9@F*jNQ|~%Q5!l-5RNmJ{@pnl&Ow)#64C{4lG}7+%Ap1K3y=J4Su@; zcCI8yT`dvZ3zja(&X%!l+J7_+{@GCv+6355*=ptytGsmSa39{<4S`jSZ=>0BLbM*8as;nXu^=l1{=LEjY3C%?oOb? zJo)GLdDx35bUQuHOYLf~SvKp}Um`zu!Y!DxAnq*m+L&yRLE3?GibhCI{ zvS58%A9AuwHQ2YTa+-{c4rZTlJ^WU~U)S&=G-YdCB(I6@Jk6pY_L)-3+M0z_fAHGF zi^@z&{fyK%EPp<|FaM2%0)5eGZrAG6Vi*5?1px(D8yzkT=?TQAf7^o&!ye=+AJ^9F z1?Gqp8uJKd;OZe!q5~_*LGDLnK@S>TcfG^P^WBXpgwBl4tXy_9DE8~8vy!r}0N7Zh za{tZFl(T4Nb~Zg}7-CRuJ469}340V5#Q_AAg#ACh-a0A@wp$;j8F~;9kQzFryHh|y zQbD>@x&)5(SPh%3Jn{47-38;t0GtcF^-E}|;Rt;Fc!cjx$>PG-#Y-TZ3# zU16CGZslj0J>k`hqEj8N)W5gix7c|r-iN$sx4rFXy!k*;BElIevlwY5U(9P#xtZQX zcbxrC%`gl;yFWEU=N7C+l*YxWweie`N#JUnOQ`pPQJ3YdES7TkdCU);Ijux4dn7RW z=l!ZY`;f1BpUNfPR?k&E9oJg@<#-w#tE{{trH{!()i2~L;KJr`WHG(rj4GnPlvowz z{73Hfe*l;RGp`z&8|DUNy}*11N%UHFdJmhCd!ipGfjfc{I!^4)pa5?>_KRSK>My`L zNbdooG26$L0!We>FtX9EA0WeQgS128G{6k({z&FTXQg&SmIHXxKY%W^j0CrmOMTbc z7P@t{ZT5TEHUT-1Cd)#mbu|HXFPz@S#->T5QJr4V6q&PFENe^LV?agRZC4*2 zPjh^vf%oZ7*-1FPLfEtUVG-)?iv(rbGQZ!zx~B*H$LRwuMsfv-wt`=-A#v~b2Mtp& z(hs(U8}Dd5C+KH~_?ubHufg7X_?^U(D)SD!Pe-Q6gPv&$11kG+*ZA65}?N{12_w@4s*zwlvFE zM7FKil|SF4{VDaXDgRDSz#Z@ot$5F3iYX$yC#@Yjm00$tTmRkq(khJa{UBK>sTYR& z9p5rR%Q~bA^C?O3ttr5!S_P~u%@R}G_Zp%TFeXYF|7KhvE&`7>xlr49B+r4YO7iAx zdCS)ljndc*`2EB^9RCc;k9QLNfTNThDesa23Tl(*G7NXrLL#~lj1l?{83o{vOfcCS zzQcL_i2W?R9#6{BLSAmDMZGZ*_S2sb*(o82hiXF1i$9)v#?0Sx@@6^8VAkjG<9J z6T{|bpwuM#d~IlVnX(ac&Zyhfk1e#SGu$96<8*Z3?TJo{l-n^WAG4(gn~eDaxbRdy(3D8r?+RC%CgkT%{+DB@ zPe9j^X321cvc5zCi0Tkz_dIb4bZdq}3ot7N!j9rKx#M67{0X6=h6aO+QRwnD)sU6Wg1oO|gu#U<{Nkg^wq2g)rL!b7)yo==n_4BwNQlDlU z4T^E@*#JPwEdACo+ySi4N_Dvw(a@8uMpzBgWv-7C*N)EDbGLrwvo3Sk>aMzt>1(-PB9- z`d^_JvXdE(k2>qfKpuz!AoB_XNavsG8*E);a(s%E4gYYk1icpIc_+L=Og7gM$a_T9>b2S%0?0IKI zFgiZ|?j4@xYv4HC2dK;C44EIt9;=o36gY_vB5Z!`PK%IHX%p^=t|-hS-HJDG&U3y;8Uw7fRIKDzv43PqvqVhbbNQ>O7I9G%phG`#OV2@3(}RI^VAsV?kCqK% z*Mjz3orcr{WJHq7Sr+*xa=!K8i`LZE8I0KJ7Y{)_S@U7#)Bl1&tKqsKD-F)!50xI-biz?m_iovV5te#wuxj*nFMf5h$@F%eo&czQsoN9%Y=CS{+c% zR1AG~KmOojZoZj#Evb$nU-c_dUBSBBz#nUqyI}#Xf@Bfo*S!X}wA`$&N_YtV32xjf z1qMrDN#$tFbJ3poSXq-E`!F z0^EVmT;eSPz?hBNdm~@Gk~P=YEn~0?f}AfbwX7afS|0%7H2Y;>2G_pu}Zv zqpz=Td#7%zRQpT2+FA&$n+#MdD!j?_ZS314`(!-|bg!4T~!Z9XBw|M4KW;gvXtgKZrIP`5YRx2=P18)O6BR0Mu#{@oCvm`O7sUy|GOFt|9vf? zYU0EU?Sa=6?Csh|K!fwtBjH@Ff?7n{n2_1sz7UP_KMUfy4odVfTi#Y_{57{EaOH2h zA?asekVt>|(DGG;M9-DKyalBHTN~n(3Lzvcyyea2*30}l$N=6}SM*AWpJfUXDvC8; zbC&Vse5GkE-G!O7S+>~`N=HZJnoe(F%0huVZGTeJP^ZXlG2iLu=m=R>2#V<>d8(`M zQGY!w49leM<9sjXb>?3^)V~vuRPw)KG>6d4TTTlGeP7bxZyN9MM&TmDCJXE?Yry0wA zNVNT6%d#r2>G9{7x~Ufc4c*|s*=GFO&VBK;veKR*)lP4b?cf!@;g7^Km@66ouDX#N~@j@4~D{k1M zS^7tx4dS*P5$Ey-W{~Ol_fqlpGwwLG8Rn@!0cNit7~cLbY)8!(#^X!3g!&TlxA%Q z1&=e{j&3JuCz<1K3QYs(!G04@%+3~PrK3S2>a6V)k}oS$TodO(tJ0Gz*)uXc<=b%D zCOJ5MxF+fY``}%*VzAVkczCAeOg9zeu{2R%ewq;LtMN=W`xPs9xx3MB2r4?(vl(Ds zF)V6t#71gPXRFfBSM_!(RV}df%c;}+HPOubwnP7&@7;PZfrY`KNqdWXg5nF6-8R`C zb*jUJgQm9=4_agxe{Ou;-QHPNt~xL45D)y}Xfvfu?WaWTVLoe0{jEM}=m23jSIEeH zdv3*U{uOJ^#9L>sxS#VCf6Jw#%PJzh#c=xnkrJKGh6-Iuj}U{2&(T{cx^DRpXjsZz#B>g4OsAnldCn;z7gM>B?;hpK(@QYe+<4LnlUlvE ztngoxHMM^aY1FyS-(~I&1ItX-dl9Teumuw+vIvqCdD_%0;aq%e$1X`Rflf$t=K$UC z$~_|K0k}uW&6q5qUa1cUFvq5#Q1@$-;_6nN&4yP zo6pnrFELu0w&1F*ud5r|O&5(BtV6}ZDQj4$1jg=~kJ9G(6A{4NM1R*{f1 z2*XST*vv z5RR;hzffrRtk$Fe{U$P;d$T%v==aSpnd{}Zprq3AKU>6(aprV>m%1wDFu^Oejvrf7 zUAOB(hghcSTAli3REoY?Abu;G;19T%1$36d|6OwZ??=989j0qsqqR29>N+6A@24Hp zEa~^!x7Pg24a;PQafFoCig!be&DjEJ0JJ{*TfDJX*hB{n!X`98|NB@76TLsmhqH#S z#vb%$!JVO(DJ6dW-ho7PVR!e9J&RjP{9H}s+nZ`1T{bKU@etDH)R4EVv6w2_IK@2E z@C)f#%iK?CG#NfjY`*<;i?sfYz(*eA{pjVV2I&@`m&I>#PAMBNxKq=@ihhuTzqyXK zo^#Ig?<_F|be0a@gJ+cefM%zbD9!L>j@|ku*Kow7_koFzL=SwZ@8B|QmEL#oqm{v9 zNbdCA!Lg3@mwCQEG9rNI>ILY3LUaC?0PA9(v`*Bp2KuG3s>Lv>KC_!caxyiFloqM| z?tL1rfH#0tphO_+z2yR;pMlLguv$$*iZ*8&-AA^+wFaTD0&R`mj~9(Ozm|UyFG@gC z7C#X%mJ2YX5eQGn0wg@@xeUzB5FnMh31KHga@aIX0oU7RMyug3(IY->SAJTFV2o9{ z%&zP9i#`kU4>R6}2R6HXBuCYmm>-R~_!f4zZbE*$86ZZz_oy71z(Fdz0%}h6p~i;j zs`sGh7m~{H1wy>11M3aTu_J!Ri%6obxK}tsX%&xqc8K|^u%KY=RP6M#6A(KWsr=Drj|rZoKmS*H4s&mN^22lEpU319`IzWO-6<@r1LGs|EmE}y zLup)Z0EUwpU=&6~Y9WcZj6Q2)XBiK9CFL^NVOANg7;%J8V-C9sBsxH+I2_TqQ>Eh{`{#^>a16m*s4<8ST4^chvX8`R`O^@GwNe zOf>Xv?TPxz-#dm`V4q~?T7E4gdgE8d>GSyN^&e;Tg8p3t%4uUcaaN0JX)>3N(64=%sCeC_bFFzVq4{{k33msq)mzV6u_+rh^~Q-K2Jm9$p3>wtG)Y`%D-=yi_5VDeoe3 zUX&gVTYay7+3wdkmM=BkP4+%HME&&%!;BUGJ8z>=bz}bKOotEdYo4n8W*ku&nf@bT zgJWM@q`sEaMAIQQ>A(8eOh1k1{u&M)?;P&Do_TnQRB-yyC%+?Fmi&7IW!YH$(m%Bj z7{AU%qAE;&J71f1d#+?^ia)*1{dDRX&v)i%^EMTL8Ed{QLrPcN1UrTq^UvUD>JlN# zeISHl)NvU+AnI{)qd`6ZhMZyGY5pmHu`~Ew65R10a2Z_$EHbasry$t3G9O3OB2oBq zFcHfjw?#VjBQ@cp$w=%aV-QY^oYnh+ikia{@LK)YuoDJ=FzYT5F$W0F0#KM?KH&Ff z^)z0&i<9k=SmuUoEomF%=tSHRx@t%Vi*Mb8*Dd&V2}+K!7Rt>WfjzZ#ED^z#b}M2# zgT~Cnq#~H^RGS<@9!P~V$cuGS@;O2fIX{T0k$&UsGLmmRB|8NMe|poAc@f^a$}a)^ zYNaqy^EDXrY&84^NX?Ja5xRztTG&hNZeH_kFZr|MV+598W%f0D9gW6db&Je+T9%H0 zIVDf3nZMV&qo^Zz)A#ygp+Tu6n{LwEx9xn0`-dMj&VBsG(iL?bp}jE;x_3&TZ9_Vt zZ$*;G&u*FxR!})NYv*{aQSJt?0qt1;Iv5Am#2TK3Ka21XB~ROW^#Mefm3h1|YxlWz z>bl39(y%%Ma4Bz1{yFOY2`$#xOQ1ci9n6aI%wf36e^9oJ)VIb-UzBRH-lMcI*|5KG z0IJz3fA^1u(iAeo_kNEV{l|@}KlNXwr_zhEzgUKqf;4vexd$G=jM9f;9Zxiqi8$2) zjw~enE$;aL9Q;Ttt%x8xFT{z%p<8o%g|>5Myoxx9qQmbHq<5rouGPOy@86j4-p_Tm z!|!}7$ZGkb;@Ra=L*0_2szLH@O-s-H<;mm$|L>Nk+~33H=DK0=hRuR+gq`dVb#)#A z@LIC*hs}2%d0b9C4j<(8U-WW3uxR9*{^DY0;B?~l zE_&dXirmYr*M`fJ&uuTJKEI*1#?J}&!deTFMk9(cy1lsXb~U$@{xnA$a1n2-htvX; z?OkAqt$)uapnmcM3hdc9HuTinS)vQzs?dK#4(C*V0e|;u)fSENaKZ30Nmo;w7>$q$ z1WtU2v;*QaE~!|g64mKqM>7XdENaCokSa_76Qq`+Lu}{(C4)m4MUhc7l^zPt37aL@ zTVtEA(mp`6I{`|YSVxwR;Yd#h)G+5^9$Qw|@?uDJYh9`*LpjX3w4&mN$Bgly)GuQs z2(QPB0nvFj59Xfw=0RF;&t~v8Z!v^G;a~EG(w4gCwD}2;Q2~MjPM>y*D>)0>9K(;1 zve@F1xD68cy$Jc9%tH!+I#o53(q8~MV+WjusyYH7Q(2UzPSgM zr9C4*$tQAgN8LIA6TN3*S!N)RprDve;7N2&;<3~^$H8$2V29+6fki3_5^10^pm2AOge*R}v3HVOB+hFeI8l>LT{)5*Y*V^fakq&@1yZVr` zf`*2NCbVlJqc+CY$BZv3kQivW`9l30>GQi~sUOOt@LTRO$E?9dm_Q$C>+5gV~tcgz@R7so_jOV2CxH^ujX%Gp z-q|bGaCSL$naGq0L>KhA`!Q`BjBbapSkKo4pw>UUB^q5?M=fm3?+@bCWC8*p2 zqWyIK=2V1Cf8h!Z4ETnx8=l6qf@vSo=~vBGR4fY|c~oivs1lOKa|rH0lOtJC^Yero zFvCp(@@yVmvEB$h=%)pKt8wD2CuurtT;WKU3-*rU0gK;4>#vQp<04A zyaGBP6Au*~BQ_-^C$~3*7(;_$&~q>osWn-f2!>)E;A|qb_hg1?DCmp03?$S>&+wd> zey+>>Hso&FU@$3>oQf<(Q>H|x?*cVrvJ>M&`3b9^++i{oiv}Up)|Wu`H|99i-=q-` zx)J8N5NNqsMqbIvH+leMIx6+Hbbj@#DMo#@wj+X-z))3F>O@Z}U_*I|-|ymig~+ZP zT)=6j;o&dTh^M6@w!{}Mw2KVr>bjwU7*Fry`9pM}DGPn_($w8veU0s+)r6uK6UUDW z-{8X^tv%~@jd)W^WrdKY4lXy%DoKj5kM1-9kJFS)v&4r0o`_^9H|nN}LZfJVmC3`0 zEa26CCEnEvrgD|n4_Z)m7pRrA61Qm{Ih7KvDkFFC{}=>XfyVcbu7W!*>iJI%vZLvK zn;Kketl#|F(&d%$=LGr}#1cn*pB~=Nuq!P5mq+Si$jVF*^&{qOxvRxPYTc52HZK%* z)g{hE?83ecG5pOKR||_0U!g?4+0kp z5^?J+FOCzdAOaXDuxpV2rpIgP%%8T0Fpx|CTrRMm zJ*XgWMSM}4A!c>>363U{XiMM;jvvs?t)cW4Yd*wkr%>dBh(R4;fgs~~A=KV@q68va z*?f54BO!l)Q8{*0FycA)=VKl4TQ1{A<-Sva-Ak>a!#lWrea}ow^-PQ9y@`Xm(FpKptAJ;X9YP^EI{=x;g+<)jJ#(KG~ z@wSSvjwCkZEc>Sq8n>PXKV48X?p~T{*}ce2yQ0!EQy>Z~DzChU$zODG!q4f|@X~e) zQ1P_OMK&deJwoi0Mk+C0h}UHhq}y9`$rXR0XN~G=_g z51}qrPk-H|eLFby=KAuT0`_IPcUasFQrOJxnk=V>j*f+|WdZ4?uS5k)NC?7Fc!0Hj zjpjSj6t%esR;UZ?Mi!qW5)1lhLQgXA427Oh{thSlSGlle)(1J|1*8`pIVZ%ce>7RK z-%#5^#0JxN(X}xS=zM<$buFeSWaD21UPCxHS^z$CZ#(8EHhnK3qrTlNMJHrLEV8xp zLF@ughf5&*P`?Exv|~I+%K(TIk9s1;irTaWwe@nOC|uH!s)XwB2?r+{H)k|SKXB+T zTL@u~r(mVR8_ss^qO=tY+{X0_yNp@YkfMCBO&+!GTd009SP78u{P{HTSErLNU2!==RaK7jgjYeymB}v8fh}dUTd&*qI7ZYN=Ekuf%!d<8PZPYfUR7)^L z<)k66`#EdU7}3nJ0*gKjU7?%x_*1gEcWOhte>x|64?mqDPXNODZFRN}IP+?&VV=KF zA1sB?(RU(A(8zuWsR$pb0PF}OM{bWOUHxbiZh_^YfB|o-o1aOq_?wD#?%&~GqL8+( z;j%AHJQtUW{7Y)-`Jlz<@?i(>2?o-2MQ?I;6BP)Wd*eBgG;VIgzN_f zYf_5+oDs3^pm+I}V;@R2sa`zSU{02nMUuV(hYxH)S-)74=ek2UUB*gTG z`E$a;`C1sz6xalPz@WyIfOSv&Lk2P!-L@Gc$fgSCi`bWvPq^(N;5q@vAsMCINPHmu z-EXC{g(q{ZZ_0it#YKYjJIYDDw2d#4v1hVAkGCm0w5fI>xK}gt4Xcay{`4VZ;U7UJ zQvKla=cxid-Z**WpLV?3ilj--w$GKk<$S%DM7>(-be?c^P0|Z!+_$x=@L%D!Fa4v6 znkT-4QLS!hrGE=BGz!*2`QJCP7_zkR5$FT9M#cMt$tbGTq62|oTZ#ezu_O~2Jt9||e9kfkTks9;$JU*O1-5=hUqqSkS0u#p z@B@5+kF-g;T3Rp<4-byHHj~h1Tf8OG3(Pij1}qHxhIxv@BF6|Nm7`JJ2W1AR<3yjH zXxF`x+T`g2%_#gCx-vAjvl z4?I?C1oZ(O(PCtKk`_HA3HoUU(1^Ag1$)k63U$^SxHdS)52X_mAdL&UoBoCDfs#*O7rfq-+@ed3n3 zP4Rq?t3qkJ*i8qK%(H+c12`4Fi%?l7!2K*GNlq)0N@AxMQrMDw}*{HCo z;mWn??ji24g&$;zYAY#^Ql&CjO)^jp;#?-4vHdUWf>(c(V!-3jps%-ziA=K^=JIJe zD+@KI_c!0sr~UC)hx1MM)2H<+1__lkJamQ)SZz*XRG<9>BC*YWvd1aHf_*zHLHVU+ za)2LR$b4=@2dMbX%*LXvMr2|$aF9E zk!j{fx2v`%lRRwyR7Vtlb1rWA!Yze0(T=7rMuZWXyQA6qF-+8)6_jB&kuNV4-zlwEsCO*k@+JE90ubl94 zhNvW(%9F%%a75tZMiM}iVsRl1pX=f9%o@1m_i+{z1+JPTr0#NIv zujtvzVZDMWgKzn&F(+B@x1(PrCMu~u$rek>RPc$yvzVB_Htko zQnQT+7uNJL=;VwZ{aoyjqeiLBLYIxMuJ=Tnkq(sez{H@A6 zabfBjrD6Zh&Ig2Zz0#q)*OKz@PaLuvL=r@2L^R@vJ`qou`+g{&BtiZ{;4Jnq53Hjn zMJ~{U^|;01kxhw`p{Jj5gGcoxog+wB3Mg|`^c|&_mX=mwj8rNmf^fr}Ca)nM#sMN6 ze30UT;!eFD`jydj=Ay4!YRpsAe25vvnq`Fm#B?N6ZmNBH-+_M4z{5%#+=>4ur7g>@ z^}}0S{_omvaQM>*C)eiXqig-N;fs$=ttCU(u*P0?JI^TMjf`x#Xx?UCu{r9thg#oq z7l^Sb13pAs4|}AtG8J$dF`v-%zA2n}E}oUz_EyjJTH&Lt<2FdnfglZAehdy;{*a@+Bwy69esRYM3dFexx&R#k?Qy9=V(3vxMv-@~Q-E zhYPFR*(6=&9;v&8_=UVdMm;U*p{IZ->Q}LM9>n*Gyos(zq8JNJXu%DJ>FV{ApV)#V zX&zhK+Im>-twT39=^@4*rWKp-i_PkQO#Tst}4~O`ZRn}k))n(_5znFN2(5jr!4MzefqLt6VvH; z??mGXNkg|&lSZcS_Y5KX`<kcaOAeA7o-otLTNyu3>wBS)=&*kJyAbbNTrrqwU9Xj;Z^2 z*Av~MY@eD+Ys`L1USsak(RNLr_6{w$)yl=eeyz1(hO$^19L;cNCY4^nYWK z5_+6`HtP>kWnh&nG#VB67EYO=WxI9qP-yJgt=K}wUDr6&dS?`$*a_Ls?I+|5#W<;^ z7sI>uR5}3?nhoM%neN}se?^^rr*#k!a$i%=5O_-HIM>{$?!PC=i8pG7ynvMk&{LfM z>}id7)LZkE)OGyqUQ#|BCC+hJH-g znI68OFcBh3j**Uke_*hnw%nMdxA4g-Wq>fV&6DB98ObRlW0lRBMa*>tUy;FIcdjSgot&Np4$BBAXe zx6SsKtx5|5*Cc7AE`h;AeK{6!J4v~&Onm0H=J@_1k(;ykfx|HY$0-`Mu)E2xKe(~6 zv!=clIAmjfA9(9bf0j8!&qN=-Y)tk+*e3s^=RaQ?WY0pWQFriv7(3kp#@7n-+P`a9$?-)m zW95feRP)Sw6Y5X#<-{;;k|jE$Nmex7J^a%-F-s zQ=gI;qSTvVIAt^AS8cBp$33L0H(Sk;%JKBSH9w$xF;4Jen;}9`60O})h{?6cbeKE= z!hCC9+Br4+qUp%s5Oc;8Wp7@=YoSiY*bI%|c}H9kdnwORQwq96Mg1pgx8%;>Qf2gI zusZL?oFt@q4$hu`^;+^i-;Ju#@8W%T{%2~csJonlB_^1SweNW2JX0xTR?EN-{Rz(x zpfvfq{G0Z^$PL$&MQpu}v?JeoosO`hNL>i-29NvI;+~M8h)w;gSIKxpR6+|JMWV4) zd(RQuna1epFz^OM^0wYa0VWabM`~>U>{X>zltaEcgIhgPDVIUofA_2My+374$nPg# z(l)Fg#^aQXTD~7}O4Ep~dPTdKlWnrpp3sBs<>j@*U;?TbwJ_qA+P1qx+zP|t609I% z7LNIdxXVeQ!52k@N4AIBldj2w5L<-DrySvNlyFmX`F)p2_F^VfYVMYZu}s7~ODrV5 zezbp2AeG6czOyneTiEzU*nvN8w%fBwMUhf0MNmgZb^DfA;NhsM`L9G2TO0q-5DC-8 zAW3cP|5aqX{`ab-Fq_f*X)~2cFV{e>CqJntJ0W%)b4_KdxI}xwW`*sS{UuJCOH9`6 zN%!}{a={CRld7hzx9^-nXW7$jk8SKEHF~8L%?fu*_`Fn)ZM*0r+11~a8R8sUI5GGo zctsSK6n$Io4KJOq8#wm!{^0eNd5ARf4H~yMa}gcxH69GP)YLeKONypglRTmS6p5r{ z9e^@#`z=py=f!?m>=mjC_^dT84d^Ufpx&5^*KZre*?K0nKWJ!ej8#z00!Ft@1d1?a z+jTroA(6xbql6!X$}=k~){kqS5j%U}XNow}78Ms`xf(Z|PuTRkH><^0EvsBQ1czw! z5HAIT9B}FUl3agrQBVD6w|Pv}%>u54{D5of&R~vICK#DXWQe)5I8IarElUFTKS7Z1 zuCqDgtA$-Z?~q5}*@FJ_Hwc;s_XNsQa%`Z)ss7^GJ7zC0V?AXyo71y$uhJF7M`kg}O0Zgoku@DpEl^ zq^EOvMJ&u@R_YpP{o&~lt#sWHTH_m&PoATBa_F9!!jAVp+K(3l!=x03idJXNS(MTc zHB-oXJ^f-r&N=EQsfckgDU9HS>5i(PfCxI_1$p6=|<^QwVpf@O57{a@B+ng;V|9M-@>cwuM^M?`k zu^GsYaBBD248xR#C0(D|o83+^a9Asw$etf<$og|Ta&qStX6a4oCMmHS8rIg71}?cD z+S*qenz`|Vp1}Duv%x;f-FBw(-YUc&JM>-ez-*O)zHyts-OIINP zG@ULNMO`i-Uv1=yyBGKYcYK#vm~(`J!9V4dL2)}wgI`x^}ZhXGn-y2dC2jfDrF z%&_poX!P`CTMa%~tH$4Uz0GahgLv(;*BB1+vU?WA#Kda7Pwb|ut(437XDj?>?t1U= zBC1FFK6bH8Pn@e}!{bRR{e>KV zoSad}0rf+lw(8oBdrX#C?Z^!6+K0ZAk>WS+It}(jDKF{y-6lV2}b4E#)thYj=}fyZ9g@ zHR1lJC|ovrnYg77M{M*&HxBpNdi37OOHjAXyiEA2@o4hc=_Qkdj&D6JS8&xv$1QSsIZyqe(!)_ic;^Ig%DqTAYtYSa~W^$k{dsJC2zQ=F7QFv=DA?E z%`JiH+7;=raiDK&DmQ=e8kYxmt|FFcYaondQ>p8FM@|x0K<(@WV?8oZj-f&7O@e(| ziom-5AZ)vv@FIY;N%v5uJI!VfJa4FQUUL#Omn%S08A*O*N{K){xA*R zM9Q7BfSYqx5*15J%MBpLD~h2K(RpwBISn%M+2=)usHcz zY~XdP1t5@3fW;No3f?pi9i;M@Y3BN!QgWus+F;o1PkoF!>P}H+qbekj*auzR2GYL$3(noYnXzYK^09wK( zppKfiX!xH2*S$^P{l2_hq?YRD&xUT3|NeA;e&}HRYkG$VS~{d^`?J%m_#Iwzlw7k$ zHyd^xY*O#XAMH9Z7=ttrt+($d)ZGmZ#@F-thRtn(ej+=II*RG)e|4Zt+YkI6T|bLB zB=l!LzlFJju5XKd{k@4AnF{>>JE-xRSWnY`Tz+$Lu#b z`Hp)me{w23JLrlT`Zeo@h_utXWmQmRZ#5kct4+q;w0ezhf-ABZne$%CxrN?71X30592aFN(8%GV@?m;7eJ{W? z0d6rfbQ3xd4}U-|Gh|-{ycY9z+#X&usK~eRQpO9CbvX)R^GpVZU5t;k?7CYhzg~l! z9zfhmhzpI!$QuL$0^*X7`d&;GQa7S;r}^{1)0MAMK!uIM>R8}{c74BLnUL1%(hoO6 zA|k3T&X) z4k`oNm_rYAP)RV7mo7rb#epy0(Td|p?wfj)e9_1MfR(r3f}O`7>lsy~^J~(EC1obY0_$85k8;w; znlq84N+?dY?;iC^Zc-AWj~=iN>1?d?*41V9<^~iK09)ZzoLru)G%#*lUk3FM|0G%= zPH8eM7qGjMkkQi8al*k+RE9{X4NX!u-5C@{QeuZ7%0kM zg$j!;{S5o+AMSeg)8f903Sh@h+(r1d*Sa4RT z7MLr>>8vu)>Y2-4w57>bDEB4v=WQW9PF!;H%<>7*RZI@hS!#deWWrj^on&z!vgxx; z9(Y|it)g${t*orhDVW{>!E55_?sRvbvM9C(#aG=?Ouz6(WK_w5RQxFFsyQbw8+N{Q zfu-Jw>~;CC`=0;RpXVU^^CTD}?M!^~OFHy(H%j$DOO_s4IV=py2gsk;dIDj(Kyo_u ziVuos>=p%|c4CrvxH{<}R>}U~+s| z(}bw<%_tW4ARMK1DtPiuD81$~pQh4<2mwiw1S{A=C`Uga?)U#v_}`ZDK&I+`|kz3~WFZx}zFWQ7B6x zmxYOU+eIcsF?f3ec+NKvY06wb7k|E?rPT!}xb4$IIBk$cw?UQ)5UPwU#I6~cm^Rf= z>_oJN`s%XorgiMj@~_&VThgIWdwLqcp4-!B(ioO`;*JUu38$3iLs1;W-O zFHz3;!sT+r!J`oT<^AKl{XZ?#sN}+MK4_uxG16N@-sN-j=&_<#FFZ?Di)HaEJIr?1 z#^Ki<&St}V^q$04-{hv(I{!QmpaUcX|6NUmU%z|e)wuHI>aFBTEPf4i?duO6>|{QG z8V zYQ@V@Kv-aEf#oP4x)85O3_H;=C{b{kCs?qVFiM*vGI~LIFklT|?N7x%zX8ho@?(C` zI!dST?|Q?emcT=m{GNsvp^%;TFvK2mH8IxB5{P65sb4@p40O{4d_4)Wg9O$BI}QBN zM_+#5_^_0kXVSI-MU5Ft4~(!c{%#5iTbnF9R=j<>@lciDH(*Uhhe&oQP0v%8RZ$L% zm;ELK>r^p??ZWVl2rZ~M;_o3H0=8xv1*XAa?cmn)#h{V!v|hReTMhd(*tDXDJZ9+3 zWJa?}7k8F<3|jo9NOZMCj&XV(pg{O3-s!)4i0PU@4$%Vb)6(5YV4|UV`D>r3tB+-? z$ABK6iEb3znhO3#16DY(!d|)ye0C&6v1H?`>(S9Cl`g26$iJFJ3tV`veNC5qAmTJb zag06{)#&J)$t>&rl`+~UoRH?_C$^PGC$hiuZxTQ?{ttZD;9L547ADzHnhJjDQQOCs zImkE?O<9vrI=B65s}n;akK-Ga#|&*)oUxTE{N8H=!zo|2dO;d>VVetEoy%)M~`CoEo(HQ z%CNUE_qWprEYBLwKdp7iXs?YeD~@;H{iT>1z@G`W#aqI$?AVunN0p&qp(F5{1A``8 z)o6by=~%xMSwP(pGvI)_t|%N=*B*}Pg5KA`e77}&$#dld(TLz6#qA`vAhBGTyRQmH zNCl4>kqaB~cVX|TkjOh!C5z9!T#gA0JOo174`|q+g0KhicM7SY40*SMyc72gqRD>bH_&opw z?_(Bt@-=TDQ)C4AB|Sq#5L1&!L`A&;?a_CDdz?)?3`Wg64Uxv0`E^IPM02%v;`BcL zHIWi{Y{9?kXi)D&c$cYrK^m4u=VAcDNSNF zoUdgzN@+#6QTqUmnAewb)6W^(aW`ED|BtM<42Y|1wsmoL_ilo_TY@zb+${w6;1b;3 zU4pwO5G1&}OOO!UB|vcZvv}YAo%`*3{`aq5bIz(7_0%X}o@&ekw&6`&ofMvFkxd^o zgUpb`C(Lt*@4O=Nl~k^AYZ;B)m!CkfH`GTq#sNL_7R+UYF}xO{Ao$fY@92Da7cPqU zKSg>m&>nd$(l@)ul_MC3vaEJ-I2o<%e5E+~+O6##r8QTb?b@VwOWtYje!(4HcE(o# z4KF`_Ei4a=E;v~CoBgkV@1JV<2A72mVruK;hpb5uzV{e7m>xvw-gw$z3~M)k#)vAF z;!^AkNETWvku5n&nm8Vw%s<@Wkyz{}P!WN6;Q)1B^AzRNJ$#i9^vUTN1J((KTJ0Hq z{3N{PTsWDAkaVZtqA!f4=8>IwoSjXXvG4&o5Qyc3Yj6hQ>B6hz-Q17g^RbjhDU z#f>q@D+uU(5PoSwpw`4?s`a;-DUd|f%!L+!xq@EU1xzi?jbVpS=P8g)Q7A9L23SNk zVhG6)QC?_=nC>rchjd2ZDG~dSJsMtspEj)mOif! zgJbo0hp5>AT-71^Z2<<@1StVYf|5QPi5THZRDWwQZqH;J=teTrR{Dd3Ot2i|`G!RP zMCsTyheVL`hh2ivj9QuEMQs9;q8X5c!39G#x?Tycz`lW_Hmp=rId+>JQ~Q-*{xZ-3 zB{gXyi>rYaFud?HNw@sJWSyN4|DKotL@jZRc!6JIH0tWJ%a5_9cAmeNZ!0KpXxjw$ zd-|hH?e0y?>;zuboT zLJ_)uErCVD39f6}g_rtJ%It?+C9egF5RG7hLmfo8AtaQF^LrtGYrSrc!MeKTxI-)r zJy8!8b`K%R*!%a!BEO|PuNyDmds3KmI2Xb%C&0Qw(2bZY>^I&vHq1972UK@?1`u87 z?tw~?>FzjSDW3oiYrLnNfLJ&V_xc#6c4Y6+ z=uPiB=0y6fU8f9jS=@?#YxcBj+N{(f<5^If;X-<{Y8t30r6br0=>RQAq__z@tG{hK zO+%RGE4Bs}DU#B8a-F}DGyn@d{`ozd*t4E>8Ar4!+ir$d{G=*XCq7-kTwZv|azJI7 z02)a$+0S)&q7A)}wn$|dq}RFEJdbZ_IZ7}o|B=kJWq1=;r@eUi#ZOQH=<{hH>K@6; zis0R=p}wLTv2YI|(NCN^L)Lf`s0#fkPP>$HtR@s++*zYv`sLVVK1QdqKMNxKlYYeg zKtRp#fDRl33?Ktd9tu-lWnAl_r{oukH_ol< z?Rchmzi^d1LwaH=QDEPlw?<_IIlz?yka4JtVi%x3eHccNegDqw7;QTH@rM2r<|08l zxOdXJn{E-d8X#xokYLU=vA>Ckp2Eh zFlE;|W^6*0WSXb|xG1k;wg85<(P2SYfC_w;CnDcg^r(9+_k<L+AxBzb^Y&utL7F%mE;Bi_$7_WM-n zxMi?ChGkLFF?~UPex4|$sm);XNkRtk$04nOpLUIUVOzHU)S}gCB{t_c+U! z61fJdOe)Ids*Ql87uaOrV@g1(vpn41=P%K`Oq9)24P}i11T&*V^;-cQEQV zO9u_>V8PjZal}=pa-fj<%WJ=y^*Un>*863jt4@)p@Q-$AMWGJa9a0XAc2@+FEAPKy zOxrl#w}WRj_qS}+D3tX`5j`62(B%+mB;F19v+H6S#6GO1LuczlylYa=S{U^z#+>+om; z!4yc$mXh#)2h55%Y9!D1p7}ibii@ObODmZbCw$l{+4e9y z73i!~#oUL*GE_x5=U+3$a+UWk68SV8losSEt6L-zAJ9xrqDG8b{^OGPk%)=HpzmyE zsShs)mx!vC z-xrkjbluARfMF>7E5Z;A7#1l2elOVDftywWmOj#79ZU^R7WF2A43)T77|{hEizkoW zsf#YvpB>5x9#`RUYd)GABM2mc8w*qdHr$a2^ z@3IJ`nZv1;C~KUem9WITv7D4yH;Z?@%;2a~+OUvOd^IaM97BqNn#$gVGxC4&%s(+* zHudfc&I$Yxye=Hgzz6=?hqx%JWgT$d3Q;=$nX^AdODwn64-pBvv!fGp20vZPm7I4PM*5bre6Fr;3XeS1%ZPizoN0=?v0IY812)Lc zDh5ONL(#2qd#Z&4sMS;@`XwEnB*%DNw?_^Tf!>ri`;6D3 z0Kk*|5drP(i8#u=R3ach53|OZ2FfS8BC7r0NrXfOvpk#-gn!zrKzWnO2T(m2G(+Ko zFFiQAtU<`i?f9mq--x_VF~bRkyCKVkJd`+1eWYZljtGOScTh!$w4tJAeRP@rUC=$S zL%3jffBCbA-573{xm1dU?zNBM!;G}Yr9V0g@JZ)B3sT01PopMT$m<3Nk@aljZN16H z3NaKz!VbZvRr62V!j)Byle24zZMi-fHT}}}m$~KSZ17W$#Y~&;Rf>&7e#l{t;cP^L?s-0e{gnZ4}a8h;?YCbVm|+pJEPr@;)+NxJ~S3273FKtzfI7W;=AoD z);fL*@j$qMx#N@>2wuCGFXX;XDksC-igW(-js7;zdss?$3^5&4@oDQf$d9!9mN0lW z?5~G!oBaP9k_(^@IDpx$?JH^h@S)LUC(2FJ^Ifxm=7y0`RqyZ#rpt-0++^80JEp0% zrE|-1+Jer8?dipFi9^1$~JpIv+-)OLF!F8;aKZ%hDtd2WYw(d$vJTyOtr7Vtlmpo7x3&`F403LtYiZh}oQ zA67#24GNQV|AHiB3^dp_zh7nTB0M0x3Y94RZOugi^QP8VAO~( ze)LH@g%yO$!Y2lqRbDaQw<>Z0qf3N$C@c=?>_!uioxbmcH|R*n1m$9QN+0zcDB6Fu z(&4IOlu*y$zj#r(9R5y~;hV;wy_0TUpv|q2cYU|&L-_Q=SI=|rX?2;G))y&(jxN*O zmoN2MqmzQidWjb7RNWGNnO%;nt|E#v%*<+BFlz5tS&EH2z*P=+_VYwcaYI;=SULO* z-*nDyL4Hr)l_7gBPq<#NQRtV|a`;o`^$AP;P6|Ex@ME)$Ur+h5OmlW~YVjVQ0(@$=mo6X&`cl*#baU$=+ zDZI?VLU)f?o|om%f8*?6|I}E#zC?_lCNB#IbC1Q91XqIQUVP$z)6whT;{iTujPMt1 zMGf$WnSS~}GT37{Uv1PDQ;dF-y7bx_X2D*5cIT7IU!n#Q@hqZsp%7q5w@O$Cj@)nl zh9sfEGnt<^zfAYU_bb^a@zbQ+>F&87uXWBVAqU7UrT}a%SXvf{hJ@BlAk+`vttr4y zXPREfL8)5pB%|Uh>f0?aY=FdB+n;2!1j`v_qeI=723tNXZwN&!frMRHB5#b-599z! z&;xWLT_g>5l0pa=hMbWK(T2b9a*?2?gXzI0&c%D_Zdo5ffn)hVVWA&J~!$MGP^f`0k(p<0^9o`+ya2+r0>6|xGBI>{+EiB5*I?_Zb57$J!} zwhzpBeAK2BR7#egVg8oLH?a~Y5Kf^9$m((T1Ye?qoahpc?-1u zKS&}}i-;(xBLDnG{b9(lqj4DEW_z`dWKC4g35PY@Qm zJ5gcp_+MG3{CfAV>U*Z4@cRg&A(URgi;k*Pz5v#ep&|`f#f=5*JrHFKz(El{7vDFX zl}fT)jWoMl*);gG^6X=&vkV~-v_rE`g44jpB6(h{4+{0=|I$bXQ?uP_Ksa@~pNp_D z?~3@9iXU;atNLuKOmk+S0*s3#Df-NAYwfIQ?b)bgFFQ6;bSb7JWV#c2HS0HvNDF=u4wM3!G-OE^cis{MD2Cpx)2s-Ho>4^1T3kX!f2&gH8AvwVkk&9Rb| z-#(4}C6o;8PITry_EVd8dgDq$I_kSEV~@i(ed(wBbk6Jy5z}Lj(!&!aOK)8YzS-Ry ze4VLah{@ni*cv5}AN~D3f9BwK@!|7ILPmqYQgF~;=s+3Xr}II^1KvEg_Lb7oeJU(z zPWs{5qbU2mpOx9iB?Bc#%D>ZjPNXPwqOX2m_-$6^4R}9d-w-0v(;p@telH&V-WjWU z;nZBkDNic84$S`MuC+k$|=Ld zT9jR)l>d1Ce)6VFX-xTJ+b$+;?PI;kJEnf`hYJ{1(h)ujZW~e9P|Q6~@Zaik&~ov& zm%K)=>pZ_--{h(~mO|b~zr-7>CGLkA5KIMI=1zxZzjGJdWz!ik%{(i~-DA<~v4!gn z-TJ#a>~p@F2Yq>yqX$ilyq#9R)bP_kf$GnZzL4ep->UsR4W>_m4Z|bn4I9RRn7a|g zunVzcJepoLV?qYRYJfEVtNXtJOO&HUd;N%_eFIHw*{OVm!!l0d8R{DdtY-y z+*~AaG73cYE4WpI^8XP+erZ4uzJfwlT?N}#;;NdU&tZqi9M;^c-U+?zk_NOO2oH#p z_paWTaSp%*{FPtD2$x~<5SZO;yLL7k%>2NylNH5URP6WI{mhW-GvG4J%3KsOnyOZL zf0cd<#gjxRzmm~tDZZ!giLB={qcGcMOG>L!p?gD@3*vEvNutG|rg5SGYa($J0|ODW zPfs_nweZroU{g_+o!!>w-X)~^PiwIeB!4ommKszaXaqabw$3S_!&piXl1lnswS&QtB`(;oW0Nlt>^R$-}w&xw`%r#bb9l@pJch7 z9+$td%`EIS6nc(l{Jd&hk!5o`>Tzi~diH9b1o`e-u1+IGa(iuUH5;rRoKa+Jc2cxk z?)E(QhA|Ld^j(7PPje>Xln*Z^FJ{g2JWV#;e6O`v6Oqq8)>+GGt+`Y2e3}Zh>v(ci zt9tLe!1G(VZ67sjt<4Qn&-<6g@^7L})bCAMIUZN|!hE)K%wLx(EZOap$GZ9Xg!XJQ z-kKnoZSPEV_eqS5O_uJ{9v5pFC|K~?NRxkWOSEgd(tMkoCC}FO{I*{BM37KR-ZRQoL4&SHy)=}*XF7$U?#r095#{HE3A^A zv?U$zSoUl!HT3oeWMC!-nI0B9_>oc;r^X)+lt=%oeyizkds?jAcKy=`^Ct^pf%+TA zpQ4kwJySczW?%%_EfA5+rwh6Z$DF|wdc-(dA|*5uquAf+PXPROQ6x>*5raLc5QW(mx$m$h{6A@lc%0**p*)Z9H!HuCN+i3#DaiIv$9zQYFae^J@oS zvfe2#{t!_fwM{(PR>3$DjF+xz7XT(b9!tf%({|%#ayrWXM1R$pS&qjrp-z38b0ZHP zts!L7?Kqf2BLw-JZG97axjDN<%d%w7%{>x5WngEDA1j@eFBp6qS6bnXb%i3(4%~lr zDZ_C!fS6JalwaRqDiTq7ba>ew2BJl3(r_r|y_h<*HjK`re>9X=yN;G;yB8nEpa*@h z=#_ZGXn94%d5BGEP^EQ7olw|>&0O%8isv9?x_1~9hP9)0Rpu&IiWuawGWjXibIRH8 zV?K3n_RAA^?2q%6YX%`)ZValWtJm1R2!CB^;<$>7Fv1x;#T=a4zdBU=_IXJy>BppT zquGP2d^~?|u-oBE<}j_!^|6r#z71WS2t$I#`#IYsSqolE`8dG`G@7(6uXiG{SD8G8 zSDAcjR<689O450)$DO&uEMA!8#@5pj(ui!@M2*nOH$l$m*KR+DqQpS5ZGs7p#!_eyh%=8hKG189Fm8&o~} zlrJn)YOB1*v&<`j*~Gf{dAhdM-P-TY|=Sj)&y3)AcOSe#+`7u#|Nvl|N%N@7Q zY4I#b+c@^*0d0m)I7Kmo=#s&kZ)7rf>T!v9Z!t5GT_P-yMX+^sbA2SQ?s2K>gB z6?BL2MqbwnCCT_6jd*N?2RyS*cCORtf~x1aQZH}Vd@w+2xwBZG#WrB_E7xzjQy#?f zu_NmNXY2l4$Nr4}hFCBSXL?4!AbF|h(qr=ZDLoi25G|Ys1YnDeZd~^b9-j~RY=}~q z_7lR&&6XPK%zF`zAIKlBReA|n{`^IqxCuA)Smo6$$b;pf)-sdXZ7rLwK9Ec6jEel< zJp%XmSHjvIjq8w!nT+vz)WTjf#JteWgknr`Eh?2XEe^gPJmg&5?v*H~uz}Ts@R5Jo z185|Lqg5!wnF-K8ajBSdJX3&!M&_i)MPaxRJcUYv++&i9u}la=VPrZUE`CF- zxAMYBGU&*<;Q95{AT`HxL%{RoX_a=>u;pbcY(KYNu~gw*gG$N1J(ktSjN1Y@SaG3J zzbpK2eakl=EHv!`oY3ycf*IlyeD!Jq?En+Iw64!mNsSYZH^b;-o0X5+-E!2P*Y zvoN%7s|$JpG{UT$Jj}|t>6Diibdgl5WMPak>wN6J?yLDb;2ur-tph|Eh66d0H@#od z8|{1oqNY?7q>6ctaA)>N2m0(MzQ5x)M)X{Ft0nyYN#A=5omM)aDQIh~K#X?S|Ifwh z!^sbnaqE)~ITsSXU4h^lDIt!qV@@A*ky(2R??=aDw`gQ(&N#s_Pjb0 zNY?f@0U#6ylSpj@xEvf8>pG?j@QC;N=l&dP47u}jhkRyIr4^E$NFoj7FW@GcRZ^v8 zMaqx9M*8fPoSYFk?#|Q!e!9{E1JyC#U&D~d`g!U9ph{{zy9#(B%OV~?PuL&)`6pcq zbwHUh+7i9(LSw4=SCiHG)}%A5vqNQsL0{hteBd37iile4cH@?z;XhgI?)+;@%|_j^X! zEY4x2S->Lx^A0+tir!>gcFas8hEX(RRG_|q7P&3W%Cz`EZ(60y_w}Sa1@)PHj{2&r z7xrxvR&SK61S;+H4yGE}$Q9*GoY2%|ef4XDpS8ac`0)7&E8K76(#BOhAF|P|h;|46 zl3i>l)@pS&I){%H5m$mg%{+*32bz*@w$*yDP6b-u&G}%Jc7n zMS315_FX2RMxEP#?uMT&w!oB?mdmGjS}lFglKDeyY?gRt|I*Xt&zF~F2&VAr!uA8$ z#q$sJwf$oAN*z^%C;A^Li|nNwWajFph_L-gQv1Bn%8%2yY#m)oSih`Sqg$?ga&e}r zO31-kc?#h0%Y(ICWEvt^`i@KX!B14l=FgvBo&+thVtohZB!mz#4Bw!~-jFM<+kO$X zf?j~v`P9yrCz524XAawg9Ij$nasr9>SRbaFk8{8>nMOqwRE^}AeYn?o8P}mR8mq3H z6FHnM@sK0dEO6|ppnkSI40nNGpR+_@R zt7`({xbYuEx5mHzw7Ik!3u^x3r2%;l^cv8*P>Ns6`+{1i8u-J0U_MKmVW8cj=Q^d} zr^O4sSVKb{GS7xIXQUeGwb^?qU6pMA#-m&MAipR7pDe1#Iahhp%3?Yg)YBLC@&(Me z@pjCueQFL}Qr7yrv$JoAq}8F3#q{-wnR=L_*w21h#Tp%9eU~@fCbsY2(dj@QY>+J{ zmdJjGxdE~WAW`eu7B($;9Z*31@bWJ{OqF8=M358GWH!;*08cbDIQ)40^^RXZ5ZiCB zb#ht|g^%}ks>MQEL*ymVLDvvgEFe@1D97&$m|`W->q`$YQFX8i{!Nrmfjk)uG zF0~`4weTPEXK`k2+H|6=1N@kb5s*0Dmc7^>Abq!f7IOu!@{CxZE2Tl7?etgtBq7oG zyXU>!bZJ;K!eK-+efNHR=kakqF70BMR##`A!Pp8gOd?#xWLH3i#Kqn~Ooi1&)ndKn zGT>i0zc#VAXEqAdV=Kb#ux2DSSF^ozA)-&aBACMK<(X;koa&Gzuryscu9|j^Tgxsk zCLQ^aXuz({P1hu})i#UmQ8)^j%6sowbR05reYN{^2x77YBSuC3+AAd=;7Y6q{tSE@ zt~yHh!~Fb=8RL$17=MxVecqSb>J54R&V51U5Rsf_%q|!}~Rr zIE#@-U*a(rr|usw%Di5D{|bjfi(kS4l-;1e!<;=Y$HkJXbVepk98NjRlVsJCugY5L468sEJZ zOg|{G7&t~d)U?NF0*FQclV^|qU!}+pBDqQ|wz{{^xqf2kZt6d3WD?;A$Un>{8G7)x z>XKj_tdkNvux6z<3=Yg^>n$R)^l ztRaZua&w7`fhLZAl+5Cb`^K$w3@CCZx$N6_$MTBIjaJL;@$~HiW%u_+!QQi#9Cjtz z)=tr8#`;|>XGre*1GTbk9!UzT-ZI=#lTUGRrGJgte7@*Q;G;41*=GsOGHOo6r|+=iEc7IbOmDrb+d ziOxsPNI3=M2FVs+>Byhhhhwv8WIR*RRj4y)+dz^CEdPu;K*3-0%l)O2_Y(#m-=;M9 zD2?FdnG{z-bReB`i;GtFl9)Su!e&0c@u*u<$%Iy+VD4yXMhH__nqgtVnVloyW4*Q1 zo}LpjQl<*a$_3j{9qre0>h$#H;U3pEaZJHF&k=7C{VCu15{ zWrKi@jcrlK^YJ^))_}zE$Nne*rDU%gaZIOH<%Z;-XFEXboh??f*Yt%pfiy@@uB93Xh=`~d?dh0PBV`JHvtR_=w1B`(Et6OBlz9{ zj(LXHQHk>WlIc{vnHu>EpV}GvbGJeHj(Nu?hZssjcPVsdQf66~YLuh0ey$D5}nSRtU3T;N|(a3<-y ze?}=L^pp_7`YwiN_R3cacWnXC_AA}lrv?x*)$KUaoNX*{7ZqbS<1T z`b(KwrmucY)|SxAb*A$MkCx+;@r;^)!l0ttj8HbnXtQT%7Bq?nn@- zONN{0mFYQLIIVS~jX4ZL!&AV|~4?kg==%|Cvo+J%WWEl=^r)i>F75M-Q)geTpR zE)pk0`tnsPm88x}#y<101|L)1L39j4q09z(r`+i?95K6JBUSEtQR4iu5MlPXR_@ON zpTW(nXSAN%L*?br#iS}w{xrHhO&l%ffKpj^Jt>vq<@5SrHW7QiROexMI1o=W<^u5N+L2Ayw*Ku5XOK5l!It)H<3pyrjacD zPGKIKslem56XA^0v%ws(7)#6snvCmT%*S!7;s~Efi#tW%1Te;5J{4;ndk7fgH5){;5Ii#8{!6E_%mNkwz#BG2eUL&cC!uw&C0q1K3iGxlcu1??w+ck%diiukLw-VQW_lU1Tud3CJAa2`ONty!hE6 z#CR5XD=-Ns%3ywVuba_%3ubxtVPes3PXF>$ljN)2oJ~-p{?A;_h635ZR)E~o#_tm$2KQI7VVm)Ca^>2i~%k3CEA(E6o4zlSGs@E9iXYA=W(STFLT-` zYYb?x47c++N&TP|2m#g(Q9MdDIxYatp+Zj2W=(B)q9Ykc46W(5gPaoVWsQXU&5Kgr z3`sALx}6yJD2jHR4hxpyPCK-VHk>VCVZI3#Knx)v>z!PZ4k%05tJi24gb0hpc5w`G z`yDNI_hH*LYJmD6w;7e0`=VByNvxvB0Z|TFQjae08m69;i`LUFCa)SQ6YnovHI@Hz zm>u(Z*v|hLrBcje?wn{VP61)MgpFV>nle4*8Q9>&CD(YIP`r)LQkNx`Jb>aHHfVKo z%K77#T(eZ?r8I3s`(@}_tL)t(V&F)NtXUSKqkiYTK0FW^fwwwnOz&7pyZ`wvEL(x=8^PC}ome=`>75Baavtx9kjOlyU3B)2|nxU+FP5q{Q#-uPYN5S=pK|}q` zBzX?}jMlHZpR)`$6A~a_x(6J!;d7!)clIEy_7)4W zEpVpcSCT9COqz1&NF#u zw|DzZE;G($=E`woi=s#f7iDAkecUD0grm=tR>}82py-9?XmwczR{CF<-Axa6@oupe z^C(tIVuJO;FM^tZMk!e~O2x&q{IbIMWg_>M7T#p_{8d6s(C`vOFIfuuiDy7yACYkA zz1H#2n_Zw=6RrI7drhTH!xVRTZF*ax1%qVPTJ`%^>nkEd=S9tc5e zJPdoE(C^%zzWp;K1kjp2ZF%Sa+FaBUkgsHHw|_!7kaVHPlR zBqtwqJyd(&22(Xr2=Xm7&3}5O)4zgFGi3iS2|aO83Z`1>u1<^+7#Q?XNyMi4qH10o z0m2;wMa>nTC9^gh|DCmneC}1jB6NV znu3_)9oXpB!>jtv7!#ogpRzx>7G)0f6mrk1*Z9R(?*cI?t>|T)@E4&{BEp3)a#tEuxA=!@d-gj9SUAgADpHV#Ga4={_&KyF)MW`8lJ z;(NqXe!g%vYFmH?%~7Q-f7?Ji?yP-sczrPRyUbMnGg=W9nFyQ@SXgYhU>Gcs4>B}N zD^vludwN=EgJKw1{a7mRjWjT^$bdz!`X+&5b_7Xpi-hG3&ybO1=6HF*-DE@ZsNo!9 z{?Hq+U+F98O%?}G7-&MEWd>p~jl&gCu%}VK4crDJCU>97>$5KuMB-bHIf{m}8)cQ| zVosyhgf=E{`iun=e``Dvwm|R>Fi50{?qFJtatI;aH#W!$;(xI0` zdkd($Qx1At=dhi0kdfLO>2 z=K{Pf$I-p*7DC4T4k7o>1@UWHU_VMXrl7{QaDaN|v6@!*X9ia?{1y3=j|k(?ChoVBH5=HCHY3mxj|6zgJIC{^!#QBq;Z6ZRDCAkrB%AEDSH<@_1V zcZKEJI2ypm0tTG(a8}D1?^Go6SFkt1@2_v{+#byK#++X4Xp+8)hv6j76i6Ga?A;>k@E_MhGLNJ=Ax$Nw(tz%f0S1_U72)QX*LyEr2L;&8OwiKW!)*0 znJ`X9$w_oS0@37?Rf6GQ_{McNAz}_&i=mho%HvQnp&~znuoAL~)I4ei~}v*7efGbeWmq*bw5Cj1SJ39x?$Pp@7D4&Ziy; zJS=1HE#V(X6bWcYkazCyXt7MZ%}6ZQYKZ*GsRqc7S7)g)#C87iPqPZRUZ7aW_aNgQ z4;Yco2;| z(!kWC>~WA(L~s2rwkf~W0 z5U|c8zlnZkUQcu>0`>m-ecT$^_5GXN7ZoxQ!|i`Q={0w_X=Dbf*U`W|+izgkCc@dZ zEL2Y(HJ8MdefC)r9L{57oV-KujphD@I{t zhJ8ss#6`;@jV;D4=IcapWi%b`m@bj9vy+~YyH{QjYl-=1QLE`D?iLznCv#HcjOqAuuoknf&u5D~gWa$p(r@GLT=Gvhg z3QpD<$-zxJPgCBGquPzf>JD^F@x3^tx#;2HdoeNkM`Qx5aB!|SFkfdc3O44kP1`Vq z|GliRe;u70pK4OfY>WrbZru?#!?5K3a_$5bh{hgyQpv}Lh;HJj#7bquCrFE{(s?~F zLf^Lt`iUok)Sdqd#uf+0y%GHRN^^!I+-P_cbA1-A+|9Vljf|$_s{(((={xvkFUrv? zHT~0wL%kD16|1Eju8kYHwOgv$JEo^Z#^ku0oF?YT*_EBj`I+O#0zZ~-E`9HYSe}ZO zo=0rl=J{iQjzq5sg4$Shx6!#u<8E`mQK?+>n9}!G6&2fE0eElznSfU7F*jh^*&hB5 zhE0SH?-iZ&P6~OfL5%wV>_t8yNA4DhxuCo;6UBiUn#hd-45D{L4bBe(- zQwC|GLqi6Nc0QNIY0%MP@1$}5_(8uAEu{N_+i9SNeO77K^)E}f!ze~I*9u~tG=-0WM#7`I5vG(o zDSuob9rUCZF&ch9`ASYi*#pnK`AnkOw}V=00%0dt%^&m-djwbgzanBX;F z`CKdl2HDV{T8VuAtKFrtk0Ua>DrQ#d+aSSOt6z*0TY!0Wp3C8!BR|*_KU4iPw`e+( zWv1W4o540 zTL5Y$AA$)s&fo92I1Gcr`hV(WKg+qj+~@;BnnNGAm6h@FJ>qc=33FV)b_*NSIgTL3+bQ_rUJMWMPvd!ei9 z^`xME82K-~y>|p8oDK=+tcAx`XJ5W;h;V`*q5~s8HaNnf6Z3`Mr)3UJc4Xtm%f^Pn zy&2)So|9y^F}PUz5_5S*Xz)`icQQ}OVxmaN+*jtcOIz;IoU9+OgiO4TmzpT{!|y)f zy+W2K?0d9Q_|Y%Z0K6?be32{yBNi8d`m!7%0wDIKs*N2dGs(ocQyB`R>7uoEOPcq8 zq5xYTbU*lQ(mK?>)cU2y=_Vx+bK~f;KVZM6AA3&{YVuFQcju#=SGv!8i=0P7a|pHJ zWqEmQ90Pgn53nDPtzKiQCNLQj zW~`skKpwE7+jXkB>Ko@<(=B11rOc8=lV$)snMzt*4po7%_*av&%iNd8ut`wdQN$_V zPr!DIwG{BOn=O*T{i@f>3s`-{KLPVV=tKOh90Niah?Z1%ri=B~gjiuCNt%w%UD2s`lgxAo(r8mGES z?T<9HneV;*mm)5Q%A?muWgbhsmsOrF`(IiCy#zI!4LO}y-dwxYjLU_4qm-vi=}MeANSSesD7E_BuMhFASZX+itvZfGHLFn`9)Mb&dS=&B$jlGi&H+ctl_TUVC?O z=~?0GBX{YdLZ*X|6`!Siy}a<1!n}^qDwDxl@IK-GOWE!qY(DHZdazNnPCAEYrIZ^h zX7JrMCMYSjBO3_1`^U3MUE=W{n?kCW_2v`cTJ`k?H$Rv#0=>#HMy zFQaZM3X{Y#fHlF$IUp2%^v%0I)9Ix>l;K^X%nO|FggB5T(UJW3m^}t%k9suLR?j5W z=^ZG^?4M`0{H>vP{`{+Vm_L}x1fu`}2?S)WD6w(ye+6iY;s8Gwu9`m0)EXH9R}#vJ zJmKNt;n@mpE?{jR?#}kM!~F3|tL52dKg9dpcun9e;s_700Z*X)deOT0z-$mZQBiW} zZQ^fji)=~pP_%O8kCHn9`dWY#jye`(V?sQD6NNgDLAV!I5BG?a-=thwtEt+~GRzEf zHqW{!h#3WPZ7K@K8UGl-yE(tS^AmCronfp?wne7P*NKw3l#9>9*BP0Snfvi>{L;WX zBK_UKdJm3ln+v=MifnMYN21Lx{_&iH7M5FN7YPsKn0~z7&6mkw%OQn(__-8*U&gy8>)=|Cw7ahk+-424fZ| zxY8yO-%|N#mNZIYA(7pN=N|gBg;}BK+7t&lG_|$I%A{(>Pf}wknk?stBdA=s+NO6w z!9>_7z&SxZ$`e$FgWW{#*?DYMA~m1tb4FnUOy_RarUHs2WX~FL#rR8yufwUX8^o3K zx;YxYA%`RQ3dk3oj;650J7j69i;9bojZ%M^-!^@loH6p11KbLp6lt`}e{HOyRy9Xz zem5Yrp}Oo}wl5zRu$bX#bl&ErvoN0qD8_#v#JRh@DJ_xbF30==nCY>&D8HSze(YwP zPlbxMv>R47Qo)#I_mRAmu)ddeit>my8m(tqpkIO{gZ zFv)WLv8Yo>wq?gCSnuh>oOzK)>HbiN`6%3TEAz7XVK*RN>1QLo0TKhz;g`K!Is2P~ zVMi%NqRFECAco;OI?9}mM?2QIojWbv<8g979r{^B-*kIHw{vfigN-=U&HTzgorXFL zv;Dz^=w+C(5SyAqwkLmw?IaadyfMV+*0O8oleIWtnn061u9>34(SVrHaRwDN@Q&?5 z@m?#oS>N1S;)r(mws;K&o;+gjMQ^ytq2rmq_M?ht%MG8|laP_TSqBwsqtMclFPbAw z)>1%6-x~gqyg3Hs4t_+?@vfxmT#o4{yW>!m#lFcu+QMy|hg16xJMVuz)T;Y&ghx=T zS%;-wBC*lin)2^7GKN{#vNRyOeLb{YP)PvV^w#A-*_BrqI?Ug_@?*U*Jn_R~o$V4Y z35PKpke9>&7CjPxd1bf2cDmj7)?D9vhAY6u+i#)Kd^}@4YyH(a?=!H_Xq_dCwP_vZ z0Pvs4)oHLH9?ufs3P&xF|Jt0I#qapW=asm3yc1N!=baR-X*srb zygouvVA)8NZ`ayvuHUovf#CBOQCYE^S04ajJ-}EdH>qt>_`!yje!smt0$LMGBfflt zp7nf<9E%j=40s7^Wc(@%0!9g^({`dhbuAiLZb!m%J6ws8)H^|6>0NNe=w5N5-iG8AeSR1U$GlwPF4o$!;p%&0l8PO=*fg%BEyFew0w)UgKjjzuuAj|D{W-QCK6~+n4 z5omW5Uv#AZB7;$*y`OWh2w}B{ryilTWjhj8YP)>17h{{(!D9=0>l!V^ID9e00uX}S zw?R{N#w)p(D+M9kjswn%b@rGs8JrRN$BQAt&sOH5?8eg}GL4`5Q4V-Mf!mi#`Fx&c z;vl4=^bo;It($F$ya ziDBsOF6joP8%0830OrxLa|`YBShT&WxTR#HdrO9fUhW=Mesi4z-E3cAEow4x zj}PYeiw?UaDvT(cV@YW%0gP8zB)M*k4zyo8L~hhcc}_6X+CB4NqSsD0pKdisuP&to z?xqwAtslI(KlM3!0X+XVq6RuA9%7w4`dfH&?H7jyE?-(JZrZ@QT_&9qR*&qIWL)pM zjO$dm&BJ4VF`zI&jA5RgqMJOVcUxq2Z|t&!PY!s}iF26aZj|nJQ<#q&zBnCR{q3#u z{>3oA^BCKe$sosft8Bg|Mu& zwAP2-sAtw}kkAi#LZfj_*UEXs#Th<4QC}9Dg@dIgdvZbeUo9~xg>?cdl^;~QY&S@7 zmU1#<9tX`czC3a=4c%XGO5;V4tPjMf2VsO@ffwH<6ss197mn^HFrK~wl$>{?*cJRg2Db%Nt8Q+opu3s-+<3Z zdygy8PXUXM@SXRPfNdtT(5C)Xk>q{W-F;OyM{M>Z^ZH1?o*q_nmj9b$WW$sPC z5KPFG>)3Vm0R#g5`6B}wLil&+zMF9|4JF0`R$jpmg&>#xB@g*c!Nwx{`eNrxdg9S| z%?i0-BH7Uk4;XvnD}Lzudm*Jq)vnC*fDPRFuOgE2oM*%wQ|AEu4n5=O)L@w{r3Q#5 zkx+B{qh%%QH@3Oe?(YG8=I%m83Lf)>(_EarN2P1WkzU!ahRC)JaW@1!nVN7)2(3Zu z{cr4-FQLd$uOQ~^o#pk(7HD)h7;P};`ZqQKmF1zX`%C)e{G~N+;FGZ@x$!W}BHs&Z zh1<{fC;KCWQ+pxnNtw>j3gym%j8TfK^Qn$(Z~H=)q00FVT|yx_2PqN*YQxn%YxYDr z<>)C4E2n;hK$H6R<#Iot)|OvM49-e59R zy8TA<_PVfsB9X;y|I@1oRY+7AkbVsfo^1ug4hEuKh}}CRepOD&rRX2+v195tK^W}x z_FE*~O~BFqhrF7@kkr z-AV$ezla?OmRoCD%~xk-x(<+@)|_r8HAuJHA38NpTTkf>*2ZRtLh|fe>MN!_)+fnD z4?A3VIre()E&X|jRvxbl+Gb4svwh}^PUD<6&yRy#ZJKP8wFW6l6vQd2k7zEIGRaOx z-pU7)_pT(M<7k40#Bu{8Bs=4T9^Pc-v<&Vom&8r*><#_?%kQkyA5|_UwP;KtYBb7l zzuklG>c3WPvz#BQ*Dd6IsiAFr|C(2{sp^gWjDoz;{b6c379AW6i{_?-vE!jPqpc|bVw&B(^y=5>llGnKi3&f9v)9t@~RquY2oQT9DM;d zD^_u&ej*J6P`v>ltgKt=v?~hF&))Fm4)Dom-_G*Lwsk&E z#nEWNo=Iux!EifV?rf*j&v8G3>E4+PL+QO^+l*a*N~-~;>f)4dyp#0zc^| z{T0WhX{V9PX5z0LAI^h<#_}+i<2cEseZHQDY_xxhM^~P>eV4qoH4M$_ud6PVOO{BV z98b}-AMfn%uUKkxHWLr5gyF6|F&H_09w5*uAL<*ET22>h8YzcJ7cE7kD|>9ZeB z%fA<1$C!sE6$MVl=ruqHt*pE!7peT2o>;WsgR{A%r(9e%yI#Vd3(nP{p_5C+?tom5n{`X1CmsVbc6R9 zQo37_1y@Bahe(MtYYHR%8%ah;9Nsn@<4CU2hN|rp|chqQMP|8|rOJMbVW8+769R zO)ABT$jDP}pyOIBHB~8Ob83<%TC*QICy{#$>@I-iXg15W)t#~Iaw+-Uj*IE2Q5Y0w ztK-Yv_J*pK+R7@JU|_elBf&D&?ylHhYrGXnD6TiW+M~|=v!$ESuY|M)kB++h+-$>z z&!Q7+)TI5dH;E~pj2>>S0*u{yF$IZ_4hY9N2-J3b;BF*d2-Ngi)OnvV(C3J~FoCfj zPf2eN+by+^n#fYn$6=G^U*zXhcWx3{!Rm@Cm6cO0to9QtT>v7kVpGK+13wmkdSqzS zXe$ry#2SgXVNj6^%MAuU{@yU1{%Fl_U1-56s(KO_@$2e4a$|JtoB{1cI^RsIGF8XJ zPg#JxIXZrF1%A-Ls?kwp@c$cVWJd1zC6tWKfUN8b(-nW{+K`?=a6q@I+~H7i)$ z#e)?x{%c3m_yI%73Q-Ia=*SI3deT?qItN9ea#?dRr9%AEeO814<7>^({!&*K|%3@~HN`oVQdE^8!kOdj+}z^Ok9u$MJIc2;30)ZP9LhtN3`p-0eCqc3vcO&SX&;|T9oi~+$r^6$Lew){`TQc z%ikg|g0Kn;sg{J$0A6qq#!cTK+n>pow1foaScX!xBWIsV2o~eUg=mw)+}JTPPSV)S zU#n=aS+Q4gIPN17H??~B`2Sc=HvY4GQ%fE4RXMOrh(yA={gSV6tG@ydrsb90n~%%db#M3;(O@2Gs}TNyTt z$GEH11UNX3e`#>c;r<=ypIQcG-ReO~-sy3Wp*U(wMRx%i=Tl?gdqb#}r=YnDs1FqAxZH?fgJNodY(X^W zg)fJ&m>{|D;F&klxC)AVWA^rt!tag_=GFJ>d07?r6(1IIoTFTKV7jA~dhJbk-ar&} z#OGXb@P^Pq=yp=y=X^AFUzOEwe&c(uGc?nRgjkpn<6mLyQ84X@spT48Wbh?bIJC(R zV?)S?;Co_Q*SJhEtlz@oGAJ}m-#s91lI@8A^1h8$F0;h*&lxmR`4V-0vhi)2LB6$I z9QvaCc5_HPP`BKQiymdhUgm7-Z=Q($6p>d)O2f| z4#pF*jJWYHv0;mxmJ09m6loik(LstTv{`XOygTo=G8Ssiw{NTTi`;v|rb>(h=Icb% zWrUwP@N^%&HsXWQ6^mZ+CJyy@+PeDvYLvyMpU%yM-;I3qxl4#^#A;d3e*y-YA)xBA zV`lKr3CvZ=DwnAh6jv^`$k(0bJNX?RHoIjjWr^r7ii+8 zRg#>G{mvRE03FDy5#1DnN^3Age>v^E4BBVpb@%#^5k(irhTZWYNDX8A)12x|`7qK| zvm99|&jufgc7o;jbSu1gnBXpqSq{&P4dJiYm?{-D^^5h^GOV*fn|5tSR|TPKbbK7k4-B~_7u_DAk{&y9QngB2yj$x$33`aIBd z-0y0sWE+(6hcD6w@u(9qMGlyO)ZJbYd{8l3y@tPH@9!*KBvG*xc)CB+3`JzqYmkd% z0+6P%j@!M817#C^f(n&Y`kTBp8u9#Uu#l*9W{X$zA zC$tp(L86@v6Leg&Bt*B71K5Ka9NWNJYK0G$)8HOo{l&xsiiO>Ww^17A!tPQCX+`YO zsl3w378axF_iztCy=nJLGku3a=IxCN_SweG8)2SQW_wXQoIb0b)o4OiWlw~BRtl_{WIq2vz8Yz?9{ypOGh^l- z60p@zOv^6_bZAW4MNx0;mh{+LPZcy_EtM{oBK{U@Qc3|&v~i_q;ZQAuE(nWWvqMcE z8VaVe=vk#kjiKz(?i|PJ2y!TX1XZbx@PQEx^~c;-y%%@Di#H?D;%*;4TG-30ZKH>R z#+3NS6!{dL4t=DAecRBxY?e?H4@Jlz_;--A+-0n3=T%w!WcesHfyEH^6r3CNAL!25 z7mg^R#})Eh#n4`)1aVOoi?r_qvd9;}CV+Bm(@ ziq<$mMRy!$ycX zcg3bl;IYXTw0_e(5tnwQb7$-};A$ zIO;--Ay7Dldn#0BtvVP~lKbKGw^`iow(gMtX4$(#x_8cRvGcRA&H-Od=#BSq)2MOxIA>js6xJh8sIh};>Mge@-{_v$!kV=JC zRiVvXwYTl=PP5B|fMv7yDZlh`#Zo)_=S<0^HLss@h2q0ug7IOuq-$32kDh;l7>4Kj z&lU0LMIg(btO5xH)&J=HRbwJ4jcc!ZYEhxj@wMr9^7iZ!Vx=~$-4ErCe5V^;ps=Z= zlA5=>E^PXK$rat<&}q|nH0O!ndL6B; z|6Lj<_}Q>3?WS=OtF>-J$|8ZN2l6dyA^4MD^-}95h&9Ea61{i1AK1jRd*Mxvn&Mz(L$-Gu)XXH@my6RhCR+w9xW(KS)xbL=?NoP__}b!!FSHd<-33UBfRtOC6^Ym zJ@Yd?Z+90M5(9VE`&qU3GFLNM2sNu{RvxYkph_z4q&HKF<#4@j^k%Js(3Pw!SOEed zeRPX|&!=}e=M;Y-02Xt^psl^dYMomoZB@)HMQVd2RM10rCrzS@)S8|EjYWx zwf7~Oxq2iqdM`~MzGvRvBJ!K;9Lqtb3NYBdIH1}pw|>W~J=+-vuG|wF+f*)A{^d`) zu_cfb^ZQqwpO3)JL46{^Y=Ogy9KGdyom98*-<^W=<4S=Q*+Mpc?NY_CQT_`D|K{!| z9dZJ5xJzu|&HikUT4Jhv00dQeg@jGX2JAa;#;;TjX1J(8<^w*r za6JzTLFVS>2OmAnJE*;LKbV@u)hW_mRSqKl0+NYx?c-Gp0#)ERH6;FiYy7D!`aoA8 z69e5E&j8%-4f5uwa~(y?S(zH%WeI9Vvf?=0ULDt5>;eb^%K!-}LW;r$>Ivf1=;G1R z72p~DLojc|`8y(U>1gruBFD(*kXPV#r(m-%;r&$I3-;_%+9PObePd{9TOXmGN{Toy zlD6z;?*`j>HJP?6fzxpE2G5hVr3k@7@r!|Pytgl*@4NjU&cP_+rQcnTU;<85ebfe8 zd6~g?%!^V(|2#F8Q-q{Jv1h@TUr}`yk|qQ-9wryDwpSFv z6x7zMp>Q<0_cB$KIO>|rf`%90+x;M z>j}(3MldI34l93K{fC!Sp+qPwEyZ4IX}G8}almMv6rf$1%l`TLdza;tze0y6FeYs5 zQ*Do?@-qx2%^8foBG%T@X{p8>!UxH2 z*fw4kpA?VC6)`jv>dq9B9q8s&mFa#=$9wx}-6V;1ctnYe7K+urhjx~9@O7c_A~iu- z4NmE4kTOZ0r`Tpmg)mTKsy!fp-c%((jw42|85*EnMkhkK!*LwICC%V_9tiVtpu&Xc zhIMc%pmOhXPbKwkpNQM~-(cA57SxuEEHCV40 z8F_CKrU7bN?!Uu~8$a_T6UFM5XBq+UdFprLVbPpXQe25Y^zE8_P1^_Q3MZFx~{K~~_cHMUOPx>l@Rz3^NYJ;0G`^<^v&-;t@kaEn2 zC^a;fjkQ<`i_`O6Zr2St`no?sUv56PR{#jA;(GGy%}xGgl_`wsZ=fZ|EU%v(OSOXF z;w{OHPsm4EB%`g}>_5C`?Wc5Zh0k-1EbS%H+C2We=mZ~)^B4ZYoZIiP#1ekv3BgkZ zwNBf@w z+}vZ38}ev)9H+{FR)twOpLZeT{@1GtbY;5r#lq{^Dc=ZmkT6JcLx^FhxW)mUKC|st zG(et3ilS2bo1ZotPd_BF%f8U7D*)=Umi1W0Y528?!{(Lh8&zc#))qIo7V98e_&_AW zR|FvkaCWMY*uN`2#;<0&gEyO_+K!O0%?(S}+PuXJdkzZ%J&t%~vkXE8Or@GkeFn9M7 zjT>insM+>Es>)lSsx16&#BX+3{?_-keO0>bL=5Nc{|l5rUkC`r6%TtMjs$0S=ti<9YW9V^^joa3<-jl21BFK>kI5?&M`K0vEHH;OSn zXO$fAMY)!Hp!2JMzSOdxr=xJ_Fv6pZNP zHIwyYbIz!Mg)=j_5>DX7CSe36v3r-=BtFY@IkTm)b8QEwzi;73C{t06Pmu-!(J0^@fL ziiEn(bu|4%%5r%WI3Sjp3fiY?4b@0OV8)CrjAHeQ7os@QZ=+TkW-TmYJ1xRy`m(Rw zeIA7OX^NZYjl)r}Sdh@OFr$=vq^`kD;cKZ*dvY|n==&sTw%2CATI|SP2j5S}U~QC+ zz0;H^GiYyW5W3k|GB|_5Iy2tb{*++v-m=ytp%(jb?Y0?(5NWZ{yfTFPBp~_N9q^z& zDsp+QzIiieq`FBqa`RztTdi5zdun_2rCPHWj~gZeZixFs!J2BF_5R1I zf|(Sb-ND7^^NvY> zxm#v3Jk{LCJ3l6A?{t(IbXVhP4kffM-6Ush{I%BH2#952#9VApOCmY--L97xoU`Ka za}qC!t^2f;E@kW_>VnUWH6nniNB)|^@PV2@wL8IZ<*C8es^>y0Mh0^;`z7|?1K)9% z;othqrByY7o^j-9bC#L1sy+tIJf|zy!x=?+r4+YFxsMH<4`n4IKyntIRPKn|{>}=U zeU94GmLl4}6>VS%MfZV}>V1DW6VnhvB9cEuwu<)Rj%j9m?6?;n*H_M>ZqZ=^s3G6r z&fxHn3yEoGKEDm5Wo1>Ig87g{vC#+K0S5HREuz(Xrn%%9!I_LTKq?7i_23d_{M_xu zKK_*s7{{H@ZlO?$P{ZL8$!Wa;(Z*qKMM^%RB+yY(^aquXguQ*W?eDL7({Ek|V5af{ z-=!jZP9C|4>1^0A6k1}b1hud~tVfcTzgnuR{e1C;&eP!1K4n}wUSDVVPO<}=YoE?g z@=#zIlJc7DtiX+ApUL#DqljseVteS;pc%ZRc>vgIxlog3JD&@m%&x&Hb%Xg8f1L=t zwrsDqQDm7sqNO@r=i%CSI9=tr0Evnis{zPlY?9)m7WRnz=&-9B+wUr)S$or4>0?%A zU?1Yw+7Ntc`?@&hT(>J(Ideg@%qj9F%(G_gHM@+oCo9?LY5rM@3xoMa3zy@HkV~Q5 z!1$AlR*_|t^~T{P3>7=~|4}9woW3ebzEQpR3OG?#hd7zQOCdG?RvILP2yMx_naW`> zso1)Gf!RqTddKPnxL$2*mpCMOu_nvNkb7kNpy1%ip{KoEdm)8QG!2o{vRx)=4RG4r zHC;?loh^1q$VuTyn9=8}NdHn6|K}BIBV=2(s7y~fxl%vF(Zn2q65UF75#NngNB^x+ zyNIE`Zl~J%A&!*+g%$rUwY(T~*j>b7+bVn$zSeYPu0H99qieg_W_W0~g1f4Q?4I~a zeD-NeKp-n}PeeTBC&q_d1iX|SJU-u?TpCHCVRQ>zIVuPbQMF+fDD9Ka@ZhGbPo06t zj9;t}cHY(?{g)&ru4k~5ykPG&dsl(BvXz&C552Znmu{VV44$P5npzNsHAmw3Ma(|$ zCqF-dXzD_oJO|SvF+Hv@O+*DECYoBh6X>)l>kG9hTyXDu+nHRS| zE+nPqOh?`k7gfzTfnwU^f;QF`#m8X905bW7OwclK$fd(qkZ~8#Ma|1 zjBDfg<5uQzQiODX)l=eMkDOP}i-t$Z1EoPMzliM6=Ou${-!L+cS+t`HAN1Xu3TXY$ zr&L>)pH#pv4m|c4@|FZk8eQzQfeE4Q4e_@J)&1EjdJc}7 zJ|A^eHN&-=W6ZTpVj?02@ZcSFwYg;a=Lc*Vy60y6+sDs0Oji6rI1CU(p_^cy=i{Kv z8iB&Ce+nA<%0;8UvfWJdHGAab_>uZRQO;1Y2+5Wf*9*Z^PHHSdjuTg&o~Dh&gkJxb ziwI4_#!lNj7+0m=wDn>-{-~R+f-Z>?w9|L{9hdaS1F>!F+cbq$~1(-$+ClePa@di6i78T zQyO#q9FqthKIMeV7}yr%0wc@Wba` z!g!b%G2Tf&QjEl}BQSIlBHndt)y1-+0bodvEk)EN+t{Tvc)6V-o+Y&UjWD zX0G$h&3H=d-QxBjNkzgUG+zERInZ|)N#b9^DT{C6vo`7vUBIH>|H?DE+30=87t>1I zm^5IgS5d*}Mwpa9(VDY2qdwyn4awW~dzuQ5?7FNkXqjIZ7&|J3Dl*@Sp=#)o zE*OoNVL9bY7_O{Xhp?WT9Y_@4t>+1|X55+?G@R6VCvce44oJ(J)_RTI*LK5a9cQve zp_nbm5D%#vlJ*az-k$yGMuaT8Xw%=?i*WBe- zIA3dbk?JBdLM?h!dsATjK9l46aK+aeFbsQnhy7@o)iN`!49HlXr|D}-1^S_bMUMo= z7uS#x^X8RI;N19p}d7zk-eX=!!fm8$tum|#z* zl*Ly5Eu`xB40!g~@-z#!_-l3jwrUFzHhV!LgEmVGuJD(Le77yBn8+DA8H6%X;6nZF zS-Z^bdV4u0)6R6(MR%h1#tQw6HaCNN?^ia1S(SmhCGCx5`_jhCA7>-Motu)a28T!4rjm=z#j8r4E#o)wVOH?-cW7c~IW~<0N+Sd|S{~_V*ju)wmx2)=G;*@lR2IR>fw^y(_$Fz2!>P z1o^H;WPl7rp83@c&*^``W#oJRdyPhi#cBQ6h{lcJf8CtL7w^j<3XvV`YZ3}SOOB;@ z7fnQq}2(IzdrN5|BT z;khT@`gbIDZzNJMz6OsTF>^Al9!L^fGyO+A=tk-%kcH#@koIDoE))u6m*9q}F(GBppA%dhQ;iskOZVCsS z!Q>KN5J?hI6y|Ic3%9RMjs9EUBi)1~YLve&RDO0>1E{Q*Y5RTWsMZ`~6a(jB?QQb) zmxI=RyU22EO&PH_CF`kV%oI)(r$pPs;bL7{4toVs>~b>xvHJQql{Yuw&H6SA8V8iO zUE>DoYAFtkE+boKf^#XweC8X;)qXy#@u9UUb!Z4|qD|)_EMmpeQV#-_XncG}3Z>2g z#0o}vR0iJ+NgjfPxJFc`(u6*~s{*(*8d3jt(`5g-n|_`|M4qE%_e_UB43ySY7)V{w zTNXTV%>!PrKwU&8irx@siX=D7p^hwsR;*nmOCiaRPfqk7!~3Y!MB=LhvDXYW-H&aB zB_)h|dRjX3VQ-x(-xOXd-M?M#n)d&_cn|+b&N=n(qP&6d)gkyt@FAv2x8jO_R1DN7 zUL{JsVAy}J3E{dRnFUcF*hJWJ(T9&C+8YkZAZdWl=56~1$Y!p~#5T9T8%7~gGU=0MoLZCk?$p5((0>(Al7g`HheuP|u55Y5irZVn zQ$#}l=7+y>=GS|9*(FV01|fA+jOlR#`G!g1;UF_N;tjLIZ3~sr^2!uz0uVLbG+-brbuXA^7~*as4SBA$c>q zb${AXx;-qD&=dD{{)={pjSK7*<(6R@PV9#09y!#ZEqFRsHxb0Nk({|ELK&Bk%agS0 zF9ItgA#tM7TYCO#tKr~A0edultLU2Jk|_xl2EKUU4GHyvL(uW_CkU{jnx#;w;u z?cn*_3!ySwx?m=ItBo$= ztDUiRn`k=MH?(kvIcnho#96En`esC}jt2MLhpwAo4A$YZc*B<=c zYV~OeH%n~@+#X)7iPA5lkunIeI{RhjFp}bYgn5DM^DNXq7MFKH-}9@t*kn#FtCE+E zwwWbLw8o8dw3u!AAJK}~r^!2P7A4m^-Cg!sx|JRLipRbwnr*JEZKO4_@3CvJX`Ttb z%`ddxgc>E;T|HO%7ET5#11J5iKHo>$Q|yy$vqUpI1>xV+Uj+y@N06>4>R&gD%@Pug z7%)Sf&bs3p^?8!W6Fd(?ohT{AfSw3h;D5d7AP_3$7jf}oSA?Kh zs8ZY(6V&%-QhzZTWEz6LPpeMEMM#E1RUWanSmG9}A-2?MEhij+!iU4g$xvF6cVCzv z4b>9<$)FnRP;!xn%ahn5wc-`@<${oFdyJzMSESN@Phu3Gv!UwI2(v3zkg-(#<>@f4 z?x*&QBvzg#hTE4@Smi?Hj=gVv)?#9Up`!N!5pTD;vqW!TzEUb68L~=B{&i+yh-TujzUiJ1B7ZK`lz|p)a*kao%+yf=0la!#BGeL(*@b< z41t84*H&J*%tL#bw;o2o{`a}F`9G?6FM;{kVN5^J{C|TeT*}c`78-HH(BbpD4@{;c zQua-jESuGiYbv%grpWq8ST9Up9=ujNuJ?^puPO4!y59f31Hgll;ekM|o`86V2D|;+0BX#{d2|FM++OE>a6a0T^Jn{^21`X`0Nivb@0 zkY=iloA_lp1R3sHs@;ov^sPEZadWZFe%OuJtQ@sZDjaej3axP}8q%f&7AHL>{b#XH zLvbi?E_Gvk5pi2TJ5#IUNWRPUrH^nN6Ih|7C09Xal|{v_t}bLWo=B^UU|u4TM}USh z$CHRlXiB@U(URvZtLnFCqmr>3?}>?i>NEhukOPB-K{h(e3;ui3%S>qmo=F7C#h-J{ z&4D~(d9qspezl9h2$OZWr*t3c1=Bvd1#yb}tdLecWep#dK2_r35V*lIgBg+%Zqv4viLV|Ar~Y8~X~|z2EDg zbx_7yYh#)5BqX8l)9^P-#0j!+JDmb3s}rtZJSw~z#kG72cJIR)bQ_W zS|QU!=V+H(ucHb?dL2%ZT7_{740;X>gWWp>2WN^%x>rbwo!yMe(7B$PUp)z}sQ<@n zYW-U2|L59PGO?(s`;r=dlNz}FM^-_GkM6+ZS1ZO-NM+276aBSjr4}b|DOEXr=5Cv_ zel-xuQ8tHh;$4`m`bup-dyjqZ>cdrG6EEc;u-Scl86ZrO1P}rug1eQvc<2NQf1~@m zt?ugP3`bSk%J+7vf!TFvEwGo*exbnxo8QXLVP5vSM~qV9yFOeB)t0B!eIJ;u_J0`m zHs%L;ElL)XiBD7*(ut~}%(!J5*JKEpYIa}P=!0EEZL}805>yD>t{k)X+mbK#-&LEj z%GoLFnrXs+#QakA-ImOM{Y&+Xn^IbZJ6;8U&P%<4Yb+kl%jRuSD5|&H_+NM37f~Yu|9>?b z0O-SNvSX^S1E4;WS`evKPVVdrUX2@<}-AKmG!4Ju)qm zfXBu|NPk1?5w*^vRx*?mBugNJfId2@p*Br{7lNe>;p7f&t2;p;j36oubTk{`E_N?4 z$CxTO{eC2v(+#yS_}qtoOiQn>W5~rR5gW+49;2JtyHg;;ajSuftm?nF7Qv@ljc!eF zHb71*_(4UjH~1KNq%eE-(Q|RVoTJRlJOpds7(-SDT6;@2EHTO~7n@i1-EldBDY*t$ zjzm;FY8AXBy@4u!?&XW6W0-^^N#)6nrNDwZL*~IzE*_o0Tm>n?vjO#8`y_q)uHzG= zpYb|zyQPiDJdrT>DOEt0NC3LKn^te=Xohh^9aghVNgNMq%3{tJW5bG-1ZFRVj_%U3>3Wk(j`R#)?l(1kXBH}q?lS{Lne zk?De~zap$dbl<{%drUY~Ba=gZ)W#zH3VSmcstNd7N3o5$?VAsAA%@y#Z4k=1R;|uP zJPsbxD22D^3@@2tVFzoimO{+}C0}KnV%mpBq7U=wh$;PcJ!gHTK@e zyy3M|GEG3+NVs858E&~+PLAu&q&8blV@l|H zwzK1r!-T2VMLqfJ1QLG~*C8}|eO)fZWN#X0b_f9LE^^&g@moyq*Nl9@XB;X;tr-DK zq#1r~jYmd^EMqn3GqFiJ?vhb1%22*KH8)j{#0xQPid{i3J47`(4xZT6`NPvkTD|Je zJU%{bTqp*BpbA75MPKxL=;bntx~cx%3G#@VwEw9AzkmPrArEx6f<#Z4h?hd7hDqdZ zT=*%z0*$R)3Yx+?I$JU9xn4c6}oI4DQn#i8X+#S{u0}WP&tSQ3;D} z%|;lKvq|Q}u(7=qtoa>SlJ;lg8MA$~o^>T{x2kaORSUgMN5~cI|K5;U^~kZ5x2z?Z zfpmU*O__{E;oGu_@;W-jjVh0_MH74~O&LL!{5v!O1@63Ts{)?$9JiAtAaXvf zp^*FjaLGrUQ+wGf_FJ3`2j}bYXHdd^SGe~IzuL#KY>t}|Y$?oPzhfNJ2n>wNC48T6 zNaWW)oT9TC$xS(~`$O~U!6`|LOX-}VYE9E_a<~Yd1pKt7{(E|$5K%r7r~cWgEoYB~ ztCbN^R_#O^hjw1)@I+vXD-^Z4m*J3HU_A@D`9eIrZ9JY#TcH)df7D{wL@Z;oW#`rU zOyhqu&fk4cSEc;cXphf<0kD^iHQ;OA#|L{FKLXbJ!UCtd^it#cZ91M$M-6=$_r>Yw|z!(H#?>5m8CXjai01NNu4fEC45{XS9B zkPEH~RwuuBy*@?1r%#r%=!Ya+c%cS|bHNdv_>oO?gkE{iyqwuR#-*@`V$sx`;?@vV zs8;x`mUHB&jnD?slkSX(?_Er;X$T#CRo_D42-}fhIhV|bo@;3ztQrG(BrHm{gkmLi zI1KPFQpNF>8g8X8srXs~L;QKIQa@5sZu9a*&S2^WmQ0rA7e0Qpo_OLEog>2TD zeR;#hr#1nQc-Rgn-YV6U?X#w6$;(y?yZn+`Vf7kFsEGEpFlWT@Iq>I!zT5K!;{-o+ z{!0YO=gS=9yvXbE`5a!vv~Ngx>8z1R zAs~G_?LU}@b{6P>?|q}7@{|F$R`h*hw=EY(?XF8g4;c>Titn3zA2(A@yEkJK2g60G zY__nAeT&UgALE2+$eJBel+x0ysRUM|dEQ_jJtJ#sMMj1!ClQ&1qA@&BHl)?jPxv_BBtAq?nq=*Qy(WbAjm z!#7$|N|pQ#?c86^F185_%(w+t#LfGR72`aOc`x&^mg#?*6Ky2pOMHY!Qt-B~UO;nA zP((!_t?w<2RYnOgh+rMEbvJt{&QJ!z$2EsGjBjMPKWm7-NBwUN5e=Xj1;dFCU}1eg zOnYKDhyz!BMy- zZ(}P`caG?IdI&Z~b8&k&Bg!@DUeg+K!z|$n36=PS|$YdX|VFJ+`+&;_L&Ynplr>n#{6)#Z>aONuCYf;&i z@YAFz(pHqjt6x`MTomH)ekR4@%Bb7)>=((NPQM+R5Qz3J$CI2#{P~_+PDrmi)3&w6g|Q4!x|$x$ z+0L`oblt_ixxbP`KszPnOCH*^dqb zweCIOMojYoelKEfh0d{{j4DBk>C-n=p?^j1z69)}y~k*>H3fHTb72pr~QuDW6n&=^<$d)NS*!(43+j{5%OyM}=hX{eH5W%&GW z)pjQ{fwO#c3rU%<^6w3z7aAgA2SE=eGXpVdPdi)XLp=-4HDgM}^mHT@f+hg(L_7sY3X37G(dhK^Q>mQF{@j5m2vlNe0U$|EbuwApp zUf~dC)C4@L+EJIDIs2U+=yF?JrcCYUA-mE>*==x18_Drbc5@qf!&a0DJuQ8ovS#fn z^aIqp2deajW0sDlFW$O+L(1P*N*UBX+a3-MMwY+wxhr2AkBT94Er_lf)V5zJb!*x0 z@+mt0_OhJDhD=)yVD3Bx!qXzA-$Mz_q5F#~u}Z)dU}_I~=ZOVW5ONp_!ZWV~ICOxZ z?-*EV;UTxbk;3oKxgy54Kul{G!?1Eb;F>zUUY{-v{u!R7c@RxAuoLth0k1dZEC3=K zhM5Ea%cBLoIa=@pNaQ{3>I|c2HJ$k^TyZfZ#nv?FS~bxTRMptV0Z$^W2Yc)aeWZXb z!0Dcl!+vaaijfa5OKvm~GHb^kltu=>3SZ6(7tKmiu3LKD^rrVQB64h3J1feM z$GG8&lXrKc>XR|k-W^x-=A>r>2uk(iGg;vLN@|X1sL=5tEw;!-$XM!~p9FO%NiWf+ zKetO8Y~~5Bo%B~j%{JSuW;Qvmnm4wu+@0RAP^$pPMPKLyeVd#(b0oKcj5+4T9G$_1LSxylh(_EkLX3>HO9qrO9x8H-hecxQ5d#%oB zn?FMP_eM~Ielq?l8CI<6PUf8U`!nD%U9s-={-5UlD1~qopN?`(+D6MpsQs#48#9vX z-JL{O97UGOPx-7#WYhlv4l?`u=Q}rGE6dEr218bRNB*<(-<-0gz4P05gHaBJJJepy zXuhnb(jYC5SS2;m#S|)2tMhfG#0^iD(c>`cqbCY=2#$nq57v{&bmz8X+T*5#Wl&@WGFZ0KKqvC6DU3HAI|MJ|^HZFSzcKt1>RcXvIvp4rkc&K7h5}TzJ=?=xl z0{m3qE701$;#cQ+nBI4r!s`)U(f&_Z&T!ln$z9X7D`Pc#Nitpl*z!;1`w5k4lr&|! zKYFRQnyyeXE)OlLaq%_VKC#1oX zJE%1kw~9BfW@df|<)g5tb=_@Rb{=oFwZ(v$WSUw*M~MY(i)scMEvhr7pV zeLt$l0VkCGXjqny$5$%bh;%XY$iwXcX$K&lp-ewUv1VckS7GN#?+?H%-B@ zlQV|hD$3D5B>x{_UmX)?n7vz|KyfebQrz8L1}JXDy|}v-hvHV;-L1G4cXu!D(Bd%2 z{n*`~?A@E(`G;gOA(P2>-uIm6JaSBZQ0^gg_(WE;8QUQ^8inQ*(SF>(|K)(nBx~-$ zn&DLi=VEfF2UdF5>}@|hfHi~~ zs~ig{Dd|4k+5plKA?%Bkk3q&IOR`%0!N7QNnn%`hR@GK8{&#S+F@r#>-7;y;a!xvm zb4`{|kkaP;Vx)vQQbX zrb!*4UX|#V#Kg3-;Vu>>ssH>Z@cboh`!ZLK9Q>2Jssk^KIYf7(FhaFlTVwy}seZrK zcJw0;I4`2Wv%|B_Y?jIfVXy&@GZ8RIEG4zxJ)8zTrL?&liu#A*xTGX$bJCE8=xR6a zL^f7zw9B}atCi^Kj_NcR2Hs-5&m3{Fw9=pb2ZPxyF?T`K=ZGmxB2Wr(k<-TQ>Nb3< zw@`E(&$2|NPUiEE=oVZI(&68D-+4vcTW%Y*=Y>D`9|EGV;}l#&_-!QPa8X2y;RM0D zzZ@EJTx-i}(YBkP2SZwd9+CKFvb;Pp_8W|PJvmqfIh{+QJwnT?-t@3TC7+q`c^M5>A&t|bHV3F8^%F$=?#iq$=7@% zV6WBSKDlet{jSA(KlcEl3WiA1Q@8Ce7nQI6C5~HI8i)JQ-#M&C61D1G$%kSAMAy9D zj28DGTPi7;uBuJ_>TNh-SZPd3KW`RujEL9^cGxMX8#Vqrk`y18PIm)@JTr1_lpGDk zAPLOf^lRu%b2ugek>f@hoSe`jCYhC-cq@mHR`(x25iu>goun!HZ1JYsyfm%FU5>2U zQJ&)L`gQLm%XaLK&oKtjem(${6|Gj<9pk~{%u8z0G4kY}W8vUg!Ax(QZQ!;3KRhMi zoy~tJl}XD;RFr|iC(_p`;0fTWmh~Cg+L0I} zY9tVzy#cqDQXOW=Pl$}D^wEH`MiS0ON{jBS@b6#ufwizZQE-9Gu0nTme!{{|@%;#qvKD;@QCv-h1* zm!dvLn6tiZtbfai0=A6)|0;k9mE91-Ohw?8x`QK-zKXWPyl!%|HhlNvEBvR{0$yO*1 zuswuh!#}5h5S`)_6)7Bt_nu8Bs>!T4BPup1>R7}~&eY>>R&|ELwq0JX?2Kj-ody3n zLXmC!ss*376ki~)o|FeG6Ziy6T$nv!e_#zob=LzBi!N<|uva$xgKTl}VOkQi-o|=U zsi3SD)kM$hfN=d?0H_j8GLpgxaC<1vglL+yZuGGKL!yh3-gd-N-IYaBnM2{V;&E{7 zV^*w)9))kE9pNN|uP*qw?KC_#nJfrcoPg+0nrxG<7+b_`;FhqO)d%O#x(A77A9Kv= zT}MAYWRu6lVh_YDj&n(=mr$di;8nOHQVus?%E5gUMLn}!dC@#_TxDo|2;o({n+61z zLByQzrjZwR=kB7j1PVdxykC%?9kSiB#UCmrH`DoLRv%x?6F3BJ6Tw78CC^$Pp;O~I z&4T}I1_Spvpx&+F`){Z$Jcr~83|vBe5Cdak3L-@Xs2Z`Uf(LvLZb_ia-IBPtzm!ZG zGB~K8?jsGMHAHdbnH!g?Xd*wkZ8*iKmZ`~3ip2U8_~!By=?w-x^R5OX}#r4ITGNZ-|L>UGE7kYPhi{Ru+?P764M0K;2w3l%% z(A+%^w(L2<0J9Es>7$HDCthka&x<>U;`&-cIU-;lqULqv^ZNW{aM|gg41wI$?=LPC z_Iq3%XhUH!kA1+TWY=Gcc>b{@j+RbV@L?N4;B})o`L&CI#y zt2v03d>~pqDc#QZkwt_vuWI*d4QFvNiFR1SX`dP`l(n*2DUKx=|Jb~(pZUA)(qH(N zKbNBoPp3Vi9&-$sv~9IY`I&Us#%XvCx|#?)W>UGQfG8**6e)J#jGVob+-`w_vp1 zTP?QO`@G#ZJ%YimgZd2honNYKH?PY_mb{0V>X!^pO@4_49S#G?_lus$FrKY0v=WR2 zAVysZ>-myCPE%2zP5fF{CE~yk#Am95w2bHSq>bTH2?nI=t^23WJ4nhcwuGryCy=_> z`HU)UGREu9y(@Qn;daL%1f1X+GfuA$u2AcVJJAlok*&voOjA|+7n5u*iy zi`vPjQE~1V^~@gUTKX6nS?_xCar||eCI4olH43loE=$EZs=^g*K10;{)sNu)zI&NQR*#A);`RZ4Zx0AXw-z3|&lG z=p~A4*FO0IrRynAJOi5UN4Y41hvRFq)BfSJo!?`4d;l|}NUVj=HXKMW7v&G1@3xkC znIpW#&>b*5EP#D=TE(wQXDdqak`-OLQ5!r*&DJGu*XWq6h|_o3d|rmfjy%uea>dZa zi8&fueDAFvooO^$>JS~zjFMB&o-BQO{>}h$+qn*8gG=3L7~Mnej5(vWqF z3>9)BC%|-n=OoH*O;%vi?Z)Iz=cd(VuLqfM?q2HQTB5#jP`dXA1N!^axfZYgnoDSy z{ukEvXAsRK51epislR+AD5D+s!@W4B$gx)F_HT5i%rY(#pGs9I zD$r*o3Yo`}pGL&7rms<%NPBZUvYf1!XSD%Mry)H)5H&YnTjjr=mf$2cUqO1H<663c z%B$nl;G1=_(AVriRqe#5Q*`h~Xdvn>c93dxj;GqDU;Mj)>A2gQb!s4*LCGATrNpCg2}Gi z@8?Bne_p&S>+ovbs@Q|kmg;sv9QHiI=k2lgUZ zC#SHplh61fk_zH$^QBE=7i|=hKYk+k1@x^wxGmK~I_NU5jxOw)ppf_L#!$G{l=dmD1NH-87m`}u}3+5SgNZ_nA^^t7F zyHtcTLjLLaY*vuU=Xs+VNFph!lI71HT!uQry}02R5ufzOLC|Z0K>qbLnp;PgKJH`S zUUv-?+rJMtZ`V_#&_uuTKQt9=de)4OV!i<;f*;R-m zSdj+9*>z+;V!cf_ia!)9-lraz44hJeVyCNf1kg>CKf@4+8;AleJiSIrg5AZPiczgkDDIbV>rX8EQE-m)<6OgAYP6%L9zx=@Wwrw2Fv%k1 zA%MLAEz|nN8699SQI>9KojIO{>5|$@s}Vn1b&g)z)XKqd^ir+ere30^GSPld=5chh z_)}-0bM0z`&(5j^1<{k=P~*4vcu`d0=1vS1=XAwVyV9GUP*?wCAG_)E`AUAMDlm&K zj_k(Bi_AN=r^$A3GVDFJwGDs#K*tyZfXd@m0FvoA0DVwCO2x^&p!E}FjtH9^;;Fi$L&&3&D(NA^{j>?rHK1r|rnm0$0 zN1h$(2?MP=!-$HBF8vLQUo% z_DcSi9R%LV2J<$&Y5IzNf*@?SR^}}ucX-?8J^82fV;r>K!|i7$(Y{eA%IusU@#7I6 z4CT@QgVzk-Y3MQvvE}h-8@l@aiE!7(O3f<1&Dbydo;QGAdW%#+!p8A#(^Wy?4dSm8 zv})UzAY`7IUi^s*wUjU^nBPbIfyYW$w~W(*T5Pbrq^%R3?^j7exn1Z-y5!XF`R2gzQudDdJ5*<%#a4 zgNe91U{q2S9iQQ_+8v8vRDH1cAY7d7m!)$)@V(Q3{tXlO<%f{v+7>_AwiJP)1dwv6 zkxEBcLhAg4BpxUZ7B{GeuH&%E7r`EuS4k=C)3VR!2ePf7qyMSiZ%nbHSBpIoP&S)* zk;o$^UcdWZ`(T73doHu=i)OC~fUr-5P_!j|WzomcS{G#qc>(;o*jK3>Y@Emodl{o^ zS``%N+I$HMYs3j5UaG}lx2K(tYFB;9j$_y&f=cuvR2?B$kq4v1!AKQYGru+yyeP;%o4 z^DxHn`N_y8=2krJ;#2Qow_ zYN0Ob#B{oq!?K`G@BoK=@u0uB7w~q22jYB=P=G z97p+R&y5wvE{@T0;W1|w5o#cFUT%&t-{#M*j_x?`6#R0X)uuMb%4pt@PKDC!!YDtg z*dI=u>ib{5f6I1D^d^y+2bA(4gYrJ7@6%_(NWi)B?+(L?ymBlsb%>JG5u+j0KK~dj z!T}=J4m>L%-%S!$`3%FJ;Cnsr1NQW#>RF9%jabk(80Sh)A&$FX z^2NaWG$(Gxzcz^@trTM)+JYl>`|GIK1RSvuOM5YtHD_r>`lFz%92Nko&Vw9@E*Wsr zf8jM6h`ed9(ug4Qf@(w|UnviLoA#M&L=$&I zEgs!8@^E{ibo^T@b4y++6Vk$8%#X3jhid+YdV$D>|Nl+@6IDtbbT=YM(x^65LNcq?7?ED`TaEE* zswoAIt5jTe99eT!2r(jSqLn0OB-_S^KnVEPmHV3kiOJIzr@cBxjp42NvY|=324{W0b)Dpdd`*4G*&7-Q#(jvYbZQav*Q?u3bb^m$DK&w zhFyWOg`rNYH*7~N1cuFl1Y&JAcWOV7EMR6Q7=XV0--~{q76@X3j+Uzm{)LOkD}rm2{k4zWO3C0b^xq>EzWi zZSg_^$pV~4&S7E%+o%`1V|R)LBv%z_nYgy)hqmL;(P!^J{=MU2aHq?KXU3uix;(GW?61_R+k2`|FQ-zRBP80}EnvYINl zlbhykgaXXtchf;Pe*s-9J8_l1^SGLFi-?#b}vc z-gV9;MdI^CPoslBCq4^{t&DDW6{EMutS`QhH43;d{$!Y^Pu;FE;nP%2 zZurS=*TP;5eC18TvaIIhC+km3&|mf8cCdOn|Fc#8ZyctZ5oucd@4<7wn3f1k9aInu z$9YZthZ#7S>^12^@hH| z(+58(B2u7hkL#t=hzB)T4VP#PO1=N6&&XF#Vnpph!L+h$V#zSKI&*kVqd?6F`D z`&sX&l^l}z#+1n!b3ZQSgfM%z@cwc2E;gbOWA>PCEIwJ!hXb`0oOrmi{-LCkc~f{Y zvd9N`uASsYKC@q}P^p5lo2gHh(K>7%eGvTTn3UXfb9dz zPEvFWR{$=HG?K0_|8kNfu@NeTfN6Q$bkir6XkWQfVwNB>N~Arh0%F7n%XcQ;{Iz9L zufE48giV^wsXackSc|%JC-%#`2XPPVLa}21kg|kz94+OdOxFW0=b96m>mrF1YCkr=IWVK+;!8F}(-|&z);VGvu zC3egk0Z*fHwnm9`rKe}J$=^%Ih6mDDFV{>aJ%IM`__{JqS=QGCMaeN-ziaim@dp)W z8^p&QR#E{Z0#NCGoS1lnI;?^;DQJz>Wvq44g~h!od9=t~VP2*6&nrZjC>xwDDCTO&@TzB%{ncc&2S>y4XhN zR7;41kT~M^r1yp4J(Pn8B_jzZFc(;RX>12#D;hKI!=&tY z4nDO5GYqv@~1F_dYhHz$@ReKl0#$FzVsp@Awg&WYSbP5k7r5p z#o#QQkYx4wnNrnMtKN9i-b$4`GiAJcp|GM$Q#O4Y&dS<|GT@$r+Z9>OHVVyxd(!0; zMv`j!=4X?`;bRu7V%!Y{x)C^|!Ai<;7GyZnpqw$5=R!Y`fFxntak@|Jvd4d8=%b+5 zEIOc`dkcSQ9(8`)!fW1UO*;{@klgth)cCg%d%{?@=K9BgHI!L4OCer#(tB}2@6AE1 zFcmn~h3wL|0}A%>ZOnV)-u=&-<-g_N=&*79Y-!Xl?OpC9G(#zyM*_if*In4m-X-Al zZE5JfNcN9T>xdF~%*}m90q0+PQnsTMK#ko7YO!nUrh~MTqvnW=o%||Gzj6x2k)QPI z!Hk3Dqs>>%bUpUsxFAh`)g3IIIYkwm!te{2d?FnGZHiQ9ig;bGs4&}G?v$y)@+UfhNBl;A`sM6)bXvRqO&KR2BFh+;E2;9QPEy$7V&_1!uaYYF{ zo55(vb@@gYbHkor*j&f`NE%;=rKzt;op=fbLB9;GjZLx;HDFhy_)5!hUn^cLbqb;d zi7BaAG!CT(xVXrm`VTN&^1H$teEQK0M+UT`f;{3Zr)@!kzJq$l-V9dvcCUgjB>;ZD ztRu`PFB0;1C<@cWb$@sXsLem2Wd9c9fCERBgjsJa@9*@SkIbIkBI&&f!^o zOy)-_X)xl!(uUoWOX3656)#9C|Mh59w@ZtdHIJn{m;2@)7V5etiig9qwta%*Mm4-5 zHIzmlb}%eN5HpvLli;baidf_G*>!Z4CB3+pfli%s5&qz3&72eXJMUvvqVx5`T6e@XY%FOq8v&;N(b~BYQx;jlh zYw}1m8|U8eQG|X}$e`pXiQy=QirgJ5Gbg_a{IB2FAH74h%@5kZ2zi{EUSsEVHX7p1 zEPMerOG;qKhg}JLhiw4?0HwXwWrq^H_BO!9#8)mJ<=9Mh!phlK4$ zdVt&|+Q)wi@3{W&g#|9(60q`|$m|Ej;$EJhu+lgsSkv|q-?(aJC(vm}4 zQrzbd&wDq3DLL0!vR!kIm z?#DD6qb}S7b5gm|VM$1e(XBrAeZX14?yMeXM%DOf7>YhXlBoB!t3^KmG0y~R6WJ=6 zP=$Ey3CAJSNxk!jQX<7XMVMeGqxV9lE_J0Q@dG<2qvh$yT~$7bdu&s6pbW2emuAwx!*#p~ z@_5j%LS7eJ#S*nPNUsnV7j1WI#$x z7fPG@Qw{f*2yejaEDkJH2#yddjS=Escr1L}iDBu=pWvIW$@3&K7GUS^W_~Z4c&0%` zxy6SHEFGweDa2{EP;GKPjBnJ1pgST^|JJFqheeNKdke6gPz;C^1@g)<_ki@ovfTH$ zZuyb&dL%p>JSpZzW-p!LSSGi@B4*urKC$uezfX9EW?D~MVIzLuTV>_(gJ^4a-&64! zOs{EOV=GImTcphX5$&j7I;YGT>JdtitT7;}J9p~~j(*xhVycpTakm)IhYzj<6H^5X z`}dNLg^e*{@Cm=O4}%kx7ABlRMwKwh)gM8f35oq0`}sKfYI!}HCOsTYQ3j6Z+?~Mj z`y4a{?sc;4i~R~@LTj@nc~Pg1RKlkO?~J-2`aTM@;;t0;z(-M=KBZKV)EVp&>=opA zm}8QM`c-lTV%A}D>vc9m{b&&$=&JOkb<8ldk{B^>ROP5-anha8$L}lf$7u>IvZ`== zRb^AQkx|CVpK!^kutfX^6QMLDW;8?-yS+zQusOyksRnRlku@m^NHyRb#O8Fw{exfg zSPa%BWOP8b$O&e`J)%4wTd3t@!kHnjpmB@(=7Q~(Ko8h}503+p0txjQEhN5Eqz{po znj(XwNz!^Gid)`FU=iwvVg8$2*va}5${uExPe$vCaNVFaA&)^6st1n)>}?akdy{%P!1zXII?fair?r*DssPt>m8NY%R1WKE{hmyh`<2ZwF8&or=cUrtxybGpQYH_Q_Phadyh^Z%h!jd_XOJw5wSG^*aF20d_Ff9MlblqcrC+;@-!>n`woJk2BYkLOXS@;1hn(@MvL0 zyb+ZG2~rz{7&+4fIq|Cp)QRNj;=3pI26i5Vy za7!ON*v+>f@wgq|cL>?iGw8Ifj39CD}dK#lEQdEuh%2<45?tSj}vG@DU zg?uLkH(U6f829rcu`pJIomn1?{C{HBb#8Qkc%JzB2;j6Yqo{Fm`KgLipQZHO zvj)af296%wh}oaDG8A}njnrd3%CrCRju}5^8`$aQguj1SchOuzvsbS-j$UyKR}qkpu%Ka#$^q2 zK%9(fhtZP|e@mI&x?Mf22tjIIW5b^-hB;_3n1Nm#Rl?r#9&b6_AI$jFS%=|N_x1F; z^rz{op2owz(0c0><)W^!nhe1WQU={K!!Fvn(5(J!*HgxXjR!hN?R>g#Zbi;h%= z*U~O&mpmz%$9(V~zF#=fKCw*4;ZFrVmtM)6U6JIst1iva8-yG^Mcq#jB9X%^Bt4i@ z={}xH<*pivAsQkBeZBqhKrs7@&c7e|yw21}_?(eVdt!-;<1@9rS=xg5nf)fI5feqg zr!l{)Ks1Y%2FS@vJ`lpa4i;06LGY&&vvYdx@vlJaNj;$jdPoIpCn6djDLx>kyUkN? z(3~1IzMbuVwI19O$6_f*3$|O)UDu4HAb)0x0?$6}if+p2u7KBQ?IMGfCx$X`!ejr( z36DiRWWcm(Ei7WDCjtX#crg!@C7H!rrm1i6xnt8BE&B@~=+7fqp%Zmb2jfR5?;~Q+ zUDcrOIO^Pyvdpb7(MQ=X@i#aI#7!KG*yTyViJil{tMs{U)-Yobgy z2kjBo!K**mR?yIMM_Fx0SyDu~kShCCVsO{LFfvoynf>}5T{Fo?i1NO5BfQ4NmLHPB z6I3{({S5k0fZdOZGjhOQgnasaa7o^6UFOcsH{+z%bkg?N>RA0`P4Zg(wJDX}zs`Xu z0q#4)-}eqRSooY%xJA)I3_1qju>!ABD{SDDa#{OCvizIBWQO_gJ6bAarS}wxJ{B~p zZCV(|&KCmC)NFyH+D*oM&&|KNHY8u~@a`EgzABpckt#S)2083-=K6}PPFg@e5X3l3 z2ma*7_JQd^48xUM`k>tdXug?Khp#DzSV=;qyWky;3!7S%PHvMO+6ufABX}KNsPvA9 z%7jT0augtMQ(}eNL3AZ_$!7#ikyG$filjL>#fe*bO|xFd}Uw5!bP>r zHb@wB5d0Bm@=WQDISk%mabMMu;Ow&O*S_QFEB!P=$ngL*!PR?>F=KFwnx~Uh{drU5 zC~WneQRJ-pHN#4Dmm_aRX8#Vwv;UJrCpEc@9|`h9E)h5JLv9Dng9iZ_;pA}qv0yHZw#&?*L0gROFqDqZn-0t!8iE(zo82+r^(QjCN?wJRJxk=s}8r)*D2DU*1eu- zfBO&E-KIsUmcP;0@%=@O%3zP%IGj(Q8qW~1c<_*;xIYEI&e+^-O5LP$<>wj?&Y)cF zJ7_g+dNBzswXm7)9;mn~p^ZOoBRra~^RnA5E4$VD-YhV%oAhw2gR~lKA}IG#h~E9( zQFNth%l62(v-=D+k3>a|1Gc#VQKkL*tsE{xM8cTDZ`XApNHz33sdF@1T?-eI^+0?m1UYwu zkwtfs(K)(BOewbK9<{KHBWTPaJ?Bx@LY(3KnLDqJOO^6}z~Dx#WCA2*RZ5 z2@2#7@!232?||(ZETfq1ypPcU>H5|04F?PN`#3m+;d(eNWPn*|&Olh4+70Nyk zx0Qj5N@ooQbJOsRQho{d%Fo7TEI7*cwN&$$+vIEk!iJ}-vneJ0bBy<=N09K{ZSz0l zNFD94s8F%B94UJ8V*C^y>$x|!W(mb$omnoNwG!~MCuY=Z8=y@5HMQx4&6KA3-pYBgJbqqhVAW)|!velvwL2Ua?6BxAOVWX*QbyhuqglFh08@8F?oHB1ab$9xmhAGrDWpk_SOuNzuUDY6G+js!=m`#5ob0>au z>O3i(Yq|z*$3xGU*f{7Rs5Q5~HvD8lv^B5UMetj12e2eDBbEyds{;^G!~cAG zFb)!E^5*1?7-q%Jlm}9wsWFQo?Sqw}Q_~Q=nSQx@z8uq<2?4ledd;cEqkyI~+%zg7 z=f$jyVr(C===;D#Eu+`p{;$6RvD@e8=H{Z;S%rT(4Y4dkG}|Mqd>Q&;Mj|@AMQy{S zk*}q_Fq0JkDI4ohrF0v=w^sZ%yQGQ2u>v)ntC$D`yqATbo0#N7LZ{Z%;b52yoNtIC zfE=kysRIEs=R~aNsfhK~*@sG|g8JjRLhbYR1UT#7drIe3o9i?h^0t~- z+twZA){8hP>dg!xyJ@o4Zh}B`1Zi3l`fpQHITB8ufIs|UTDq-~W3QCHPk&!}Z)&RZ z5r>9KM8wKeuY!;ScFJRPHfItXN`2R3{Dh@q_KL0-k}i;4ckNfSVJwu<^PP0)?nm3J z#H;5aDb?t&+fScV1ZH6mQLZN|?>5A2s@nsId~R{8+}4ZasTn?n3O)d&ExX4v>)A#? zK@iQm;Yn9`us<`Z+iKsos^f~Q>vuJ-+T>u&Tkk7C^{oIiy)jX8i4t`Pk>vJgXP&@3 z*BNB@hvz_E{|bLsv8es0h*ihu>aGTij+sjQMYJ=quI%+12;HR@)|EeCWXqW{${Sqn z*vWnmdF`gWg|n6EhNiW#GLe$r@ACM%!MM~V28be^Esq2EJVE&RRnL!K0;0uOAyQ$C zUySam=Gw=D=k_me`Slf<5?_syxk{-&?!CNUKHM&^+U*oual+#?F+%u2&E$3ha!F_3 z`YYz4h$yOyAg^2G&7OWR31oQ55c0a@z4Bbir7^qEP6o3iQ2l-4$%yNAF<+FlFICl% zu68?q6{IvRX$FkzqOW1VfExZ!WR@6o-d#FDqHivJ)b+IAAy+@ zde#9o*s~r<8CjY$VzIh`qI{a%Bcm@E=W(aIIIG|zm4Z^8$a-!{5T{{#&Rh`d^yj|( z^BNSh@t-67W^2sRb|^$+R}SNim1bEw!Y+6mIoju~ftFkqvg7hX8H+_77L#%CtRzDv z0g>C2!D1e9utbxAD3CyLtHN~AQ94vp5En)ha%NvX6v6}5w><}4MFL<5z1^Eg{KSc% z%78IJV~igY#evY2iJ0FN&8R^~xy*Y;B7CTMW9ZE|$Y^VMC8edMgw@@E9%934d+y2ld_vB52m$8n%=J+Q@_XFf=Xyf$IizWFW%l=@H11_A2OdVk-!qAh>sPjP+Z(3qHDgB0?v)YJH+PM@-CAC$;Vta zppzz>onX&7yWZQgpF%X@Cy}jsexnT6DKy2*1VtD76Q)UML5f5ooj?NlPd-wAJa#L| z$&M%1Y$&ho%~vzgnC8Dn?qD0a0Dmt^RNHf(Q>DjEr-@v<{g*xpn3BVsCxQgulk@ep z2?Zb-`U2t4e4{c0mR-`^{L!4kQ?`L-g4?2)nrPy1#$}SBC7>V3{gMrr;0kX zI!o5W$@$f$8Pi4mLGcf$)ITb;j7Ob-?pEbn0m*rZ;=yB)d;@;=C}Wnev+xQPgZ9if zQn8f7=>|Ju+uq1Y!JRAgnVA$t_Mtfl5D8dCrR4K0)%C9g1w~O$yaNQ~_cP8c2j9I& zTrJdehCr0BM+?Dl#DU{te2>_D@t^eF{biUB`F6K%lX%=tHUzw_Z(hx*=eX z#WK zA+=z4{I0srjRMe)QR1;4-N%10=-d@l&{HXK*m^)vnwi*xYmkhFx97`K4+9?F2OwTP zohN5|5HN@41iXht8hCdgQh9iZyvSDi!Qv0N7iZi1n*8zJn5}CnTXy_Olz4*U3gKpu zK-?$G=LV}4JT0Y@tWMWtyMN2ZGDJE^^0Yljd`Pb{n5`6Iu{aSFsAC?XGb3>fAPgKB{v5lnGQVTzit6Tx@0X1nSLyQL&52-*KKRd;48 z7wZqM0O)2)8SMWym41sTH0%xe%b94>PMjZ&GU0LlqdtY+X?hueDT75FXy0j21HJK< zeJ6hNjztKUL4eqRWYtp?XRz#7H;csEwB_0+dJyc8rbWtaQpi^!2|4d2O=)>~`LiIK znA&yBg3JL|+*OT`Gt^4X#bg;ijoAfZB0sj8wCHBV@W{xGAWV3Pn_zU4w=O)F8rC)f zI5c?PfXBD7Q~}fEw-Jp%q7-hf=*WkV6q2inobEYb6rzn5r{W3H>3I1jOq2n@4KUL^Q^)J8>XVhGsgUk08w6d?(kzyYw@$H z=J=w?moOmrt@&m1EiK$#h2pAIxRQ#L_2xUk_3anUjs>!T9 zbX8^fRl$dSeDyVt5+`%2zc~UW;4&oq=B|AvtsV+QR3p8Ug(x+5)@d#( z1UaYDvFq3(0+Koyxx~4aP-KXh{DM#S?o65QzNI>$2-+Uct2Sw+k0ITFj}TGvOrVn=Fdn!#SFpZWC!Zi=aI_ zy59CT1sRtuWDw^@R8aBSe+6a#8*0sOWTY-AR5yE{cYoo#Gzh9!D`*7%_FS2r%6UMy zsNygdnd?2FG_>D?RP@JrLL<=>9U z<%Y|_?(~Z=R3m-SqOZQxGiUXjq8Jhfe^t%JKGt>5#@?LO3A(4po#U}_&!Q_s;Y&L9 z?6;^Z*p>&pqOIpcBf6}<1tAuUwFvYC9bE%r$gH!uZ?)MZ?WB>4bl3opTAudNEW z)&6sbHU2Q5_A(uD3R``&%Ec_QZ-97NL^Aa2)h_lC#Wx#|(`H~dD2M0Oz>>C&2r_m1 zK|?S1FR`yQ&}J9~$mzwo)H)lsx&ELJ=zp@0rJBeZF2gKu`r?8_9Y7*f;sUG@GqOY) zj^~TZtJ<>QxMGCB^y}au^6SG{g?d()OGpby3Y3y1VF#FFI{W@>gM*0*+JO6lF*)_> zZ)vd7@ay1P)0w{B7`LMC-jG=z6oE0ueBmJfmGL^Y@}mi_^SwHzjwkppZ4MaYFV}2k zx?K>1o8Uja8A6v(GzQ=opHohl5mNiRdhcFOSGD)?@76e3x=G11=LLVS9QAW;g zs&r7vmzcl4=nSgibIk}@Efx{`JtrkuvtC{u0b8EKner+}F!Wj5)eAV4x#xXI?9nLo z6C-v#@XkHE{xcEvK5+=$aa$ubM%_&0AsV<5OESN7-~gd^5}fBm^_S0qW^6O ziVRc+r+@VDu`LZfm}t)Y<Asx|fMa)E3YodGyG}|B5-C7U7kG6C$_XS$B+V|NfuR%hBDxL7BJ9jBS z6T@c7syLhG`W~rpCM1==@BAG#9`~_Fz9F$4F|rzy?&8E&3V%n#=s`^M(>qxX{LyFV8-STi(&WhtU~2L+p7?GAS1;|Sbrz_*7YKKTbCUMCTu<{@72_o{lBMOgo`lG zF5KVC-pREu_#db5pS8a#ZzJO+{|uBQ-mvVkVHX?@3gt?-Plt1xUh4G%X5G2y(E*CH zBBfzI3)PVeb}HP)K%1T|LiV%Y$Z7F^_@XA)q@KB$iW{a;P1CHD;k1;qH~fHNl>hc$ zH)tsBkSc&`4GqB24qYE_W`Frm3UOJpMsr`#0TTf|0QtCp?YqP}mZItrZYlo80D7sGh7h^4wDZrNY!) z*>cqlOI_If4SvK(4iJEF7ZEh`LwZ2?1ij$#TVfux*n&AH>V+Q5B%D4&&4569jO78> zUVhAj4!ZLEk7ajWP#&-IM0V4Zi#p@*WJAObhf+8b5D&SwI|J;tmnODSO5Ab?8Wq1A z12Q1ck?y3XS_4L+Lk{D^iuYN(OtBdI7+6FpJp(rkQ-^Mps-LL6}haN^g-ZX!% zaNZtk5eP-DeUz_PujEE)Mu8-Swry=jk*z?NcVHj6GP*91kzQi zvp-HGkn%GDrb(@+%D(oRaZN7u`SsTHzgrFg~1TQ3bLU7mM?)2e#-rIfe{kof9 zj8S7yhkf>5YwkJcTC1xs18OQG5;ibQM&$2^b%u4AcI+Lxr*gm)`%GEc=VX~ohgEeR z=-T7X#obDkpkB<^dy~$P`UW-|xQtvHQ2oWPotJH|J`uV!q)cXy%WnOHO zPTz$3pGo;R+vEJ?-ZsUj{y?d2ANU3ZjY0k6&C)cw(hGNSM~NhNKZBMKbWznejfHpA zzR5ajVR7U=U+C;OTVv3LLGEpQ7HE~r4Y8?}M#|f{?$i%=zw|PQCQjF)%MUiwref42 zN`MPTUFN{bL{U0>uLFk){zF39HOLAOy}CZtql03$f-5cpS{UO+^YEMN&L>Vf8XYezu519WAsG9G8 z>zPafWY26LE|J2Vi9c0^SH1~2T_9=Hej3`5dq-%u@IC1-GVe%$sdUB$952RR^y99W z2VN4*ists@$7)vQ7U6@CAgoJ>pv_W?z}2wzvX8XH{VAsF&U9poq4Cjda3lf9GhZ#A z8t}c2xqAjbHo9@>t{60ZV2sVZhtKiux|v>Wc-ve#ski>q@J)vkVk2o)Q%jJF#okqS zmIL<$q6Ksy&p%ElI)55*5^HV&Ae_G1278J)Ps(^XY@i*Aoxd_9FkQ+AtH%wZrKo%5 zN1z)D8+rM{3X)w4r-NL%^^mLm9s6OgL3L|5Z?jS2k2v`%ji}!qXo$Z zZrz}ymf+r_Ro>ElE1AtGL`4TO8{JrB@Y;t42WwTL32it0=U z;ia)pT7N0s-d;Z3n>4Fwd@d=?_Bg{VriWIIZ}a8~Ex9ri8c0UXW@V~yUF^KZoT;qc zC{PdB<{_++x?W5hCl*@#anaspj#`1ViT8^;d>nOhbd`{qn--<2iTm+A>7{qa9gc&< zRsY8=_qXvVSX3&&P~Sqm)=nG0aMzvthJ$oaFs^AvA&3xjeE(s7mlQ~o_cq7kyRcI8 zOXD&?2pYnVT`Ra%STRq=vyQ`d(@hzZv*t0_D*RUd4`-weSEUZn$u(DuR}R%>fAkA$ zYllfDsm~5`U%V7JD!1HPEJ;2M;xX4a6Q|)lBIZ5=<8x|gX=#aZ_B2K;NT#30Y-avd zY-s*%5mIT1+LKHWJASkk#qWFeVNL@o4_^lqR#c)s{R3eC8;;e6lzw1=$O*odgfQ`u z6+a9Pe!7Si;uYC6i5DM<-oAP@o3Y35pk&Seb&LU}2^PqUhW6r_u;6UDRni4_4b&2y zt-aAFDH6dXFxq-<_K(u< z1Lw|tH>p|nfK62fbaPRegA;+5!H2p^Mg>kHqX|21BTb?kR}H+*Q~Eh)2xEwGn=u-` z$+O)$!t|&_r^)MBN@6(=v|x((5aU zJE9~#NhJ8e0Fy0xTHSYnny91FIpY$fhF<7|^pe_;r}p2i%nwDTufEvbI{NF1wlAD< zv>Xo&-w4BqK*|B^2tgQgPY3s&pWN2v7ErUU_N{oaVPxtz?Cl|y5GDh*VrmT!9>@j?*p(EjGkWP1U*0QI`9-Yj)JOZU zVi+S5fv?VdV>vWOZBIN<@iw> zXU@-olOsK2;;P{;-VarG-tTOTYx?ZM_*L`abtR>Wnz?1J*Uzj8=4Ibui|-e$?vqc`)zZQJ zv8F*73zP-R>ePPKWdkr)8b`7IvFyVyKNTLvUypE{i*?KA`lsSpTDKh7{%*j&paCae zT@`;G|K7MLwasrYX+2$`Dzr2knd^OF*|L(z#wrHrrhsp!4Jbq1`0`Bf2K|rwXfX7h zjELOIB3STG{Ke*(GBLo`^p#^8LV%+a7TiezVxY)W#N+RxHeXMDkZb)ZHc;>`g5YPX zw)1$~6(p3GEWc-!P6<8pD(oHD5($(4`6?RE$6mOho<4~{>%xJa`Bw>=c#NvgG{T9K zWsW~*Q6MJ^Y-e{G)?#{jkCo6 z_ZL}0@I4fx$J8wgNk|X`Z~Lld?8{iXi|i`hyvE+2I)AQhWCC;r-nFf;Mam37{daLL z*fTr*5%hE!f=G!=07r7$_Z{ofUrlA@mKMjAlo*?AR4h4V_oaDL)%$$FyF0c)#*$=# z6x}QiCD*U*J*P4hXS67Gq3YAY8MQ3INxJIFRuP5fbRl7$YZf*0K2-H0;FZlBTanj{RSfcM7GlFFhq#zrVtq0-n-eYdX zW#)xfSa^(xrru?%NA!(t$N9P1ELU8%YDxr2;9g&n_=2LkAkR>rGu2LK3Z5C&R?n(T z^S^F~`Mvw>_J|RDyjSBAlUGurkevf>+CuKxGFW1q#fUnphBGAZdOOXQ9(zHuzy$wl z;bS##S42u2cNL^U2}wvF1hbI0l|Y|R*0#8A+^9F$Lqdot+p_)Bp7a-X)@#GOMW6uL zuR?bfyxGpWsX04_%;h9Y%GTm-t_u-Fs9Qy*Xo8g>OhP%;iUO-BCBn1+JxN79&mMmP zcy%126IH}zGrpKw>)hCWB1y936wLzClRC?5I^@Kn|6p!#mWhWE2Uy8g_NwM zG7HNa<}JvMek@y?QCs>B!M$6&s-N413m?T5er~PqZr#oejpZ5?m*ulg$?v` zF=_$)_B)CV)cs)X5Z~9qJUr$oGqM1YBJj-iMegGAinvHZHO-66#iLvf&d0NtM<{7(}xq$rSlH#5qc4N|}TZ zy0EO`2JtO~SHhliD>y7)B@ojKa^ozdBE!viDx(-fOhP?X8yQ>W<@V>!6<0${bqw)R ze6XO-HwjgtbE@MwNEBgzpxZY0aqKuxN&B&uXA*IBIR;c3>)TiX1~PjuCJ*9GmrmHi zy5A?i3K;4`oOoy%gKNsI*Pz6tG3^;lgbX)1$$SVFYEid)6tiYH^|`cCHuWE`OcLh8 zL7d@Af@i2C5N1>0fj%z5Nomg?0Rd1Ha7qGX{NNAaED8;Sk%hE<7Xp%k)yGfawLENo z1tHzyy0Kj}QgmneS#v6f=$x)A4!7lS& zx0NhO1ZE(omC)&zFtmgq(m^UI1O<2%eRy;f%+t|AF*NcF+AOAWn1;R+$-&zOhk--q z(asGsHzVy%PrgiHb5TMf$xKs5qVV*tAs&!0nvJXi3#b_+QI(~0iYGm*661s>ekfu^ zW09_6E!M~m3^%x(z+t-gz&Zg{1}M-M!&gVeZp63A0zPK+IG4$9bFo@#4!*V6-*LcW zfGro@EbN^5s$hy~Be)}g4g-b)0-*&$V4Q`I-60qbo#{R~!Hl|y4waZWgK>LUTBUG z1)awgy)6|{yL~+x8l{2K&lIxEKlkpoi0q{zzA{!c8Q)1{2_*BoKgC0KT0hO|$L?q2 z?yBcxg6hD2yE?E3i4keCewnMzYN+&`9V55;gx@&;`}W#M^@F=z@4iRMN9y>t)cDz(JW_gOcOdT|hQk(+&YR zuh>jgXcydYp0+Sz+%9eZxv|PSIm8i;Vx%M|Bu*?e1uv?7;YrYnP>8EE;3rdC2;_~S-Fk{mYDaD(nxf;bOncOD z(1=@#yJG!%$)^krM2KfxTwT}IB%*zstWQ5xQu?~4!XR0`b=_5m_fxzk(r-}tAD0pm z;8MEdegAWbS_dMhA(BrWENjZ+uu$7m%8?g)*|m`KKdrYh-Q9j7KjayjEWqz=&rgx- zTgj?E&~N$Tz03u{R&YPc?EcbVh&{8yVm?Q7jh1d3g?i-UQ=c!88|K&DT%P= zZr4B(B&Qinl=B*84WEK&U!t!vzaR-s`f&^PGZHzvepfk~(4m8pKc=t~j zqT}Pq=xcCI|0n#HGd1me@lF8(HAT+Y#H4(_L>zNE5L+Dk;6DF;e}wc5e3;En9@_goB@6zJ`YpovX@viYH2FyDp)`V{A5s35a0@Bbi&pgxujWj>j2^ z?OwfV-Mw0DeSdZd%mX~4-uqR})jshZbHL%w7u8N6>=Q+UZE5dDG5smFEWLb6k?93G zxif8`$PB3YQrpe_>s}K8q_T&e>#a1tq4Ne<=Wq6VB0);02`owv^p!i4izfopFP<^KkRW z^A8`)poZa^DsDgQzrECHf*I;Xu?VTlcrwZ+YuR>7{s>`AHrtvSxDy(`Uy#9ApJVVB z2A+mnTu5^E(+HR!u)G$U1yJ(&Tsyiy^)7OVY&X9h)x-Q>j_~Jg`3hh^1q8l#_5f0# z(_GhG_C`ir9h;g;dZQV^=h84{&e=Fgj{cAgk=ak10(w&yLZ6ENin`^06f;y4TAqs( zUGPBs-rDxc$anF(&)0QTA!o`4ho&1q=*dp?(kB1#Yl`d zfRD!^ncgLJJF)xgHAjzd7k5})Jok4b3|{%qdRoDOXi3ENsV_JS>uazfiI9kO8;Fkr zuDVrzI)QR9zo;wXPYwlk|9m}`N4)IDZefRz%_~`eM(Q2NlZV!le*2&9qN1ArEXvqF zivr0FZgWGjFqP}j5~ej=gSn98o5o_V60lNY%|2*)uN-tdyq~ML@u3`BPr!RpEbulq z_J8PJKDH9QI#u8t zn##0!zS5dnh1(uEQ2@7Jvz76?eQtZ+Qf_}Q0fAZ-a|6&1qysP;>_@kiu?R{KmcJn` zkRc49rA)T@G?;|ta5K!l3VQdZ|1obk{8L4CeP+l$>oR0rbk-;Jf6!>qYBV54l$Ku3 zWC29_YcY?!q_x+IM=pfM2vlbT7SkW9Zq%)|WNk)_H}`&3Do-f%im3N94M5{egWd}( zUF=P--w&BtwvBnxM|O&T-?MbK>1tqB`VH47t{5k70$fX8OF!RA$|jJ=V;pW&@%&l2 zwiNLikTL?pe@|VlVkzMUP!bmGOGeZo{9e@bVsY#?wPH|V1&)THp@Pk|J_$~Su>K$0 zr;~dD@n%z9{_gcuO$6RxDhZLQ_0Oi3v|zJMQdHJP21yY{h>I$^dhYC)GuXZ)M70IZ zuo?M=>oo*Qp7H^m_wYlE)JgbC+pVX`ZMLW7Q9STpmr<6Kl(ez1Z^eqa{lV4!-%+ao zgBOTP{&~RpTdNNiN4?1nA1c~ki@Dq1pod+n=~Z}=AhL$-8%Ctec819dyZmPNs<6h5 z>fd)UD0k>s5f52xasR9azDuUl@G+_xC@kT*YI<|=9nXK6eI2ie#!cFddNoYmBL$8D z5l{()v4QW?De5P*7@}$CLEe{er(2Ok64EFf6q}id@4^zri&fVJNr^C;4sixXVS$}< zno7Q5U{9Oxn`D$A1GVRljSv5vCNk5nb#y=nbrbYCkC=kEyf>C%A5kNCnpNUz!+i0M zG`lYm^}r0q4MlF-K(5&=l}pz(aISnpkY>AAYGYCnjo zK>|M+ zpddhzfykmTkYfu>tM{zq@&yU1Afxw3%BSN+kDD?X^|5>xez!)KtWQ#cODHte*4DFW zf=3i3fwb+n?eG2RUz1{!{tH3d^w#*k)pUAuEaD9FaMicZ#)>WdRh_>OmIu!Hlm0G> zZHYrb;x}L7_oGV|$|3b5rX9h|pm-FAbMSHyolW&sWQ&-Qb7*gWfd66LzVCgOD*Plk zc{g?Ym!^7Y$rmES&{@7#2lwRp2)<0?I!NYEjQ5{kM)CLZZw_!bynirE3d{v!R%j^D z1+BdN!M7A3#ixi*g*Vv8ju(kYcx%xRx6nvxYtv&L2=qt5Yu@c)FZ$8-s`TZqmfg9I z(o%-@>jrB5#}?gzR`+b>{9#wTXK|#VRuP_89DQHKL?y5*^jM7P_x)J)b$U0%aE6b& z?WN_LhjK{U;(ea6@;M?WfE%?$fY4s*1GVhkMA<{)`D?b@L-lcIppZ!P9i1k%AWrHx!C+%%n2AyC12o$So#LT z{?*6brJwcjnAM)}shx?Ubo*yL)uHEKM>uJ^dpUT&TXGJ9w0_#6x%1fc($FyZO_VPWAVAYV zC!K%O=)iNvXn|N2h?9H9E}y|3192X?yT9|Y+e8<}WWL$F9GOy^Py_s32Q{8sbut;e zao)RqPffdK8p?^0av==s;E4(yEwMGxpesQi?RZbDyL+6NBSg28rh2U9S0R!m&)b{E z(Lf8Z$UMUivUlQyA}$DgD1`7@@LWkADno*)ruz@cvFA<`v~Nt!gtC%SGf2lcww!!Z zYMeD&%MzprwhOsG1+lqj!*ZzeXjuuHeyd{ zBIONTlj#xyKY7z5{$nBvHi7g<50T8ohkXk;O3xeQ{5bjrEUvoIq5b4f>g~+{c*(;fu>R4ntHa-z zb^qD8L_~7^6R;2e-=QTZVAlF5b2D!xUKlOO<|TUejMR<&n6i*Z>5O`hFm37UH(sJf zpK2f6LOn9xkT%#z+mP?JQALF;dgP{+TpEn#<%RuXA>zAZ=q0=Flb+_;H^)hDhTHs7 z(|jGGLdHvbSzz|hYoK?WeSQsqQ|V}`nUD4uw*&n|%0ML*bf&l__@ddBHFmi^naPfS zHB6S@bloho$u#nL+oQjzqP-O5wRs@OtaD@aeQ$f(QquOyWzjja_5=FQkjl4KYot@B z3q(WA$}3aypZaM2V-LYqea=5*b=TDK>gwd4tMC8R1DhZCzUCdS$B6=Mk{!mUAy8HZE+EYKNxeQvUttU%&eciQx){@&`Ls~JSLg&rd zSwm|nV*9hgh8a`wZ`^tHPuxLI_f3yCJDjRkQS~&$7#Pan_mhh~(ok0Y`Eljdk0lEJ z^#0hqA5LBB>5bu7RBqZ{l+t8uI=+ROW>~g=D$7lPceA{mFI2FCivU*JPI=XJkyq zW34sS2i$4X$HLMs4+_}H+5^r}E--sF#eKxpgsD96Ofg>X3>3COrI4;`|QoI{0EH!oBagFMf?RsNMk(NeZ1NO^kg;J zZo;7w9S0V0#HNh_oy!y@Lf-i}(cEHL`SLhXIOKelIoU&cHcl_Kle3O%94))z%T=|#KgvCb$FXzPuxd;&VSiX&WM#-qiKJDB|zUaFGO?er(j^Lduk)q+RJ ziyt&c2NjtZ=PdwGX6LQ~4ln?luhO4kvk;{KX8V|Gf7|lXi7(dGZ$opD*Jyg{5n5W& z*4I{1B0-LqR}zy`4xb9FTR#=qnD`Gn`3-Y$NQ~vBy}95U!9Eg@>8En?jI$KE6J~PQ zwS>6nV2B-gJ|B`y_`Gre0XaV-png5@htz>Tok)aC!~-j1DJPXLY|RNJo8W2WocxhC zv>N|$snmw~+^}$TKEejrz{Xb6v|}sqR0Rv^O6K_9;ab3*iF7T38luOS=mV)?Ei6S?$6(;spAsVEh&4Cf34z79I7IeRWt3ZPC0nzvqpU0SZ=+LDqenHm2+}BRpS+s z6~iGwnjLF(pGH1!(eG1^E|hsK8*+nRz-Yz!^)pl^B;$)Zjvz!lrK z3oq{WpL0ciV_`G!#_YJ9(5f`%ewTL(d&7Lhe}b$uy2I=xfKK6>2d^CjbA6eZ)7fHK zMr4#0Ez?hBF#6DY_~L!e@c%1`qeS=AfsRait118SOKhVn|9jDsNAMMQ+t>FxYQHKo z_-1u}e>SFdb+~9SYXJ;NQ<*z%oiC4gAG9v@CtyA&mO`k`G$tI2T+3r_(KGQ)+526ojEtOLPvCnx$XxGs@a9dO1$z6_<6Sn%_R&&%7L!W4WV`p(mXEV@ z4bi(ZITOFjS!;Wmf{rDp(Fh(0t3{Xj+hZQ4bW~9j=3=XztIa z)Ut6l-x*r^sK?`iuLEu`{tkG0pbPHMpL4{S8cxUyke66qicVL42EqV$m4G~iX^O|+ zN0w=~eM`-S{ZfAH>08WSJa(Zcr_LM<*Q&Tn>yh$3v*$}w<&St7%Q%s{(IP@O2ozpq zOOjnO&)!JdT<<|HNszNZoC8SGN0ja!J3b@ix4rjt#59KdE>Cv-`~e?dLWfq z;{*1s^AuoAdI;8Oi~UQb4Ny&SKdfK=lZZ1x^uk`*jQ#G#r$)e?^m>YRIDM0*5!i^B zqiL+3D<}VP*jWYyjg)B=#c3uTe=*QTt}r&Mb{60Em@=)q`<*3cSHIDmxX4Ywj1TmtHLAP9k16=*EElk0uBdKpxY@GDKwZf;yl?Uihru@|fF z?e1rv{V2Ao6;Xvwk2{C{81e;h*#KObm;y^#I7^e{udO(-!xeDz^;)#SFVqdnCi4%I z#pqXdm2bZ96h?Edff2^#vcn&n4_p2=*_Cni>-S@Xke*}4^{lMfJkw6#esU$A9n5o% zQUA>py(Zr>ilP09BGk{aG=Y+cKU9BNrWy5{UtO=qyUZW9Um1f_^YX@gL%f1QLdAJr z3y?Q$g#P@rv*bd_Vbc7|_jNSDhtZJvkbc{9x+%XrZWTQvz}#%m(~aWSg_J_S0`JhW zGsg#+Jjub?3Eomx+AXlH7+UvJaKEhHKQS-@>KfCF2V6_jN!YEV7ftJaZad}aFIM zW-d8#gM!K2qJ55Y`Sz=qBRIj$T zH8$ojxvUWF)RPV}p3~EK0*#6tv1K_7d=NGPvczZN>}^m+q!e$eo`kpo4fmPnNUCLh z#>mCM0w19nWJqvkGNdf$D%|X{v2@jhX00XbS}5%C_WiT>vkTgsn{NHx5zLsJxt;nB zf4~}LHaqoL_)=xs}6ad(2@Du3{)DrLqo|s4kHHujnbpxZU zAMWnz=yQK81z$PB&SVK@D`SundO%CtaW|$4HH6BMdI&f)X^TEiZJuXahMX{ZP;dZL z3+ztE56rTi@x_C&rhCJzcg71SuXc*!c>%f9C?6JdE1hH26@+Za95QLLc8UIlZ+3Rp zzHP)I8GVx!Prbom`Ssu5l|}B$zG;!KxZ2MNKKF@~5yE)Xd*LU6iofEiW*2K+3wTMD zF3%nM4O|`a>|jBR^4$#U+*PpD><&Ft-`M*=;7TaW4r*_at-;xEHV{XZjdB9QG{tJS z3shz)I|m)kTIX%9dwO~&ovbFJpN`5(w}Hpo4X-44 zqTh7&`H62I?IKxt;9*Pe?Fr$Fdj@z~jB9j)lhc#DZJ}(`NhxE491?z^2vo$RZpO21BWmmqQQAYMp|@<PK`uDC1-zvowRX&R5m7yER!+6l}<4-e?%=Xe_z0 gQdi~cy=9Vqa$b~XwTl! z41_ucyQt&Lfd<&>JkGi#XcB~m5F)XPiSBg`PZM1hl$Qb(eeXB>_;&rv*&>wQ2WOQA zJwl9*=U=7XxP9$+QF}fLgyLiJr=NN?Mp1IDa5F#dZ(t93 zCAqr2(|eok1r*(8Wqec^H&&bM*)kBZ&DE3IBF7kmgNSFX>(@5ty@(EZZz8NfPcl}7 zRL9`a?hSyKDdKAtzyb5$Prx5kW0!xKHg=Rdbly5Re|9`sEI%}PW1b8X5Qubr+rl>^dUl24!)Rn>We@4jxHkoB@AK$_;hT%C$8vqZLl?iXLqOBB6sU<2Wkr&en_F29Zbm^}9Qdb! z2-s-cpzM4*$7O~MvU+8hF>K~}C|!Vla7*wC!rCZ+hYkr$#=&^6sXaIF8!H_{L;UUz zTaSbtBK=naQJZ{dX$sm4Rf3yjqe)IY9kYNEo6ZSf;|~^-o#W1hfDvd>Fa#9h8;X|O zG4DY55ZV5DCcJnVc#Lgn4%EZiaR<=!J9bH{AsB3>Hnb2&J7`?!@v+QC@=OO(>Q!|A zXfg0hIp)M(8p1?^eZtbyM?ho|wr+l_s0+nEg+EP0_S-j~c(nKe`w{2vq%;ZG3D9^o zO=!s?f>1})peNfsyS`g$Y(Nb+2S^Q%OY#HW*=GrA=?4f@k+t5F{`ItZh)3Z1>?$bf zy)c%D3~5q-MLnH4vRN3Oa+|NSO!FuYZJr0ZfD|~v*FQ(CH{=KX8Rm_$1jcHxBF}3I zD79sgs-{qOxG-%l$oP~<6YH{Mlo$2hHPZU9Ey^WkJ*OkFBjGVwG*aYw^2g@vXAelIX9$VF}+}OA+}H| z;xTvQox6EYS`xfYx?I zVMzm-X}u5@gc;d2Fq#Z$lj13YeWWBw;D<;kHy#}a8Veju+%bm%F3|jEra+bE;Qs8l zj3$I`G&E5)i3k3NJD}1VIlZ@6h6(lqIT6lPmRubI&jXtmOpd%k|3wCsP_NoR*HfyY z@xM=)3d%;L!$ji(1paS!QPD1n^(*J z?q^a@ZbwTwisEoY7UObU3CJ=m)7T6Q8-add(JU;(lC`LNd^_|kODp0{9WS`w!R5s} z$&z2^1mYnoJV?IudV08Ojvr60sP4!#`V^QS(h&s7LHpC@$=7Nv^}utP?kzx`P6Dva z52iRKt&k7=cUKKAB{-kny$ZqsVkTF&T*5 z+5$f)7pZcdP9qM|29*xEPD0_u);5>gG{@?3a>xo{pl6`)_l=tg$)@Wh3?V}#_vdPc zDJSBeZM!FD@PiaeNQl*|q#q$B5L5ZNFducCGYN%7?Oe%N$poGAFBt2A0Go%1x&;139ElRL0DhYxj8}D?!F&@ z)HH3TzKx4$-2(cVwEV4Fw_||?C?LG6B}2k$!WaGf^_zYml4LFm#csX|!uifb@E|w9 z`yAf?Xp#nonvUkEAe9@n3Ie$NAVS>29(aOSMM~!?c5P`YSs2aQE$RjlR02v8^%ArQ zH>rDcpDYY@eN+FwYflWJIC}eo<=%Ap0!h8m-YRog*H6Of4@hi!RThk7EHe>z3@cwM zstSRyn%lSxSVVO{Dk>^EjZuJogbB6W4(9sIW55Uy`4yvjd8z_KdL(9eG;Kdpjb_JVUAC-KR#2zqeZckh|1vzodL+=J z%pDS>V#7Wnje(rO(J#k-b5Z;f#y$IkHuD`kH3K@7J*3!?ogwlI8WZ!B`cZO_xElnC zK=Cz0??D$hjz;;Rf|nMPCj;pYr-1Q0_cCKeb{P)@rI)lx{pCx?MfO!PLUEP!k8>gX z>f(~E+_=30iFn+OIG3UC=!-FWz1JClzatnpY& z-olBX46$g^Om{J!2PChte{eBL_ml6;?`*SAPdD6{Hi_VU^rEchnLzf;bVB99*gU=O z&hBZDHa;P+458H0kudY&aY*1gMOBoG5S+wS*k*jVz8lEfrs4UADse0YAsFPYvRHco z%HG_v&EqzG=gm|_&TD`M23&;Sh|K&A(f&OoAADT=+B+=lJft22J?X43czmyWW zDTo=wa^3s=xu8c5DxU7WZO%=!)j1)Lo}j4%#g}YHLc)XlfLh&l8ub@pKuxpz8ttS@ zOfK;?8a^m8pco#3XyG;h%c-B-n%fE4^9MGb_Q$sEpyP=GX;ilhe$P{jxei~ki`}X4 zL9VBVBmbK=fS2H%K;#URl!G5ca_t0I2ocjMfJp&|;*CXlc(R~REOx%kPm7L9Rhv#A*UNyP6?jMPW{ z$w$j%IxiBuLl}PCV^>l9SSYNO5*X8kA;yA;S;2(F%FOkmVUyCjLC4(oV2-F|x+gtHyLOckM# zUTsCuDG|fG6~gj6f6PK$mPhe+V0rnGH*Vng4p8&@Y>=;RV7}|$G)BV!b>kst%O8B2 zCDA^&YXJZWb(RlI#|A*yeHs3r3fZy*#}1i6#Q6Ud+sBa>)~)I>mZJ%_OOJC4+FmP& ze*R6)@4Rt*T3w1#tEn6pNSt-^=Oz+>CDPY+D2B2mQ09;tTFExOgCCde z1|zpw%QhA)P35P@hnw^!eF8m2V}j8F>2Uo}q(d*s+xNe}k|zlvIVK<>V4ce4I#M7t zz5>FX{R^mezIL!VC_jJbs)j3@^zL*n$nh zJOJ%1_&+_U=t5|^zE38zY8lFsEY;)Y^P(;Ng;Fl|KoIm}e$C=*H11QrLkyi;-Vb8~BW zYpjYCrUjVQB-*}cz)fc#-_=C%5=1YP?PP6Y@S5v!iZ)~L>hIv{40Bo%*= z!ev@Zhbjt#-U{zAOR*=k=*V)5q+Zfl5U8_vZ_>%Mt2;2=3cqAi?{gYP;_26T-*vNa z)z|Pdw3cOTrXJz`dxOs0AX_vwaWGl;8+*6pVIz6ZRGvYQdx ze*M*kn)pLgBAiBOGGt&WdO55j(eEFob`s24UepdqO`wg*v#Utl>Z|EM3r@jAbRl^B z3`-TZbayBH1Injxrbtof3RzI7qxKT-?LyTshF7POUK(*-H%nd~ z^GtWf?@T7|;^C2zk>xM&_Yh7K%FZ@6HXJ2x#?l(r$xiV-2KvOPmxSw=J9my$!!Mbb znOpCc1ewyCS0oxs5sY+8f?|p{{V&afnKhlFdi+&S!_?n=3B3bM1w=T=Xv5wG3K8hM zs4=l`F6R~Xpj}c=)x)Wi{YGLLpSb^KRkN^4T%|9f_PyOdsd%GZ>bNRloNzO??gGhm zguOzY)ZA_L>>P)489S2jVeu96r>26bG;ZMe`%!B`beM)1B{<(m>Mi-uKbSnUod0ag zv)M?-Ejx?mqRjBmK5J*PiQnmb$MaBUM(%t6;C}9HhCpb}%>**t2WKnrb=(GRaT=+7 z>v+f{`D%2K=rV2vl>^_cdf#5on7r@G)Zst-%a!qi{z9&0aWLGbLBt(s>#Xu6{hJ5v z_P|9K0?)MmNcMcHV?e3zjR>QKb)6D6ThaC1PY0*D9U+Z3_$kTYBizDaQRA@->4kPR zw^Ce*r(~S%%^EL^WDRwi5eW@j)5HjAC}f!(s5S2Q5e{9V5(%>Mzrtp#s-cT37ixXw ztnDHr@|joc!)uX)#D){87QBHdjQ10nzxd6b>sY51GEa90>MvIO8apNT6Y6Al`%H2> zpJXo3!tR%;1cEY+KhgfO{~Z#)Xnkw_WjBvuN&-Y$Fs4pQJLPdFx^({qw)UN-)ASjA z=su$lPtmo1!SCbiGeWcf|4p9FrE&UW-SZfd|6DPO=Kzm?%vESU@kjZ`agy*gEhXmd zAKs_v{$r74a9KPhI0^um^1&#bKzp;0%6Cm(EU@cH+<-KwI&tw61I@noz^Z9yjU~MNHxvDs@b%i}Zfsgzd}I8}Aw8+|WsdYl!`aKNb;h0RJ1%3- zl&*Pdu1K`wEh?HVsaHD{L)G!}^h*w}mVCbMG}4{vjh<|6Qi=JuN%*$iKR7YZfmw&&=6N_R z{f%fmIN4)6LGU+Xi#qG}1{0;f5csMbjM_d7w0IxEF{e+xzYQn0km*BEqEc^_j1b1- zt5HaQYp9ZD=*j4NcV;-(;*R_9&DhA;LjtIuCxX5(`~3@xYel}N?H!jAqj4zaAmOI9 zJbR?{0!ddT)@1=Z0s)*9lRlaBVgV6dm90AUXH1lxXqt}CTXvkq-(F0A-Xi^8Ao??F z5~!IR-%tbBZD7eiAYR~yQtECvUiR6=k;Mf@(4z1T=?E<PwD z_eR&XOJ0ou+Ny|dH1?SoTq*mcqWi#>R{M|m4vMG%WZTRrmn~dEgjPl!|n3_LEZ9aF*&X)h+M?HgBocP(; zD`8LYqY{PNc z^qc6B$aiS?%of!9?%u7-Ph(4-*9ICE-uDlPM+a4{7z(@rEp$~;1|uTdlr#I8E~7yk z?+sl>$-lAW2f)J)FfU(ifFgRX(jBTl}uvi`QR^ zF}>tgj(-zHay)AW(MqaZi+Dm{-GrYm*`jBHv|dYrWpv%B-Mdoy@# zz(>%ns#IF*l{FdFQmC|@;oERCXvpubG9tfM8|18-Ie?qQJO~b_`eHemH^SG!CJ(eC z}Jo!#!U#1@o)$vgz-&v=ccKT!JbpNJSy zolIA*iBLWM(ryN(YTLDpD(G}`>!_@EB)8;$vljV_?Jd|pWZ7U(TUp67QerGwpiaV+s)X#FhLBQwVMza~qcCM0Q{?*+^YC)UXeQv}X`BU>m42qB5&pi|7D0&K zQoaiKtb_lR5_@KRyVq5MuH8JW{S~T`tg!1Tt3M8M#y*O@;8)zWwDGY0ZJj`;8qz;a zJG*n=E2y{Uq4sEN&4I(Yi|)%t@!vv-q*M8m9LygYr$awBEuDT7ughAUZHrZSSL3s9 zNV%-uYxn)}2e!_>97fG*1J>ukxELF#~5^K`NNvkpiJV*N-n-&f>o*vo`Pi z9+obDgU7=UKXi`~N8L@D5LlxD;>pQw(nyeA^gIGfr;D5z!aI1xgU~ggj4UehRJ(x_fnX zQ=T9p1mI)EgqSf9N1%Dxy$I(hFQ5GB&)6$Dr~f7yN(EzPe3cIaI+;jCd_db?zFzU2 z&3FAMvXO3Xu|G!WY!TIn;?HjKc->yg(}#Up_*2AL{tZq+&KMP!Q6hcIy|_yl0V%cT z;VdOBQXTpx6xZ|R39X^LcyU}tG|2~}Ee=+t)4=lrikHLAAy~HoKgH&pf7vy{yb59p zO#E1p`}$5#YD`b#e^+kA<-R&^5t809ER+eQBf9u4;s^aabC#`=Npq6 z`KA!qb&x&S6=I#=a2rT`b#J-+o8=iQzxev9 zs3@a`T@F$#5PHChBq@@LhWn^njkpiWj07UE_{78TQpRXIT}%K!Pi^fJC%5_;VpZhW+XTfb(|ize4Uob3brNK z*6=ekQf(iea|Fez8vm`tb8T7cN$k4c=ro{;U@7CsAGUBKX!K5$tl+#RA^Pa1(S~X} z$FA#5j$hFMv?*VLuD65M#OMagDq;ARJoaR8b4xlfJK@Zm4@@B<>tbrXxfh-Sx?_<9 ze2NEXJh5QZ3pd_MnR3O_PyEw{GFv0KI%Co?Y@?6AhW#RJvTFpjXv1-Zgw(Teb)&l` zxNqu6-n|y4jmDK%L|=a05FKw7$<)1Pb#`C%Bh9y;zbMqbQ3Qql;(q+BbHRko^q3IW z8iYx>h}r00@8hZZregB5{hHHYJf|a)g=eIreZSGa50y{tFWLZsEB{^33z}Dvk z#Z9^8ASQ!boVK9oLJld2@P3+wYwiQ=n91+{>ep?WbDs@{4xW$gx2SJ9;y!j_XG!jj z{DU`#tES$tGe&W5_X)LW$8~STa|ypKSmjUb)G79=E2#{XrRn+|XA{lA6`^c~9iK5QORQM8ZCgTmm1P8DCF&*KBd=CR9ii zxoS#Ra%Rd`~=yb4W-x%Q0=t#Ty0e2_-*SDaWBtF0cSAo<*I%A`nMn_U$0bN}mdv8yX> zgR^eHSBk^Vf)L;U{H%|IanwduS%PIAV>4ZbMp z%NZy35Y5`Dfn+Dpresd;-WcbCB49CQek%LHcExF36HDa8StoqitaZKU(*J?Ml0v{$ zHlvZV_d|wLSG=#FS&;kTikd;ox?)(&z5J!At~Pt`j~rm-F6ljW?n%>)NyCnbkDVz; zdBxl{rSn|8>un?xJu6?IBJx;}iwy2wW9^Q=pd$jpR|F+F=?^_0ps2Pqc~=3uy{wH(4wJ_5Ap$}#f&mKvvIY$0Fm`B$rdU}Io(nzkjI{{)57rKxmHWXD z8q@(&9NaIzRr6D~5~B`}(zvW$^}lQ9kE>R@1z8!Az;tZOC!(EYKffZH5Y=;^_Wk#%HAsz@Fmw~@b!JeFRTryeiyG|b*d-3V50iIeiHb+17C_#;SG0KemQ0TM{aIrh0y6i`ez zT=%JS8Q*tRFE6i18HZLttiN9UA%UK*tHg?3f3R)Y!g3ui2T%yfC%6s$D{bD-DA9cJ zekAKSmdS)~rvxI%445EsT=8*V&AqgH=4n(&zbVnd%2l!iG9@0CAq-;(WN~kW8B}my z+>KXYF1rW>DXGfW^He>89#>?H9etql#XWvMKT2P+$_SD>|6O+Ndnc&e`{FryykZws zpW?n$ol}vYUb5nc}vqQZC)lWeh(J6VBV z1KmltXukqk!a%f7Kn%wLZqqb3@~6i4pTR4rZ+4IF)@a*>B@Eq&T0 zWSib+xcKO|V)Ke(O0y^4RuR9oxUEenb>T=OyW$lLb-H4syF+E#M0_s?8ENzxwBgYV zXnFGEpDZHO-UvYa)3|jyu>LS9PzH3JxP1?t>3l{wY=d9BUJpITZ(vJMOR8pf1@+50 zydqh3&M@qfo9ew+1_!v!EI+4+V~I({iXF;%T{r9LpuX1nnn1!ju{Wk?nqqw_5I-&N z$4+DdE^az*OHN?C7WyM)`!LPU7Z<-7qZWa@S6qQNQvff4wl1*m)g``%v`{>;aCSe$ z;t@Y>G!%(sbM*l@lTKDH?Uq5=BCwNkYpY-IzC zhG2-AtoXI)wzF;H(7tj+;BSOLn{e?PjYSc|)4J{xq8d~3dw7BKWGqe2BMqaJKlP-% z;gj~Y$SpNPL(@i8w-NYKO?hFdusmJzTMHzwz(VWhXqr;R{v_(=vzKN5J15YJ zXxA`ZVtt=y3ffJAMVW?wS5oyXKIL&@OI#CAR<18lSY0}_j>Vf0KNxm;#NW~MYD&V( zSlP*1RzJ)NqWgXMb1`CTaD-YOA|f*!7>$=``FXb2V_APy$>3zyZ7RjVb5-Z~n;m8d z%`o9hT<~@l3uR0;pQgxZd*`E5l(oqFRq%$|RL`zCd*|#~wodo({nCC~dq4rD#eh*z z)~k5yXS%qbI1T82mdac|I8ju7@E;-2L<)z9R#!T;>x+c6Jo8^|ZG=-66p=$qdmYhl ze8%sf59e0~Y>506iaME2mz?au9)FCse8x&_$6SGOaOq*-bk)Y0)lRbP3SZPDX9ZS-M3bz*v(2P~zkf33T$V*HcLc(| z;u=t6>i4EuO-y9qx>l(lIz`n^!yJ1@{sXI?#{WOCiddgog>kfBKx4^EM|$K6N&1R4puU7E5^_KwD!b6Q^$4K6VLVpRdH32&~T6Z^OpHo$G{uQiV98iZP6qn zr?TUxwA16;#?T8-9v}63w`DJq1CmWdbIV@AyKTEpy}`R$;Mp^B!B4%}J)mLu&~S*c zd_N3p@i|4ioByCVO?ji-NI0n@z5jqSe;(CW`A(YgDYV5w<8b)BZ9M=DdyCbdU-0yP zss}YKTSF(~B$>Tv5Aa5zitL%K@fObUb9PaayICq63z*M$p`koBzi87No%5SE?bu>C zrIw4kRAVLJAC_TX;Z2VdHSO}o^gx3zdPNbp*ENV?&i#DTRL4w%KJepL=oqJ)%`qcB znMzx&7c@E5D!9cVN%5sHQZR8~12T*zpvHE?5np(_2p?bT-6%e-d{m3WUQ)obVI)@S zx^s7eq{i;=$6H(NIP!Kt|=>hu13ttg;R#eU-@y0|sz-=g;Ng0Yi(oqnp;wCx+{ zBy15KfC>(4XO};3VW=KRVI$YX&o96qYm_WYskXl_!F2E7n&JAo#>HN}Pe8rC z{JAf*{`#iL>oO@%E+AfSm0e)*{z=Fy;Vmj0{Jay#qIs0bMU;kkVB{T%r;8Lvo(~@q z%Mf_h{PaPm9uHomJ9Py&Pl1xT7np3L#<0;DZ=9rNX9_m{Sn_*X=2W+Op&u5u-*WQwbAuLoF zQJOKwLO&>)ZB&R9-#mGc0P_a60%5|dQs&; zsq1mhPNj0Gb8w1jJ%*8@izA*|G0lmYi>No>+s^exz?G>T<2!ds5`(K9!&5fLs~Q9E zXdUviI|zeLzCqOt0>-s(qFBn4aEQFuHev1L_k%|A%iJ_*JTRKJpQ$);Z$l8&#Lmi3 zmK@a5fq#rVwglD0uPsP*-8m?F;2*c5hWAT(1WQ|mP_2e~r%bYe>|0BEIv6#PLGgoL zX=X>c&SIXs2H^EtKR>0ZHf&Drf!`lLvSBN+_4nD9ZeEpYK*n18j8=yo2wRlw$vUVv zOI-aRPQk((Ro1E&i^{O_!o75g07Nfq!J3^ITiLEds@7(Sh}q%O7rXED%VY;_x1lj_ z@aS?jmme{f1**_?@tt;O7a15B+NV8gXyCma5O~k<#Rc)mZfICbZ++ld8OCJy{=GkR zX!JtM#MU$E>X~(4KB0wmZKA@C+WVItU^afad>#(-^zbKXIB4+8rl3Py9JH{!+>|nS z$by@^WSgytnH^ko0oi_ca>%7HB&H&Jo88Rr!P{DK9naWwYP4517b#KQGAI*Pc5dJC1FhXhJa;kXC>1qWm3drPR-v4HNPs5jZE=n zs*zuF(}ufty84PUHanJI%VmzwIBIEb&EX~$G(AC&8o$gwcO1ILS+A)QICpUvfve2= zHOb7D8m*2ScCSD+`j159t?kaBqkM}77CB4Owyb=go%A>8foe{OI!J@jrSrVEH|y%W zad%OJQ;`5-Y(jGN-qlYippPJxO*%|K?D;x#rGyC}Uucd(+Dy^Wi@}{9u4Cy?J*ItL;~O0Uy~T<$Qcr?87U0{6WmUG~kI%7c0&dpQpZ97E})t9lfs;n-hX zQ)M&TgjQU7YEvo7z*miz5eORIOXlf2Dy1J=7W5u9Ei2U%BW8;@`NFkFLi*!0?=N(gL$fVk4ihf zH9}SUiufKyYdq2>yZrQk*U$5;IF%VStSMTylf5;`aKKlec-(fir`vtQ)3hCyWIzL=Ko%5=*-R+jL zBsPN->7RT4(25Tas}k3GNj~*oc4$Vr{^BmpuJC(xIE11Asn+NTl(WjrDj^*Jx77r7 zAztjwT~19JPYW}AZ*nMf%qB*h3wYoA+qXN%y>KLMRzw14hTF+|Lr_Bi;}T+Vk58-K zS>2M5dDf$HjL~;+bvbP)u|XqtT!3D@6DwMkmo?~XR=5Bll>nrr%jLQt(Lr}Kyrj@` zbb68)WmCR3fSeuq^vL2fo^8Ifvn{){>jupxA?NM36$QUZ@21Acevgd2Z_FIs8G(18!DzP&G7(A zgdpj{ZY3BbJaTd2*i;-lBXQH&6FnbI2s~D7nMj13^C%Z@3}#4P3=U5#axDSI;$qIG zMzS?b>8^S;k~@K+O9xf_SWOfhd7hv3sS=8qo!JuN}azd<8X5 z3e0qXkBz$K+27kcd?(~EDzgZiAUr9IlSp7tN%`zo^&WN_@SQ(pqh_RgV!_af-C?CE zeuOH(h@!h#!cm*oUJ-3fVZT)=sQ7*j%O+Mh}Sc`TRy0E}>O zhIV>_MO*ih^ZXyD!uHmS^`Ii$c$*_|fNJdVJw}6D>hyK@RIJkVN{7+scmb7u?%jv8 zVJmpfHOclPw3&`96}!24x+Z4F98aw!7vV;BjlJ1%UAM#Il+;{t+OAyQH0PPPiivYJ zpfo62>CB=<|D6e^-`|?6HmAI*UVmP%!#%ydHFVAd@#|&19AFU9>$ngbY%#Ld{OJyb z2gFfNQcPyZBD@1M=_X?eZhX{v4=iQavS7d-U9Udt_gblV?VYyjzz}&*2Qg8w^)|m0 zf*N;XGaP@-6x%7}x+jo(ZAU8qn**b(xC-P6Z*L0(A&~r@5h1uR~Vh>5p!M)<>Bp}GV^cFVfS_3E=!I>2lcQLeg+ zK1I*!V1{)Rl+Ez(N|XJ`EjBw1jw&mBFTtYl%D~8|$`3jfPNjVLGu6}=-n;;>B?D2n zf)5Kuiu4KhpN@l(lqR@ZoO+&n#IAqvHkd>a^3XnQRmB&~oWDvi^u2E1mgAiUdV0ZY4kEv=G_vb2P0tMN0C*!qsF?;l^&guU_j z7cpI1BMm_5a2}+cs>|lh#{4#b<=tM4^w<(e%l~ALjl9{zF2;1{V7X zL&Bkn=cVfLHk)4B8)Ooy>y#{-3=B0=e13P`(ma-Pi_k$f&T<~I18V|8X5syVWnO;& z!5a&!zj}0g7pbO6W4%+8;+HPQA@-v{*Rye03}vyGTkR8cLz(%Rk&$N`xa0<0s;~go)}~45|vaAS0LqN zyLz+kY3c>|YxP#-YS0(XL4Q0>mNd`0E~k>G=3kQ`s$_N}K}SzpH)}>jBo2pN1lB7c z>nBqw_H}9}N{NhP@d}hIWR7k5pr&8)2Ed(aIL=aa1w)xz_`}x#vFtDGF zG~-&wjHt3&EVad9UOJS%T9+U_iClX?i1Cs^6uV)hnMRZfYMcwN8QSbU5u;UexCvUp z*$h76B@945sxUP!V^eFvbm>0*qH{_YC&aM{XRi?*@ui}vfgR1WrOJ;qb;u+KX+Y9x zBcqDv^QkT9NfZgmnJhESy`70XO^cEMZh7L>@(#f}%e8C1g_!?M306 z>*GpTp2WN2;s%R@sfNWD$e-637ES1$SG-EeBZCF9ocBpw|G13U<=|L8&4|&6IOhnx zvGiE7@Pxm+1-RWA9u12vayy}1Osblqyh^7Ir5#0^Gs+V^0EWlDD?7&L>}Ea=x9JD? z9rmbHj>N_(QB#AE1Hv=+wzC7_M$4U8in&SL5a1H@k8)StLG7n44L8!rhhL;_ zrFHgexSXB|bJ#3K;!^ueee=KZ-pFh#O_Vx$thr$yc5V85V`w51sb1TA$&7NZ0trAo z`B9>TnQG4shT>$MqwV8A`08`q{cqLX1P7OR&p^K0=Js=KxgVC1^*%!6{nM_ZSR1bV zfBR$7c)N%nP*1J>>Nc#Y;U(hGc==0KT|8a1_aG4Yh}$+Tb3fzg$k0i%Iiu9n@7+NT z3lXy!+ay)OI3+SAq>>ZKY}kmIyOp|A1DTEqC@eD(zkYNhRp=&>WPlalHZg38uvnmhc7vaCloCbm}Pu6L^L=tMXKrds-RlJ``N41HzUM!2I zg^Nz;4_1S>6rSt)Pwlcy={=tVD;zG20-mY`_s$&*Ri=^8Q>T0Gl$ELvjQq=irlB81 zXRS6SzVV>CF?lFFNeQ|Y!N~{jyaItDULd+Nb?uG*rJZQi!*{PLN@#s!4Jhex`pQcS(~F)F6`Oj{##88J@w3 zj9o%Ot*bTOwK*+c@=xkEH(XVw9YvTlllvNO{`i)h6;Kn0T0m*DOzJf5C7;TC`Oij@ zS5hgc{s$2B`)?pv$E<9kSpCWNH$bZM$5tA7+hOaKhU1~W1W`36X!);j`HA2&A zhF#^qkQ*lkjhK49#x@i|z*XkfvWcYzU)G@y4} z;rK9K6dvZ$EBIsLW;JzizJWR36X@e|ZKBhOcU+d=o)u4NnTCgiv!vHqwIHAPVA>FW zG7nL4qS#$}{ozZc*R(G@Huh4-M1v*c&zaz~Be`B-c=U74>Owcb249)^2ohOyavz<- zuD}&%&owDnM@|9<(b+cGb#x1OXlY+8nN5pZHaa{ptRrJVID4PpCp&4_Z;>kEKyaLo zgGo^;C~)N7i{2P{!QnSaU+QK}e2(s(u*KW&W0elvuM%$qq<3JUxx-t7K!#VgeDFV}midoiAy$>P_m>hmTw zoqL1~JpZJmGjv$+TAy_Wyg&2tZvH_&zm~f8bEa(eh*~CV^-Alkh{iC>K@QzN=DCML z^8Fam9Aa^HN~U84M;@bUZpjfHj+V2aoM~*P$U7y1aB`n$lVF~46J&~_n!@N&X{rXZ zn}rc70QE-w+GLg9jRjbqIF8;6q&vQ7JwTM4Q|rN>646%r5Is@za8?+SDuFGLhbCTz zoL}_vFhtD7bMS!dyh9skW+?8^+vfEvE=ny%jYQg>tO`ZnyI*zGo*Bb&7oMT~9Y2=zFCR?$-$OO$x6WApE#=Ey3 zfAs3`y8D!t`a}Il(TB5rpOoYtrU%{5bYgW0b<;kW)WJi$JXu+FoQDbrz}N-Zyuu!x z&NtMaNbk?bGo7^N5$^x7^?Pkxsq?3L%=knqMZDXSSFbPU{qSqjh%NR#8`T8g3;57z zhB_BkwFrYg(Dvx#nu>G4K8kuhNc>qpz*hTY-q1MQUayP(^tI(7}qCO+)}>j zyBV!?lA7=jf7d)s$3yNe-|68W?u5fy#J@ttk53!L!z~i>Xe4SdmwkcTz4qh;9QK7> zdohD9toWf3Pv|n+9^*lcKa+}M}N>`T;*mSRb^+3M^?Z@pPqyM zU1v)*9mvWxHLl?pZJ6e3-LzSLn0!nkFT5dMAlTX%;k|d->H#Z!#AatTpmjq>yIPJs zmOPtQsjS>x`+M?TeEnx_&&AT|{P6@q!%hVn} zJ}RUAiyv~*VbJ!2Lakt0%-nW8S5sXrv!k=Iq~>U;LNqTH8vJc^dh!Tnc{#p*`e_TD ze$-r>w+76HAqU(0tB!P>*xpQ~-pI?zHUu|D3(DCZ=T&}uokA`F?bO`|vLcKz9hpHP z`WaP9%=q)-;Q7z?$vCXi%*N^$YtNR0&J4p^Db3SI)%LOO@B@xC|5^Y|0#=HQPj`S| z`9I#){{w7VOcnqV9`J%?J6vs5=Q(gcE3Q>`;3x@?#0pk?2j0lAp0+4&vgbM*Q*tq)~{k~lJ^YMBp3+P zqybTj%+7`W^WJBZ%kRawXE_><01Q}ltsHxYq*WdMWQjl%gYPrFxoo~T^{LAfDxQ?U$ zLY02|qr~H1S>ekcykL&s28W6267hgx|LEh(*Y1C@lXxR zPZ*TGf2H-CKF62zvEx}5a`xQHjQAYR6q8PW(c<99@qfBISZn(iVpZNZ@8NMQaSw5w z65RA{lo2+XCgw>^kK!f?w~yQ*9S-praucFF08`B2sUs||2}y-vQI*~>bXRf_om?k$ z4mm@*K%;&}qODXT=WsI;eBCreiDa)a7R&bYFpLUy%d!w^9G~ z^6UQZ2^GrXp1LE}#fl=&e@D7X|JOD2)zAVQZ(8`Ibi zFsxAm5QcRB_jfZ7vJ%?nLxbtOmbtRnnu1Cn|H%W6@Q1x4B-B?jufa_nlzvAnc5|?i zf7Dr$xqGmkQoPZZ`Q!$G3p^ObqQW9${imQ3q*(WzooyHsX#U&3)_M5)s>!n~*wL|` zB$bu8uH0idl<1H3@4wuI_%fQG@nN3|?hJNn)w){ws%J4lis>( zMSiVh7Vc!LIOy<<&=ZtF5;A?M(OPMzWtuvDyr zZLZph^{12-`9~ZHVEJVBMP7AmE?!%Kn>Y*45yJ-^UciFr# zP@lf#|3f_U0QpdpVHN%rp8R1Hj)q6gk=nB4K_>iXlrA}>!w-0gt@b?ZudSUnoJ%On z1ZJ57o1**lT%WUyn?812omYlr{4+gbi$65^3f>XL1gG(OZ--?0 zFdlv9-R(4GWW7t~e|1Jz8e&5{tu~qa6qiT#k)I9vQY#@&Ff7Pu{qP@%>ebX#c(IV^jHx`au6? zMUf4=`!Z}%sTTPxb$M&qwuaP;qppW{@%{3Jn|;K8u8Oh$zAEaNEA)GQM-ocB0*H%` ziND{m45nF9Dl5mnNG|AgRbwKL^1>p^48o`*nsh;Fe_=(qhAeF)1izZK0**dSA^`~D z(ZdYM68bq1b*^Ni9A!|T@@oRs^&vGJ*L3bfgWA5n90dxv3a#?PphuOp`%tg_bej*F z#m0bqFo>(bO!AUQ24aT5%90Iy>sn(kkjbet9a_>;UX1mb^t(mhquOGLo^UaEgMfga z@DKqCcU#bk1K`hcJD5@y>C46$7hI5>Bhgd-zGBU}$UZq&uFY=>l{_jdY!Vyi*Hie~ zat4Iy4Aj^7w+L+k;$+bcM~M2F&wfD_OY+f~!Cq?gZbb4a|5ZWTISalyZssn;5FwL$ z+4NFNZ)!Gt?qJCxZoWk`jzzoVa}LDowrkyE)%KaodiP-f8&HnJy?=K^wvfAy(o`6@ z3}6a=?j=)<>cHCZ#2|Cbj|MJKvDA{hJgL=-?I_Ou`?wa(T3}9@rO#%9y>-rG)_)km zQyXGOt)CBZ`zeiVyV(OY;dsOMQcY_rB$9@}IBp0Ifu9i5s6@}&3a4y`L;P)?x}f81 zS_b2~;h&#?S~*ike=f#oK|g$_Z9XKfBcQ|40*JHCxWyEvtC5afvd|2FJ@GRWUWg8g zZr3x%53`Vo;Pk-29iAacXUtS^NO>p*UuZG-zr}RJzj=*+@I%=@(l`M}YX2?E+-(48 zPX1o}-ZW-MZ>_>ZGo{Uqmffo^u$Uur#+RKc+Vtx)pP$`X7aW=iX+U*x_;isntcZ}f zKl$r&cfy*&%uFZrNn}mn=cr+TT26`*`J#d7TPzD%cE&2epI1bt16uGCb1!@9pwJ;h zp#$AsOt>VKGpAwXr^XZ4;kl1X8v$47v8(&bXECX zFyIG6qj46?T!3S$3CJhk5!HW zQy_nXfK{)fce#zWV~_fB6j}-R^{PQ1EpJa z4Emx*2~2n~I)m%|D#$jX(Q6oX_4=SZK&)m#LT3CRD}I7m_CRd2noE+pdb##-d5J!z zgAUcrcIHPqkHZu>gGs&?LjI3?>3IyrTvow3t9tW(KK8F zJW4J8wZ)I>)n(-Po7156PcHR5RCO>DhvT`E_P1UPRON_+usX4AdnNnaons8$5@WK` z$XZ^lsq_CUQ%;Uos6QwU*w`xwI9NEEGK`pE>Ha+qM}Z6rIxg#ayPsFTN+;P8z5H2b z1~q=y7WXFnLGjiQAxJfaYybLzol06hlK7BUR*9CytHC%N%XQ9WL?vp$;Y0ewoCen! zSiKwC@CZo$NeF1RxS_-m&1-$L+sytWZpA!*}<&nYMIAco$wE*DzSX8*jHAQ4+_haU8#oywxQ(k zp51mt;gY-#+E%2o*la`DNkBToQkZ?EBBTc?Cwu=dZ2;H+m1qqPm(1+O3dVpx+ha|b zwv}*Ngc6m$9ZCmJ5FNz;{H4DaMl>^m2_Tv<(Egpd1=^-GhhoC|@8iU%bEL$ihJtTP zbG((6a3NF(-?-2T-;Q?YN!SA{1NVqLcyKEVL=Crm zb;K9JLq~4Au7uJ$=svja;p?qWU)^wO4 zlv-+?Mw*+d3R`0)PjaV)mr5g(1`F5=N=LoN{)Rv*QL07F0-S6dXS$PK{_E_ z1YBW)KMjE4s~|uYXpeW2{V71C_iCrZ`i&(@4Dmoc+;wS`Hw|7O#Q4kda(u-X%C3UK zAH$JSQF5dAyt^wsr?n-N9)a&iHm;B6l zLe!!7X$3zzrqF}=u&o~cqf)Bmxx5;G8E?1R!C{)rCf$~&22qEo`E1vnmXF=#!SgXc zCETi;Hzf>7quoLlg%~0;$uRG*l-!Eos5u=-Ns8)8O%eztb77Dxzve# z%m!6e8M}1F*f>Tj1Mf?+*orzHn-);JA5;79BR01pQq@QfTlibfNQfEl$1K9^cW(u z4^`uVW7onr$q~!&+YRE<7)e9o+R|+>h4n$h^y7)RpbzDseLP5?kSHLwo^geUYGmo;v(&E6}y8y}zX`|6d^=B@LB(w-+7M&;rk9Kk=++UmJ) zBpoZK8-O->{?|7G45}$_4%Y5rJ?9cS@yp&1r&iv#qVnr!pB(7*sLX8?*Fb^h zrtE5Dly^o*v$y8Fn7x9vQgN!bI_&n%!sxAZQ9j4V;{eB+BlF#01O4N_f>Z58rf*+Q zq?BiapYCRQWRdl8o&+AIllAO;%E>Btn4{q5NXL~K7^hSc_l69Feph#OAqe!cvckarjuH)N z2X)+Rz(8X}`$@yZhoRS6h_6s&J_u4rNjZ=8qGnQE%XG>D2S4ysJ=g}S8B63)ljKf@ zglfLrhVF+FnUsZ)CaEIS_)^>bRv!8OMao*r;kDG0l;809fCfm~_SN=lFJ}-(L0gVT zmUBO9#oL!P03*d@|+%9~Efq^pHi=Dh-N|695CzY^j@mE+PbB6lHC1 zZ?A!IlBwq*M7!ih62BC}*lN49baX-Qr7#*=G8?cQJ2d8KuoXGxcA6JWCgz`k4xY&v ze7{iTNuc)8=^b;M$L-&(BkLmope+bTc0|7@c(eH8A(_ioqy0tG@tWK8yv4Ehn^5M( zH7;()U55i29PV?FYQ(g^>4J`iiXP_wPd{tJwf>Qg5>(i<3VzKBrNQb!xa)qSirab;LS5Ay|v2KWOeWsRemkS^2$25Taj- zBO|W5NR;6AOW1MS#QO%5Ox=bdkJTg2@l{ZLd&Njve{^{T1ltc&*gC&m2xWwn2&xO! zx&_gk=9y%*-fNmxn-WVx^WJxmD3NIsP(MU##kBOz1Vi?7{^kX1^rOQRkL7(FjGCqxB7JI?UMIKEWJf6{ zpzbZ-ve=f>YmN!ub-5M2dr-5=oa(H&kDioexK3ic4D>BShUJ^+QqsT}=Umi|M;JTi z6OU6vgsbIvP*iOFeLdY91u|@|)9(#^MmO)77EOyIWny5}jw_`*MVU9z*q0)36Jwu) z)1w|#E5tdxr8@DgD&9o31NSOoNrsamH-?Aon8}{rUDM26|*WYK|Aw&SX&8 zZcr*xD8)NefWA*o{VaRqRB1xMeM4>zk@G5+vsbX*;B`cQagJx=dC zSQ>%bklESe8MeB_@f$Z$0ajDrHG0S`cPkxJ`c*OK+A#&ylXagl`4RKLz$Ve4G$XfO zRNJL}>%6WqWZAY>d>%6=p2ndX!NuzGui$H2j}0GAh7m|yyFM-4M^+*+7%2H&r>AsV zDQ~xd{>zSJhV@?T6TTS@DxkgdDI3AxI@d&D(KGVQ2EBP&NCRB3c%RV*yzS%QH>vpT zo4~pR=9AocZ=Bc-=siVP$BqDYY6$aSnvbOR5I0fmU@oAHHW(kQCp1e(*i$|AbT-=) zv7p_o3vT$RKXvpWxjr$Zj%FKoD5su@@9xe{Ltus=X24{`s%I3?-QkpFn$%WEi)}A{S96q_JnAMXoGmMXIwBaIeDu#_ zYTdM`5}<#oaUP;p(m=Q;^qG)`s1*U*E0ePc0*r%1va_mN~F++%X4Sf$yxYcOud zg(}3JMrNA^V_C(2KaHhQswjWc+2OVH@X-q>Ox&djb%3y!g$OVzpfYOs z*P>B)!URC&7VlYHVHeTQnP|K=#}Dz|3L9Sus))n2M7Y2y14`9S@=qb35prKxNIwEC zzA#?+CtHMZp5yA)oGFS-PXW(u;UlVe6duLj$GVTR-ndY?%<88b~m%40OA(v z7L8gVeOS_3-Y)(q9?b_l^pMab;(WQ}54a(w)&YNqxENk_i~XU1>j;yX9mUcx+?91G zXDot71rC=!o0bd`knw8c&}_VUSEZ)BuXHR$xvTSLA9DnU4J8=?pQ5(O1y!GZ7ugH3 z@ac+;9#dPgUP5;p>GG`u;*QnM;95QR?=OH;<%lvuAkF1{Bujpehw5q)AO1aGPL#5S z(yF|{vF9;StV30FRY>>M1~orC)K|naDNQS;QvlwmPH)9Es(Z1&3#)EL!J0RoZwVHw5$QRS?SJdnDTk3GVu=MI= zLA;fNbcyXpqGNtf_x<>b(k$BN*`8|E?M{lbkx;9>$jnwPrqL=#!$V>16ugW6?ih&! z%GZ(q;>jOEQ<(ZUtDaX}6b`%2dw6VI0uxlJf8tOAp_Q4M9bITYnNyjTZd@SZmA_4h zHb(z=$<}$3P5)-uar2tS8P??x0KqJ2I2Gx|qMXiL<0?A4;A11o%6HJt`lFmf*k8d* zzc?{k_Wn&*H$rx;ZB#U#)w8ZfJd1Dro3v5=o6#fL-WB8&Pc1a2qqPv%Hsr)sT%Wr4 zot`O(u_9VKF$e=92sax0TIZ6BM%AxK+?bk&T2@CVx6bL-_Qn7HDV(8wgnRtn3ZG!H z$zoy3@y@`rtaAZ8H~=k~F~?p6RJz3j;{i3-)T0=b+(t?rIKe{U4VbS~a$@B1bj+Ri z)Bh}J{O6FHnfkAs$0yAXjlTaeP}U1*7-x5lS~(^kqx%X>RJvZq4~d1xXi5){zPt#N zEU2iC3brw^yYx%uD#5Gbr$OcHSd0qWdXpR$77Z>q7d;0)bllPIO!+b$IK|-V4rCF}Z zXS`@UI67@P(-2eDvCwJeJ1J@&obV`C6g^!cBdT94Y9V}XL`2hYaTk0Oc)yKGVB70@&VMP=mR1EBES$D_3Q?$=~^l%X9tzrqJXW+oW zvoX^R4Aq!G9cejK7XcVBCf)5i-*(DvBzi<}aN@p=+-th8QrBEFk*rR2cj;k-3Zw$3 z(Y|?y+O&ZVDO%HM#ByHpKe%S1VY#@nuH$sYs3c*g_Js`vc9akwaO+~J zONfBpAEJG}k^PG39)=<;fmnfXkiRCfHqwlD=+FD_`%T6JkGYz4TnA_0)YSJszRyN+ zHJ~j1x>6;9#NNp&(dFzoPMge*X( zTc>zF`d%+P*;o78+6QO1g0u9q)Uz0)e8E-Keqb7t-r;FXd;GFWKNkuI@z2&U+aITh z1kr+2!8&@30hcf2DRq{dAkNT^R_lvk6*(Q{P6CHT$K<)fi7;nUPn({YV*Q(rZ% zlSFO|IZJ@UFQb3W1{3Jei68F(gMsKd1ZL2jbg=0wyL0e{LzfnvCb#_got|m3HiV_3 z3Zk3YoJ!UYZrT~3k^P{9T!3LGFwuWwZdQiqH*{z3)c#Jy{iq1W1?#a;msG=&ym&Q9 zHiNa785G@;+l*@YF~?;qy}$9BjjqSz8Y!TR{BNK4$DsEASAyhYqoPR`W8-*>SZ`14++o!dOX74MjMO*RxKvz;=~HP0AF{xaQ_Hk zQ!Kx8j^+EOw5H%!E|8UZUP=?B2n%bXAjkG$NAE239~&?vJzk{4y#rSB{5Wgn{S@JpWPF4(=`2jRxm%?N*k@P-fmBK~OyBhaxc94YkfwOhGJ=+}gxwE*B zeI6(wj@SO!w}*H7^|K6xkQ3}>(LnPX%bYlgdqtps%~&UhX7Zow$X3I3%X{^Q)Yxk% z8n}<2$o2M$vxrQJYRej}0X>8x!g!E7{LmuyZiwr^mJSxg9J2jG-zGcx*g{d|f^#7~ zb#4NM=fYaOE3$zjNFOk#2KeuG64jEA9L(2J50r=&rKauX4}Nub8;p2aqH115GF5FC zy#AwxiE}(n^8@66arPEaRkmxp=%>;l-Q6iM>Fx%}Nr%9syQM=~8YV3zNP~2PATep_ z5J6g6CLj&xomU8SuVO-1l|YbFSHc{d#i|eefwfZlj+e zfu%;$UV~P{@71+k-nqKpt3blaMAuc6m;6Y&L$!o= zn5xS*XJ1idbi13N?oL6jj+;(3Av&1I8ybdK2mW+jt`jgSU2j!=T8jK5Kf24stEEi$ zRYf-8guO+BA5dOsb8j4f=DzAm6!~}gnqBPYe9!kl zWP0(kFU@sB=qb{6^cp7dwJbqBjl(9O0;ek}aevw>j#b&?^D+p;hXkc)`Q9BAg6beK zxR4P8!3EbxO8792cyI+(uCFIcOEguCF@ZTHdXBldpdo{X2aguc7FVA z^-)9p#?99IhNf51-}7p_30+|Gd}J*ZXKzfL^IgY=_UR9QG-Nb7uWH?#A9LO{zL9_H zEq*;t1mXzDrs$?ZEQv8p6;oKz&0o5ojcZ+LpaVyWoaP3roO$QKfEKC}>bvS@=Bo8X z!F2??C1>qxyM6p7H4*0$=XV;iQaDI@%DEEfSDR_kAoF>QCCCANd>i=2ZUkaC*ZqOq zcREJdacT3*i{gbBm!lo|fvmM%k9qBSVS>P<2P?MG!|mA2wByHS%zq;#M7P||S72#~%a9=V{{E^CETnwXA7XrkbU^#G>+j_E^4D*lOnz)#Izx?Tv zmm9=Hf}AnAQ9%E&P?Y-K|8Ak!W2cF9>PmpNbHVwSDH)}syLyq-sjsI7%cx(}bH>s* zjO9z1nduM_5>^awR_nbD?L=Gj!X_UJjE6omf7XYdPva#S7g7**gH}wA0*a)OiDXWG zZh_E`Gpk4a<)>d%#wxKrs4VrTZaF}Z>pmg0Yy0D98&IRSOcc`{jkx+e%$UQh*|+@S z86xcvKnY&H{|ehVK@_f-1$2VfP&P)jo4F4&RHRD>(!>??hvA}ys7iO0u_54(gyGlQ zP8AuO%}~|RqV=fCuOs$hhfsfQ@}``|y^w!4wEyhwwad{hpu@5qO_%WPKvBg^@>ohN z(vx}bch-%TOPV3`&>%h>#a`;mNW|>B4mQblfG)(od;X=4k8Q^sGGdNpf{^_8mzt6A z37~{eej4drgfG(>&j2C5$-YS~Eyo;C=1&y+llg}vux-KAf7o~TLrdQsGc$A23oKEM z$;}{CNoUYfP~5fD+PBF#PLDY5XSqzlJ5VSpNo-KKEk0dGZv#gd31Ey85FF&C2CJP% zoqzGo)?b@rW5FXT8;lzqyPdth?;rO#24Obc>=2a>PgE}VfuHW+pmLTdk9D{zt8p$f zZ0lj=Z0(E4yr`oIF`Y^QPF1_SpXUKb^>U-Gz-vazcJ{^Ddfc|h@8+P_BUa{0O%!R&#Dxd&YNh5^rZqjQ3#{a#(teBE6>+w2AK8S1wGIvH3vG-ktyA=KtFz z`jQ?H}*VfmJ-}3_v@(02Y(RFRro5e#qf4w8&vHo@K*fyJZGm#_UQsVIj z>@_@?s?Z$F{H|U4_4BgbT+eAGAs^E;DL*>OrGV&NMNwpSv&xpET zFZUuuH@M%M?^SU$U3eKUc;U=W14Teo;swBIp4kY1d#~cHeIdgrn01bU`wmM1LGKMr zm%RD<>o4rftj>(Sr&mu_u$xg?pHP;B|T0X&U5uQ6c(XMx6d&;zXx3I55UK}WF`;_(U zG#C1TuH1^`4xAgBCq*xl%m>wSvOTb@3ZZBB#byxQ-XemLs970*HG>KGfW<=GrnPEn zzRjenF@4F1c`@HM*M4kyj?_d6lBV@^rw}M`>Uo#*Yr(|#3|$=H2!KFg&EbE*j_k4a z>6~wiBC!5po2S)IXX{nHzk4`Wp@Bj&dMlUD6DMm~Uqz!5pT4)bbu3JFJw$Ax7~-sG z*k0O&2A#5?w)NEvD>Ozc;kc2bu>u-H4hb-vR1 zk39lo^X1*v{i(ob%!cb4XCnelMzer!w_(!^wH7V$@~|;tmekTA;OA^6hTebpYr~7< zK-Acy+3|<+&G}J|kx}V)o<4Y@C4Ui9l+AA7$EXM!`!;&A1qM)Qy{z?_==fQet5Kje z2kN*Jb$?t@c18aY|dmZ0o`-kuZNjCzP!lRkIh*6>uzz}TQ> z0Z$r%yG15|F+93xA^dFu&DCOsVpEwlx4pTY~EI-e@r zI9>O3HbpM=$zLO-7&Ha&f65u=Cz|;>_=y@159cq=m?Jqiaf?tKy+e;c>>f}4MRc2u zs<_OgH?aBMjg9(E2m4BG{#7`vc^BwNGVOK3f~Bwkc#KWtdxGQr*?`qB33bk3(kmXb z@AaG3o1{Z?a$MZ~yPLu(dMiFi;%h*Ru-}bxsaEG-zKkO3*8N4?=D-?_+napo9s6}@ z^TX&oNxzkKZG-pV=@Rb&ZCN4kptxK$HPkcM?+Lhq8n%ah)C>dD#P%J6J#Y0?t}DO7 zXPv;-kR&MFzHEEg-)q^00j43qZx{bpYHiKwm4Rv78E_udS){CUpWYd#fcTxmH$dm= z0HaEI!LRBC+gB>TZSWlit$|UYC92d;j~Np}R#usH8Y4$KH4M@t;#|34VSy3;QhEce zf-UXyGdBcUWHe^^TpG;X-AHVAWM!^T*e}}XeR`CK*^SBAW|C^lOV|_e?F6I7fjN|? zj<&|{f6VGlzt6*`kx!I0^@F7-Lv1Z;qQ)qap#wH6?&iumG$v&|+fLhf3fF#Ek)NOU zG6w9twKfmM*GYHyP@a?Q1Tz~JEPQ#M(JaYHoZRnTvcmf21mg8 zHWc8EvUH;MRca3ix);KSz;W2;5vt>A(Noti}Wtj3?)Lq@4iOYG3Z_W1_0&TotxMmw%^2d6gv;3AX zxI_3!ctQbx>H&5`%xbbs5}+QGy!qL{T9n42J#(wMPm5ANK!u#-EF?yjs{9C}x@r`3mc!974UA>$2@<#W%@D9Y^YN555+V(wta$k!~=Y*CqS z#!f+Uk6X>35&oUk%~ce~+AoRHXQIt*&o}_GJf#&wMveB2i^9pM-`t+Z{$EMY|F_D{ zi{UCmak80l$bZ#zo^_H>r2JJ6%bN_UL6=wM>Qhc%OsWldKZ;9?!Hq-9sM{|a>zK#A zAw1xgImd&(VUeP&+8TWznH{hff+M@pOt!JfzE|8Vj&Oa$VieTODQHIK#?1ITWbsH- z)im~YIN|IfICoHkj0HB`A4nKH0T`2pt-{P`OUvvClN>cNIx_?BT(ypxP*X2N5{2i3 za{ysk)!?VB`*wdst3*+|ANwxzrSlmEqo2(5s=F^Ha1C~#rRhNv+XUkc%R5G*Obxyb zdjbGSBKg5zV;@-Gyvp`5$#^aITnB;z&{x1&VLvVG^4MTi1loFvy<<&@IMQ+SFHmJq zq>=y2I1F7+W~A zV6768cph98#meJ^>4P|4%W%SF3rJq3W6@S-W7oWRP1KbIny{|I!3+2XxmCpt=EZBU zU78@m6a&MaZH}!&D^iQ9Zq(irPb8L|Mw#Yz?l%rEzKF__Y1u*~8L`3r4-}j&kpQR3 zKUhVy|9iys?|yy!;lKR4BN_=suDEAB3dfh#0nE3$oBW;XgD zE%E%X`qO~s02)z<5)-uw^{~S7=hjTul-fM*`fF>mK_lWz7hS}Gq}ta=mD+*P+}!oM zm`k2%8LwuNJ^(Tu4E`_sEpdXI&3;=?4s1aWbK`~(W_1;kYOCFD*jZSP-(@o6QPqwX zGy%6v@r$uHVpim`a(F0`A%72_xnmHC6pe~L?2}WO)%O{V(VYe0PYZuMyx~3zv*BXg zkbg1~_{KGhrE5*SEH(oSZ1?RED9gQ2`Et?=`;btIz+YlM1HaH`G43X)1ER zHe}Y9+c!m(6rW&)v9l|ofLislH_<>2D{=U^l`z*T*~GM2ixK7qm2_BGHdOxXtigs*9+4NY^KAKo0Z@pkuAqvgP8+jz|=P_-*T3VRLi6iMz`(dbX z%$h9LQNGn09^97~EeFuJFt)PByd)Dv7YJ#!jUnUd>;Q~g9-p>HuCc_nF2!5})_ae8 zG$&kwKnrKAHgP%nnD~qBCZwC%jN!WY7>WW#NW=9h}UUgfO0)ODWGsSC3kiRTOh^ayO* z8BA*dXrR$ELrp2`yXk?7!9sPaz~d*!G)~t+$iFoovL{@|YGR|;5QJeaX9b3@N>Rh` zN58INKu_opNxYJu_In`ijYToGFz31RFD;1MXrdNM-3OZ^NGkq_cQ&!Z#UqV+0A~-qEkHw z9CsB(@mAZh9pjC%%q^SDd%d)@}7A5A(g>N|QZh~%tQz^y#Iyyc`* zr0iaRdYJ(bJeZJF{vI!E%>C+Sq0TwP03}|8hPXj3^)7x04=N^< zeR!izll~5vF!Dm^5^4dCqZuFjY?`kPrfuju?N2&hrJ)+}Jl6gcunpkPc52z|dZP@8DZ> z_qoh5h8<%P3~2s-`x+UsE@WU>bMQN2Tdtm3-@LU2KMjeE1#5Q%@(jT{`Yn_|IjQaT zEHUlQIacq9X_NThpEX)n*gMm(-qsyxYFm~BRi;l4dh#soSor;V;_0+B`Rl(gG4L9^ ztg;Tp^gq9_$Jlfit^WS~%N%4lVr6fY#m-!tga{=7`7fn#O=gUQO*1$HUzm+n>kqx8 zW}P}W829>&QT|$?nH$ij3m^{Q zxQc(mQmv8qul^abro?YXMsm;EZegj zJ-Ed@pU!I?M~Fz;{|Z<(cXs{^Jvnhwvdqw%OJl)`Z?CeqU5KxGXcrsdT5TslLX9;y zhuqQ6AWM~nEtv$o4XUn~1sgb39h3-G#je!r_;srZ1?MufS55f zpO0=d-QT}qZ4)Ue{+@fd@&xV{ehqbl{Xc2l&m+>FR#F`f2N({?9bOC%p2DvIu^3>- ze<1o`KV2%odwg9T>IQ0A; z)FFv*{QRi2waCe*@^us@szTK5P{#TLNdAOwVBNrg{cGQuyH0Bh2Ig)L>Rx2jri=95 zg0Xg!tTQE#Y!KQ4J~G>!|9{%_|Ir8l@jGz!h`vxfy#gw+9`Mi}9=Dcbz^8aLHujil zteYrP5a&{b0j^__4h;OxEcJU2740R4uR3tfH2i`d3Oye;WcGybEcZZncmlxNA{i{a z`5{3?eoRL-+C{et<)A}G^w&Ekxei?sgVQznO9!+hpA2`&AS+!aQ;lB8AOUfslwUBM zsSG#^asPf6iqnXU{%NY|$%aOEGycYGUPLQWz@jF9A%Lu`tUL$;cwFJ*^BK+`iNn?L zAcU-R6wM<^lzJ=1` zj>^1xfFa>w#y)88?5nF+9k1}>71#UVlo75LHi+@(Q@+2gK?5p)qU$Pz^-J!IwQ2GC zeprX4h2N?e2%44K4T6bA`+sk3QS!F8D&7Q~maAf27F*nUOi&d9(*ihs1Q6YSQU)hK zKyA?TY}aL;=nb0QK{nrB-vMR}xM;^=BGFG54Y0I1Ip3*l0H9^Y2_TK zlZrm_P7RoH7GJ|mb&(zcFazjgr}0jD`pDQm+J`e9HWk3O%3zTZ>ZXl$PFG;-j^x5i zKp#O>lPbQ>>BF@QN}_w#dKsg&b9|t}PMza>+N!-azUwuYr>6wMjDLB`e$Zk()?jKidQ#JVL z7+CVbuq~{1Th_?OitC(2u4{c#Lh#J{4CRx${9$62jEIu$^)`bVWvnyhSSu+d^S9}Q zzW+G1cTJFT+@l?6I&7tWpjSHJ6qIE~e#|ydLc3^R8_3x3D|77DLTsEprOuc7Fz}ZyF>f&;ltRV>O`lU>$)N?}7HXw*WYR zuLNhm_>YFbrS$*Xvi@zkUhWCowk4^;C8o#;kaI}$a()a2v6`v84I(6Bnw-#5d^1o% zzX{xZzs284w~BCIQ@PMFHqNm&gX|YVj8&61dhAdkC8uSOluWz)8UoR+q$(HrFFZC}QaI}38!ZI?^k2I)r4zEzz@}r) zg|ol87WlZhuK&bgborlep0q ztnY-OmsrO$w!DTfB7L+vkr1+oWY9oOIk*t5jH`uJb=lnbSX1=U)SV|zdp{ri#O)+s9HHaAa1jm*C)9dZE?SEq}r+(-XsP=V6K&YBEghQ zdiQ_2Rvt$6+x&I}NrbnJ+N=i!+irJVww*I}L-V_#l{SG)E$TOhg#CD0VLs`P4@S7+ zud%=51+hTb=;%mpA7564jm5q#5fvd*U}x~$Bqh5RL_!?Gmj|a!&lNo2^B1$AqSzBb zx9gibRngq=F%qEe=FZ@@Di!A{Nt^2f^p6)mnfb$~0;af@C(7&$YF_zr3lPPyvQ#N4 zdK<)4`8d5>k0y1>>Cn@lAUAmDDcZ1VvaoOxKcdhk9&XSU{ZX|a$4~@bX_t2Ii{gd0OAHy|y-AUG{5x-7P8Qrx{c)~#QhIC(?tWB! z!Hp3VBE(vTw!8`T%qH^=u{>#J-N;vAEyobS)J&Qe54z{zxp6Ss0fl?h6r{!9Z$9|&?$h7xHl zZu@S{&Ogr5{&cI3)*Jp!A~Edf0|nz@L(=Fqz%&O%IjeT|7dQ;xTBNZ>SVv15g;@^R z*tGholZYj!Q8GgBaP_ z%x(@7Qs)rw6@A&3x;;4XS&SPOROQTaE=p@mQBLDvS`PL~b zOm)5@H~IkS{dGJF=%T3b7WmGbC|1~PcGy^M(%(Dz7+d}x-qk?&?0{~pbb|T8?KQSX zFRhLG{oM!l%Glu|OuiJn+T?+zIW2wgD>es*;sRR6QHRtU^m!|e4?r6PMhm0y@haWG zHeY>xyDl25gRx)r7JW*6f0xU=?p&*nMzyLNXYo4vr_t#RTyRp|W!i&v8kJ1{A9>qA z$akLXtqi5mFjXikEQC5$6Ai?b6p|~&Lf=!R1BOVR>ua0RM(9g~k4Qc++tF5>*R}cW zqaTb?HGQr}tC6Z;u6hY0vn`FF_O0T0g%&MFP{IOD`6Vcei9nFVmqbO6Bl^BR?Mn*A zNEB5_mDXV=@=@<%NXWa^xkT=w%IhvIF&xDYUcuENA;QA)=I#Sum;6p#7 zY@EM=87dVi<%b7`j-K1wv z>f_2u<*{r}|DcAIg{jhNBpH;~mc}=~M-8K%M$<{MBc_KvZI`S(El}YqYk4yz3s_ce z?6=?#0)8p)f!h~J06_vchAX-Dn+rj}MQz5k7sw17LkFGiUeh3wP@#84?-vCCT>@X& zH&jYUONOt6ydJQ9dP!|2)k&GoDp(<4T9&04`{8)K(VMqgXqKo3W(Ew1;6H5Ozwe$A z(R&rC690T1U5C~hRu`@v;{RTy6ex)a$b|J3Zgi8lMy7 z#7ZVJiPlVY8d*$TEA)J$Iu|LelD)OHBg=}{UfgGq2#iAmAxJ?O|JAL9KQdMv8_mRb zH$GWE!4Rs*AGPL)S-rb2p;c90yYIx!qWDu5A*@!CpDo(&fAm!>Hr!*Dn*WFDNRvJ{yeQt%8hHM%d#AP^zTHqSON zc-ex0<@x*hqZl&z4d!wsWp6vTg11@*dD$(mG!v)1!>Tk>7k@=F=%dbBXmpe1hzd|L z6J1E-0NvX`QR+|zJ+wlx$XV!d;w z<<*|=eX*ie$BEBnslfQ~qeX}~>1X5MPC{wpkNLllM-YVMhh>9~tMYf$cN8vIri)88 zr6Q@L#}Ei0q)4d1j$k7sC!)%|u7>)vH%_+i$qF~C7N_Bl95S-qYG6Gmgpgi%bk^@0ZRlY&v( zjl1%D?FD5WcyY3X!_l-LY1X=1 zyTcAAU2HSElQTiofpa%r@Hwo_?FXj(q#-lUgL{&h7gSX?qRq2ue>G+W8AXe~8Xir4 zd=NA}_TNWsdjuv&&x3n3iYC&T7agTjLVevajxAT-_=v$jd+yB`8G0YnPj(4QPqb}G z2xpjwsRG5UY9_Jx|I^IL0$CJIm)@cnV23TKe^&^*U=Z%Gn4M3>6F{6Zs;Vv?eEpHo z?>_s=2UI|AN%;fv>dFCa*ky6Q@jQp5f48OFthhh^WX1z0W%23~ z4vtvy1tsUkspL6x{T#gn8IZ9-FHrB$h{;qI&p%DsHdb>xehZ_=E*st&dXa*G!V>}3 zKrKs&NJ+Zfc?~VG8c<&OYNHeKQ^h5S5a&C=>iuD`HA~`ovZ{8H)Cvl&MjXk_mwWeZ zOaa&p9JCjFWe#7>$gdPeOCVp)4O0yy1T$)$L85x>H3UO(V<>O&(3f9M4?M9TGv#`H z!u3yw)Fz8i1t}ttd!$iXLX?>`;RB4a7y?6be5edVKlK^N3&S4*1dfQV6%Nwfhnp6d zPZ1G_%La@+$HCL>{(4y;pXT$uHTCw(A1d|mvyy^NV}RFVIb0No++w_9`BQS z>W}YD(9xPM-Plu(S8UJ}S%3~4j%GJi{&CE>Rq~ema9p5RrZx|rNH}f)BvSOHv-oK+B}`jr|mvjA`a z;XdRr8~qL^6h|qemf?N*Gs^t(EN5&dZ&mIi5aH@o#Rx)dlU>jQ-_WfMO03^@yG8;; z>nnZOC-ls)70-h+_}Q#)&>6Sv$cxHJx~s6>9*;2TXbOZ(H97@zzdD?*#U=EsCOf;f zt^N%yViiRxu<8g~MR9ArbS$f@qjbqbm={IRz{uYwgJ94TkAloELc=gb7)(Btp`&Qv zt2)PTtz^8K$0t`G8w-{~Iw|`Z7*YH6Ks+DCqT>r^1oqs4yA|r=UEF zTN0M!*k8{!w|FMgh|B`P?r7*G2DjcomLFG49%U;t5+Cvgan5b{PdGjuB;-x1uyiR% z&{QFj)!hNNQ@!B#^AZAv&Q?#c4b(ji`%?w+(6pijjkr@;qO^DZ8Yb6)*C2+XxVxV( zhFMruS?UTztG^B;M+VVSb`G&>4{@zW(}gorojEFlex*_zcy7Ovf$Z*#TTM6GDJNa; z3{e#uME!AJ!%I9!RyGTI{BSs1AF*=r^PN_*>PW?1z*1$fgwUbckP?4_zWar#2qc=< z;k)kO<6T3FW%q-5=i&0FCeib;ndN4=Ne*vAuxkKEx{ z7WCuC#V}#4zon03Ikt?$p*2Z#f?I}K=5g%^2W4XUJf_xJLQ46ow=!qcp;_;V>?v$7 z7xS5h8xhaKgb;*alc|*4+fYx}YRjm$Gi9!M@o*MUs{g}YhFQvv^Gdz(6sYH=Qcg|EP>Dhg_cJOsFq7T`4a$D(Mz*Yy!S&VRC|g#UWDq+ z`F=FZTtIk#af1(;5OXQ38e>%yf;b%^mDkjyXJ^}ItOd`Jp6<1y1<+@|27*eXH&?fcVpOgumO)5zV*8Wzj$CJ|=y}W2d z*}yC&9`g=20}aGWdNfJQe^t>*`fQeW#7hRN$B(nBDy8$SDxspW{P=?$VFt7qA9Bv4 z9G?@=?;Yv$ML7RlNxc3e0H>~=^&v;INAH5uSwgo%OY_oxy`MmoW*(@mqu&rPNeHsOE>q_j-N6(6oqVY7ae1GqiZUGFO(Dje<71Qr2 zI?hIOm*9(Mlf!C0t2FAE{+j9426QE-t;+(NL!+ZET`V4Y?5RVMUDQ4Rf9_o~BpFxx z(}TF)vG1YgQX5#MZDWvZYe_E>5S?hKTN{b?S$&OT<*pKYu}FK48VR^A|6SgEKypY|@^yL1(X6E3Pm`FQ7P!*_Z{cMHCxn5@lo#?b>o-r zd%L1e_#fu17K=8~n~gK~NxZ~8Q5L(w_{mbs?qnVgL9@VvtaZ*&|NJ1=P}45pccX4&aPA`&z4qvg>OyhrWiU9T>Arvi)zMUlo~Hk52xQd z!bcjRS?r_IgNK)bUJ?+k(i4__3?cs*OAoXQU!hNqe!D%%(4`QD@qCROwgV(mdfy=p zf1MPkGmCrV&`4Z*9)>Y8grn%4o6OQENdehXpi&-9peOoS+_Esu0aMkA1(_rI1D>NT27iHRyyD~6D|Jd5iR9EtzEJ`uH z{=iBVcwt83x|Ix<=s02xyrH0uqVTEOA|M)VYFRB>pw0^MJUxuo)2S|z0ZMY@cxi?j zgYiDAg$N(SZjLzb6GV-nD&L$LWZT5ql2Pt`0{xDvlL28RWTCw{9tgmUXm!V_oyPk- z1!Lc!*{{jd>qzu?E4Adq0lFfk`CM*Ftg`RzJr+do@-U_*ZvKPFC{{Pd3a5|5l^E8d z+499N4wqoKn`@f3T{(Ulbl3PU6O6Dn{Wu3Al;DwU^|OrV>7$Qu!lsj%{}^ z_(xG-P;jK|L<;$nj>qcl9nXFdYZCFfpEWz%G&Gu@qs;Aie} zxR8~L=$6%#^a%y7#GzdG`bO8yh2c3Nn_&wEhR}lWgVh?W7f2N4_g_0L@?5F5O7r|r zUWs%(Q0?zrcMnmfJ{o&89RmDty`pUoeME)*GbNXKAYil@a{1?|v1?&X$P+JEh4VjP z`2Uc@vxYN15C_Dd{Ue$MJ1Cbd52SM7n0u*J&(3Bes`x#{LLJqE_o=QYxki!Qv=V9t zv~y%(=gZ5a<<-4Jm|g{d;(YXH9>pJHVbrPHhgrj$5YsU{NQ%<1+E3f?)-8q#2xhlu zZOyD%&4ym(Q)I>$Io^>=kHQTWuR52_Vr9Uss(B4FRvO9i2!cjfJ(C(_&M90Vlya1K z%QWkiDxd~WrZ534DY3R}=(LKrT^%m1aw;mxMK7}Z;*GGQ@$G8hHViKzW1v#cu1ye)rKMFdeH)^Z131pv-gj1V zuM^TbE=D&6uL`>K$vZ}9AHj^ybQMY z9`nU0AdOxF^geYbEhV`UAW^2HNuxmNQFS<90X6p8jQ2ypjb`!F(G=AH6hpIYAeZyi z(4X%fLSaSWc-WH0-(yEcwii=hPd^yD-CrydVmlT@zU%CHWysuhOHnW+a3OKWb?rNd z!NSg|SA+p$%2<0VBiZie(9yurt@x7E6S8m1iXA1Hkk#Q>K93whe2!x;TV-u}tJaMMf-Y8zsN`Y>jbaSR%n@MQ9ssS?I?xpo>(4w7?O}!#}s)?hfXdrK6_qu$VIf>*|On!WPIO zdG|y~(O(g=-53EhatcyQkiBP4WdTDpO-a2Lg=R#V+g_I@u zFEevA z++mE|T##g}KvN{AcK&Jm@o_0ofT1Kr<2fygA_Ynz?OD(9st8qZ6#^PLlMZ3x=xJ#l ztN&qF%V2HigWuk_u8u8yB9<(~Sl$;2r1XIONWkT@et77hv^fVvQ1;KLjf}-0(T9OQ zLvLzs;2my$2s7WQ0*m)S8a&UQy@24Bb%v;h)lXTA^q~|pGjF^Z<1(d=V>iiSplJ7m zrXK<9c`uM`*M76#gbPbu2|lK%q)9eZHc*v5RP4W^&ZMS-agWyPD(&WpT`+%o0Z6XL z&_XA!zj2ZYq}}#^CIZVL-us>X@L^bMB12<0xjj!`DWWyV$}d$Q2#pJSD0p!8znHSj z`)0FvIPl)naq5I~Fdn>}E+F4idMT+d<iDbyjd$+8tx*enRsA0_f&Wv8ta*z|KsIBOC#JJA#+KB1(KJy} zQlJ9~RElq~H_*#XTe$$&H#6^5EgQryw@drfa&#q*n9`~Y7F4|>=bE0DLn(IwPPS9;CPaj=QX;`+YK z!X{$TH3mAo42jrcvITzN0cK7BBAEIs0s@KS)$-ul4J&yUe#RODjmyNf9AP@tv6vb! zfcDfQdU4{n3jJ-5$^I=O)i6FH2{}cYktr);Or~nyeNHF9E?d9beLFJVf(6Zx?_?Db zaNwxzyh#7Fn?{JNfZ+rr;?32hN5^W}^(-QA6;}K$`bAtuosX7BCL4mZr&TlQ-}zIk z-th(+DqA6JMLR@dXb#|(H5;Sj%Jl^KVGUtR-EZ|bOr!;W2+GE_*x@u{>;Pmj@`Em* ztkogB9v5l?C3(K zQl;bB zFZCf&>@v4vKx|68ET^7vX$A%6Pw*6AVOgWlM^;Q}(bk=Ll05Au-2+@{Z-`F{wv z?gTWOf1M1=wBU0{p z$B`NuWqNpMFZAmt`u3Mly|&*LLatxwCJBCV1S;w)naYAzi`F%Iq?c(E^ z1D@r72nkcQ+Rv!?+sg9M@Smz?{gu9t3HaV;T>?Nm<{Et9Rd7#~m&_N-{8o@IJr5 zW|pf~v0&%tz}hpPX{>O6MO2JN*BkKXh?t>F^y+Yi#S1r_fluig9(4lEJ%xEp+MCjS zmMq*Z0ET4g(rP*`bipjWft~)6&O*L`YW9fGyQd`2&MSkWidSZFBy_77AEn!x|U8JR=2yj^J5aP)86ZgMX$U3hKFtPtB^dLYT5k?8F1^gJ0s!4jCL1Z`KD)hhGX#*;Fv#Zhr~G&#{F0yHQ4VhL^tZirs% z`lWxw{9z+a*TTV>IeY^V&!&c(?sM`Pf4voZSn=p3haxM`uM=0w80TnE&c6AxzotHf zN6&aQ`J}PI)YjJLrc{Q0A>gp~`Zlty_3-8(jryJM!(EO`+PAZnQBH?^NwgbtIAm#V zoFagFA5dQV56Afb$QoFF=F%fTLg3p;?9C`cy`0Rl`SFTP7pXhu{#UWeS4KXJ)A*`o zBi;`{e)1uFodVy-`{3IG!1R zRFj`z>!$?V&4q_3j(x7$yJ;07{zmCtsf3h^Fbg@@k&k|4B^- zG%l%vP9w;J4(lp(-8U&@n&O5`-KI6ncQho3`H$2weJFN9 zFn~-9L*`el)k;8a%*kg1@c;21NjavU2=&vecbBA zGG)U+UYAp2^ORMJI3I6PTxEvK2TSa%vGH(436+<6_=!+hx#C&5Er5!rohHRfPM#sd zm<81H1Mi#2eOKI+vD+`4pOURo#dQCzqWCXag?B$IZ-pP1Gl!l1CxHB=1mnt4@9ru~ zWxadJeDHw_Q#OLR)0QwPzvtg}SAc51mFsv_OvagvQ9>=N@`Ry4JS?ubIhZ!TrT=Bv z&oeY_w6HPJXHq*N1_rpPEBaajjpr)YkSuh!m1ohQUd}IdS>t~vT8WUz0tx$6rI3c& zte7l8G8^#uTox}h5X+(sj+CRq6(9EVCqvt#x;#Yyd2?MCo3`g=)mFu9%ygz_!{TOb z5LCl%l2NM6sITy|yS$7SqFh|p>D=kgstkpMVY9Nldo8T_uv8zhoQHhKiC3zwz0p4w zfxh)i`*r#UqZTbb5^QGI6|4k?Fw$YL0G8%PL`CIK6egm)peaQ6Lz2?Hi>i|MRtwEe zTVGwe(NK8jP}>e84@yqHywe&&2)y|Ec5kU8k#bQUC&- z;uJ0&17v#+=eTdBGmjq0hs%W@?%=oNugvD$R$8CfQy@U{l?NNc3b&((xGJ)6OT=f94-M@-DCQVlI=$`dS`{5 zfBesN!ba44#=p5emqHmL3XS&oi4(uDi9xpxzCa|o+=GXd&oGo9Xk9?QH3(lP%1iyx z;i<}kY9h47xMcG)B&}k+1xO&$XZ{a^C6 zkI&ZvrW71HB^ZBP2<@(N7_gxWxUUP(<`7HETmpRy{8*iZDaQn$tv|a@Ia;k85i%Tz zRxPotbWTdZ{oxyw=F8s{N%vpbfppr*8$o<6Y~&b3 zw{VSYsg+|Iv#T}<5H{pDc>)32&_WtMi<_lfP~MnPopB5&wRf5>lXitpD>9g%?0wpt z&cT!fmsrJcwTa~PUnDOO2$MJu73q_>UD;D`WmW={#Nrx5TUnNVE*4rDRMPj9DF)?& zL9h#DFh=d)5E+enrz{N zw1m#XxLd4g^vU$V{K_|3CxB;pHO$G@>(mI`gZ zPT%=_uXm3J$NtBwzZCh$R{g)^j)Z+K&(q%mk%0C^Y{@E_no+$S4DvD!T;QC*8H@^a z+G5tK95lXPOjQ;!+-)0#XrZDg@X~YXDRo5($DS9odX?Sl-YBjmassW0du+Rf>5#>k>+BP!T>&eOll28SQmH9X z&s3ry%ZjWfDc?hyjs6(R?kCPbgb9}Bb-w*MFI1goU)u;;Eb1dX_}&EqYOEYZ3KX3* ztGuhe1&e%KubEzMf!-O{o;cKe0-LfBt|~qlTGGuuSv3&Ry3TO@H0t!1*;OwsoGQbD zKG_j{R^ikl*JHEmu8=RcdY<^$0P?P1;U7_3;lJeia#sQ+F|p0UdPUYWTvqnD6Gm`- z5?^PzZ=lzZC86=jMZUF5L*oN&HbJ0%m00-EYV7Wk|L6Q7cz~e^Mh6^$4$cwPF`uv* zdbT7lwIL8>+Q&FwpZxh-oBAvWl(nUiSfDP!+R^mMrUzZjv3En6G{6MqOkdX4g??lZ z0$h5Q8}!W|48kdtgZ2mLsQIO1qTGjnzk1fNMr{aZqVyxpYt^PE$0gOJl8e|+EoTLH zA9;-{CJ7UaVMH@!t_}`;z+AmnTuP3n=k29pzaL7VW1x?BixNfys%d(pCG`<8EJM7D zgPqlkN&0+Nk18dVDpv2$56} zdX&dlaM>rLUWHD6KsqBDtgnMO)jC7%sX&$*4p`q~BKk3Hv+A?u=enq|;CxKZ(*E3n z)jWpxb=C8c`Sra10{QcgaoTvZX|(gTVI?sMy#fL*4FphFN#)rJ1I^Yr*E8m@?+uo3 z23jqtY5P;>t7X^o5){s<%85)Z^3#QDSkqu&s0aiz`-{@zBX>A*q583yqn)XC+-sua zrRj$t*gwg`@(xTmN}yg~Pg7s+Zp=4rjT%K89O*4c1GTSGWMF5u>_gK>E#@3sW zw3kkRRL)%%3W_TK)9%Pb_t>HjA*sHeS0XDm;?80(rg`>DDsFNA+_=KtWD{M&wmG+w zkvF#@FjQB8P%|kXRhpdq$DjLqN?;#sx$3F z$6ns6C(5M*v!?~JCh)*xLHhn*R&~d(#q9*jKRA5qa-twsg;@9m<0IeHB@F6`52IaM zuoU9KvRs8SkX>?$k|l*9Lnw`ij$7S6TlLOmuv=JY;>NB5SQCt=rSW1^sUQ@wsM0}W z5{989yr5rLwjX?x$~PuNajMxXmaRV19f9hqMV|_rg1Gb+(Kig6ki2jJleBP@mnN~f z0V+;9M|%~ZM11<0=%ipR1<%t_vB#fT&-Sf}I4uOtd|p7imDR34MksfL>$(ZoE+xs@ zM`J_DMMSwqXmQSXn5pUt*0I%cJE^4^OtyCiET?qTzj&*ADo)wD-M8SsN8j*NUb4KV zmE+C<+;D(508@|68Mj&-15PuaxDAo~+cEyzi<4%c0T~=l1Q&S$r%9Z&{4;Kf1R9W7jt+u6RE2#vEUN6GINxra*-2JR2!0F*g}`bGF@ z8C_vH9g`xud}Pl%TM0aBFfs{4iXc2|Y6Yx-5Lp4Pm8O`6JOPS^IlKViG<(_gtLhtv zc`F)z#I#0zLAQ(L?km6!d=u=I<2dP9e1^c&!Ng^lOe3(XS-6cnmK6P!X61Y2FHGOw zov|Hk4>VmNpP5Gdh&T60&nJ)GCK(f7h#>!m^Nbn*6{-3P9V-ND%K?r93P?DBF91;G z695~N;djSHs)~LyAngip+q6@E8&hJoa@7Q(^G0TvoLlLraHs~OlXh*PFhBw)Gx%NE z=$5yPMxXmchqzD#gW4+i0h_u8WVeM%5v4sp6Ur*veQ)dnLalro@^is{Gt@E?4wE{4 zO=rk-6ODPbcm4o%_H)YzfvzWnMp6rxU|FUa&BH~$C23;Xz8k{rc^<;%zTSxgj;_y1 z_E!YqeZ7K9nP>VTyGvo*>%7&^b9i%?VT-AHyz9RXZdni^F7Z!=$^is$r*AZPl5JGqS4Av~c2VeInOKW#G2qe5L<=OUm@VdO*@01bsgu7LCS*SP&JXzL%nj$i%$fHxK*wu!#lC@}m`tNTVTqb*CVkP2T` zhF48lT3S=VSndD`snZ=sF=N3Y&kdoJdStQ{!)gArq%E06!JCad>I!=w02jN&uAaOQ zKQ(ftzJIGNO06I@uW8Xks&)YX0DhvU7i$qAP*`Ky5z8EkED!h$Yv|KJxDQ@Z32k1g zq85D(#Gf6iJw9J^7(9GAIbNLu9tTb0-*we*m>|izWYDD$*t+0Oz62u3U;u%Oo6pzM z#d@52`)N7URp-mmf{cKP7~%u`UJ4pKw$&l~!Ia zkveI3dUZos23@vGnii&U8v3yZozu2Y(&;4!b8~@Kzum4ntC0wkUqb`J-$DcWD-NW= z6rGvNe%!av#&5FyiSXZjA)IRP@R%xHeTX6OR3I4%S%$@l$Vd|j5;RVcIaUo^8&pPx zKv98m_n@}I*msl!OB~xc^5TYGI63H+u_T&F)vl6@t@ReiY+@*bjjZWzdZ}^7Xd83% zv6)?$wx*_AcodfsL$a{lp;WIIDDdwWsN@f41qe0&58$u99P>ai2K)t@gc{h2@@ql@ zwde)hD^H6EGAVDk1aq{&Hvw@=qSM7*uOYxWn>c{9$n+_v%ICbMk7eWR|_c=LU26qf^?veY}7BX1tTtc7cXM zrjP|?p^J$mf1_|C_zW_=(tQAoi89$jg2aZjpG0a)grKrJ7*9>Sny*}`Fg?(_xKBp3 zsUa`)xsoC@GPSleUm06J*5VSKs_+A%xS&*UjU%TM`a-}nIcYD#v3S_|4|l1d_XmJ? zB0pX;Knt)Pvx;|90xYTtCh+IIc0qhFQ{>Kk#uKshot7;xHc@L>(74rqAhsWF!eiSVEjUC+s$o>oAI_v}hPj>HHjX6=ry7_E0G#}I&2WPx|4Zq=8q=ok@7gY**1Wre+6pq)52FVaFIBpNS06?j zWS{h`t@@IqP1}zvXpHXv9lM%y#5OzX#}6)S%Yh2iE-0@m|7CPpRYr>{_E_r2;5%O> z0d2$_hFQWsIF#5FYotm46XdC$NLK=FU5q>1uea5!>^E|BF#9N@)D!5W&nFfuiMDfR z=@4a^Nx-K6Ty%F*QY<9FQA3BjwlMT;c(k;qP9{o_N7R@xHrGD8ik&f7cb}COt(py(uaW2f1$5O z^u?542AxSsowgkJRDLgiIkRa5hl$JAZxF~QU+}B@FDS;2V}vLnpIsswCLiDWjMeAV zOAYK1`Z2Pw?9(@jAD0tlD2M$NuOV!L7ntir< z^5b06{VcSqy`ELyS(b5Z5B%zX4<7u>o3s*L2zqs|y?FbO=UC_kx%vaRtY}0c?*1Y0 zT3kSJ@{_5aP5r`iF#&I1V{*-}Xp@)d! z51%^dP_EFCOuwCJEwdC?TQi{V8Bd-_xgpDW?0FTdN7}>!8g&6Lw|ALI@k&yObF(Ye~bsZkp9xlYzQ?a+&O_k(9A8D^gFE6#sB_1PLctV-D^rKkFC+~$ll4zub@g`v^%+{{OB^FH6F(avdC92LeC zu0~0~9cH%pFn!TJNeK1|HH^_$-*_eynA7}lH%_1hcWJ ztM9+0^l@8$zvK3$!cKA;`syYJOh{e42f3#9d;}v9I!LaxQteIE7AW)otcCFmiLK*f z{;ZD)|DEKOzc4Ve#MvJ}=|`~4<$GTgn6Lv7>BNk8Q21#4jP`YH_l!kt>PZmERYNJS zb2RajcD+oz>)c_T770W5C(gUIP9yqth+qWTyw}FB$A>w@c#Ut0y$Ttisu*7-3ns&S zv&dK2H%{fCT%4;24tV&^Hc(xU+$=HO2;T)Oqzb*2B?0xnIsIi=Gqb}`ouAu4t*q@h zR(G_~6Ve%O~m^!K>gU z|7QH>_?e=7YHl2(x3{99F$5(YI+|(~S1kRybV_ZvflKVaQShI}`4P2WOVCW?Y41hw&*jGdK3n2X2b!8_lJ&BJnZ1=c#<|i9>d8Z~t*7xr_HCl! z2q4!vbQ!(-A>UXukwoqEtg;9y!Rtk7Mcx2|RBgUidK+OCJG4Yv|;z2n4#QJ`I!Y2CtLxy1)x3d^V^2I;&N~1=oFY=`8o$ zN7I{SfSo7A4s4^NDje2lHS#3=)_Js4T0rE{bw1Icr)53U@wJWpeg4WPc3?R{bgbVS zdun2RFH}*zR<{(*@MzoO8ymGEi$DTKnN@lj! zCD-+zY#!G8h$t#roo5>A4whsWAGX~#6KS2dqqyZ(eV_6HmwZ%VelNB@Nw5=fm)WJnzWTD)!rFqOiH&RuryoPdrNQO`C72$aPAa9 z+tXjZc7gkLYk-}kLD2$Y7xMCnXKy@qkOUR2i*T5>_AnJ=xxVu4x4ABs#Y9(>!Z+Wq z@2dEPye8())c?^&__MX}SK0d6s>FOC&3~}r(LNB(-r>F`z*a5a34?*otUs}uD9jeD zTBInEwd)Z^qHuPh({${Y#K^$B=)@lXyx>9kKuhIb3ncS-Ho+_(Zp|2e&zoAMdZJi* zk^Eb;VQ#q-kuW}hJ0yEKO1wReo24ZgYh%};1CMSWO;RC8!*F?FHp^A9yBg5Qu#D1*Q zY~zWn;tvN3@W@IRw2{qtZ-W1i$kja5;johp5@)&^S&B~(A18vRAqlcEfl z6pR};&dCwc%Ld!+58N}#bImh%{f2gtBCcIeYNJAW*v=Nsygg6mWm@v`T2e9X z4=}#^dKE%;2eK-!;qvkA-}}il*k%B?w1MrlWcU}>PEJy^WIOa{YT!pv0pH6G+Q#jv zzp&iX{$Xhd-w7jLEzQ`=URo`djU6dE1Kb46EOb3@HO{r&OLn)zC*Nh3d7?&5TEk<% zNt#82nXXi^-;-jxhnuQ57@i%-x&c(;>!7ssI&jlL!3An7Lg?vXz^}4hDyHv8J5FWC zU703*+WLZ?eH;!kzl@<{HHLI{Tg?j@>`=*WX5OqaHt*BprMjR*42{#6e+Kvl43>eb z-ME+`5t~?O`+aHKd`6NKmfpy?*X((yszd6;?-t!-Elzs-=yEjWYdUvIh>qy0 ziL%VR+Zx#27^l`Do6H^|94)9UGVP7ZiSKXp=Cz0+4dQ}6LoZoLm-=`yKG$PvOC%v_ zc-93RZeP%zKU{4`oq8oPyTmpB9*F89`?$5y4p_0=m*p#U;1(S$nNwkIIjWfw z2m}S6A5M6Ut;tl@U>u`P#pn8=-gAThVPh`;FZ`Gm;Kx!CW=YWuvynLL-zIxMZ{A$| z1bB}4xEU;#hVB90k0Z0}78_7$t?zV}tNB?ItMSF}Kw|#G0S-%v15Qi(fQWyE-fO(? zPt@s@ttQNS<<-O}>Z)mH-#B`s%HSynHGB4>%|CpHyxIe8LCbC#6@3MS@&MfEJb*WD zKJM@;c$4<)ft9wxU*NJU3|&rafQeRF6%Gyy6anP)x&2>nR;!@m8-Y!O@Z@S1!^%#G z7hRRDcNVYw1+yx>B8gN$c3=54$T{xxfP}-hjXur$`f66L&3e4@D}BS7PAo?@!K6*h zid<{}{g4Yca-0t=QtVQfn8(v`V^ZSB$AO~$eM{8jAG=1NqOUFu89qOt*@0>6F@L`} z-Y3E9Ri&6HG5rBH>}{uRV)24k?$^e=r>w+RXOgx)d9;)x}vO)(HKFad@tU-n{Ah7GZ6)HAHYi_G|cpumE%O0Pj4x{ zveL--Z8cjKG=9;FO-L+OS1Ig(?o=hmiMEb_g~CoOkVw-`67o8>ZKRilkthZaicVxO z@I%z+TL^XRTFPFQif=v$B~)*RJ%*hx0&pyQO;w{~v%Gyv8zflnYcyJ`PPHVR&_)ui za?4(Pw_lcj+jo_h*p90G*;SV?ewAlfPIx6)*)IM?N&Y__UN+{@^{1u{0?Sa#S1Q5k~xG{YbxeQvZnn87s9Un?aN+6*zh#G%y z->lN%Ls5P%vnL_MdF1v9+{+U!?`E{Ff`eI7H zTOAr&`ZAFwK^1H}rIbEu(>;%$)LjIDfqv^jHsO_7vYkVdl{n$9A3@K+xw`W7@XDLn zcL9nZ^Y7@=dp-h?8W=?5P(M?6KjRFfnD)IV$|p+fY{-}Jy}#0$ulGP~K2O#w7-srb z)pWu)Z~0wxA>}2Su*gMh9%c)FNlOeCj=Ay_=;pr7ZuvNmJX?h2BtjWDh%#lWV7>Fp zG$u~)dR}@F89I$w&}(44FAI&fvdJd@^y6AU%`7;qm0u-3;B-iNlux7X)?sZQK36g| z!FIE$;n`4$^ZI>v)Km5ZFM+-~_Bs>M@cfY)6dPRkn%@G;C?XScBW;})&kbl{XK5K` z?}C77&C9M%=Kzqa0-UW5P%E!d>RK;~?ZZh?mOAv_xMk+}U69GqcuIgYNr`Ry?{u5r zO|*6Rv&Sv2Nt=qik`Iv$Kl(h68>^g8+KXm%)*jKW>`%f93g6#;0pt_nG|!m-UzCQ! zr1<^lul#{&?#K-ATtqk+y*yBmWygz4#L{`c_ys_zW4~)mn<&)NYX03~lk^KB{^u(s zv9-~@=slgXA~-K)IS3L0;D;#V{fLvJ$#4u6^6G;`?H{es9R$!waR5%Q9~j^zHVQq8 z)O<$RzMD0i&hvLrM_q_{lv^q;|3%>qludrw#v6k|XZ;lUUaRfEe!|cJGcAItl z%AsAW2Ect|k$j%--VTW$wXyGHn%sS=g!&3>btL2QMUcpsyHyDhy39JPnzlg`$o@Xz zQMKYUc(*(F=4J_Capl<_H{Yx*;G#M6UG0&LEYyv==C$*5y)Q~;dM!t3&$kE3xMf!% zA8hy?kX^9rwqHx$$h*s}9Lr42ZlX;drn>G0NW#Y~zaftlgVK*slK4b1E@y1gLvP)d z)A)qsW0d836T@oEY%BF$hu*R~UMo<&&G7^MnsZc0%?g4craNwNOMpZb7ASy%GWs5; z%hR$Ca1)1a)O=@58?(iJ^X>Esdfg$<4n2P&$sGd;KfZ5twcU<}C5?rjEF28cGR)}+ zX&zvvQw>B(j3-cywSFJkNe84i|1`IF9lHNjQ{k@|PJcBKbXA8=yGOD$O3q@CubHI< zhuA_wBh2nl(f^{pA)bhZN!4GB;LorNz+7co4F%0+uWbuF`cORI{K}m!17t)3VWJNM zJdvUi*)_0Ysk7N@cb^~VWTBz(W2qGd4i)ZZ_lx#x=R=@Nogl@aGVK@JzhFpZyT7K~ zTUix)g9o_~%MQ@h6h%N|?S1@2p)_~%U5A%yehk3C9n4l!jDNB2XgykLsDE?1q>}0G zq3wVadw7|?A=Z3;!v6;l!cXr_!pw5(;6AQF_9kQ(!r^>{(T()q8T z2>|Gj_zRqTKyp@}>Ve4f8-|1k!94)0oIi~XN?L;8-BoWtaI_ zIJ5HrSRkXH7*6qtdN`H(fz-FSwCS{yGlmkya2=Ruy-0y8yY*D_rA$?T6AzK&DXWwD$h~#u4PnMTV*h{RlP~ zKiC8fInD07?&N6cDirC#VPM!1rwp?ZYIrQ)%27dY!N$@!kme(*CI6@|iPknmW3ph; z%LSN~_-T}lHHZG zJu_;JT?E#*gn#ySCjpc_^T_}x%*C(hcQ@WDwyLMQ>M*8Zj^N{Pw))@r6m}mNOGSJyADJH^ZwrMnsc>>hTq3+GT}k#FSxrF z@|r(dM6UYl`}zg<<#hk{j{XLz`QMj7EO}UF837;sh5UoTAjiSYP^p+HOAi2f2VRaX z*mdy%3d42ra9m}yl5${iIoA7Y0b{>r@BSz{4fH8^v7k7D#JROh$m*_2Q4AIG(Atce zc>lfiY0gYo7~A3y&AN;6nP7JCJy?a=jqkF)}t0EdGE*%r6Jhr#7gBr|r>H};6-48f8I z($%Ndz{#+JhB92&?>v`O8h!2uC)zUnK`H@kNMSf~#osxUHGhOk3xJOOLE!OW2A`7N zU1XSJDX#2%X6a<}% zd_WJT(z`7F;DCA;#b}m~q498jX$yrV069%q2W_hyANPe6&51ST)b~8iTku?(bGYOl zY~cur?7%-DEa*+}0vGM~?20|pCVeHwuBL!BQ{RwkPDoJ(1v|N@#?FQg5b=|w;0$wX zLrwXkGwW-lgZ+mI{r@)7vCE3W6b2(I(W^Lhg&wS>g4|f-VpXm%m7>`KBjeB#g9E({ z=_CiRMxP%5A=al}Cj-crKu2LGQa?vM~0`ydLU6Co)3CUYpDgc(*9mcv)sr|JyTF`Vm4|Ux){UU0o z4=>=?20X1R766^*LXqPVB^2NKVx`wT$kJrF8MOPs+qmhIw^VMmWu*4Id?1uq%PgAv zyw#MgytE-yqbSYx;~{{5;)Etti-b@4DNsx;~dpBzm9~sAkhFVs(Mqq|Op@T;z^^gt+_hzPwbsb_~H#+K*-4s-Q z)Y-Ut>6^_{iXIh(pPU0zD)OcG8;5=4as{KXDwXx)VIbDx9{(Pwz*r2M$8)(7kKcDu zk?(vY@1zn%moD&SW%@d}$jbDe@bnnEd;8%SHDSf_=kf5!`WC|9P~2*Js!#V0qL#G< zhqJHy5?1%#o0{u%+I9?Q9qF_-c<&pZ;R6?ZKdk?NRx*|iIxWRQAwEr&X=tgHnm_Ir z^5KV+%%CySSM0m)N1KZ!3Dxc+vk)__Ru^*&%E{CY)0$;|i?=RG<)O_kv*e8=?%rM6 z28%!i4hnOI`A4O+e`!8@bL#FIy-rkf_aW@jfF}TFR}5{3DeX8l{i`LfWrUogT76%<$ zPC(QV-%g`xsa-g^^`)+&yu@YJQHLIs*m5%%N|3g>!tr@Ye$TD3(1;TLZds7SQe5>U zQ$R6!zV$O{7C?)k`$t;Lf2P;|{xY!hSqMOu2ZGWe&O(6#5*_R^-Q)H4R#^Y>s8DUc zUhH_AE)Hq(yDf2PHZS@kNw=rls4mVAwlD533NG$Su`k9WeqQVk7K|ppiu7sDTb?YR zZ(nR*{Jb!~(3ugtGncz)3{<#@5RRejNwNV+ZCnA!Pp&<#+Rr1<@+(M$SooH20Z0EYZk z4N*e(N3z5Uau5h03kQMJ_BKA+&~Fb=bq?aCurC*kIJp(^D?Mw(Gnc;#c?p|vj&fP? zO#3J*Cef$ubgday3{TxCC|I=Kj0&{bF98DZbgM%cHz@S;x6_h9J6rRgiT|A*|43MU zB^>~mcHrGJpTXn+?;a4J1KjolfQ<(*heojY%{;eM65o8gI44s2=(p0E6U-tg^H~ik zbO!RbG^osyhN`&$IUi2=Kg6m^S<9&Dpp#B~RKF%ZN|Z3JDz6KPo7UuVk@21g)|NNu z&P?@~&%;=Tr!qR&JbL*)N+rvGGuIDt!v-Wc3H}nj%mr|)f$Ox0^@oY@4R$i^9_RV5YN_;^Ih1MBoU%IF#3|B#czICxwEbt334jxT{wF73 z^!EQcf%+$_TtR+nP&liq$RgSpttlPcPHVq_MiFqC5gssO3!g&Be{c31BSJDn2s#<=uTu;m9~f z;7-f87XT7y6R7|6bia``=DhzGFvYN|&ux=6y5^n<2Dw^SQKwTkYofGGucl)km>}>*e~< zJRk9w=d0`fbL5!y3YTK|{=_vO1OJFIa4Yds>i2|Mh=71!pn^+B8$krhXw~eLx-5#2 z($}%7%jZAU7zw(ZFsuEwbqy%*=mHg*u{6aD3h4`I)zIYov1_IcWtIOKtVn7`GMGUhB5%;cSXQzb1O)uaCH z!pSEUBVlCMu^kG9j__p6m_BauafwrD6OZoTlD(zeOb6Yj)pf@OBC^69t~V~(+`tus zFgE-X5fJmo@l4fHvUm1$o@v}3)kuW2oh?G8op^hA*bd*hCvDM`QL;$9!shELs@FI; z3bRr?1~91l%P*nQ0_kHFa#+>YCziZ=D^|6d+QuSZG-sOJR5g|nKLbZ|?Ef!Eqm5-- z{D*qpj}&#_F$f?t_1BmN%^C%F6*OYOBf-$Air+v{q~Vc~$qm9K@J7O^kd}}sM2qTm z-5vQh=`Ve5n6Av*3cH8HQ`wNIE2lm0@qhjho&C;5LY8T~L3+{B6jfr;1Pq=ka%7uB zqK|&z#YzwS3qJ$Dt|gwIx&;pj2}2_8aqEu*lmx@B8>DgeF4HLI$nBj285(0Kz+j-h zuTuOmM7(|-B3i?CqFgMpKd&MaG9_qgX$*eud>h+5nld!>PE}upDrs9oRXre2nLf$v zPeDH17EX8Uu)~u}4wZaqMo~7H{M353MlQFTkMY~_U&>?47lIi}qX(ANdIwZ*6iZ`` ze<}fOk$b&v%+_ULBr*3eY!^*spZ9xY0?)@#d>#6TblQp@EuD(zsZt3qKh0IN_+R6QDLf`B>l7CYD5UhX03;w$vEiAgGK=Y2oy&E{{o~ z6NRz9b8K`~Qg0d?N6=BjOo8TP^Vlk~HxKus!RG0Hah-hd`v%aA=+8K@fo*_)S9pf_ zoEz1dSo0q`V@k?rIOa8rnIEYc`oUCGdy>D0AzZ~VcWsA+V<(gNFe`iui)Y9U{$xIs zJW!P7n=Xec{4j5sHSl=SsuR{< z4MG*7l_qvkKWMgU1>cInaQ8oZ^nSkEy?B}DM}#dat$KO^KRxy7#93Y@o>-rSz*12b zlY_>_iaW7G`w6zMPKo`+4nFQA6rVcvBvYsJsl0YJjYA7cIt1KOu!VOwbFTWjV|V4_ zm}O?W^(6H|lcQ(8GsCaJ8_r}y)zFC8oIjCWzLipYER zuS#$8Wm~x?%|E`C8$0VNq)(E&GKLpcD+wAtF7S_(D^M*_Z&uB;=MxfEoqDbW{9pnNDmJ5bYy=LI+|SRrMZjv(bH@(H%VdKbG3a!5^9ai- zitt%4V9**xz&Ti>bU(;Vf?^R3^yE}PPl#c?6CIpW8<=Oul%d7T$h%i@|I$s zTr-5C`#R2PbB&PwiU7TGHrUU;zDt`r@oR8_Gy#tqFBD3VND;zVB9%#a1Xdajv$T015(tE8fyh{SB&XDv`R#wgU0%l)pN zaFjZm5B%N{&E*q6Srl2?+Ll#ITI7BbSYHpvn@ygf9(GDhvy(;^Dw>J9uX_Ef&Bo1* zPOr9{{dZedP-GZN{TQ14I7D~2fx5+Om{8c6kzKyX?l;l6Si`tQ^lJHomn%2TM8-KN z?4@25mg?rwQV#O)C_5J)gp3Z4w)f0yP8(7Ty z>wL4%;~}0(p4OUCL8UI z3~`I__qGNSe&6k8U1BNXG8-A9l(Yr_cU0&gED#=J-zykDPd zKd)VHFQfSynkrveCVmry86Vl{=|*JM1$Z0a^7v8kqTV~abzFJ_)(1sJrEHZ8*`JY5 zsq5)Y`ZXq;(mVbVd(ih}@07cgD1|t1^c2Ze!n1>a-El;NBYF362aH{7B4GqiDL*^$ z0W|<^^zp(;|*_-`YGKj4gO|rI>mo@P6b5- z$l>SG^j;*NmM2_D?8e#>6ft|ljVj^W)39oJ7@C)vgU9W3W|upyWb2*6NsldB;g!WF zeCy5Y9~Qk7nW`J8pEP0zSvl z5X=h!msx?0jZKk?w$>KR3@j>48CStty~Wwu69}3f?+bws`c-c;%fsdC>#P5XTrxuW zaHispr`+%SC$1d{E$S6m#zd(GL#8X-Bsq92I&A^q!Mzy`Asa-WlwWp60JX&aZY`I? znF1l7d$ak55A*EwYK2HH#|ue|U%BlK@1Gw|JxDL4maB=?*adHvKDJ3bG}tco_JO3X z4yHd|iW^+)j*VT38?Y8&f+gvJvupcpMsXJd7vY2pZaa!pICHk0nOVj^IhLj{?c(GU zpWDfB^;H)iOXnl|E^Lln7{x|-1Bf&-?Qpj0vfAJH7tR1?HYSm?gZ>vT+gtre!bskJ zL|Vaqv6$kZ$cP7(7Ork^Gyjv3M{`VrnCQanU`hmexed~lZmEg8x>=#rpr6CPR$_}A$pDI zL0?z^ZGJIycbOIeJCAo0{y2~C{h2P=n>6rKKF?{w_h9YJc)ET@ z@sS4qCM#E$Zkr6;OJ@{EiUcoZ2dU#esuEQFNx}jdniwCOH^A zXWi`rhrEOB`>9+!j}z~IN_gHzT%`gtDyr8#az(Z@bz@Op;f4S*1Z($)#%Qv`;LrHj zUB|)BnKl$|d=tgfh>FHhPLX3LGhtC#HiCNIg;51L>}ywBH~m4JU)gAi^h&h7I&n5k z32i3f>qP-e-f;hUOt|x9d`mV{80tZwsY+-M!LDQ0JYSB`$~k#;uWOxB@28&F#Q`;Q zKA!-rw77%B&lk3WJTp7q%!oezSf?4Ed?r$gVc#*jY@VABIjy6Hf8oWErzAneqk$jA zLlxI?)xVpOznAJ2#@L`XCAkt6+{UwCqO6=~oZ!NT#IUq)-xlCqA@ma(%?NTGXo_`} z5jN&u2)0EpM6Qu&tu}ZUa-GABZV?I9BhkERxDj<8bQ2QEQM5wMRjH$#{zBUr0S#<8}M?zMw1i=&B%cl zD@!dWyKRaw@s-KRkl;CZAt_SxZF)_DgKQ6l!)eSlzM@+`e2B zgFFLAVRlD$fclQKq||dxNlzbf8Tb$d#vtP_MXy;dgH5l#cDww1&I zZ1B!_yMESia~8}4EdM*Wi)ozpHP}}y*|=GKt$Y}b#MR|n5wl@YBc&q(u4jD z&iQiZZKzbH=K-Cs;JY3;n2*13wjllUjXHf#oOMq-9+-E@;`ze!2QT=pk^9)B51Vd3~5 zToa4hOS6GMCbh><2fIz?w-wM)rdv->N}*6=6doipG^=OD?7e;))FjaW8 z3I>x`D2dxz%GZjxM*X$|DBU5Y7#AB`qp>>vFWZF`;A4dlDU?RImx!NSoH z!J3SNl`#4#mLM1(l}S*K&mERwalS+P3bfq_4d3kelx>NT`jdinfbFrKislcA2GrSY zD!sXzjIll%#;@DiCEeF`rxhoO8$fv^QwC45G=%-RLDoJn%UG?nfMvY0%B_Y;%-S`@ z%}5m|iXLjsm=6vI1|~1!O!?C+-;uZfa}Ibdvi|^bZ6w#}<20dPen?qoZh?*FB<|tt zc>Hvk6(I3aXSPhR;CrhD8^H?y4f^1_i)foI0qc88PW$PJeb}jGB21`n5Y-LaQy&&m zd%angL*k(d(OA1fgu%E{JvTF|d~c(p^?7#vB?|=~lt_xgYlZURNk{UrSIaHez>h_V z9gw=;fQf&2#7AMTYGzTu&tlROvu$1HXmv!y5#HS+ETNq(?zm zkSv8U%D6B&!x&HNxcDa!ly)6Fq}kym{09QY!?e)`G8rF9E<~5}eYG`hru?Z>T}_G2 z5qKy-xIqjJc6g)^TX4!7D62W>bi(>t3A|%h{1#41ZkV?{5cCVkt$CX6&<(8w6{X zZL4H`cOIkdue0YZj>wYIGvO ziMs|+^c**}Aa*C;7nb_lb!2g@Go%a)c6(~TLdyo87a+rl@}k#+Uhd9?o+dmf083k2MF6NDyDuo^J` zzYl!mft^2v-Qw=inA-yNatSmt2PAa>zFwdO-Op>7>d*Lp%)qp)CVMT*37 z3JPWplE(OSkau8Px5tgEJg6ch4vuk`Dq14chOqb|biaC|2=}Y|NKA2Yz6+&Nmw4Tr z=&xpHgBp?GT=rHwg{!H&XA12<-K_3*1n|fFrqOo;I~1uwW<>||sXo(vQO4bZ z<$>S6KA8S2^mrzoS%`qb>`?GyO4Cr*8w}N>s8{U8`+mPjUn{`+8?w3lU57&?oCta^ zVWDWCvQRJB#e9P;A*9Nx&kgZu!nNDMRCpgH)HhiF$i$G3;D{ckA?MIEB;fv_(^NmS zkcG0>9Aw-#!x&pQvs)C#f4E@IswFN_Ccqsn^b?LXJXR`(G`Q@o=*r!Y-5sL6!GUT@ z%kXHt34a3lh$T;}=p3xqEuwp;dY`Dpc(~6NF8ax1w5hP$mL*XVy|j5Jc zT2&Ad7=9nwfFr>w1%(rD756Oe-WpXiCObX^Unne)ia^n;~BYqv3%M&%?end)Er3N92kg{2q zL_fP(P?>tSSDaN z8}zy}zY;wPaMNEHcNp<6%MPZzjIb2u^ECF`=3> zJajfdZ@b(omwEQFPUsV^U!f1LH46v@>r&7J#W))tE--u%h}X*Z!Ii~|4c=F9E%p0I zoVI%X;<$H;E2^RGG@4N?+8mSY<7={d^qb?gj>evra%M1w!|y~R zYdJwfR}c#ze1A7$dH~d|>VLp}gIW6e64|TUX#r7SEteL-PsXOXaDxWgGA$WQ369gX z0>_~q^%>rnvRLDUi?py_RpZ}Zjghiipeis)hv^u8ggf@0=22BU9XMUt;wWy0Hv_W( z42>hilaT?UT)}XVbQ%TFKpZ+mH8bA$gIy225ppl|RyhFJYR|*;AmQkd(AGUAlDS@J z4F|iI&Ko?R;f^t{FX!K%M2av{(Ozv=4>#(o=Q$Jm9lys}z>mRR4)Z6O0FSsM>0^;8 zv=QXAAtDi+1G~3cU&s{85B!v4^LnUpHsyp25HFKwxUGsGVg)xa(5Elvskt&^E?Q~` z($6faTzcPbJ-=&I`kh?q$2xtOV-`~@Fkk~PeeT7M`IDcl;;-mzhPoB^%S8?=L!C)a zopTLJGZSxslR$|jEl*5qQ3+73Lf}g{u!=Jh&>us&zY~ogSkP{z{(c}G&QR}{#k)WC z$b7zQWg)on4NXY?N4w5dtz+E|w4aR%e&X;=3x z$JLk{ql-WC2hp_4(65mU{H}IREM(o*T(V`FxlY!y=u9L|k%)}{utrkD(M0I6(I_*m zep^-%<#kHZ`94%dY|Z5G3RB7Eu*Jn)gD!scA;Xz<-ir9NUqA7$bv{gld>)< z<_fw%_r+SKR^x_>Ttm$>mnB;5R?>ZG+G|p`bSjsAA{J$TC7L9|4tBoL$J{GU`sS2XDGq&<^^?URpRIqo zF*K2OAOJiha{F&dADDYfn)_FP;_a`N_#^#*)$@M!ssW+3ppT)^UX855R2bZpBCqZ# zaJ61lvzRkN76=O4Neq*~@m~lo04k?gu)vEIiaLDF>2Xl-tkYJ7D?{wiEvINT&(JS^ zL-n~K{!!70{WoMu=Vp2qw zgqw>MQjMQ!uS;kZU2DlLE7%2g>YoMQ;J^ zCySukGZGKHhSlN55g;g=ds@wdZj#MAv#{3GvUG3BbR;{Fz%_(1)b%m_6d^)A%x$4e z|D!U27tJ!=8Bsnlu18#7i~Jexy$3WEF%lO?YynlaEEh^j`fzz>C015_4!*>5IZt;@ zG#NWQ&6uJE8_nc=jk_-6kI^BzFBE?g+ZWOYFBRnEj4CU5Fe(R19PFo~-y-3+1S-5w z&zA!#WCEW2#rty*iP=ikkW+Mex?L{btGcuisYhoK3?bCiA*axA`@ow=l{4(QNa3O^&7MuH~*YTVL5fB zVJ$$~u>g)#()GkQ(E6Cg)j9Tl6I0Xa=U3pmKeLg zC7W+4Y=KvB8kMrPJaWeg;I|b;+K*??9$!=yxL+?}QcHQ@InFgj1Wq-uXta#%RRv6j z9SA#L2*%)Ul4PZSA4p8$8(n_J@c8d2%f^6nYvGTp(=qmU6pe%;fQXIy*YehjaBu_Y z3$Ev7+yNF-6|Xw6${uc3->&7@4JL64`yDVhuJ7r%Y}nJb)eQzrp;iMo68@Z4Mxjqt zIP6fzw7_!il^X>sYo_=5yh-!Uot-}0`?y#*#PZ{`Tx-sSYQvxK66XeeB5NZjKs`|= zyIWgc@_mXJ#m=19GLF_IoNWU8K*6MvVGU){g>ga$^Su}Fa`Rmv%x#D3Y~Y;e<;Y^S zZM~$+3%Q&fm^b2z;^f+Ci+n-~*E%1uk-?|A&EIqVk=brNDaw6en={b7V*|@c5O~7R zHgS8+GLcxMBIx7r=P&fEq}iM(Te{Es%X~43@Xol-`r!{ouNlQO(Ln*9>l$l`I^2!( zH%F2~mjn#T58H0j!{0NgfT>ouJVu0kOrmZxD5w~ensEMCi(}zOcwJ0zmO7a8_@eH$ z8#3;d*`TpUYPj~_n5SQ)ln2_x6*Ua$3s+w2j3jsMa5`=No%g^!gqaN&yF=m;76EmaKEKbeN^2G0m1IM2x^Y&BUWbcHEd2{ZGFX0Y z+1;eI&mvAu7DlR*%*tijC@k?^mIJzgN2~mDIQs{@Clb7G-RW72uLwh^eH<_vO(*do zhE_}-O2NA<+AFN=;^HFtm4wzn7h9GcE-yE`;K^KhkAkdzWx=S%@#gZx{r1`?Tw%dC z5uhGXT_T`VEXOTXXY*!_YC=X1HT&&_g9NGY(Y@FDmE1a&~-E>kl(&=;^f;A_!Nc6C7J#P1SJfym(W9 z6M&_PJ08HaTBMz2Re^W&wq6cCL9&m10@1|a#~YI8AS!o}WMuE1uwR$EKgYDmnCQ2% z^Vr!7l{ZI__;PPbl>$X8%f?L-G@EYo0k9$El;jPzOv{ME-8FA|J@!d$JV)pJvwxai z-!rTSFEDMibxC<7YGa4|Xl)oxs;k-qxSnYD63WgG>pi>pOemO;|AXV~9ljTW8fMWB z>29{Yd{3zz5RUIz7%zZa?2sx9JkTyUvZb}*-yuPVaSk2JU&YYpML@}OF&{qVAA#V= z6^=m`n0YC>A0`#CKmWGO`M!UmObyAKSwu#`r@J!stpr}8cbw3G^!=i0`RZWoor!Hb z&DZI+?^jL7>vAil26_3b^qW&ZES^{S-v8zJJBgmhByv;OUQ@BfV7hf<*dT$@cuSHf!gl5u#&@6Z>?0aZ~7ZbhQXdrrKW@kLD zoyB23nt%M$ZCGFdSja!hkP^#tDy9HZRPgTG^mg#EcDWb}T*0rwjC^rr1N+5%v(Mzs zYTxhS@skl^&kj!0Z+M}i#d}Gywk`pskBWzfco19_ZM9J(3v^u!0)aC#e#}9~6MQO! z`WMu{Hb}?@Y|R`LFtxVTQ#}pJ3{xQN06C@KlJfD(`Lp7LD1oUX#dqYw7Uyt0ksY7i z6;FdE7Z%?F4T@5$%P)}|507zyw7^xfilm0&(pS3_i{JIX21;eit6N2!ehmD`rju`^ zznAg&aZT@x{uU)g#LB8P*&14FN5hjXsUc#u&Z2Cakx~>*xjv8nUvak9V(6vk z^liZPTn8m2(jeCcH=7HOpK=KYhK-_eQI-ORfZZvN>*ldvZR4b6r7o5Aa9OzOgJ>njN=zzxmN7tsbu>pbaql*PpZ(C3>8j_WtfeubqMnY?(>*zlk^u)Em!mYB+fJg!{qPR&rHI%`M`!I6 zN0|1ziQVqSA@6SO*7QPM-*6$cyD^jNs^p`wfQCF5@4Su&Shwffa{52;3@o56+=I1yAKk2GID0c>$}^S>tdD;DBiAj_ z5<4M><%s>w!jZN1gtc(#FNcgbn4qJ-ZFumate4j zjfY@Dy=68-t`{&u?c?G2n^T_+`LbiwK^NAnRIdPU0`sa!Cuc|(tI#N9fssWQ?@+>J z2}2^MoeA>-4FfWR?lWr)4$xUjwJbd5t(i=en(Sdmz!#uCzP010ZfY6s5N9zIAOW~s zTYoC`i-C@73d@f3X#N{0w$DJa2SO6@6eAm#fZ#>@#?@Y%*OmGuaXF+m_x-wgnx`5o z5*;DOO^gmG=6^OUAJgD#>2d^?Lp?wGC%t!Dgoo9tN_TX0NUkIulg)Tsp6(1B#0$KG zfi9zM-DdE;MqO#OPZBQQnqH2rNjw@$PPN193f;1~KJg>P{KR&ou+1p>Q&eLu!rKKg zQosv7K$#}ZPy6SrctV_lghWe;=hUHF&<}wnYVHXM>jNxTy)u%6H>OCxS-lch18LOg zo7(M&&3TtNh$M^t*1{qzQy!oN@DSs80988=U|n&K1B+u6H7%>xFV*^|QlY zhR4MhLrrTVOn6n}w2O-}rw8{$5t4cd7kL45R!a6;!2u?18K6n8URP!?>R;66)G8B^ zKJmKkD(2|Y^N1}R5;)fo%Dl7$%MuUxa7e)`|L}L&i+bxl=eEb$`AX}4FrO}@zL6&W zH@}l7;ZkK-*aA35zrvl@WiSuNA7zNTa!>N~WI}saz6S0|ENJGVD$p6R2_H=o;ViB< zR3FDROguX}d3291N48~tOXK3l4MwI^+%SO~!|$D)Iu7$nEBDi>Cm`+IpKQOq+;=ZI zPL}RRi|(@{aTs@|Jo87pUyUaQ^)_mr@*R~Pg8`HE8{?BNFda`Goe7nn>~>c?rgGE4 zHl&4(?i;wSdV{sO1x_e;Q?VyCo^zpbujoLiP`{#~4s(ADymPhmS8?9fcQm^qqYGE} zjUEE&z3Pi2&e#TXN#9H1HyQQ$*-AfJru#G~F3yDv%XJq83#c07IU{V;8Gy&5tJIRd zW2{_{%V_J!a1Z{v43OiA1OTb)T^C2ksSOmDW5-XT8+hBM+hp!@=p_{nX7*J!k-H|E z1rzyD`+3yaq?MiEgq6FCCzdJnZ8*3c?j7bgGl{4~bV)&e0$cr#XQDNlb zRm#&kd6bpwX3HAQNncl6p9NAp2N@f#cz1Rn&B7H(6T%&J7ZHgsD`Fo{K4xun*qz~f z$vRq4quBUi_gd-N<6p%|aN@o!zv?aL5NVl!;dtiC3%3@rABaIdcc!T{D>Z4K6O@XF zc`u>-ULDJvjp1!fDE?{3AEiud{$JYN&qYd1PF@-0TVZBn6*_#SHmKFs0vQ9Va8sLC z1u_!d34lm&<>>W3T^!m%$j;Si<#ZIxDVI?42PtiirYS(yPNG~Wj%EO)#24zw{2;nw z*Ur)I?1?F8R(d;f?}(0JAD}4y?iUIw00O+Y-57jp6Ogy$CHAEf+>ecqAdE7fd_1Y6_bpaY}}REYZ> zqmMIH=JOON1v?D1VT7{XO`%v-8qh*}d3;EyyH7Ry8lBUSSl}-0ipW{ZK&n(nx5JHTaBLwS+mC-oEk0G%W3^vl%&wiC2xEwwLEteimRrVH_7%ehU9tVsJK zu7083^eG*ObBFx);JBY`kS8l14$;~*aZVu!sPKUp2QmKUxAAC|pVey;d;G-))n-yd zG*>RWuUmXw{NtpJYvO`4w3@scWc3t;V#Hf-smsg%Zm8^*)3fpYbeZ{Ca-6?GBP)hR z-qC_R#E;fn;RanIXHX#7;Kc7;Xo*TiL-+=*>qm_kVgI>5;m=6q}SiB@hcr%Os^=MXxP> z`juKDQx>iK{5xr>AKwPLc`uGeeRn&7-E^(vc713`(K78$TN9s&94G`+KySd^d`-VL zG_T*uST$JuewrnwI9}_Jo0cIua!yBM*Z-S9*k%ul+V*!u`byXija}C+fv|lBSo}3i zUpkAqG`UH2c-Wk05AGz^iCw{WsOkcwc&;7UO?9GB_Em}F2M>q$IvhZo$)*k6k}1~K zJF!LTs)y-Qu2D}!qhe*`A(bhG!@ssnrIxq{F<@eWIDaky8;~&>r1WW&Yrct z9lWQJSE6=ZVO|Iism7HJPIG%H9(~4#S*T}#F18z;?ju=r5o*!g?0|U`Wq0}(dSOFHl2WP}HX*%p z%Nc#ZejmGuC?gR9dyj}#@!RHvS1x)YQ9Eus7aaZBH%s(T>H@B(Ht-HUSnBM{O<@Ps z9rK;R61TI7V*LpxNmD%wMU19!93ymH`OMcg&kG(WvI!(XDCp`l8HW(;fs?I2-U}2` ze>c#vqnScDrq<%6P9ipb~j|X6?(> zMR=hhHR@~^+eL^%zhEA|&`8m1WmYQghVqSko^5p7oGf_gu|D&L&QP~)^JL3q(0ec~ zwdME~yYFJMVSPEUv2dLt?rx!yQ%fhI_iO4K-5Et!SK}|$LBqfs5hS&KU|xXgS1q$% zvWsE!%1k+q<=9F*@6u2oK8T4}z=|M4=*9>B6|znaz;fSA1b6QtWF$lrG`JQhWnXg? z)l&VnBdX;yYe+z1T5v#Y7HTlg*rPQ7!OFfqd~*T@i4VanHDr!mp7!(8H*;P^M)Fef zq+gE`{}pk#{!hf=rm__Ts-q>Yn16hFuoW}91fH5_uxpWTSH`b-DL<7qu{3aq+eHiA zF0lPK-gz`1y@qhaMjx{)=yHKNpQWDHS_7DG{4w_xg9 zv1``Ukbwg3!97a4Zhh`NdVG9Vu`;?&w&=AT7|}jvVS`VrsYEs&HqjXbz~Qd4ntdLQ z8Ivv6sf%pTNb!05072lr-?q0-;;93XMH>NhaS#1UPmb_$0bYyAE^f$1ue&-z0|>HE zr*(&pz@m_-6n71|`-_(Akwf=dLC5KLRDo3Ns&ON_7A6u8_Xy+$DcD6{y4nw9cJ6oj< z*H|yUy-(`}i))^2l?vsCn+3naza%4M1tLex5!Qg$e$8{XA-hL2K_eeB*q(j{^;PSW ztpn8drQ7P^i#ZvaW4myxIS^**eph`3)uDN@8ME>3Hc^J*#&wxL_Av~ffbP);ImT#{nJ}^DpMkbjfe59}a3F|04 zy1WxxJH5AlZ#jlndULVJTAbccznatcZd;Hd&x-H42|GA!^EnzRf>P{L&Yx__WoP5J z-l(3)aeaf-+D%un)2`+JOa@}@H` z(c}GN5}C|ieN~Q57p~_OtKvM*yts~h?an&guy#rBisFZx2kLVRpfE7BcK4-&{M&HC z-~nh!pUK~$**SriQI>dox=iD^CA&ly+5{^S0ZFQ6><=LJPiM6jXF)=_!FcS=u7uE! zhJx?2{@ja1xRELAp4E@451S{Ei@xv|Y)XPUkmOU7pp`&Dv|p?hY8E zQSVQ;zNsgFtYXMDvVd9H9kTiPoq|*|E950G+3qCx+tjAFzVfGS?*exWxn+skGg=$a z`Q)3wXEjU|9c5KS4UoZ0Izn$=ozNHQq;Tio>GUDxt)IZ@DOW`1rhNp+)a=pB*o3{F z{Kq3E#6>d#Ci|Ow>T&Lx|xwAdcRbus!PP3CcVRSa6Wb1ksfKE=> zgaqNXtx9_EODRxo5`NWhBMx7JNg!)LBNIaOlVXS-P$^CKZT+lW4{C1Mrw3qEYFuqD4x3+=j-cJqJ2LA zC@y0uhemoQOZ4sk#YbuwgUHJZ90R)~9q5{nHhMlvs?=eih>i0zyGMvP(J_2GHtA33g{;&g{rgG7s#l zz%69tGO1!Epp#1XWRUE2Vw)s-bZ?o)7VAObI9}wtRn8FIp6|)#uCpV1fKTh?F6x6a zQ06TCl?!PqWekLKq)WQ7#nyRY6XidDPvc@b5)o;ep-tD|JR*(IgOiB6FJb6d9;*hU zWiqt0CmA-rx*sSL`FUh>ukE7>9z{Nif)AbzD^!|JDfWmsZH|FiT(T|)*V9S6rT6?u z9ohlM+nR#Ey(E&qhvxP_O0z_a{x&D7AO8SRK8k${M;e0);HQuRjmIC0a+XB9KNYx| zO~x2maOdVgbkPiYwhs zST0LKlKl-y@L3eX9a7Sm$jmR;(zeYv_RK*^el#b2Qi^R128XiGq!ZkH{;^==gh~Zf zKHh}fB`Qio2W1{Yc>E#w&0v#yr7%1*_{h?4jQhHGX_nHoX4z3C0F<^Wt0HohNN zZJL=7E)(FgyD%FO=$WMg+{RPPRpx{IxKJLY@DNf(f#`aBoZE?#95(mcaMCSeW&31oSfloqj$ywnavGAv8*^`RGkuJ^Z-< z1-|4=&=>E=cho??VIxQ*-q3amBK*`1N_!XZEYjAG4Y0h+=I{#*{hB-A1ewAMzN4rX zY2J~s+xHXamNK&U>*EYc^dqp|Q|&aO@M)#qc3>f}dfy#w{Rs_5LkOC#gGXK zG-usT+w3J+?p!anzV&^67X?2ES;u45LIJd8Nc(a@S=I&<**nSA)R#QQBR)Yg;YxA9 zVf#jS5vJh9j-)ptOP^R`*|^+CD80@~?Ck!OQG=POlTeQyB1&orvlwS&VEe(l67Oz2 zgWl+OQ}1aWPt?tQ;(F=86cg#u*Pl#lH1Ne5-~ZsmH#}C`thH<9BmOe>0@i^FttZ`O zatF9OAO>?p`0u;>{@fNk(P?~-{5xI#u5=0b*4tU83$E!04xcNj>w{6WGI`>G9 zRR%mAFQmdaKiE(UADb#Z7kq>E;V~E=Q%{D1)yL}t2t;w=i(P98@1s-ZviV_UT{nN| z`a5Lml|^1I(TCIzr|P2meV|Zq`T4t1B7^-RBf4r&%YMOOy>SH_;zOBKXZeM!A8PZO zH4|DE$Mvi2`6&UQb13&H0!(Q0j zzKWD-==6HAR^;e5tEMB8QYgIUYa_US%b5*=$lqbL&}bD3w(fV)ZM%$Q4*C}IU!bju zg$4UQOQ#_bTTApTAjIm^|4BxyThF4ZkLAxZ>@ak1fEdz_Cd&pHjdx#O@)?2-u>e^6 z3k+BL74+KV@cM)A5(o{#9GL+mv-x&xuj-0D=qh>uin3?`hA#RJ8b6#97H3JqiS4N9 zr(@EvtVs~&J?ZHh0KuKCijUA_>Uxo}Gx1$M^Kh`yf#MOpOIt@^k{=7#_xGzFWj8agr&3K(2oWQMyUR5L6%|5p zeds%w_!vkE2NfA1lNYg_q-&7gNTCTOtW3>Da$3!l>zh9Z1)$FKb92QrrU|-azZvb1 z(TWBScGWg(lAq{QXmE}%!@wT>7MBsD@m7rJMA;aq1W4stfQd|^eXnV@1;S8TH22g8 zQw*=??jiZiKRqmL8Ef12w;4!z&?B~2ebb2=jE1zc!H5o&q=*&1KoJkhF9Iwf_gViS zQOKICVD4x&x9tX+K>YMYO4@g|9@`gKB7S8L+In2pv5ym*+lU__JGxz7p>c%X%9&IW zxO4R>%5BV%vTQ#>QcKbH;7r)&FVywCDG&DYb0c48QslFBMnA{=VjRK0Z%H zD^05z9y7o24__o+)7kK^!f9{SpS3ZU8W6QgGQg@{CH{Arut=!1GyEmDnv*PhpObTl z9n>&&dzWtGz4{-f5e{tav*>dz!psrM9$sNy5nzX>H%6OcxT{?+xWA`)-tH>>d0R(& z$h)#UNJiQKtZf-p6U2K(P*QY1>4-khV7uc9g(oosgtB|*D_Wi_AbEnr$f8y&gb$fS z=f{*`^kPI;^R|dZ?IS>PKH%s?>-J#uUS>kOeosYpq#At5f}I#D0fs#ua~#&K40y5z z1Nl6<%m9uoI)E;SdC%k(a0PGJhLh|exJ0p*UZJgH?kGs(*Gzh%Rdoc`aP~JJb<<(L zh@`6!)Sloe))vW)kVv&LD>6V@Y=fk6wP`@4}lK z^TNE`x`+k~9qo0UutDyLW=a6&O0Jb=i9^4^{bMz|=M#y$p*aUt{uCvz7C~qb* z?tRg)w*-P|+1Wa=`)ki>d{=y3h9t8PZLZ?fdr`IQUQo@rgbFCneBzGE`>Ao6j-=ak z7-%P5`#4`2(=1>{WgxJ|Y`!u775i-bWB0qUGM2W6fL>HDnt&?)x5-o=YxV%9@`B+J zbKp6ptKd~7!Ig`NqVLu~e?i(Al+ObGPd*zz#FwY9 zbhXV`+zxpx6nJ^-V2bm@JZR2J*fry`OxWk!>~j@TiQjODdk@72&g+SooN|IsN?XI} zsLVFCgpMeIb$A3gCv+sQb@pWZhg2ULzDPrX(*ABTMofIcCit%56&MHep$!442RkxZ z%v-y7hQuHiH7ifSR#c$pugp~tvV_qP6>O0s-9%LCXx>Cru=2G3hIZ^9BW#;n=nao* z)vlB3P;WEaqDN`N$Ii3JnlEdg4aArw>|Q7~<-~fX+D^#zP()k8u!yFAjLC@hLvXlt znc(jM@~W5z5(!4ROSPOlQ4lJu2f%%!kq|kmb~^ZA#RyuHWqX7CX@)hVwO869Vj9zg z4}%__1y4Zr@$tSba+jxrU;#G7s^}MjVWG8XWfcdODP+^qnW2Zo*nGmQxtQn#Xs)^0 zZ05_@K*%B#N^EJK8byF4-qG-^HH8v+Fp>P6mm1D=LnmZnYlG`!x238|o6{NrJh@K2 zK{vm(K+VwYlVnN6w7IBg{36zj``D)c3mbmONl)VwR|SB zo>3Mh=+Ql+)Jq(F6%U^+t?X88S<*FURJ3HMzSfQAJR`%O9%ASv{Yd!E>u?tb8F+?6 zRy;BLwrvE{&tduqPjypt?#R$vU@Zhb>iHhoYvp>M~_fV;vdSliC%bne9l|FFufVAnsYkt}xH@ zpdN9GzTp*b(`3L zKhJf#|HQS9za?QN%5B)UNo=-3=FqdcX?M=s2m9?mVi3Dn|7CwQ z2}Pj|AYfDeC>vwKBAJW*MFb;}N``l$XlB{S+5=ELWkjQERY z?46e;qn;U*kSG&MjTV%bDXQviSf)D-0moM29VKLTa3@M(bhIKf1^ z3ME=~%7n@h>|88Pm8Az3u}R%qxHsPK=8~>YHYLWKxDdb zmCZCHTw<1)E-c!NDWDnfL3s4+uuod;o`g>(`PbXGAHJAsH7$mr^~zgNz`nA=^sNuX zimTrkj2P;b!8#))ptWlFF0B+l<69HO)s2HFoZY^35m?J2NMb(HsZ_nsD-9dm=x*+l zc-dSTPYJ(>xfh!>j`Z(r)sY9 zKzarTeaa!;QcaAI6~Tx7IXx1qb(AD@kg2GSb}W|_*Or6~fDZfJnFczD5}<>4_4pZ< znfN^GpsClJBBSrT^)T?v+sujc%X0hc^0Mqh#(2XP-)Me$R$I&p^ivjKoDBgXH0)=m zDI5*N-mT~RwgmqKgCiOEe0x&qG8*hsM#JA(`p&pi@|~I@niU&SI(;H0szO zS;-H*!yo;buHF{!y&lIU8pG6XSMY&7^JbX3qK{=AiQ#K@`^-voPBo{d@v@hhnMGur zs>|=o+M-NT3adDOY&)M>aY$Ym>lR?Oh#{u)n~E|H)?_R*M)RAeA*-(P<* zkBxzMFa~R$)BY40h3LVYjcu{CDztL%)98CjsktMFq8NT-Ucx)>)eN)NXjl+IV9S!( zt2pfu8_s>kR!knrFg3aYhvSsGpijdsaqk_liJ7M>wHV32d^B779TJA}o+^bQA7iiZ z-?QmK*Xgh+aysnkkPv;*i z|D#Q<;W2&|wfuL@z}a|r*`J6$P#!M*_o2zv%m4Fc{ImE$6injyw9y@~H6@s0rv-`o zK+s@@7QPh0h8WrB5}Z(7h-$kNnJQX45QE}OUI0Q-D4`-eQW(|{ErO;whR%#%&I~2V znvI9^q8sVTUAfE|sFy$spzUKKaQw<9zA7CeeJpVc=PCsxZ{q4#&e0U{19Q!AhYZy5 z+dxD=lEO(Y;gpfj^N-xKmwxWb)J-YCQoJKtST2JN7k!te1&zzI_{qzjlevBd-lKT>8^PDr>YvfVwjpPH;Try3j7bx@yrp$K-?@nk& z1kt|cI#4lG5jcK#SE3a<#+|a; z?B3%`D~0X|?GQYA!4&n7aB5+Pg}Wtybyp~oPp3NUa{6h=8+OBac`$sVJy z`}1)m%T(HCyMB-qB<|mV?CR6M1KC;27?lCyF6Y!5zYtskPEl=qXX0zOiE6}$_14&F zs5Fib$UBx@YQ3sWvF}a$Q8za9UvZx(MhN-eAKTr^6s{+C&+UJ#tpR&%Dpo!V00&YL z(oWopyiI215B>6vCJMn*rPYrVb%~av!|+Au;~WBEArpGG4xK&|AHVz1t>M5sW#90i zTc96Wti`b+eH0k)m9PvYUP5xCgl|Z)iHCFXXOun->8X68Azuf7pYh%{SrJ9rOO?&Y zL&`E$*WCLmVs5gRmgT{euU7?+RKDCny^Frv_a*Aio8x?c= zK~(9FeYh^4tN@G%PRvM(CCy#KBpd%z?Gm!a1dr85hOmc8cRvR1Whl^lp?@k7 zcQ#GpT=h_Fm(iMRhG*dzf+PY+5!AQ1ve-WFLUEhEv5N81;4=W;6Dz zD*}^l<6g#fBflz$u_FN+i&OM)iy|snx=AM{VhH0xB(pXb@Q~Or6MbKPMx$bT@`&L1 z{;w2X-Hu$$c6EM8gu%G@44S0DOH5=|QzvN9zV>hRdMq}Bmia+Ayqm#=pVphU#cGa3w zl>*`}U|vKUTt1XlfEWzmyZgkX8YaZf81867*Vt{o$F5BK|GNTbC=~UNq7NRaxA)5! z!S~?q(*uPTtv`FRM!hJiIUDC;*D<+p2o$QhASbOOhOvM?0+{IQMi^(F*^$omZ zav$vH1Va!Ha0w?DJ(grIU@=9CRVQ-NW=N9qEi6XLcG8v2->$*;Xd`yui4cpSu;ics#a@YJEle zjjLccXK&U_*)!&cG7I~vrI*eytf-O`1_Q!c6$40&mstWLcOda3PR$Sn%VxEl+?Gis zYWkF--wHNJ`;brsUwn2c;N%}m$DnkE6U-G=--~P51C_PXrlKzsVpRG=vFCSh!p-zb z16psX)_<6Y8~4iBrDAC(C9aahdLazc6#oDxe{{oD_;Lfn@EG7$;Ox(*05r zS8uss8KCQ)g#L6P@0MriuTa#iHFsA>JGAhsk{?hYL7)Tclh(STn;`*`FG!|x)?pti zP?67i(!47*-eE#7-_xtDZ2fdK)mk z)|#{xHbu!{-mm@?5`1%<50=mWxH0(ap|JlFhHo)kj#|(%6J5k&v%nkMAEL>$V_}5p z_)QlRAf*kh*G|2FB!4l9suQRWvCkZomL;ET+napcZ`@27(Db4j8nd4y<;i&y=1~Kw zzyv6wIMKDqcV++JA58k@OM+LX<8wUYSBsHr0UaAVFfL+H<1bc5c4{fHi7tzw!HMmm zB4%PZt=vp6=0h6Fjuyx0?0^)LJV_{dq~n~lp2Q=-ZCW6BXH$OXHetOXsKAbUGJLzI z|MjJoH%+I(L5vV_?2|66i~PR{bQNzcONEgQ*dBqly^N)Q=h^GcD4fpS=yH522^4sW zuPUKy_BVw&SuU?)s6<`}CVzgMwS~-0R6}@8F7|Yxxx`QA-Kth@*!$Cl@)pj`9q=Kd z+vUcHCSzUpFkjscUFrPG-rVxRkm6Aw0kIp9@$_+>1u+(4Tda-jcYQ_vei%*4-?AnGuysnGL&<(l{M%584? zbnoL?1d+92(pflR-d&Dp1S_VE+Bebsj7l`jRB3^*b0oaPaNz0i_yC=_o?>Q+wH5P4 zM8xCnsc-Auh7w{xSkx$Jn%iF%<&2zDjW^t)Ndmj)HXZ&`PW`t|123rsT+nzxeB$}H z*BWm$QVO3Pt&Wuv(7^g;*Y;(aRBs1m3SIG;E z;DYKCVfZme`so4EefwH$_sourFH;8Ob-2v(B;TZ4$1e$uHIzNlGQajc#JCFvoiGSw z76wYbE5pJ(kpB4|8B>CctF$h*YpBYkFte5+{e4C!CNCs?mDHn|CXcN{JM5|%_4dBq zOpQHl-Kc)i$ohu9y~GsONM{m3dx3H%7H@lqk@Hk_MqWWjp=P#t-**vsGA@ElqZyU! zwtmbR9>~87=behLGkQ$@w*x=$2N3(lBkn`_GX@IvTe*|^PtnJ zobIy8K&^iCf$t*-nz=~ui%dw4KVmO4$tPMJTIBpVbai!Lgb~Ja;Vg_CFq-!Ry?vP) zOjJVAc4PO|<}mwRfiq~(2ZdEzPDZ!+JHFIQ=UKU%U|}Z1!Cj8>vTnc)Gww;of6|Os z8&(tBoi^YymnV0x{I45Ele~Y*=UoK^#m@QQ=J^j=|jwPmNWo6b_hh z_@b#+@*7d1C|V@q5_rj>{|#^Dp;>gc{Sn&AqbrT4rK(Rhf9EZ%bA+n^LO<Tzto>C)S`@*e8?06-`1ZzkoVASdOW}L5PHPwnPjwHrs>!#J9 z+7jIiX@m(26aN{MFIGnsfK#E`!>3d5;u6hC;%=qF6%mSnAOElzw4&)%1}46GWI0U| z^m5Nr!4UoFXNnzq4@x$KhDZEDvEArg{Ex_a>UWc0{mTEiHssm@xz62L=gTx~_0)Pb zcKD3Tpio#x>>KFmlcIzG@e10x{0-(L*qLi7o9`C#Y!$4`P#H~;qJb^qOy z_q7Z){Joorll^Rhq7tCNYww8u(~{HE+Q!`Va3uz3<#5b@My|WzsMG=6Z1r89es~{5 zB0THBwMWTaf8Ic-w%E{&fFU05UZPYc7+1f`Vtw~Tkm%xKtESXA82kYO zs|B3#O5;adw*5HxP?UYr@4zdb8}n?*no(LZdK!(d*{`+0d#_}R_{=`>O?>_MyS({3 zfF%o+y@wskV4FfM%ZQKxJl(--o4u&B^pd$oS>E7PS--6>x}zo*@4Fa~$iAFBcfr5d zmiCT2B&cMHGk9Mx3f~L`L$KM!(xM}9POf*B)!vE*xG!FMlp3{& zOIgM!$eiv@#J>9d8F}N?HyrymLtx}6%d39AvMu5n-%G2`N;C_rA`y^1FZ!UYBhwv# z9M?)aTm-@<(Yi{0)syU8pX5p?kDkhXR1v@kz5Wyv$+M7ZT38+zfsl%%`aMs@k3jZ; zGYtGsYCo<8Vdh(1-X)`YO?zK3iP-2Vl!`*D$3+5hi-&*7X+yz@{r{i2&}&+5^=aQ0 zqZ?={nkuS4Dbc(O=}IcS`+u~(WmH^UyCqsh;hx~GL4vykCkakSNN^1vtZ;`SXo3cJ z36KiG-Q8V-ySq#9-sC;sx!>u&w||`O`;Rd;qiWPxd#yE}`OIfdKoFoFSE(!fz=ne1 zd?>4I{AWENoZ_bvJRIMa6iWhh1n%u9ms!~WUy@4jPNA41Q@192U8|heI+3U`tL8p+ z+F3}YU0o%CN$!l)34Hje7AA?V=!bY@vV5}6>$)Zzs889Gp8h(NtZVa7+%w@(BE1~% zU(Smj7H%5(U9!xS7m#iyaaxmxk&CA-$BbPUo$1q_dv#Nv#uZdo{ZUiumtjlFaG^-QoYm<)3XbuC1s7+d0p z!I!6Js}lwIHh`5GRLK{D-TX{~6r```wTd;lk2JdkI;Bz_D)6{G5$foT&k|Vh3WC3s z5@O`gHD)qhCej~9@)1WC`6TeEeG`qG&W5YoVSG&vd+GCbf#O|>esdDXM3F=4N2q5= zyxtlYT`_8J7rqkI`I|*Dy$^^d`Et*+ia>BD12a5v0HYX}+zr&6u{c}=GC(Kf7^EJr zySq_sU{G>h3{3Z45%cZ~jkaSDKU+Svp0%e+Yp@T1Z`MB9S5!w-$?-VU+q?JUVg?k% zQNyo6F9|;V&aUqWIess*B|VO&EF7HEP71g|fzjmhBYP)*4vxuZrXFIGcYF1+bAsmk zH9!e|xiy?Njq~+iD48`P;{f+Duh7bm&)Ecb4S;T=@XlfV!#}mD+W+0!bVd?+Y|v_| zHqm>oB-PYgEC@Z0{gt@Y;%hQ~+wT-8IJXCLd-QgZk_gm(^hL{&Z`Y4&0mYcXB_DOF zq|s|tD|b?99vI#=HtEZ&ulO`gO9p}OL8a{sj6A3i=*nG-U#|T?5~jiuF$@o6QQLII zFp$I(L}3c*Yt}@BVFD)5Mks9@0#Z`M*{=0=M(#NdW;??bEET)sUuw_yU)0C5?M^TAQx^Q%vFKuMGK&U zn0sx75&G2_a6dT&l}nI;)j>Z%!C+U!Lf&pXWr3+6YY;o96aHY$6T;UxM6AscVa+5U z9q*~iSTOt%RV^VKq`p;Us5|aBQ^6tVe&Z0&W)h4{1lisSZm0XX7cHIis_RPCTO;{Cbx&~Upg(ckKFHW&ss+ZMk}6%}dMERj zC7uI3s_GV1xjHo^NZvdM^lA&Q8Yzr#6|bvV5?M7kfc`}>8FgaieC_b*zW%bt$D~}( zS0STc95$IcL(O1sw#+~uh*d%D|KPfNzGk#DFL;;~DfLiaeHS8tYsB{CR42ca$@i;Ws2N>^}x5(>ND|+3= zs0@Y2NwI?Z@tL&)m!EOqyb@z#R|%h$0LQrXNr1%RbGslR`hmvnL@sP6(j{Yn zzlXu;#AHxuD9mg(Y%#H->74#;I}wCK6FwE;9Tpq`M;=I49k@QPv0d0+OyA+R>EGSz zlR?MNa8rmsDcKS~qf0mI#X=;GUJ)!Y$j3?M^-a|Q9k*xa)Q*YC`DCU=i(N-jc9v^& zh1-+Kyh&8dd9H46$k0yZ70RE135ZsX@D&(L%^zzu)v!RU>bZ>@bT9mri zj_)Se9@SlEBYV?2X5&O-e3=iNIb-6-_o{#OOWOaTU%G4n-bkm*HNDL_pT~EzSMB$hrQ(UXuQie&sgA{#>g>BtFc4zj4!;k;W(R8DRaho%9=%Lh)fw)%iz@qXLX^ zNrblWS;7k15mH8YY7=GlqwMlwAU<9!=|-rHAb}nA2Apj*e~HmBf9U0nrGYry5~xI0 z6Z%-srnNO4^NNC`ki*D(sM}*$Fu5_61omrF*StUPo zpzq9;p-iGJwfUr5zS^&IF_L|`0+xvyj(Hi#0m5&XS;Z2D?hz}c@`dzCdx7*pC(Lhi zo??*-bb0~btIC_Xu}OFDi4qQh@_y{4H%rQ^Ca(4535MfbTOHHR)Vtt>CGdXhlOai1 zKp(?7E*CRpMeKaZr5M$uV}Z_Z9nmAZN=&_Y{owVPLTyRV-(e&OmP+VV5tWG@LqJR= zEzJ-PZ(Z-RpCi8Dy!vj96yac07`|tbhXylbvYMXnIR_&I5#l{r9M+<*4 zc;_~$?7q>#Uk=!Tuv@mo>vN8eZ%xB%8h#Gd6ZAcfwx0o|;MVZAs?Hn1)n=L?_-H8< zYen?lk0ga7<=ZFzPg^mTotPhoYgJL4BU?%)Z5Oi9f9I*D24YzN3cs`YxPkGrL^Idh zGXbyOtNZ=SroUaY|39cef4@-rH7rJa801NA4?Zs5D|n&42?%i(1iMW=>5xQL@zd8} zI(IF?@k563>UqEg+MaQDJSA5Ey{?}tV0pAqxL2hXhfv?ige{4HFaU)Y+-#;%HiCdd z{z08nyvW6;O5g)vHUJ+6&yDber5Dx7#%-8yqH4qDCcuWFH=}f40c;cRIZum>1ioby`gK85ccx9SsR;8{`H)Z>$qG#(^R1b&nXWCKKt=Eq9 zUYsu21b`t6O+=Sc!;6Mv;Ws6Sf)FS!DDjxeyE*(hh&ggIry1nT1*V5&AmJh}eFeA@ zC^?jUiCWPGDHv4+%^-0`iPVAh8;1c|pogN!lRShR`FD>qS-+E8FwLolxcPpKEFBv$ zKNpSNMPFL=Ia$xF>5D(+)q=eP;b(>t>F>WZyDK{7f@N_cBflY*L1Kwn=K{$h&prui zGaD)%!?`}r{ESq62YOeGV;3?gn}v!8m&EIOJ0efuN=_qwW|U20Mm3P!x4Cp%tCXyA zsb{O2BU~ZGPmB?)S$CjYqyM1#69i%N3dFLbNlv@FewVtT;e1F06`_oLq4bI(E z`3H=on+|)FGj1b z1S5e>TT|q(UVj>AEAnL>(QsqBvq?7)bW^ zi6c$b=q7Y2rgpB_c|BVWT@9=jp+pEjUnXZ`m^?Bt%!|TTB65+ zw<*q0N>@ecWH%1Rrlrt0UMP8^&Qu! zQtV-O4V#-iKF_h2!q}Mu#aO<6PO`J&^8J~ZndS5ACyNxuI)zPj?_5B$VR&RNk#_rJ z&wO-O8vdA+7Ie5y0#Di1<#1h?fG0IN?D2i;GtwPH65DV(QT)T(L~L$)5Hbki!WYZF zO_kEXFSW0~365rt6!~)A_~TX!EYT5x^XE)P+MI}$ECS5@V>%~a02gM7i7G-~@oyEG z-KT;&SA8@^ZXYf*a>W$t%UFn`Z9qvTO)*yOt*wBpThQMZyQ$~(!8=LdP3dG& zylvx+vb5XsFF)xB1hoq<<_(YYo2VDv)jEZv%r24B4dju`$JdsE-Bz=NcSn}9XLffj z8h@l-&kiJSk6aA-5SUcD0Jiz=tn-^}+6nPZ;ZZj_C*qIGB_FY(|9$UuNW+D6!cWqkJ5R;0O{SJ zt#(xXj{j3Jcu^VAbyB*$^Eap}nAot6G#-~485S_*)J`s5&L)8$z=a5oL7@L!CW8rn zZ#gb;ja?&(fcJ4Ty8r%OKu#JFjMN?(@R?}5f!Ur%&X4%a1PdlJRVBvdjoeB6(-4LU z^REq#bTQ(kg5jc@sEC~hcy9D(S%2MpHH++sSD3?&B(Wm2>{@$QH~X9#QBnGZ;A`#G zQ~HhGN@3f8*~_ub3K2hbe!|q<#o4s^XUT`D)^cj;E!zE(q-&+e3KPO^s;8Y3y~8E& zTTTYp2L<~wlTjR=wRyC3B5j{qD&fPu+3O6-S+fNrV!7qqYRZkt2@cD)R@D-<{%@$T zUlz0cCx<8N*#IwGTcA}H9!h>{yz(pS5>`!tjK;i&fB8(aY?)`#r%&dqj?YAmUufU@ z^H_V7#Q8pO;?tt2eQF5(=Ra?!mT)h|IKP1DEnMmG_?dI%A!0xJhk_e(Boq6&m~x-H zrZTzeIfw54{!JGs%^HK(LvAot#VPov@jA+r%Sa26to^Qc?=X#l z1*5sOQ&t+KW7&@P3>PFRqSs7t=6A0FI7=TPr56k&iKo$FiM|v%79q|sI461ERwU0HPWmdi_?ai^GjS^(NUIPLw-jRo^A81H7=VeLaR1F|u9(LlbB?Xx@u zL*nMBV&yWUtzIcHy9EjM=I<$-*!{Yt_baDbdLm-|udc7>pKYYN6<0_IA=3crV-t|U zI5Ts;g&*7sYuM%{jjdNNk0uSIkIGfa%BU@NZyzOcJXdyd^1`++0%L=bCLJ2VbxE@e zu}nA&s|dTFGMjm+!J=(B;_@l_@BVnl?EM2+J>7q;o)K=aFSiBikK$rZJ58<}@mW3} zK9bvycg13mZesdVb(oQgeY(Q^CUDfMZ@;}&h?z%?B9kh&Lm`=Le0%G{qnt=z*Wloc zLK5Hi;b}WX`{TI^9H#a)7b?d*xMUupSwSm4X?Kr{mxrv4!IKTcTtYWV?CYuZ#5#_m zdcKKNA5gdaOe7;%NRNrh&!hVg3VjP=7&nm!X%QwwN1<*e*T$#)V;!QhtRT?{K#I7X)hw8HzdCl^J5S zhtU?@617sAK3f-`F?DC>Dy(g~uIVDI!k3jx0gP`lmYB#E+n_Ly5;rx?>&SKGMsCfOsQa}qL#I*;d2yh6{;bb{nq{`L^}73- zTY`}^8R73Ol=JRZaUGkxE)K%|Oz+vOB=WHzbVl?UPybVSKOVx5+eI0Q)n;bwT+I@R zT_I#aQ9r(43Tdmpt(%wZT)(bkaY8Ac7cBW_WlkoaErjSZ%xZLVKxHlH<6 zFxp;hFujj#Wg9PxMpX1m{yIT4X*)<-Jwg0362=Wff?)#4B{yNJ0C`#vHYv z*d#2wNgREzeYM4nEP}7u`R^?wpZO>85r42f;t%;sKTK|1<1q%#ulor8)5MhADhxU& z>M$7<(-&7dDJ;rbKFlB{<=KaS-%amu#H=2Hi!J_L>U?7(@F%cZI(8_)I{w>gwFfe0 zJyiIG3RV1+{@VCQuZUy#6qw~+fo5Wp`@hqo3a|lhQ5Q*43YA!54iX}dY=!(GMxL5h z>-$234D!vMUoHbXpaFA`D|UuR@)MZgAgN=c__Tn5}7=ZjNX}WA*KJ!H;vpd3>9w*ekal zK^`AK&NvZ+zyQ)SoWtBJwa5H5Qfk%M##N}wnU~>5XRb~0X{pSNI0y30Tmnoj-cbEbAn9YVa^C|l0DXo;p3Z4-8ez$;nX9|vcizXo> zV_|n_A*#r?Z+o8CNshr3PM5*6TwWI(+}9m;Gq!Wqg2)+d3F4gBb~Q^$dB%1&&OkR! zC(DMzu5Lio->aANNUH69hl3G26$l@h`Wt>6+84{ld&3`?6vr71Gi`PTs3#=6Wo?4l zqjzsa7&dk762pnUI7y_^J1{ZZ=zLYOYh3B%ahNyxk*b$2mY&FcLKI)hAILRVtVff$ zr$lCA;T)GTcK*?AE0-~vZ*Xcx%QXMc>&#F{JoHO}G?{|-;zpeDZ0fU_AA_IVQQfPHbOb!hQ;O@%YTde$P z`6Ln(MtD9Z0E@q&@_EnRc#%Y#L1r0oSXr>TP_;m7{eqGJ!k16=&sqECfV#{>h+1{! z!wH3PWDCu$pcPb+l(h%(<30Kjsyu{qBK?f(McRS*bzXHh{bkLX&*O%VrR->WIUPn_ zL-X>Vn&Q;gc*Y$VB)|P7stu+JR@Pfb?$Sg-f_Km^hAEG#4GaIy=z-Lv+rwp_m z-h-t$yGW_g4g4P77czxR7LJr~y9VDiy38@iqAhCXh!MaoQc_K)AB3!WH>s@ zWTr@*u{lJF6h`Df0d@@rqc{v1hSoT5q)d{gdBczH5YU> zC+8@Kn9fwi?t(r3IiJOjd17C5?`dIo$GFB?p(MT$zk9y`1TNAp1xs=Rw)e5iZ9$A~ zSfnMMJ_NFXoNswd`y)rPWcub^7TJK%kJA2has44IazPbk!DWAXTX)QR^gEWn7`bBh z{{$mfVRLUZA77Q{mg>jKpG?5h@qNMlqL6NoIYEgSe3grkv2J+lPVa%^II!8wKaa z>X_;9{!?VZr7gqdBw`*v8A|fI>f5}JZUJx29qg4voY6-?4fwBV+((zb1(u+#qQ!tg zXPq)s41@&UBFb>fjE6kjA;)uol-pzi=bB`AHK1xC=9#9YQwmH3@G5PR zPUnvPZ2=1x`f((*uXQT(boFO$f>VNCNaAEp(SS%=BKPLlXmhc0-b4Z2l{UWu_AX<7 z51%1h$FaJ180#A-1kYF?;2GC^7@Xu6&(RO-r8 zpMBdojNEPXi(EIilkP{ck21t_25MGXaBU2VPS;{=C$Hkdl|$jKDfMDzu{awJ?TK#% zL$s?zMc}#g9f~>HgXI6HYX1#di!VwtjB#}|N6gLX0C94q?|;3QGaX2(XuEPh-{}E7 zdtC(6auwqVc>#NM5+VESgTww*1~!&D!9NVAyD>+N`N zVjwsoDnM4jGb}Rx7$ApI4hWGtO9C?Dx@m&QOCh&@`Fbp6f`Es#+r6VX_S{FU)oTsCTB0H?srNPX=$iSxl{Of$DcR(7s zk4P21MM!LPJE?C$DG|rwF-OfdL-p%MGc&(_AmHS<{R{WjpyBA@gyhLY7uB{MxhSRU z*%P4*`)4)B7qi9>@&qQg@%lT*1VTb5TVqBhO;D?g(PwB!M+28UCf$(BzGg#dr$vdt zdbVVOpI6WoL^qFHM*B7;f!?3+^mj)v6(#?H8oO?ale;+~xw@Zb2ctRmi)~qJGmT!n z52JH?`^~7-3gon3xO5~1lZhFcoq(hz6cUH&Or52#RpMktK&{gKnJ>?HF4?yGT;O7o ztK-gOrQ9~BYd|2^57o$L@@LS~W|`Dal`oeQFugOIT*8SdTLd#3rNhPKwI~)OeQ~bA zXDx!KZgXTseQ0JWnFgrmqEuoI^W9u;dQtEiBdf%&G_He=kwx4$$Bs|hPlvYua5Y7U zotAr^{Ein6?B6vS5{s6Q;75r8Q+Lc5U{iXV<_DtFzQ@7stZKTYhdT-BOQR;veibeP zRAYR>5cC7ajn4>}t<~YgJX&H&yl=&lZ^I_M#)ex?rpJ5S!|igg^t}aiyMTn#IG8p} zkr-#@{%8umw#vcSlWgc^CweD*+|QFEpE@JlwW{Z%E14M7I_E`v&B@#^dFTsS(_ioH zJ-=(c#xV+L>wl4Xn3e9KJQ-XVtJ0;;(>UgY((dG=Q5Kq7AUaq^`SYe>NL*n@FQ$UQ z-XHA_V1QO;ZRLtosMd>-s}F+Q90;&I4hAMG((_~Q4O5k`&f&)Mxfn_sL~U(<>i-5XDztBL>mRuHH9dDu(sa^Z;z#@LkW(ahM;5|habyhSnY}B};#0c2xtw%B z_RA1<+UcfnnajM_%Veu*_QkYgA|&=|{*^rOf~;#EPA{kZ;PHc>1cEmSTSr?+Ax*CNeRe`}E`_`&9_xC=hOZ1%!@7iJWc@#RIu2=9;1&7ctlr!draXy9N3k z=R+}+dHs=!iJ;+ycOy3`1-Y&^k5zMFQ`P^%Rc^J2XoegY{zGh0AE^9SVr%q`j62i9 zbgCcbktD!nojrSTfCu7sYqE}gHEnsGl~JXcz-B#(r_2ffeT1Zvp>go({=V`=-9=^F}hTuc%2`niWhONl9`O6O;Sy zRZ^f)5T#f7EnLd2#${YK;K^(M+WWT~5{XToHRVDpPzx_t6wTOp#@#k?b1=z8o`xj& ziQd%W%@I?sT2oqz^Sv7>14^~=ULVIZTxq~HE8B@F3oy=_A)4`0o(!poD2Nc2`o))D z6waovoc@El=ZwlH#$72{z9WfmJ#=uBCDh@}0?ss^VF-mTqOO`oeha)d)R`nQ*M?SU zR=a;#DTdHXComZ#tI1FnHe=Y(0LVI@CR^C4;pHa*Yg!zUi=QH=J1nCc-E`%HaX}UF z7hO~=A#Eud!E^@X5{b)i#Auw7_gzg<_=$*3Js&o(*z*ZtCQTwNZ3@;+-I)(Nfh8TYr zFXQ(Kl;%m48;i&sa@1=Cd2V6^sG@f>T zy(%s-N$DQDpE5eTfkK1Yo^~a4T5Ijjl_6t)nvl06(jwEI&bdJmwhjR7~&Xho0?az!$@<&+5D}9 zPy2ZYirv<}$*cjW>-ViuXXv->F?e4Wl@DuMAeJf&i7^2P3EtVXTkY)G`JER6drro0 z6{)W8^#QPcW}$l>8cF5-M{?F-x1j15UcZvSJUxK&%JB&a3BkUye#lr+K$17Gr%x^7 zNv;=uC?z98GsX42&SMe8fDn@W+CAuCZwinxJqN(TnFSzFQ5bA!j<=j#9lp!clGc#3 zCKi7?^tkUwv;M?0)ji3k^qi{p3$1ThUgqS02x?ztoM#&p#x= zSZ`kU9EKi?4>@mAXBle2*r4KEli|=%qJ#R#t{#7)ZNb2!QPC=BwCDM}V`7={7vU(7 z-Vs7$pGkaAZ@rGw5{R?;++-)8>g*P9k;wr%@b-U>yS5c>EN{-@OoPflG=qW&K{0vm zbAW-8DSPdSJewb-mCGRG_)w@b1~_w}VY0z_EtA$u^ToP>OSC{Z#4r z$@NA)@z;m7WD*|7J(lsGG@Ck(nXFnB@|#P`@J=QJDIvGnN_xYjEtCTe#x$=u1cf`B zx(^#gdb#Y(F!5%tohS$l{{n3Od@?Uf0&@x3LD7qkj5m_ZX0K?K?-{v1?#5TXE@5kz zv$yun<{>p?zdwy?wBVj1j_w!urY;SX?AEP z3;U^M?76~&w`RSpVaeEw^rE8#-)H$eJN#x8)HI_-nFF4k16x!kedY_3;* z-PTwZlMmkIgzOEK$tfc_WrO0oHai^RxqZq9w%W9pDV+R*?HWQHB)DOe_2w>Q(_i$pVnuE+DMFIrtkIUqhz^Bu>Yj(5zV1835RXnnqXi5 zh?#(!f!7&m76h!AJqOx76@S}34iv`yIY>DaZ$P>rMVPwS-I?nAV>t0we`y5h_Ebq$ zpaHGo+t}wH)-sNoxxH^)2OvwMKrSj)$9RRMI?^X-4%oCs6gen9`i+<-4dIT>OnGjc zj0POo^~{P6GZt_Q5}h}w^gC1K7SR(;+#=aP*pPCc(EXp)^(8-?IK2k<9w6=cnGB^{ zHl_Gpahsz!qlk*h#W%~L7l-w<2ONpYJU{|RG?2UcEFIYHj!35LXb*y!VdZ8(UbK55 z5CXDOIhL!+SP;)vdr-T}OXZcp0VJb&b8mQPzdCfN{PF$17WmcYF6ukj26c8HSFyPZ zy|UJd6zYkZ6e}x;4jX$!QUK+?tMT>?W@&iG0FQ721;CL z$dOB)&{|NiZOY*|2U~$uVfxCu<8qn)Bu)3<4Nz`?ZmNm&mP(agLpI>jbMpIa`($gX z#9pG~D|+#o6{=V7P<#Y~;;^$W^lJTGot%-yRB6rxnkxde!|`u*x%&w=bVZSSpzTof zb$#7W4i6t6jA`zpC?j-wdun2$9RB{-Tj_9-1Hl@nEtMM4@`B9)0cHtqt3Q6t(>%sz| zQRQ%2mm|(6(Ai~oQGH(8JSfnadowZ;v;U9z3gR|&uLS*R>U>TJzSRP2jbT1A76~m1 zaBJ<;dOkHkVP^!fY&b__2bpAk?sm90iD%e^-<@Siy?P_|P+xz1e{+&*uK&BH zd%pVZ4Fo!R6X*LjxB%-d703i627~Ik9qI-cgEUMBlj&+^D}VCH&m@{@dKb1zlt=J& z5StWeGsZjZGK95~^XTXMx`^FtY<_jhJ*5hQd~XmePjMWao*{bidckjJGCuz^LW5XS zu?kJ3WLx02*8N#tQMPB;&dG)hRd21ec7+YOaRSH0jpbnzTv^h;`{Ude&Yj3X=vcJM z%W)K3r(-^KOUlk~xbn?vMtYt!zUnZWG}jBwRF9n>>>HbfPuhMTzt(kconq@5m~iTx zJJtPNGDj?g0+xP){q?x!)E6aN>U`(=&h;iM!Vyd4Va8If#`8g0Xp@|R@g}@-2byr; zfGxPu%2GI7YSn+0HetEI`Zr#4Z*N^A1569eJ@3{&7e8VKrSu;hCw4i_Bi{{n3AY#E z5mIbs&fEDC7=}8e6~Y?&`9=K$v83m_EZMDQ5b0`UR2b2&x{gmOxu=GQXiyufNvh+7 zru|OM=C3xSLBKNaw%ni*6G$ftc*&}WJ4g>hlu32`E^HK2prg6Ls+S+mA#k<}KkENt zmd0GQ=v5Wh)8~n??x>O#$PMj|Y#&%pRM%6h25W$Sf{DV$28CplarQM5Q`u0|@|gy- zs^WkXUw_PvD9BmY^P9-|Xvo`O5(!#AL8;!aZR?yiIDyU(LN?8^*UJclY-H!Q4 zq3&qmP~(+W7cMMwJu|4cN_=p%QTh5po1&~r>|U0JFo0TW8YIf-BrMZQ`CeKY|E}8H zx0OM`p^V`EZ6q=$^BSC70(%3^b$#G+QC;H4Aq@NuZfB(VEj~{LaHY#k0%?NS_S`{< zKMVm%ARtmiV8PA4Xd8OkZGiJ>5DD)LD7e#_Sw*kUV!30VTclt6i^b->d!>wV z>T*iq(#^b?mUTK*y;}S($A)5?4+{ItAff1AS8F#+ojdmPTcIVnvqn?(=vmXgT54WV z(~a^y!NzgwaXOQ6hYo^k3G&q~z(+WRGbkp0-_(GQcaaw9)Ty0N7^EK}LP_?!XabK6 z{E6s>h@u&HNkROdhQ9|WrU~)bJ#BrJYqNzijJIk8CEN}3hW5jLk^$%)@XfRk$9}Ov z9N;!Vh~Le_(UhOrtD%biDnJN+r+&GYseZrbwtAPyZNe*}y5WW>D?N)^X1r3%TkHqT7P>uZb6S*AX!x@df{r`l zLQUQ(LjwZn_?g~nP0eyLGl~;7931CvDgJ$Hf`*~#q_16Sk3Zttcx09V^^&R&UuBod#0FrK=*%(FyMX|aQ+J;%~6SD{8wkN5;$| zlWXDxEZsn#UdAv`O86?jY_0!vP_-GDQOM>QjJzG**i)gzrbDmr=Z~Ti^tGes{T+2n z-g9G}C)^7n!NR#~ku+`TC73aFKr#%$z*eRhHycLUzcL=B$GRLl-^p`IX-pH-y}3Mq zuM&8JR5Gm~^b7Y^_2fPmi6@8h1lhYzlf9ueWdstL)*DeDomO6}{;)T3A^p>87& z^}My4#@&FaC{uag*9fCn5N8ud($tSGp&Z7~3?%=;!-&LOF8C=;VcJ@qRQ3cG`dMZ< zDM^X6=h!TT-4e>tl1eY2u2pHlVvTLh!7^dmbHCo#7wK^@Qq#)38N3=E?LI&&(J3W6 z?|!K(7Yui5vX0gc#Qx1?)=>PKn1$RJGaG>kjBkj&#nHvu{`4xb%qNbS&PAM;)=tl` z@9M(vE)frNScyu3ts}wiv+FHVI25BQD!pTIBXSFQxV{aM?lf+JK;bZk!gR)XaT zgsIQ4Paxw_Y}Vi-ISJa}QagJZ#i!J0?7R*e{Ueq^SY$u2xo@YeKp2Mcm>SE(hTpKU z9-uA0>({$Q@m+0x*5^mi*Qqtj%hB9eaU~bps5vCdzR4G>)JDCCCa67*z07cYA)A{R znH#V-YKK4U6KX2ro_!mZK=;qLSx=9UMi?Y$XZnak$pn-migfGLB*|Xfw_RAwgD7)$rF$R7n-%sSF;{@N=Yx2hcV83)WRVt+C)J(j>o zXfQ_Cds%Cf8>E2g4QOgi50%!&Q^@@4JYm z`*)IwWI=N@3ihREuCrO;6Z#x!_d1yUbuu7A2mYe zJ1m^DOa$ zPbeq*h{k>pqN#Vd#ClE0TF#99%&hDKQGm*vWvFu4B`^J*y}6veIv&o-(4>}dH2z#f zn==HNn)94~#=y3wKO?y-9deQ;dBRjV0x&h~>bdyhs=ysv_|F}C2yhXh)^+|t)f<2= zyZopb$y|yybmXlJMNV;W3q^H5C?Q){eWucyg+ka-mi%3QKR^dF&5hFCu0p^yOCY1~ zmWg@fx-)Jn-oY+uun|3)1@ZpJH#*jRu#L#DMH3=Z-8WqVbA-k&1)eFF|=H0|GQ?z4))lESy|XoPN=yj2U)OutbQ6s`-Aoq4Ihe@q{*LNH7(b)NFa@ z11UEeW8y1ADh&t`snqn*Xw%(@OrKl44lR~erK}V=#d!6%K$P#8bBbvmcYIea9q8Of z7e(l{AiNal2FYXnZ0w|$#;R5GPS|Dt7YH>HMFqpUOE=(_V0Sz!DBVk|E9@A;`BBmC zo7>WtF5{->vE!M67iG@?Q6R8)w+jkc%U8o9#NM0xLKQ=2EjFE^rWFV+J$!KscPG_* z1CNcYVEPGo8$rUXPd*2`qbV6|xiD9FQZ6&Qg{1)UC z_LdUuGU>6MKa(l?s5}tza@|usx%l&DydkI}v~Ofux3)W9$iY!W6GO16SPlFUWCX>m zbyy2FzlHjKZ5r}KrmsCLky!%-58s|-rOT%H6B}7F?w(Ms_dVxC?0!Yn(^hLWTB)?c z)ILQ{)rj+O*bH&pcNRIvugRX}8(mt8ds_`}f7bbTBX2)#`nQd|OKrr>Wb*shhaG%r z)J#RQST3VZCKYbC6Jt@eI%jL4qh<)fC-9FJQ6!l0TK6}1@15~ndjKs1|KoM-h@@56 zaG@nCGew_bA$EHto%q6~kP^bwqS?7aAr2r%?s1Ilu^I=nJF9TE>S>kEW?x9f{pEtk z8$Gvm0@0vk3>&sb%1Ph*>XjrxZ$;_0t+#h$x%r5cwA9-TfR4^F!6fFCz+1VFW7FQb zXH56}e!gtX1BAfVY<&+o(~^1)?qA|R?I6`*EsRiK44F&&m>HDwSyn&Ij=Gb3uB<|J3(nXVuNSZAghTX31+vk`*DZ(z>ETTg7Pz}3 z5J%WPjd|H*zMJ{S>@wwfjP zrP9Wr<3uDzVlv?2Semr-z`sB{2cNUDb;s6ZrsOP>N+3eGS>NNFLu8=3wM~^KtfSf| zY=b?c-ngBCB^r1;T_Z^59|ymh%&>R*3!78Tn(Urt8D*yA7k8KhI&1k9P>n0p{TRI0 z6z9Pn_szmteR+>$cuQzp^kqi6=Gc`ap-rs|LxRy0nPK_#`(6dN*MimjJF0VN>L9Zh z&iQS%D~|U6t-^;Xx8Hu_`L{-Bb@TP=*EDmX6NPUJyAM~ z2KUGQEqbc;ZDH9kk8jo@95eBe94@nj0cO7hK4q{Ajh_Pj>~C2rOc)7%rxKK&_ult& z=9E$%(@At;VN|OI=Oa`GsX}?`FNc-u3(u}y8%iiOA42gaXI(k<%OmVo1$Qsw2^5?> z^HVfP-YFDA-1=!W^)gFv4|5+{P$Tykzv$w90B@{Xt5vFchV`bftvl;8m zeRvkK+Mm}dd+b)pQq4}TO&~U;RjRU)ubJzmg%{3-G5KoU7W!8a{Ji6Z&cAcNpckXm z{Xjf2z7!2FAZjHCVmR4X5XOL>>Gn)TUZC=f&%+%-{`O*%Pn1uqKiaz{Z?6FPPXOso zr01F$sPw%*KcCgf2h6!Wo*OOv>oHCqEzy{>?^(RgcXqZ9d1@a(aLqDP^h1@->+@X# zgUp6(KrYck7atUAoo*3Mx12v@5EO)hzN;TT< z!%8?_>xqUxIIY~bio(qfznxsQ3!xt@TUF!^BeZlph|xJbnf9_he7G5S6z>8PFYV0U zwy{W-i#=^;-+z$S?YZWb|1ab8g=nGbK(>qEhywKF`rk62yblrjp1|Io2w| zi4QnY$f5$rdj4Pz8%~nIq+S3A_BtpK#083JH$f0)Dvv_oHWxS1B+-GiM@{QN0$E4Y zS$tl}Hk0~;NiR3ZMz2o#D&ukbpSDStQ#8YjjXJmts4S&N?^P~V>a9d^5Zk#W;ZQ-*fROG^1?Q|LGOUqOQcUwuur zOmo}+5MTroX-qm!O>vVXLNac=TZc0_sxNn|`stADv*LdEZ_`qXyKeN<>Dys;w~ z9Z%Ns63H#9%*ml#Sz1hN^S<&`QAI^7F_+C*OK0;2%y~+<;=DtRCcf2tAJA)b3R&oP zo55nj%2F3~OG_&Xl}21g8Jp>-S#@BYZ|=1w0yxUf!?HOQFkAPYYu-%3l|sQ?pi1*H zuS(B}`y7}SjW|q0P2-KHspzZ=Eou6&+CNG{mD~D*f2p>88vQVrWJsr)&k?81BnTi1 ztj3XEmBTOl;yEUFt2+m$&jsNL`Rot_R8`}v(Ia`%nHDSt4i1i<`0d74Cx1Tf=?df# z(4xi*B{elenunJmmHraFgM-R*j*1iPFJ61_TTMlVhESMcy!7V@3xl6K?ax#QJ8xH- zT^QLi)eJp*Z~*pQ-Wk*VvtAp-9xR%f$ClzDbDJcC16H6o_iW`!e#`S)C4I5er=)P? z(dkXc=$w#vdIeW9l(EUKt`K&u7hBqk*ArC_cX4J*tB}ufY)*H_CH_8mvZT`mExMuns_=RG#_I9FgaR`1YheGzkAyfC@e0x6!`bq~-2Q-&j|2QmWC{ZI_Rl%C-|+ zn_#G>1TV62;@H5UuPiMt9cj~ZteqYuX%tIQ!QbY=S*{e(D4T>cu@ur&{K^yFWcp$} z3*=>PrS5{Z#j#$MUVZU4+V0|b2nt2(sn0U-oFr$+CHb{{B-c)Oj_WK=TC&kfSveTB zUeIuy!cTEe;FA`nHQO<1`gvV5*u?EFAtfc=GeX*Q`)nVIVEj=Y!qZKSXWYf+L&{D=fYPS1z-#{TNcNgEMl4N@lwYnYb~UkC z!OlSbx_)0<%+K;`9d^z#ZOnnpwT7M7!t6gIITgPFJoI13JyBD*K zX4B3}3DBmM*k}!L@B552-I^|cPw#fJ)-%>9FEZ*twe$N@GV#yvB6t0j`|=Oz=T008 z_dlwt_T?8ZxrhFlth2uxAP~PVE`c*V&ej3`M-W`hPsK!Q_2WbJHfB9bk!K3d3Ku8y z22T_&xEFOkJk9VWx(IkX^*<!7UKsC)PZ1f-Gfkd`j#R*>%Q7Le{nDM7mX7U`Dm z?(Qz>lJ5Q;#OHZ?=KK7I8HeGV>s+z-UVE)|;X_ejpna&6ZAH4lsV1JrZdsx@R4 z9j3yFS?Oql?Qfk}Ahy3hek-5uGgYLr_&}lMr$< z*Qc8dWEMe`hVL8^BuFnZb>|N0vRV;+5wAY|5>hbLa=P*75AyIq*$pY<*R`v{c*7=G zKV#DNQiH*=U77sA*Z)U4VqY6Q8DTbcDhouSLo+TmQXU2r^Dcp6M2QuP-`%KCx&9l=khNpPH;Vprq#WptFT0i-nA)_VgYLn&3=SO&<|}E|I=b&mNPU3NzM^cj=!QnKk4DM2GbLRX&BMXX^>?g z5_$(r!QzSM_#?h!^K1hKsA1aO?1?rJH2X&e=UKP$O+#(;=>yD6S?Rcnn^*ft&w&1} zBM^%A4vQxUTb$N9p6V`lMP3K&+h0ZA`0ahSmgRR3VJxyT;W& zTgR28x#?s3b$FI}9(F-|4x69#LCa!@QYD;ggAVByzSqxw9Xf=RE<#A+?DK^J%=GM? zsjx!lpa^8S+G6tS93l{|{qNHAb}a}OBVl3~SllZ>>o(A-7g1XIvD-3Ly|zl! zxz_93jwW~Bw^y!fC$#~`gFw51u6Hnuz<>2v1OB7(%aAf1EX?(ueERq8bjDuc;)!x0 zQ@GXN_*nzr=HSiIEam7pn*Fdot{Y)*=RYkO!M^@+8}!WSBKIB8_7S&TB8%{qs&_=k(A<*2yfZ&#`7 zN8r?e(BwY`+OWdB_S+lKr`RWHuOzpRqA7kz&7zJO2zZ_D&vqtH9bRR;LJ$q6>bEPK z6o=vLn9CVr^ns;86%z^xeYKE|`t6(|Ko>1Are^UICIcL>ItOJ^)O&{cyB&#EkC_7r z{Vff?2GSdkSP@wHF#h&%)4|*g5}-|iQp0F}KUby12Y>Pt>#gV!if5((wEl{GKxO1z zRmomgmv8AK;aZ7cSGEhHFjs}{JQmLdlK@Dpg&0dEh#r|8kT0{f&!7wUq{w}G*ztv6 zfoZ^@j;<9&MD693d8n|LH4$)%W0`N|WDc9s#rHlLm4!C@2>Y}D5T|)HD)d{MTn$?1 z-cZ-{6cSk79$t)jt1zR+d(;J@=ealHxk?C7Tc1S~_qE6@)aZ6XFiu90Y-S^5NHv+m zrIjZvsmd%R;^1Z6+$gxL@2lTa1}w%T>W62xDp(-mSBp7}(^!o!NXrisk>B#rZh6y~ ze+aGm^eF$V{crwHXs7i4f;g#;XEW|gf}(~&O@kr62lRKv^z&6%>UBHa*ZXFskNZD= z^8tCyj&oMg{15MNp#Hk|>WH?#x_n1~$H1+jW|;GQ$U8Sj7-QS;u9_@Nq%7v2~94bFKV-0|A};t^0=h>=iOKH&%K|{F1v^ zn%gn7>t<~HMqa#5470vBP=y=)E@156T9%nz^17)@7Zif30>KHQdRpO=2NF4lw6VYs z;M~26IlJzad_pW_qeT9YO{C%e>nX4iwLaRD zmT>{x4}RL|W*db48|HncZp!>QRz4+R6ZJkudA-pPol02K0BqUU`Cf#(g+c$ad=gy| z+$S6w(0ECnuq+D`y;=&qgNYm&D!4t161CJU_y%We9;}b52Y0ek)5Z>Zi4x|em3?=Y zvB_$RWD#gNIJ$WhEfZlkSs;FmH@N|D`g$M$7UpN=f*vpBH{#3l*3#djJ4Pvq1p18K zrI(k+eonUHC5el`Fmt4b*JO;yhACUi>E0}RY1%PZ7_O&%(F_%OGo^Y10x6ct&|jM! z3;P@GR(&eo&s%KB`>k79cGp}6+czwUWp)p=os8BKs5bTzS)-1#Jo-!|e;@(yugDy+ zdq~UCkOj(S5vewKan zW$Js)b*3n06Lpgp?09n#q0v&jx?0Z0hiPM(+Vd}iB?viYBv+E8Bgs%v!y)C;k?rNg zbxlc&t_JSZ`R^GpM1Q{8&=9ap8{e|@`1DX&i1f-q6znk2z7@1A6U(BU-*Z*cEU}q< z&5;}AHb_r(PsmbD_8n&se_H=O)6=Oj6ly=i8V~|f_#V_s?&&{NHn||t^$(7o58#d6 z?G3!@ea?f^0gd-eEj=h=fo`Q33wBp(fhtmN?!z*4{vo_RkOf7Eakw(_OIJ*W+pc|dRF zBbsW&ME|XJ))_lF;9Ng8L*V}??h`MKpYf9$`uYjG9=n_px_J%GL~Xg*PxqKH&rOgU z+Tbfz6%61_Y47dmW8?I(2ik?PvQ5_#4~n%v8cwfsf(%0;*;|m0*~$Hp-o7J^1mR8j zy?s&?GBiKbWu}_NV-}0Y3u#ZdTQ)b@0pxFtFttQ*ydVoq%TL`RFTPsW9zCQLqll1J z)^*l+>|H3AJZBq(+WQCh$~#uZI>El_{Jb#o2|nvdc*n-lCU0iT{6BmPK*gf*cEda& zchLX6AWp&XT*uZ%Etf_~8(PS975$tTv=~l&8%|PCx8xQ%GJYngdy3j2E_8gbJ>Uc2 zlpfVY{Tm}*e?(d*SEQS&BFi1jKgJePTlG0YS|14@wW-YbH`x!yHoAQ%zu0#GJB`9< z!onQQr2L#@Qc7O4QLyFrb*E$OR@3y0Dr2cht5R=_t5FfrfpEzJ(f5^B>Ki8EcPX_y zZum3L{S;zo+{EL$S&u$T*cd&RNmg^Oh(7!rCa{b2wTFzVK(>9ZOJ3r z$a;Z?_Ubs&ttL!D<%zV!a)QcXnpnj#q|{fm;8@X zD(f44@ypWjOVx-mkF@sR<5+)xhQ)w{z`8bDf!t@bV)IK+UxooB4L$J<5jpQ~MKB}M zb)Y2-zC$vt70C+7e}}D(gDy}Jm7z~3R9&)Ze1cp0y?>R8Wlgt0JgJ>1m|o~X`V0Un zu8gX{hld)yrUfp8nll0$`!UrMW|>QVUWwhxk3_AsEB2=%^E5t&J_zS{cPzGZ)p*$- zn)Djsh}o`*I?WMXyJ1hq%w`sIhOUc*2$aeh$4 zu#n4IhC76IEjmrIc!PJ}NUG_6N33?O$=$eRgp|d8=o55un&-Z9s%BT{@BoCd{DmKl z;(9iZRs2Ks!fR`!AU=l*tpcpsKc@rZ6>@RFByp!_-42Yi&C9en_oiRmT9Ct(wfRq#{W`Tjn(4r<0K z*%k?uZ8OcgXL1dv z%|NO75;_u_+AsbFgxLa%7iTJhApu7Mqee-nvvUc_qGftr-o`IO7K5-2N_dhXL%&Df zcd1aycZ^WVl^1baaL8Eyv_LZMeJ8)mVL=&-!u{ci@;X~$0KMa~OA~}HTFu{>rCu-c znwpeKGtJ|{mFv5~d$ZZ%)C#_nmxCxDjs^-!J`}X`Y;OU?0g6Ck=M=jBzNox%uL#j1 zljh9|a}Bdcin)B*So9DHNe7W@KBsPI&Sa!$>)2QlPXufgPM8je9^oDm0`20RMalUgHPt*#F^E?rfrnc|qi%qKYPzMq7SK_h zcoD&M`vYO4t)N({g($@oq=y5ghdc*^Xe1|*Jo5PH_b_(YF3YJC2bB3eIV#azcsF@g zrzR}xpKl$DA$)3x1Yx#wO)>)9nifk5vn1~8o=9O6kpWd6r}t7wNbhU%kh_VgjPsRa z&5`yjYRL2u5l%D0sM1s89qL8 zj}$6+r}UQeumyzn|28B{{m?fSvu&B($nd`o8`hgbvQIWW6pepkE$Y z69e0o{U+|qmL(_|FU+foK1o1F&}K6qh{pR-V750kwvu4zpdIG13a4IY5_%D72nB

8`l8w-bsr@?0Q19Y+NwAg~Z0s2V8t8`x^;Bx2Nb(CDe%Uf zwG)_&tpV>JG;RJ(BP3#OiduDsx6`j|&4AlIR7@!K;*sXt+m0X!12KVJ;bXjaf)Na1 z)vDYkOn;nsKXAf|?h0X4aZp2qG+XXiUgxV+tf(n-=+V^nrlu2wc~>rlyTh}MrJfp3 zmCDc{x65w%29Tyj{yyK^@aI?pNLos990E6TtzVvWKM1~2yIrWY48|5g~WmmB&%OyGsk+f5Wx)1r2d1R8)#gila!_CKazg4rM z8ctf>~bEHNO@k=6`&&Xxp9W{pCmVCpep@Ygy z*LP1?6m}b zmt)XVoE$+Bq%JmW;7+qEBk=CpsaxKO`j!QV6yV)QkwNrB5T5jX8W|Btdr;1ap(|dt z8aXlwMNXE9$K$e5jaI1x>J4h?Y(B)_8QVDDFCRPp;T}n30oe5AlLYq(fOMCA7RwOd z{ZH>AMK_28i3`OyjQOq!rlok)<@FuxJ29?H0P7&lOE&mWsCjQ0@JRW0W5CDU?C5Kz z0D+<$xR@c)2XO>}G@Jp5Ms&L?&@E(X?sYFxNZP}xI-D-h)a&IEiNsNG$ z0~wFp_pnLx2AP2RRyCL8Qh-BX zo5*e&CY8jkjQn!ii$(v-y8oSDJw&dV^LnJ+-cNnIJ#A-m9Lfcury5-Ub3;)w<_$^6 zxN@WW8{nn_sGmZRO0H*#Ia$*_$r}p@j7qT4Ug@EG)t@3QtZIbPL!oI^$dBNBpucw& z95xo%h)mWwqG7-K6(l08X@z3Bg_G`z1J-XRo~W~7bG?a;k4BaD!31Ft%cPKqrMP^l zx{Z<9GVdXs3!&|Oh<(!+h~6}EBi_OYNQq*v`{Vaq#$mcfT98jb(Idsiv+CksUu&rj zZF@8gDW2b)X9Ai+nnHd3p9~Er@Y+0BWiPkT<`6%XF@$u6RxiD&#Ctp8x{6m|@mrEn zqFK`_QgMi1vpZil!5NfDOzFt4tifKq63DA4Lm#O;CNbdo=<>YEm5!SRigexs4p_VqY@k!Tjv z!Q_IU7{4%v_y%IX(C8PQ((5dbU{*{fD3b zWL0>P_T*>Sy#FHoPf6+g>?`vZkGK`R82@;JZk9RVn3vPjuA5UNKEyX~flAE#%5q@wS4Ebkd`$N0*>9~oO8_ui4P*La7PJ9m;gKBtH*zA^M=ZpF;i~)( zg4x5#grC`xUw@z^WT7d)8+gYLWi+))T}A{Od=4+idpDrwNR$XTXh0&6>s(EnEvUII zx?~jg^hMqG$3?#ltcoAP)~4G^Syr!e*neZ1=G_$Hb-<-wZCOzF6;%ctVdIS>4e1Nyd%luA3cCXe zY+578is*9J=qmJm4jG#MkJ zb>)RB+6~OtrSQDNT)EJ5tNb zkr$RA;a28=2ob*#(#$b;uy^(ruBu+jv94F1CFo6Um9w?kLS5`ZPI@AiKH{J}j9vh* zARX4wjvqkOj~?dIOwaU=;gYjzJeA?IhRe;iD$f@GN(E(MW7i7z@fV@5C za(>F?rho`Rs@)R7997^jroTd{c=em98j|XW3#}#Z$0i{ z55ONlQIyEtF2ra{jPVhU)K>)NsD%Sio(rIG3PL0nHh9n%h%VOIodA^bsk)xgcZBi= z)V>d<-4Rj+WGL|n#Z4^m_%GJc=Wc&nl=Lwk-?uk9ZpIT8ruj# zn1J$Y0EZMdhvDgE&z$oWb2~uP^s$kBcUo1?cWr(0=ZAFqT_&{McFXm!+k;7`VX(p3)_0W2+-h@|@&?Nk2Qy zpaG1DGa|*M3zr)V`gvY6RIbV}C%>OFtcvfLHC!WMSSl&E+g0-m@j4 zL!Ex)^==|iEB$ds$RxP6889ORbBS%A*Qy-&;K6Db5I0Ki6p=nq&3`Ai_!(sE1Ez*@ z#_$&vb|2G2h5H2~mZAs<*KY3y`Wb0QPBtpcdbJd=^$_hpLm^F&FW!OuAJ7IJi@6@a!)BL@>I48C+Pg;JP#olzazF$ov8IHn9 zySZtMr4ULuVwPZ)Zv0_812d%bu*>IXwoOgqMZfHgc7aw&7$Oe$yIYCvoHZFfcDvJ{ zW@04Gsu*89j+QV{9p5$UQdQ=@Y9K&CC1}j+x=7$P^dPcX){cN z>6=mh;X_yf=+4h+RyXPgbps%oXNmqoTZ$jjgrf^)T)erAEx5gu=cHtPOv`5bDx%O6 zfXcFWV(jWw+!SLyv#aNNa=o=tdAS@d63)*JX=GI=4iUe1d;X`|rx$sW=iPlDV%BD8 zRB8nwc#aoZYWhQ~Y-~zo$YG6ipbzuYj++cdrC%;HiX1emo-(L4X_0=#7f$ry6lC5x z`OHq;d}=IzjVGK(B1^~#h+}3Q7crffdLXJ(@82lLxQm9yFlNg{vcjb+RtC2rfL>2u z>=Z?G_L_o=7#L7^Z@vmMJ?RyzV7JpX8tZnuzLdkRc=(9I03sIDo&@YJ@tpp;H-ewb zyK%9dZ3Qtkw3`2E2;p~Fl3@r;+Wh{0(`65O zKtv?0|HDw0haEET<0TSD?kUm+I#Hfixyeo9k#Goc7{4FikHMOCfW<#;_j|&Ac(OJ< z)bHI~_=t8mFI{xgy^6NSwqkg)^%$4wLK5*D!>;0$lmI)%^bsujS2Mu4jqSw*a1DUt zZ#;qp1-wctig1C;!lybM_oU@MsW9ZJj(0;M_-+ii+$a?vj7#!^S=7734n^OlH~0X= ze;IxeVh=%Wzg*`VM!=s9=@-J@(apoXQCqd*#?T05lWHL>RGs--Gja|j!a_&KeW%ci z(4Ph#C=5-zZ)nh)uq&L+tg3`rCYl4>pPRJ`|K9+d=N8!C)~GqLA?@6;3TXeG5XUt-H4i9b0a(ZY68Kxd7k8Ih-4E14ik&!=o7iqU(UNlPsHm_Y` zpE8sTJMMhX*lbDziCw8l)Da1L;^1>g%e zIxGzWviuX2jFYgNWE8M)&&yC*6UFNa97a$nj`&twr!e2` z0xEN8JkWr4_}Kn4FhDZZMA}&V<7xf~OdrMaQz;3bF3_Fglpo?{n^J{40+G8p_uz>+ z*Rm79B)ur|h+l11bPt-4i^#DihUW62@?ijUpJSo&EqoLh8AD1Kf9eTX2C9*_7t zAtk6@z>Q;4C}4#Dsji?(@u~|WH599@h_Bsn>w+gSCSA!zJ|r>DR{^rSC`bwAE9JE_)rd@YzY9yQUnVtOoOU!HM#qz2VHHbs<{FWJ3{kMB%6mImcMd1_L zCGYM1SrAR0#L~ZBM7#ixTb7^1Z*27ZC}fAL5Y|X(R7pp`M?b+sG@LHgj!C9)B_*~T zu`pzU`);%njLG7CZHGh*UE|vyqTP@8Iwh`^`|U8`JS20|X*$PlJsC_M6H90+=^4ZB zeyu=gXn}5-zW>P*X_nXaA*U0eQYx7^S+?KMF;NlTH9_-)SuCXayuFVE{8^xip@zS) z>38}0G$>s+o-Pi!>w;n>=L5EmeJp|FZf~z!2IAZ&dn04JwCi%NO8&X2P3WG0I+LNq zA`6I*us8vSOfqb7!<+F{zcikpn#Qm)ORWJec)@23&FsHoXnztxB_k9-c*roltewZ> zv)>i;zs>3yYGZH^3PAJ$y0b_>f4weu>ZkO~lWCI%g6ZSyQA(Y*aZ%~xZ(c2EzaTvI ziG5!*R7^)m`5?7GwqQ1FqfF1fg&m=AYSddEvJJ7Ar2=R`qlPgM4fXPWTQbhFYxGqZRJ!E?zrmvZB~?UKRRP^kNI&rQmEb;1Az`!e2%K__js z!?)NwuGvJ%YYxOUX{T=w0IQ0Rx|WS=Ru_5SwM&jsH_B>b^m7`|9?TC&&cL;$7>IT_FU7qF>O@j zco^*w0Twrks{s&iD1Qhr^wa7CE??;F8|E$ET0Mow0+AJdThJPy_uW51g2sakrr&>U z6kX-4xbD*Kv{T31J6ENyEqNuS8|UxYFRN`)^~>*-CFaXH46cpe#U91#LbafGf-n6t z<_Nef6ZSsLz3(8Sv|9bdV$#;r{Zn`*ez)z2R^N1h>ajeAf_M!#RAkrMnFb<-P_Nt=UIpZw$$EE2=LSt*exrgH@|!| zJ(Rhl=-<2f^@zeRPyY3#!4Hy;Oz-4; z1L+4i;+W{g*qXROK*PY1%nzK=@!c-XpPe~fYAQExd3bN#4e?bU>=!pJ@@Z=;A6|NT z-Z}&7r#i{Up}bvp{(HXT1OlmGu>R@Y@#(ko;ndbxlYa6uB znJ*J!QdnuyjF)CVqyxuV*K**mDSg7c1+U1mvtSUwmb*n zsPB+@I_$f*zS)TRA>XOnmFBRX;&c34gyOz5lk8o#?fet*pZM4 zvu4@%t%%Jryyp5)ssa1X&WDOf)3b@7FNjIB89Bq@5!rxFDC^3jR2rG9V|wDwV{Yrv zfv8uYoU$PjienH**W0xir=B$T%7aU5tU!dK)bd4!f#)Yhe&>U17FAtHn!0F@jV}B# z3UCdb=h2C5ruWq*huvkA9nFs`T!-dqmxn8@jnnlA@l(&u7P0YK>Oi?9SJh49L`He8 zqcJVyLJ!?iZC>4lu#JOZE`h4++o8|7FDD%y4*ES`71!NHLhUtmUEy}9HHE$8jOD@y zXT0#~n^uM7;zFIcBd|xMhGdbZ0vN5Pq3vRGj)r!iXN$0Df%jgy9n`(5Wbt>`#Nzu#S=>Jt+h`OIec z4t#`U+2SQMayhH$uDw7ZG}3coYYut?`(+3pjN<@Ufq_6Wnp@6D8)s5FlXn`c4m>^Y zVtv|<#2>99Hk1l@#W={vJrB_MzxD}5W^uDA3L{c9C_pllGQ4sA5;QDN-(9uXB6)x{ z{z>&9+>wG>z76eofehSydM^52FVj!rwG!&GA~NT@mzlHtx^F(_AzOExF6esd9^X;K zj3SxGM2LREu9h6h@vx@oQTT#?+`#qmmACA1WATK3;N5jni!@!lp6o$ISp$|E^|EXs z<1qGRcO9Bh(D{aa*K}HQj>}TEqRb29*NBSTTkoo)wnd|OBH32rUZ%$Uln+^)Uldfh z8z5`JSa$z}4lcp>dPACX;;eW;E6}bM)*b7(amd3jHk*Cxtnon9m-4mr#gEm~PF^~| z`8cqD*_kdLET33H;iBX$6rs?H zAFG)jaN_S`(br91lPGfAAi^5;-BWyxtdC(63a}bUS9*a{2N8%?yd8iqk(=od8nj{c z^M{|Tgx;!ATC+ze*T~2VDg$7sXfJ8zWA-Ioe)zGk)=E%k<9E=IMx3z6l+?ZulEC?K z_q%WZgq2>PzpVHLPo2P8ndLQ6-%KN8)6~kX0asJlRh?lN!R|Dnq=8i@4&eubfzKi4 zT(AL2TOo%zcZiZlS@^+0rFV1^30Z}V%igYG2gCVysqx)0w|&DEJkE8kNoW%^-mupS zlBr-vFD+j0J5Tmyv1{BIJ@ISYxCp81YsYGu)0!7UeGQDQ@m7~92g9`k1uTJm`)DC~ zP_!1Og`YIj1o*A}v?RV}JEYQlPMWE@CD^cF)D=PYV7DhZD6wYTBY_$qCo~d8kKo$c zA!*OZ;Tx*--X?)6&$yOW;zD&;y6X7u3ltF`n>p@g3+f!xll;wTG}9lx1DpkZlwqpv z^Wc5%>87-zsiSG{{7KItG$O90nfr!z-Qik|Q8=nV?|`lqaJ!1d)oDonN;k^%(S>Hw z$`*d{`Mj7j{(s+wg#tsdT*}pfqe3v*+FLK>u#$s?U~z$`|Eh5GI+p_no2jrP{CT*= z)r)(Z&bP;WAsKIPZvvVh|H!@ zbZO5J-j2B!*bFXL4xd^+JUT_5zS#%-VNHPRKr`TaQcfKrRz}MoI8`^57Kub*aTAk4DNR3i9i%M6c11R-en^eJ@^G_ahRv!p3|cJ(y=&Hq&0zcQiw9`QHpn1 zrQp^z{PLP3Spc87#21k%*&zZ~T5vNZ#}_d??Yz5|C1>}7iMywzVvhi#&p?qGSaHCh zQ>unah;|j32lGwzfd@;ubq+g=u&4VCeu2|0vU16oA2llX5xHJSw)Dgd>%#GYRoA)ewU`{5qmg{?i5T7&i~kHe#Pu@$=T)jM1gtTxLYv-RR6e&`Og%yzd^~@)_9o z&02WBx%6OA=kCbQC~Lk*h`HR&X%Ssgpm~}0UFOz(X5Ajq`(b77w*_Y^_v98A+4AFT zp>$Iqt9-jRaXWj%J+N)BMU{C?#+#$1(@K_{YV)(qldtnP^UqIrnkGB*|TDD-M@$qmD`zn2|oJL#iQuRj&P=uAS|>hjg7ji&f_Z`EVsnsvLonLB_0V&x541I7Vw(>+9Qs zG5n?~*2e2Pi?O}tGS>S}>}uJwt10HtcHQp0u=Zj1vbCdAsa?)H&s2-GqiG)6wcNwU zBHe|^hpPhb_@z4(uz1PiiRtN$z`X!idP}=H1J7Ay?cyweQ9JqJyz3R#}n$qjGy3Fs9CQpj&9V+zBwLK0*Jf!jF4nQ1l~`D%LxoO>y|tB;JfV6n)Bme>#T1K%>wt6 zm>ikk_#Gv03S>R$8!vn_@&*k5Q&J%tb|2gC<)& z*kui!AfqLpkN#22W=*xrTfY;8fD-{*xBW-_!ACR3DuI~Y<@ZEk=l55kwWA-|(%RGr zRbL;W!Ny&ic!t<+2lZ2T+HUIQKeF033COs1q@McT=)GY2{Ac@{6dD9{`h9|Axem|T z9nMnHy~=j`e854PxZ9N#zm&j#m2RC$TiihJxt*~rMcjDT_4KlRpJB7Av{bcpCqs9j zR2L%nk9tccl?QVY6tQ)W&mDR(btQ@KIs~%)g$UQU>$G62RbUHf+L&nzUL_}f&LEd6 z8sbSUn^F2Fnq)lO3v2HmV${(}b~3gU;ITDRhAR&~+AkgPAY{eNDa5WxapkhXX5vW1 z-;W6i$h>h57r#CRp^N^hN4kn`U#C)O-A55X_;Hh$D}ldjh3N@PNKQ7R8GM_ph`>s^ z@?3a}Cu>${qP_@glD-cc-7Unoyh@x^aIO2}I=qs@e{cbcQvI|I{noBJ>2H*kdiu@> zha_)#Yzg-#`s=PLJM|)D&u~IT*HpT05?FG>DH_iW7l=8wPH}6tbtZ>RH{ujiM2pkN z9<(Nfh!do2Ujp7n9YDDRZCthiht1eOnw}9)yNYo9YTTiAx*;Hk(+qc!ca;KPMkLMo zS~o54!5Yt4Tf0$Bd$F;sLcv;f>BDB1m$8ILKDq?AH8tp zl}o0Fshoh^+&u(0e-j72`BqE(RY=et_t{BeO_tZfNwS=tUFR(|p^=P~;L}{chfF6& zmiR(z)!pc;bni}5qODF}GS3|9Zk8nwzbj>t`oMx$&23^_=lB>e+9;%XzuOe6Gf-J6 zVubryeCqd+$gRiMe-aU-$DvYrFM8Z~jv(eb&JOE7Nvc49^L=5)1A3IVc%fZIq35m< zn@$IB6hmM2d}FRo+R4(X7?rhMBA@EKD3!JsXnFDX=$A-zv?wn@PaD=xbmI~;Rd9&U z)BP389lI-4ERF19YtR1BEvjQ>qJ0ii3Xmdi_mJaT-tGRr66yYeV{kYv1Or4&PWp33 z+_!glKknj3Ycj=O7-GQ`)cT4n4w80V?K^XsWyC0GmT=N&>9XZrZf0HD$ZEW~OU0|jXb7gg>smxX{G0qR^ z?O|b?gFrI!1hTtfN{*Awdxhbw8$B#{7Nd@01cAhQ!aF3aNj?LdljU~NHd0?>&$HsD ztT^xyn+@VC3KTT+lpYyFu9k6*o7r}Q9c_ySzjk(j&mUJlqJ%L!ASCFe61sj|E)GY1 zeh1U3qY~`BWuCXq-A=({i2HAv?=a{PflPcIr%pLcO^Mmf<)>k$6;q63z+mIXnFB% zyqdcwCBbJvtrPJZ94@Kfb@pp=^u5=J<9kwVHH7ny^?iljgDU2z3i}+Wm!Fp0O%^@I}_xA`x(Iw}wlz~Kg2G8I0 z={ZKieg>QQ8d}K?#*lHy#9Rc5ZLAxZz$WzattrEb2zq0Ucbf7|WK}De($5tSdCEjr z%O!7!5Hh-8cD8>{8pwg~Pk`hJSa3F*j^0S}56wE)>@L+S=STOV8KW*`hAIz+HiaR( z4`1wEK4H~&6Z(k%O*Q=z{5HlU%q&kP%d1=|!s6FA?KpIjxX28?DBD|&%wNGU6W^^y z)Tk+U=ua)FiB3-c#)$m<6yC-cFJU(}#Quf=);roGtJq}`@vXxV!Z-@BS|6cFu#=}c6R@){^3^$1>FKLW3)B>e5e*Bj zSh?|w)_JCNmBr-{D0S_5sc)T3iNpVHaRJ{c#x~-dUR;5|o+S08 zHO!ja3{nr9HD)035&>s2e#<*^=WV20mw#eG!RdTpC~df^cKanw)_VHf% zC;8F^q@IZ_;kUFtz0Y_BdKO%F3$8rz%IdFbCyVkmopO_QLSQeVX9YDlJPuCJE&??M zFi{}N*(29*yEsc1zLtuSr(6>klprn=lpr%{o%EbFfItZte?|cHgmOKVSwP)kmj0(k z05L@epfRWBFSG6ZNzFkZCYnDd1q{d}>JQ`v{PMW)pI8LaqeP-G;8t!n0}A#(e??DW z`)NNTf}U#2KmP;D_WzR!fm)>SBsv`Oy~Z>DtS10E%l!B4p0_UuB!%#&B8BitWQ+N_ zqjaDB@7sz0`*!~h0qdV3m{GqP&%2QZ%B$xmM*sVE{|+o&n!ZG-yeR?Nm!S%2{LbHc|Kl(pk zlvRa!<1J;C0dTM7p8KGBpAh3iZq=UZnnE9GiBRQxp2SlqAX0?U5R7s2$Bz-rs+#W9SrEd$&Dx0;WPZ9=>NwTU`jG66Jea*c(dyl?qcVbB1 zn@647q0xd#!ha1II2z_P%h4Cb(7;Jm2q~s^W*dzBGmmy1wYtwybU}hC8?Gq^J-VOf z#-YNs7~_?{kat#Ug8gKd?c)fSbe2q=xpBZ`YrIaFv2wNhJ^MMU4JU7|wrdVRL_tjc0*R7y%o#f&ksf>YOFP1 zjGWjb&h5&Wyy41{Nzbnx7f_mP%_l0R7JS9UYjE?Kwm;239#To2uSqn zo>wH@zf=FbdQCzM`UZ1i7ZSQ0>HR?gzIPK1;0X|cmz<)K7!b5cro z{!u19*E^jjYyE^Ql2)1lT>2l|Hhau3@c+CcAU@BvFeqs}lf^w*zUR^(IA6H_NzG5*kA2U7*}g8WDm&@s>1UL$fBbRo z{JMWkK*!xyqS1f%zmXYz0j{cepzNzgEVAhR%jS2VvUQ7g2qmWv?+VU}hP}RP zq?Od;ZrpC}xti~``t3E0(|nVjEP40uDFgrKl)=qN&~?-~(g*ndLz}yA~;v)bCGpH=n80Cp_I`mMi zdtk+$tSxG2E%kUrB*52)VdL>~Z6E&^RRA__w8(SzUYM4!R}j|UkrkiR&zL1PhQza+ zXf@~R#qN!g%}r7sWhZEI)%uh;41KFcXCAK3+I1QzYB!r6LKq9>VJwUbe~aJwJ~8g3 z)ea^soN0sm&;dYE&PksG7;uXJJtx;N(CNyUvO>kEY9e@C)+2-qd6EtmE&?1*!<@4T zf_wK2LwNd)<3hTM!b@XfuT>eYF>l1iQs*Dp0*!W&SxS5Df|14cH(OrdOZ3J5&*MkY z6UNR^kWw|JA|!Y}EK8ebDbi~0I?Vh&{U+U#n>5=}Hcv5kNT>wKw3FCYcc78%X(K5) z>NhPYG^*dv+(dApcYjDQNOk7^4pIZfxOezl8O*zJPg5~J9j50S4KTJD#WyNVBMK5?v)VBQz6f>)@$Ps@$*UT`zb zVp#*$=!BveF!c zSxW|Ujr*26Rr_1Y{o!nv-Q-K_9oY(0ntAg=+5FU9yK0IIr4{MEAwGywkwam;4*%3R z!20QZjtYftT87dXxJ{IvmvyksP>)NgcG%7sC`8TJn zBg+7YEKoW1F+)JqH=q5rex@~^8);4X8E!2O z-o|I&H~|2$Q@4Dk-dytanfg;~{+`{otL5ul0S@HuA#^-FD-4tYZ8060W(Spuul@9- ze;5+bLpbLZb60DFOMv8k#yesJHfp0k`9EMujOMEXw&yB)dAqjRdd1!C`P38NYE3!w zGu8X9f+O9Uassx8!aJm2?bzT_&+ArBQqOCfKa!N6ODa_5%?;h}BOevEo!#J)oRUw^GbJUT?+N)|d;{U+G2j~ODJO^Pd>icluQd); z8mcKUYK)&Xk)-JX-s~qs*vve?%$Gwr^X~Aabi0b^{QSqLG}nJBPEQ}jCVtSG z?g!~-e5ncsX%EaW?~|?4T)^$U$culXNQ(fv@KYe)LmS}PwARtdW7}c{5Zq zG_6B$mrpJ?SM@E3>YH1Z-`RHit!wW;m01&YT9_N^cHcr{ceswVd0Ajv%Eag!)z&+UhaC}H=Lw9nX81V&^JK@`2g z?1-8L$4~!MGC4rA{b8|!B06DDi~Vm2<;8&tS+dVomG?g6<;3pd@D7_r+E44C{$yaN z`I*e0X1h`U$O2}Y*3Bt%S^f}ri|39yFOZ}DjFR4Oi46@6J{8J~EiD?UX=$cBPeg}n zjQ=@l#i5!zGQBpRDkJ%ruF!M98IrYY^U9JS5QX7@foUF^5#+A<`T&%LT&+MtV*T4} zI%5Qsy3NCKSCeIzz0>o1XJ7pTy2*f7lcR34Rb|b^=9~mCH&Z=g1I+?j^fI4>O4s$N z-8|vyd+D_AV6ZRd%e|fR^K%!)PuQZ3rE)&Mn6D?V>9i@B6QMW-cR*8!>i`CTEvB9@Z zZr;4fqbmA7xu|^fMghFlbl^!Qi=IOPM1{CwVq)Tis2ob1Sy&|bLRWIwbR-G_sxBeD z<%^^GJm%%B-T%5~wE}Y_)p{xLoKfkZ)t{ApjHL3@s6~fMN;zdtiU|LZe>(&g%Yt^Ts|C0 zC(G9=aKJ29dx9pc?8TF@oGMS$U3GT zGwQN1JBSoo#1N|`;YIBq&Qsp}Cofk|1@5<62Q)Y0QF;xau#0alf7&Z@QDl%w8q7#S zAYjR(gyInImG?qx@9D)XLahcaaDdji2xl+moKzm#K?M%=$Hn@Oi9u6BmcrrZRH)Z< zG(djucX|q+2%^6b(u#$zjo{Hp<`f+dX5y)pHItQ{tAlJ-&|tm=Bd_0soPrCoFEXkc zK0sB#`D?Fq%&v9N3mjC$m}@w8(17FzMV+8wZhSF&r8o!4>_N8U_FAP5?tC{ZWTw4pWSgd6c*&2ha>A}o(lM7-$yA*4n!|Q@-;J@rbHq}p-Jk99zX(D=2fO{lQ4ix6<;On_4_nt9 zzu^@Z(JFRe@0Dc&lu4kA)y|8#2#?{|YdE|fa;0lw0k{<=wX$uk6U5Y=A@4(6jl`-g zBF6XAQi)Zl;EwrZ4naR*usx+CgK;uXEhlCBK9mP!F$PqY876tlI?r zGgkF1H1m{N8A6CXC*%fd*#A6GQ(l<1LabHwb=mQn19T9{9H%*RNwk}Nf(JDfop1ot_}0Rcv5#tmRp*0r0+ol zMoMWdue7XqD&$=I_?I*cXae5*ci{{O3zv7qsPQ1mvvjxE4(qD#m5S)Y1$3jkU4({` z$0znZPiq}RKr<^l0KTTV)hK`mW0{|0$0VhZOZ6u^XGp-^%wqzxe(E(K#U{1$BAK=K zZ_ZRuGWq>_-ny8sR_(<^bb!_)TaCVMP|Uu#2u^vNe+R4+pSe&%FY>JxgAdSfU@$3U zb($K>8jtm)q?F6)1B4dOM`ngfulspe(}@p?C-k|kl6k7W zY*pCB4fXZHR1Agjbv{0(KL7T$fE2FW%y$!IAVPxIbq4*yui+T=y^U?$W>3ah1giu!6=EFq0$8EtD8- z>A=OA_SV-$s|r!u>!*6qIm~psXvn$gzB0GE6UibMD2G08*~q8=b2+mfJ{sIlk^MB9 zj$b@Un#P#l3UhNis7c*+3xljYWMTAndfTmw^1^yEQLpcNW1#BBlDOdKvoy0$ArJmk z1D#FuUHk*deTc_zG&j~WJQJlRuga49`($`IixU_pY;`}J@L;=t9rF{g*WRxgf!*Z> ztqHp^>@3!|o5MSD29}7!?2W-qwRHJ!sPQG`Xg7k0SD@V@& z;1x<}HQ!{bIsyBzOF)KP!pqsA=5Dv-GY*Ws0F$2lq&<*Dh1TFP=bow8X~B=;QOvx) zKWc;9r)Ot=Rvg#%<`n#Vd~b68CTyzP^l1x$G94=>mwRQ`41xZXWEQj9dI zNb+_2*Aqz?Xy4c1w5?trzAox{RT9!CfdybvH$MZazny=t{?yYk-QNJu#&0E16Lm@$ z-D4MVc4wSdsAyd*FK3}wohXSDgxy)OA3J$SFM%Q&J2*ATU$d*r{h`B?lEL6Vd)vQe zLG(XmL4f(ESNQ77g0{}_ol!?CdPmT6}Fo^#zm9(R|X+_^IExOr&V#S%0R#tnA^oF^X5THIe z?O^FSKkgh0n>?uTugO~&f9%$ZZnAlbpRDKo`?=jcv8@0s{P_Hj=H7Q$*rt2ouA`Dy z0jggFrEtL*dYyU;xQP#$W%{pm{K!-QLV2D**2@sk49!lUSoV(_&|E2# z_#zj#R(YemV@}ce)f2Co_L;wP1O2J`BhO6Rf5oGuPfI`OP}Wpn>_fbcc!sG+C{s?Y z@n)Ff3zY_Lv=7!XL<|_#XyqGe2Q*xsJ?GQpUs!Ji6Ff7+356D=B_=Lt&*LOk%1%5H z%%q574vo2MrWDWdPjA^GqBm9Bu4?vkK6D+j+?(V2-Fu617w&E<$m)N8Os5HhRMmFoB0ikewA71H+YsJKFY7D3f zBw;vtaJ9OseOR}+rWP1t)I9s6TZp>69%9$ykaXTjJM-zteie~CRgV-)|bx=0GcGUvoT zwU4+5jJgdbtFk-Tsjj32l-J}SG+}1Bjjb&|8x)9vts+rrK4zA3uKrba*k#9A0$Yhx zGd)d>nRe5v^r<>Yf-$xN?IfYA@-FQ2y`{`y`r>SWKcH0)2v#u3%k1Qa^56Af?o)+r z@AmFP0j_JK)B3#++;~c)-|5Vp;XHD0T*OY2mk8x)W_@m1T<0eeb-;xu%gw^uqBocX z%`S#AlYjag!AHHas2gUcc_T)5XfoWj=q^5#dM>XvO zT`Rj?`%=+Q@c@3I>w4`(*%X5bR@iq_M z;!A}4eNa+uvW*#GvmW@YNk%b6LhAjn-^VN~4B>BJVNQUd;enH<5P2pArwzLliniCO zxgTk)vQ|*v?|IJt#F?Jdsx#@BWWCnh8R&j|fo7|6Yrx+A)O=Tv5zQqopnaL%72A5Z zLA;=Bi^z=)>8o=7?2mo#V7ASlNSJSK+RoR7cttfROUjFFa7u@0ofo+$8|rxt<#V`L z;?ScUxdmE(o`WG;QfMhv@QT6~w7x_kDjDIth}uOZN2L$8Og9~+=RFwEXhD=2n->bv3S z6Ve}Kv?#ILPoF9!v~Ww3P0;05s#m-Z(nQ*QJft4QdUI%5e()NgCN!YBEzEzGPFpt6 z?km!@sa1w)H(r8}`w#FKMk*Q-tV>C{-cYaUTW4=~=y&5QR3jYaT4*FlX0fR;{OXBkyPyA2!O5 zeWTT9iSkd^H7!&orsf`$mEf>yiH@F8V+K^7RgOPtsxsGI$RVQdUq}@{Yf$^P%S!Qw zFX{;XK{(YA|2Qv+NH#S-6_N6CqS2E(mRB*n;_S6Yje3V5QXssJnw&Vl8&HTQ2tU`q(dZXTwaZMDlIALy)>`kpd54D*$|3z_fs*g846Ir0 z#oYKp`%~u8!@paEdO8-PRBlj(Ke_gM#WhE@pSIN9EwD?`V0PqJRmcwrBfLiJ&X?DG z$D14*{l1abHpAixYu)-yHDoC$fYpVV!j>%e%*&wTEGCT!N{ z_kot#|M`T?|8oqEHcoM1$o0F&07rM63*ysO_wC%>UO41`lkCJVP85*3+e}p0mJqDs zB<&lwJmg(`V?(Ev4{I}Tw>5Ig%xSjku9nzYq0!ZRL`^;*iHM|@rcbnW>z)xc$~)64 z%EE?dHj?#i`+M@=PIBoM=T!xi9_C=mjT4$d+1lA}JZ-1CEnU+3-SeS^W`%jFaE9`u zNU)8KrM1hsJq1|-#cnf#qa@XKTsD0P3occW6qf{LwGYS$5K$bN63T+#=&=;3=S?Pb zg_&r(tDevvS{PB&R^{M^(`oa1AzufxsyQa|x73Rc3_P6?mK_I`_T5(Y<&GMoY0Crj zI-wj@6le*Pl*m?-(Q(M_k6y}=)7H|(Q5^a*YOS zWb@3d>XQEa7t#!V0jJ8Xnn~W3ZcGh*qhM@{MsWJWfDhc6PXek-7d7&kYmr{_Et_G9 zTQ?Zj-StRr+gR4L2u06R^(pwJPTA-IEmz@GXML_^he7rW>bqlF?9hps#1LQrNU~Aa z8(uQ|FdoeKXec<*D(SMIkP~sIt^AA?LfJH}{ z_=QKSP4F$39DRr0hG)_N*Tm&aQ4c1h#k~1N1T*G;^C`pwqvO5AC!h0e;4#&zvvUj* zf!RuUZ&?%YTflrg{FKjXQ}Y?@-G%oji}@|Jhn{Vajy}@SH!AxETZ+N3g0vU;+w?LQ z@YMI!t&SeqAxYb-)zV^u1aIvr4Ow0ApYK6(!v`FNg=Mu*xWqZwd%xJ-TtDB-^yrli7>?8F4a z;rlw?&feK2SH{yPr-SltB}(ly;@TU>Tjn#&n2h92uajY{i~(Cf8jAJf7xI|Cs91q2 z6=r))AAXfGdljkZr|!M_-it60_^44L7LJ~sp+7t3nvDoK+OyG&C=no5;I3r zI_SjJS&cqH47!&b~t3A}K+#a*<4r2z1pp>NhtyQ?tqky{aUH5yQRj zG(F0R^mlE1FnLGx`hfn}b~Bg~^jhKWW@Sb$TSh?jO>&aaHihFxZZ`%L8f;>WF3lEg ztmX4P2E}^fjwX2bd}-o@-vyv7LEkz^Q_%?s;KYtPbJSO4Rv9qL2Rxbl8R8cg&p=!yiIM>LJUL|zvi@E)c z4JB^=q9m1@8movTyb3ds{;|agz>ZRV+NmE{Ws+gpN1d2}8fr|i;b+t?(y}P4> zK0K;@5aAF>xixd>IWTOXE{mNgAPP$u=y_KhKDM7bL-BcxKI5_aj3%;tHi_TD&)q{| z+O){YLsX^eRRX8D_~A!6Qj+>3K?XaiAI%C(+hBQ|&EnZgtF6nP7JW0dE`@{}3c~7; z2$Tzt)b5tF{8FZ3F>lvMOa@Y?cA*nd8h+=E!ORCl>{iBzkx|-05MRsO@84u#nME#v z1apd%BJ@wY>gPjQ^F}#3&{7ErET+7;xvq<3|9nytZ5;*- zlA6Lgq{8++G0r`$E#24^;}}w&C$>mRkqrKlm=AIws%Y`RK^*%b3=vZb|b?cUDw=VH}OhSAUc9Q%1?2c_bg?0 z>S}=5wA%EfB;pLn!NKs@Wmzx5TZX_M%1aZ}&pIC7#ic!|on~i(#;*&+MST4*@Swr# z)s9Bxav9`;__yM2v2iRhDJFY!6pZ>YNZ?7tN6ohUYQkRn%z!S&54W3|$KmVS;ekz= zd->39N7FkwF}HXPW*vehT|JI~OPq{r#qcK?`<^)@lPWVO5^3d7n2{gqBVflEyh8JSa>VUNd}iQ zE(}v&`Mg=4^6>*}eBt#TgpoVRP&v*b)E1>I1zvv=k!f>l-=-9CV4L)3G;3)GF04 zXutCo+eT4buRhc?R|S3(RAR)=a}~@FTNJ2N2_Aix*7<(9!OPwCh=ic|RdwKq*m()U7)um1v(PBxO6q*T;gc=4Fzs3=&5*hZ@d#D| z#*{u}oGaHIFl$&vbv_48q~O+w)VF;z`NhFKJ(5DlfSn7`QEwU!10(F~Uv~-*C9q5R zTQj6tXwNkcvZsZJWcGds%LSoy2zzQ3{_+uj0!`+V1WJf#>?Alixf_|E(oT~H1{Jzg z;Gb3MK6fmw)Qk^!EZ^u5A;)7nKvWqE|4MLJ{^0TjE_=9v(#qOMYS zE@#sy#7{yZN5z07eN$T~;48{&xu&o4lZ;TBRcUJSr*h{7&J-yv#bWuio76(odeVe1 ztf;UE02)gfMOB`r+z)eOw=T~oP9~_gk@>{GFM}^D1k}4kq!{+PLtwop=4Ed@4z958 z8_b^d@8rrl2FnM=-tI|X6WY;Tq>sXQ&v>5jSks%l$0ODH<|71vLXUK`76drWRB1^FfMoWyzJ?D$2D_ zIK>O}Z9pV*0l&e@Su(x3y3B3dYyVvGN{ND0g*LyL=uqL0CL|235ij6&9)ceJO_dQ`k<@$>Q%a6CRBUTP`!ISUSm9Q=7a)g zvJEVqf0Uby_I-Ds+F3>BS7++#&r+)=v~E(z5q(Q%j60l*mdygDY>){ZO0(wFFg{-* z5p#QnuV#y$e}S%sXm-LM-+HTt?`xQ8=cb!c850Z$M3fXTsX_&@=ReR#PlC?`Pv{xR zQ#>b3TslTZ(Mrs#M+&&!mEdaMqH=oxM|s7rWo)YZX=avP5v8;}C>kOjO#f0%wc+Z2 zvLGP<2|O5^&4jcPJV{$^*~3+z%EzRg+|GnANM2S?u`wuoP$XF7I#`KGSN-}fm|84^ zDb9T8@v-f3QHZSLUE@S(Z64UraAK~CCJ9V>%z#ilHRH8ezzuFZ*@LVMViuJBXi%qi zj<41zmR+G#lXn>U*kXmRI6n#MB6s)^uy^OyUw^cX^oonw%7%o&sR}W3A9f*ihIO$wZLM?9bU4#IFV^zSW#~ER4~$)n2dfhKSSkGFo4yn<~kR z#-zfKYe-IIE8W5$WSpnZeNG3~Rk+o5xOQk! zk+{t?s#)#ak#FsB#ip*`QpQAt(bDv%zy=<`qu$ZnVtrJYTfp}Fw|J!{<)!l}S)z(f zqj%a?V$9y{inMv}1iqKyaRgQRO4Do~n*V5Ot4+}(yXs%X@=Zt{b%*@#mYeBDPv+IW zHP?U0e%Bx6EM?%ljX8dl+tmH&?=m)qBE_6CPW;)SyWE`skTTz*_^2EaB33;nBCc0^ z!SoTLq>Auz9+*<$5>gFX`ug`@Zos>SlKJC!yDILMN9sm<7-yHmJw!W*XLFpDD9#+r z(uCizCxvD0r}q;sTzT{iK+m+bz>F0dp*(#!HAysPrvY$mbE#zOS?}$r=(#Y7q{-JI znjIJT{*9{1b*$Y)s#E22tS4z=V?nL2fRQ+t6P>c@`+*A6bY~OYv%y8cwyWnfaBzVh zH^-fGg1{8QJ!uHR>c~Klfm`oNbzZm0Qek%H$ZoT$4g$h14?6v;m~%^EqSalKSpVb@ z6Juiy1sVKo4uL?B8!w5WhS>ysA|tu?#qr7luQ>;7HvXbtC{A!|nXtJ)xpV!Xyvmhb zE;+wEu{&?Nzqb25{G-=bpV;8RW zJ;UFnPUelAF82AWThq-eBJ)p0MB)J_Wk1c|{X?MUCG%0C?{gn*{<9ARtcK9NW3vp_ zK)YY@ZlDa17~GZs+Vs(is~g(3Y0u-1?Um`84R0sKVov>g?b^_{7WLjk(bR5~w$ri0 zfX;Sn-vIc3NCV)1SN(66=eEF}pDuRI=0E=}NNzf7p#5T3Z2H*^r5G1}(Gda+{-tAF zzu4(tllYE0Spd&Lg+?EC0RE)x6sFsL;wb)#U6|3Szb>KObL`0BgY>k>SjMp&_4D>^ zy`lxcr!K93DzCfxS>IPdw-JrMekRwrQ2(A339wi$?bvT#?IuTx*5tbuYX-ApEV3`~ z;L&<%PO2nZaj;LtF1IOhlKdtAsl!bV9~~E-c{5`d{&Rz@FHGu zJp~$49+-YOAYi>9$q(TIfyj}4TmnkE!EBns9(0b0>dnH~$g%A$^@m8d zg37?<$2lZpv_8tY_MD~Y_X8~Zp~~Z6udk_mjx|V(A`VTy^@;mrut0{H&P@GX21jI zs+c9faTULix@3T}{^cf*pyD&iyWYfH<60;cL;E@HAf*H-ydvK^LzxM6eQebk}`h?DR>NLTC z)}%2wWj?*mnlX9v8VILHlDC(vwIV}&R+f3#Is8EpoYH&mMMPS*WgZL1zYYOGt*KP- z!i#0r_DW?o56dP1xPV=4WsJSXM-^qyt9SaUQX{CWP%a8~Z6V(xrG~jS(v52a3VBEk9b;+@77iJh(G^S?r5?uIow`b4RZp>9tlgaGQc^6F5R42|jX_^!iD!+WWTlbBD zv6|G}1-{|I>Yp(`Ifnz;-^s*6(y#8Wq0k~ z!R1d}E&CwL;xwjfG^nFtEdV59gn&;}l=a=(*s8s0)6?DGC?xkc+wjITpV!!C`=-L~ z+2EtJ#Q~Ef)s=a;iiwHM!EQyn+$^(|Q=8?YLM{5;@lfm;Vs&;pyDWHB!={a!V*%SX z@5z6dFACaQA*&kHr*1Pu*GQi!Xc$hBo$G4)v_t|gG5s3y;S$;sl8sZ1)#wN0$5^4L z*5aDetcj$w*1@JbnrE9vJ2iR)W$~;2OUyGr1o@Swa~@@N*!C@FlkGW?lZZ1dac!Tc zyd1;*=VBnkmwX3q+>==h8GE*r*5?+L2FA#va0T^`0$n4BP+u3v#AX_gnU_`2Jd!P1 z!>IQi5gNN2Pa3EVj@kCZqxGH#h;?y`Qy5GT zNwLV;qDRJL_^s5>EzuUM%kCRmcr{=4_%s8Jk*{Kujf^0x^0(Z4Lmew6)>KS6Sdv|4 zLyXLp<`E51Aktp>Y39|csR`&K72lA8Ew@6L?QYQsixA_r;iA^z)WHf#!H}t12&64^ zaLL4M8AGh!sw5dBU}UiuK3@cm%4t-GM757z(bH1ZafQ^iAH-E>T=|gYk zaLvOBq^3U%<;Gf9tdJ&Yj3-*)a{=Z2txvTqkO0_1KuoJfNNLZQ`qKeo9{2Xj;Ox+jLTni-nx^s38mSz{heB_H7cu2=n?cmsH<$oiTt0yjr^Ipll zB!&|#jv1>^4hizN`n*q1pmh(#F|23F!tnvGSL>o@EoW;Nacq%>>D-6*s9i}_MqWs( zIveX?DH2O5O+$Js{bB`&Z9Bl;*H2tJ?NC->>waQ!@{5MCZmzu9Y0+Zbc}>#7n+gRK zE2R3iUN8=AYCZ8vE3-9lY;I*1pujNP`z`HP&Ns>_{7SsE)-Ue_EU&roiIsDNqxD}S zfpZ|R+f)<%HwMETX#O;|(ytF*?)hB?4<5N)1MT~((Mauw4Q|2Z)_2r9{`Aa%vr*dfXN3S+6VjoXTd~!XBQmjs%-W{^<$?MW! zi@u@syn!6O4B(udWV6YJrl&mW#LBsMOL}u(o9;Rc;us7wYscyU5C@-!tH^~sURcy*1d`BlY}b?1zjLf_wUn*%tTHeYs|Lyqm) z7@`nQUIjelIBRG%7VB^Y869{fGZGZT|1i9fv%x`xt5IcyUa$A-J*@ZM z&+gpTJG5VDKMxPjA^n@zO?i0sfq8iL*#EK@IP+mIxrvA8A&>s`t9L`~=f<8j*tx(| zBeo;=Del{Ci8hselNILK;1u!dRM^wKQnnATEn7Bz6gWgb_ETYJXtgsiEua5+iA zo{9ydV zvWW17i|v!mH7j|^7nLnml76}{DOIvu8~$Q;wsZQ~LVZI+_svpaX!o+S< zp^>CM1Q!9)-(GobTu8!k{rY2WUsmJvD)Ac2Bc?n&-sk=llD={FZ*A-n`=jfLd1fqS zvYv@dL2e^08qaXvoADayi7=ysmc1xh6fTChsA6jbQ4lFQIwWmu*lvXktFtd@+uVvE zwR+<2W?Pk}K#m_DVb(W%{FH(nzL_*&(l1RFl6%WZB$5spY7IU5SiEaX2_Pe zOZehfqxwJMCLk90lXrv@WdBbZ+odrm4o!Ul5l^I9glK1#H%U~iH7;B1jJ$~G5q zn`8^q$%s8#lO5-x#$P10Z|j02+s69;xUK2q5@Zsf_<)iggk$B5JfdDEd-Vk&-vYNB zN5e7HF8YkU3>UY=XIs#VD&)>8n$=lmcdzW4`Z&&}@YBeVr^dqG*YvHWHC&D!AX9AB zYU}N<|FwPl_0!#X(L!8%qRGqnqBn+<@RBj}dPeNkT!+hyOpm49q9Q`XG(HT*5iLF6 zAOQ|3RDabt#HO%&SKcEwNh@C-i{pwPqK;d1ej1#1`hm%Z7p>>P#Yzjw>d6>t0Gba~ zGc_r;Z5vL;-|nHlsYrkaV=c3-D%T{zsN;>B#G-s=^LqE_#Gv1tb-7<;SfnsLq+sL1 zUuSvUIh2{1W|AK)W;}itqG@!?ht7{WZszh~eCd*QJx-{2n9NGCS-52kn>|DOecxc$ zN!)OqWtqWeTHMABbzUShE3bc*#BmJI`8uqRjQTN69=>kvr6t#3;w{{Hm+&vWy~RPF z&gd0F=$M{SdyW2qyAZ^{MdiKeYr5HO<@-cdwt@&_sAc^;=d$s)Xt+43dfTt&=O0~3 zWWXjS?G|o%Cv=^>E|j1VwpNp1fKLTwkyd@jhBdxAXJnm1s_`#={21gPzTDw=%fbNy zb+Z53aY?~^Y!1@)nKVt*E_d+={y)K^?8PGJ$8pbkuPVci&^ocP#xN<7%I&KggKt&^ zBTz3;{&tBbrC@x?L9Nigr5wxZk9VUdPa8Iia_hmAUYmhInE z@$5E}Z?(NYH#c^o7R&Nny_`}%Q6HHoB;Nul9tVNzc< z`QGV?`a5EoB;K`*A={E33A3V1$~4CcA>A#O^w(~p4*KOhfEjf6c(w7p+xR+gH+`ppYtZKXK;j@d9ee2|$H}ql9m8X36|ZeS`8E8xMnkYR zr%lQcoO>Q-Yx72GVMya7r(Q{gS9KAw5av%MP(yqqG`j@VDWu(hzCN#^S)`Xt#YNlccQ3Z(rt2Aem@>N ze*Kh%L3=>_RY{RaY)EjKhD|eSM}hCTUC?MY5M_R_C_r*1xSk(9sbO;9ipJ@bm0?nr zeH?h6+oZ;+fbtgu7egPp9^DMxPrL(y-@&($DbVJZ^=je#z%Rk);oY&0t8;-(X=c8) zoB3&b*B!JW@Xb_X+vg!tX1k2dN&!jCNoGvVP7*cYn6m-K$~I>-N3gj~a5py4!8`AX zx=+&OjaoSFK;3Nt)ROc{!#I;bdvu`b#WH9jI~vh#vt3Zbhw3y3>@!9=rgyp;IaHy> z|A4uaQx50U;37AFA(AQ5KwonPTl)pY!3Y>hoc5$0nRFrVPIs#(738$5js( zuO%d-95h4BOM5Vff&Rv(>Ld?_fNw6)GR_A`8aWO4FtrBo#Ay|vx1r>9Wkq;m#JAou znujr{G+a^5oF65xwzOSni@=A?(X&=(;0+T-9edDQ7vU=-cE88O!Il)oJ;HY$7Y$M`{d9Ht4jpXm+4a zraxk-l8fYew6wKzw1Mdra@IyFSj;q4KT7OWmRexn+WgW8qg+|gTXP){v5H>MHGFh<(Y^>@onaU!I*1znym} z*74G@8?>1jcgmp5K72JFRG|;@^}`d7h0%HqKGcnx85B~wPsSF%#_Z9M$E_5|8x)J^Tzd>!3Pff`)hwA!_#k2`Z#o3OyFm&ryGb&#ywpluj$QSme1s?AcJczs^& zTJ^a0!66izJ=OzP1DNC)jX8RaG9Qj%%e$bx&_))HG;nCctqeZ7DA6@a<-@- zV?P+(rXS-ovdkPBlu^#C7;mIZuaPeyPPkYhaKDWZYBs2VL;5a99R7GjC8zHUA?&^! z7%C1%LWg4dquHVvIr~on$kn=iYmcp1fne*z*GMt#ytgt_c9P3y{cQzlSVj|jddkaz zJuIRds1nl*C&WmoPG%vvR74JvQTp~^x8l-L?4V3LcM`;Ps<7lPB_JAG)G}Q&b1x6z z3E8qO-qdk&SSJ%Z-2~~T?#NCsS5;OV%&=LyV_q6>{PFI)_9;i@ZFLeS+gGZsCSz6r zkXVBMjS#6X68zf}^4wh3dZ%+iFjmfn(zIr_|Sdg~dAb+>%Z@LId0A7|h z&Yd6cQ=?`jHnDJw0`@{fMV)cfwUwvBzMt44L7%T~WVMSSBCtzs!c6SVHwm}x>H}P` zD}9FjZXh;!V-(eQ79{j(%I3a0E8ccMMyz)05C@xGsU zwXNeU=bt%TbGuCfkaPvw`Ke!mKQDj8ubHzvX?9!9MfZjE$D7>_+nXeu(+*uwIC@S} zU&y+SZCt%m{?{4yr7raZZFYVFVrvq2v@k()`KF)sSIZPq)JFvqOv;N~6u7MO#$xV` zm~eKAlQ@DEf=I>qjoUZJS|zhiOgXC&qyfxYjq4q4w}o#^;piHZu}ZG8<}I0&mBL1c zEn8B#T_y>~9%J;hw@Q3{@#4kGgh2D<=3vbVW4(xWk&u-hv%2urS*%9x~;XkoDlh zuN$ZIL?Cy$8?9`4#@7u30`BKI^-`jE((Nci9w1!J^SD{_icXSEe4HZ$;{~YlE3GA{ zb^89xx}|M*-zkVH3!oJE7hx=r=Hyz*@kAT5Bowh1eJFazyn#PP5fx3DYa@Qf&47sr zR1+W&G?`tE#;}#?3}vt7oBcyn4g-O(5BYQ^@>?Kq^;!7+*ky&z&qF89d#87*Vp?x@ zOZ+scY0}s>fPLKvjvb1fSUThvs>4V-E82dy)n9%{(Z&Dh)C0+YyhV;tQXqn=F|N;M zG$Sf(<9d;ZZ59Hl*Vi*$TGP|XZiR5aT?EnWpn~SmuTKQl4EiL}VkE$L!ew~MICrfM zIx<*hlWM>GGIeDED%`(Yz@M9nn{&avC}Tj!*~=>*&QNr2)kRRksPik~fX=uZbNAI0 zOM&zBx&g<47)@j|5J9epVT-L}CC+;Qb7z-p8TV2?x^cv}L3TS=K>T!GBXl_tnWdLM zXxN>GtxJ05pFlWr-klFtU@4m??ex)I&`OX6DvNpffYw(KX44ovDvmDYoysc84jztm zik9%nyXzg_6-!k}_N#A`wZW_mswP8LUA7M99+Z3Hey)>{0AI=5uw7+-#F^q(TKvNB zHp|#e3=BlKWJ>`MXa}I*tVNn*b$|TK}*k!S9(I0S_N1J<0qAERY%xA z0pv$8l?Pi;wxb*YpXlE6>s8jedTs86JB9n=2a5XR&N2lFp}KMh=ug~Y55_@Uq`sIg zbEX`)F4}srr{mZ$x`Nj7qLco&T!h1P=Es??*o<$2 z)t5dVXI4QOxR9KQI&Mbyv6`@)P|-b}d^m+EUr9yrc2THz_~QG0Zl{jB1W3LH?06Qb zm0o6BUu_}O&at6)MiDPB5G-Dh&jfnY&ndlPiP|xUP+5z5LX?2pu;$2Qrd=18&*x? zBwU_|qPPz!R@Z3Cx8Aqnl93!MMudL^AF9eBegE5A6*lsREy}EuDO=O3MMWSE4M&Te z#C_bD2$PhH0rCzNL9z4x2?K#WYDKDA56;2mT==}#7ebmxMo#pvrO;y(mCMC-8wQJ} z3rqi63wF??!prvi_i^zxoMq^c3?0W2@ znp-}??Igp&Y7T!zXlrb4W$dFJ#a3L|ViE{X+$4QcWvpJt0|rL(rAncq+|+zDzjS!} zT&Z`dKzVny;>V=cePR(07}(~;C-N(6BTX=<`r^Bv`X^drm@~rxIiQV1K*JCcAkq-G zC!|Tq!6sHyc9wJm|M;-FL~~>_m)_h%pICou>@&(jYfl9n&HEjX&Bhr6k2PFi2`J6U zdkT(dzBdZVzEO_8-rh0-VJ3%l?}y@L>sC5HS0Ffa+@#aE>7%AKbVatwtO#Vlr2bOe zln$>}lAf>j)?$kq%A%r&Ag-`{sc`?{f=bPm&pJ%6WJk1pXj@W)NlDw@hAMJ)>UM^S$*qCI{S73t>`6G%o@( zo;TfzYz|hQG_fk|FH5kD8x0wU)o)3&j>MTV0REhvH`AnB5jyj9oT7)ri&-7U@M?<( zfcDknIQX?A1z$E->27>^FW%O9onkjqwfx$=ZkU9vk=^RJk zKUjb>qX+7|VRwruEUpq`+G&U2tl}geIITG2PRrylv#8&4<#kI{>RVPsg8TJ~!A zaj+Q$PK|W|<;&qyr{?~4gaKMOIa7k`gTvrZ=LS=lO{qVjJK2V zjpb@W%K$rPLOyz`aV?rR&DFpDaXdUv&2ZOAmoId{*$e`Fu(clZ_-9AUOr!?!JydO0 zeV9$wwm|dx4LN5gn4~4I_WG9~z_Jw`kZ-IE8SPA#4moKj)SgO`#CuiXRK~?s_K#o2 zBsDEvF4w@c5=zQ90@ouE>yZ(*e1)mD0s4N(O+|Y^cy0B059H{x6B~e=l~%D`21Gou zq^Nd)Xm0z3_h&An+t96EZB0v>x+fFj;#SuH7Q07y{e#T9Co1gK2bJ+1Sn?XOyU-|e zISP-eXi}H8!(Q?ked{0jNNJ#A>V=+Y2g?KpgbYmWDRqPQ4G|m5K~u>Vh5XXsm3VoU zlxzZ?YqVOe^pRuZw0)Trr%FjHubV_`DS=aAxH%M~UuIV(kioPX zRUw$Dj(DjkC-|L6>|VW(@p4~q7)*5u<48p3-HnS_ZxJhJaD|JT!A$jQ9M~X!X8A3= z%bNsb_OGxf>Xg67@I0v0;MMBN;-)4m?tE#NG~G=aP-|B+`BQZKCPXEnqFEQV;t8{X z*_|4<1&t`IQoBh&&M$=qb4v0d7v*H9&nvsxj5khCPfjbV5_a-L4HI=g@&gZ{n&)k6 z?v_?~r6hG%SIn7RG925iuJ^?_R@qCb6mhc49{4m7$o5`t86y>q%bl8s{6=k*bxYbf zmgB4@*8C#&K8B?|;-#A4voO9z?p{;+?(y(7*MBDuIIpr{Q+2u3L^3BM&yt?e`qi6S z?&a%uBC6ZVU0dD}7HGo^V`AwQ3M2GZbSuX)@0Q)$2lwUf3dq}M#cQn%)dCiU!tF+y zhZ)z{nOTW-`me%VN1DswG63kIh3nl$-*NxSRhqGx9QxYoi!y;38uR|MW?`w9wqJRs z5fhzBzz1MV--dC?@o}6TRkbZ&Xx`@1L>QrZLV|+nCOL!2)NZ6r;8;&Q3go+Bc4-{% zT>;?k3Us1QZD$2|bPAHH0fgqDKaE;#{0ewLi{JVEx1dgmq0O5X->+%N-2v&n8Reom z^Bwa4od5lQ_f`qw^iA`Eoo;GOc7C{GS_1s7^xCm$JWuUw?wVKr(cAwfRsxzX1*QJz=AP_qQRg6OU!sc8Tr#Yp6|@U^ZAkQ z*JFont)LRk^=n=yj)1-OLuAI20-+b{WXCgnt+ZbRGNb3s7wDxnRm8r&>bR+&M${AP z@-tI>dXNO0R3H61x$!JthkC8Jd|Cixvtg=D0?c#r&YyE@=;&CY^_Gn7bv1PJZ|98k zombV$@_p-dX>6!t)lj3}D-gR-?xRMM8MB^R=)o3b0$W)lmXwBKW~dW&scogdL3PSD zkm|m<(9K(vnd(Q;uM%>y;YaZtRQYq!?HC$GTl05UVWzQ`I2W=dc8MbEL2OGJ4661t zOV(2jKAzA(AjMi)TVtvs-Y^9tLIVvFf^tB8cH0l!##ULeQ-lW*DQ*!x&^&Xu(w`sx zp7)(X@6}1?kuS7Q_BuC>dnPH=jWx?AM8xE}`XpV>EYVa+>D%n>37EWV4N~dVKlUP} zIyi83BchZ^wiQDX7C^yKN2dX~q_r1L6CkWWinRHzd+)xRH=dz+*USr-qKU$cu_Smn zVa>Mhdb+hdSWh=^Al*RR>G%eK@k9x8QY#34GTVTzZlz%z=Zxbn$Zo(#OrElh8BM0`gtJkut3!)!x^^64rp{4WPDyiy==c0)c=6dS* z7d=d3Hi3!`0oF}+oFWZoQXAQqo7{6m-|y#ntpD9+T=AN{Vc66@8(m@<$40fy47jF? zpM_pg0VBILl^@*d)dCysiRd27?C-{-5a$$_F)ri6KP|GqNBBHfy!cROe>w?FO~>HR z-u^xDuRQ*R!~fETc~|E^122_Z)9rM15EGq8Bt#fp^qSmg3$-mnVL}qdE^#hLqhG|C z7bYM}*2FTqrfet`fn6EC8Kg5m?I`_UG;996!DrJFU78?4+$2iZUueoxMjWMgp~Nvr z$Gj5wpD?+EwIr#@aT`0sN+uvB7H`2779w@Q%y`n-|LQAW+f?~jf}x4^;M8t(O;b`C zkg5AFQMS#b+ed@R*bIklo{u5n?umqzN|W2e>aHD6~p{0y+~XTB$q zcwQ+-6N63V%?&+-5}cepC3W-MWySB%>C2Ow+D=3y@u{LU8rlN zEZU1^k+n7=H-Eiq|GVcuyot|3t)9^0!nN!ylhXL>w0eK3vhl3GJn<1I9S?WGenAVP zT(pf()%k~aj(=Lg&}a{xoV}V@o-nB|wrRJ6Y9^5(IrgEtg)y2Q3V=m=n)Jf~7?c2y z=)->Yv`WQ)mble^ff-;|7h1;Xhl+|@I0+Kkb1!G#qU*K?o*9A04Y}23><{hUTq=1O z6F9hS%RYfx4M0mR@u8qs0Ds}2tKIYHX|fz_Wk5$JX(%)vuK@N(bgbe z_%As~x_HRsowiXG$}_&$hk&ypX$~M}!HSpw>|%ut3D@qaCYM@0X1Z!&D0U0Uoog;K zhEH#;#5l+j==i2)`?Z_ajpP0OS$p>7^G3{!+J2s%YDmK{D zf!Plx^s^^X%a@Fgo}$v6SD!9XHaZ7dy!46;$4w`uVarUKM=ae;FcpgJ^o{;IXtcz; z9U&F>5OeN)Rt(#=Wcw;T`P37&alyGaBwS_i$qZ0O+_UWtqUWH-`p; zIvutk>a#6S+(-^g zl^>Spw3C|1(l_(I#IkGRy@ibPGB5NwiH|J9EcGyVr#uBA*3E_XFRaU}Y(A${*J>uX zGzQ!HB{fX>H*qgFX-rja9R?hZdpCbL2kt;-F?;Jm>+F3*fVUi2+$86Qhm{dtm}OVC zzZ);tEH$s?H-5?O{lp3}rckf6O)byTuHn8RH4BJ|^DN_N4@rItw=Zg{XqEYC+H0A{ zxpw#e8nRZbiwU-O3q;E%T<;!fXMP%<6)K@ewby^@-fku@aXOEm`a1~bovE#}gF$mK z_AjSrU{u)RyQ;bSR1+Ny4EIJ{-_EH^2EK>9=uD!bhz9?k(Zlhk~h zYgv!Z(?*T$pW{i2tBY}U97U1fj|v`)u$Tt3BVy0>y5xpfynT_{%km!cpD?!N;Npeu z;Rwd^&;Ot`4YGMv_~??nK66_Bmfoo_Z)=cxP7IO4*I@F-r_=y{@^>L|0AOzbk+KfR zmoL1=-d6+^0?>|)n805*s{%pEgw+lLTy0)!yCaN4ZCWSR!dvcYZ@W{s=(-n}N$(NC zv%<`~9}lq9PM2BMU+|0BA{Bu2*>kL) znB6?Iv?-~zxaj1u^b&M4P+*~7jpb0(Ym`m5tpn@YdPUu$aDG z#&B!(Fcn&n0w}GJy@pCUIL_O6pXkcUrI4?=E4itSQUc{wJ~SNfH_3;S zXwaHG(kQ_v?WV9g3H>^-N5X$n^MC*Z!_ZQdwS4O*hsg7`t9(_H1(Hv2jiX-N9w$+~ z7L4|2;@QdD_x~>K1woG!D3b@G{wqu7SJl*W^x~$WuY*%ef^@5}_npj3idHDRji7F;%HO)}DjlrD89Lddp_Cr( zdQtyYt6@M&nQihQ#9lT2841y8wk;um0*WFGV+0tPMHhPd`3MeahGU_ew8G1sgQ?9E z%43eiZ3Ns62-tti2PO-&THUJQ+k?JYv(B4g5JvAa+EEnq1l1(h$;LD%K(+82%)7;z z5;ErQHXb(;SQWl8Wl^LCwO~1LI-fXH=spW3)CZAUl%k0z)3Ls0243nF5Vi=YBD$|W zrmr5k9yI(L_Pnw3DYFeOx!AAE#Z$CQh_l??^ zNhwgzX72HR3l^xqDdxa-9j+|S*n2sEwXC=NiIkSZauQ?Tl`~|z4;o3v#rqhwH!c2Q zo6@xS_IRIc9ajb^zJJWHX*<#O!}QT=1)~R4h4$xYmC!6*15d|U0fe;*9 ztKL%U+vk>kz%K|Fcb>#fKzcTZikP+Us|Z$N97}5@6>@#btZ0c>dt2bdl9n`Snw|wm znujy&sy09V({7{#&h6Xw+3OXSFz#3RF7Z2fpv3j9Z|+bzOuhu{t6 zuyY=KRohGG?JkxRP{)yAR9L_&Gk~_PG#xlml#s)wH6pz6t#tQJGtLefc`!B(^o>4= z>D#)&E@)VQolAu%lqe@W@6H3Gb~7gY9CA*~L9f8eOxw7$v4eJ&gx1xpn zM*;5{H@%Q3w>f>j{!ZMrAiIkpWufQhl@Wkj_O*#N7YMR2VvN8zXgUh|8qTT~n>Kl) z;IgQH5oyhj!I`2P81BlR$_68ujCFJM&GqAf#X@<`PD%8wJ?v>()b@(;wx%6qzHGz3 z+~sFq#ws7m*!v$!l&V=>YZDz8YktE``5HvAOn4-vXqUalF+9yF_qXm2g!ULH?~qB; z?Qbrng!A4_ESj^n-e`z2Mo+=ory{yPw9z%li;BkNuL{>qlgi3bwhjium>>cP#wnS} zzR6o=jV!O{t_5|lDh%qj=O`H@J4!*Cm!PM_}9HZw~8qE33|KiQj-i zaXg`S$u=NGvDUtHnw>fr-Jz&hJ9+@9<*#z!t8~ey9X4aOdHKdWLZEKVOSa$I+=-TA zR0)!K;bA$FZQ<$oz;?*3@fe6}2MH?DI?K554@a+Kvs($7s;D%mCy8|p%2icI3z4m( zYfG-j%}<>&`8>#a#*FoT))#bE;K9`B>Z95W)60;Uo@YH?@r1X9oAm*&g-Aq^IH4P} z8AfKb-l}Jsvi|2%i~L2#&O+3y%Itr(@Zf5G^Vj`*{HY*&6cOJ&CrP?8T(J}&H)r9|A>+EZery9Pu-ahrw$FDrpO^U2Yv ztvrQJw9P^ab56=+w-gxp@SiHGu<9#k=Jd0>Odg*Is&hbb_a%^Mrd5XS3#(|NY3QI{ zddhChs2SQ@n)+!(ZSq#3^jkNWYbMeCciH+HK!@gbg;FR#@v9n523f zx7)3|P|8cS$8aqW!6ozv8MCMM^QHY${~Jp{3r!aW!=^r{>NKAlPe7ZSPOShbH}=xk4pWe^|80bER`b1hQ?wLVY@F2% z6hZ%IPpSiWbusxuo~4m_ZfyNLcb+!u;P)n)Se0Z*1Tqab>*Ex{(NY$fHxi=eno324 z8Sujz_(O>v^D}qD^J}<8A-)E6vLmCi;D)@uz>@Z<8eqrn&;09w62|ih=$oxQ=5B)h zkrZr1Aogocw}<(tv2y1?uav)WJ4FlJz_R0i5&RMqIfmFeVnqsT#3l+a43A2Ki(Dg& zZMAxktHahjColaFZE!y-Gl`K<#2sF0Y>ftyjWKPXk|-1cr#VV zcj#2b5*(>Dg5u^*V=dXX2-08>dwEtA)t_fp)U1?azwdu`U=Co4HnjxAGd1cbxjrn-lO20N4c*x_qh%EMZxhSZNTWEb>(-s<5INvyM5y= z%fx9BNOn(BI=|GIV=15&IS`#&W8$OU7pjw^^2JDdVTPuvz5TdL-}EKe~#fNRRN!gNZ=9bhxJ0VWvp$=k() zS+lv`bZ_rNE$c0G;g5YnRj-K10qOe}8jAg7#?_vE5`AGgS9-}OrNjkEUfc6<7ogfe zz;>@LbC#0m-xi`@U^!pj-~CRsWbLXqT-VJrqUkSNgC&k=e`BJqPGa-h*}^8;63k)G zR(TS^iKArg(A+{5=Gkk@h5G$~GSF+~pZnwgmT~Nq9qzRMZ&a>)*>+=SZOy%z-QEU$ zWo<~Ev+4i!_1?;dXK!4&QT*ta{Zjj-etGopNv_1BUp<1Z`Sl(1l(-uH>n>A?-;dlp z(?Ij-$E12MbGzwUffs6051n~FN6BPIEQczwZmR)_2CZ})dz7X2Lts2SnY;c|DfH)= zO2H%lDGB=X%)giH2;yJ4e22q-!0>tM4`k-XhTtnU`2+lodLvNG{jtDm&FZK)W*sjJ$Jz5v+GCWPT=${(Lpa z_}f~$F`|f5>a+6;aM7(9=(+EKUn|teKJVTXV@+;LH7(o5p9TMH|%u9HB6Xo1j%O7`1sH{Ma{wW@pPoGSU!FwL= z&Sf&HgjKv3Kk-*(a(@D<2zS~Lya+4PVyw341}l(z1ot}S(_c(9M(g4$qa z6%pyQh96_`?1C9z|8q3Sb9Z%uGxI&S%xbqQvlG7*;sV4QvQI~q$3cW0Uu4Xq!n0w( z+sixc^p?it-ao}AaKZW2-d`ulz9zLdKtc(Pvns;ciqDRQ3{}4P`RX^bgJO`nL&ET) zxKof+R_{@9M(xo&Ush9teSkSfjW%Ml#sS_=+-VrqKNsHN*rC}>etE7=FjU4ofwSZx z=kxSX^GNwAxbZPoaMT&`g)_h8W;)JPZVxZMV2yl!9YU_W&l*~EA4_gos1y?m39sd* z5RRZCwttjLN~i!GKX>qs+7|Js0V6|hS8R>*bDWY6x1Vs^)4EM@w^!e@&E*MFEjz)| zWyBaPnK*RE=Di6!$G=%IrhNBWU-Pd0KyFm&qF?Pv_1VM1ip#+QN(vp!+5V?{d9c53 z%3oo+z{IJ)Rlu6?Wz!|2lKhUF520)_jrhU~84O zDEYj`9-yA{%IgV2?IB?4J_)?spq;yY-hRSm>j_pN8Y|(M_?dp>kp3Se8LR5K%_+|q zcHyR{S@l48nHOd~CRoNo^>BJdwF~G0&o_~5SPuG17tc4)io?SNo$-yWq7R=dR6d&O zj&H9p^WjgumpBu@R4Z>?bqF;&a$WU0yAA#ZE|lv5TVF`sBJ7!KlT-@*V>d9HJU2lA z+sjWEb%Gn(j0XZ>TTjBn_y{s-e#Y|Un2V|4kx4us(`VGQ_0gr;`Qr0S+B>rwo_c#e zX6{)lbH3vVam{-`75Qwlcylh$P+;U*%nh%|_?)nG*NWYI5+ZQdo`|NSN<1699D!Ht zU&bVt*i>Zk=GXd`&oeO>caC{Tr(tQ2L>nu&0)L9yY$;w`v3*_VtU|&NU%F zAjnwXQ-%LVZ0kbO{I?&GMV+Rlr-yzL4fL67OgHIr1sEnoMj*m5L8i1i66H41(&SsH zb%*o9j>UBrZg&2i<#Lg8$T~8WXOB@N;lZ-*fsY5TaCNr#%xDiM%BXDD|3DbMtM{t;(oe>P_D|4?y?I@40S5-&bR0Jj;3F5RSB%Yb=lSzikZ!$%1Pb z8%w+PXZ=9){~$(uBzy28h<7%@XM7`{QrP8Ddp^z;pKD2Z{)hcVlf+U~vD8i@G8;R8gDG{&Llc#3z13>b-x5SGbeEz-s4DAX`ag#T z?dEy6_D7D#_i0C5N?Sb6dK!ABO5N839cT9&e!5Q8#wbw6zt)z_7irJloOT5WUh*2q zmfZK{HQX9qPF_H6S7|lnmzY(}g4|{hTkAY~Zbu^16IHEGyYeiH0ra4K1dgnwxO5Gu z#jOp_oe)61ERnM~?=Pfsy*EcjHNQF;nBiDu>@`J*)#Ou&!I9{vt8(D+ETT7~%V#at z{HSPVww51Yul66+YTKbQi}fRSdO_Mdh6C~VoKudq3W4fnTUk!N;i+OM_bE%LL!`_J zSg<-7;=W1uT7ILKC|S%aShm*N2k~(>>dljP-gTVis5#-`UwA%gsIzapI1wfIa4V8~ z;B%b!&8F?JZEFFXPfX6w@i^;5qA@69xX*XI|UKg>uqL3sPtY|w{&y0+BTK2B^-s6 z{zJt73%Wr?ho0RjPl|SDGmj*va$%X#%XeR#C ze4?_EtQ@$#C($xnz^!z4-vP+dp(O#Yk5b<9{{=~nsc51;^H|mJNyE%|N97c?m+JMo zk$??7uzEEW)sfXM+&Zf+fbwsf)%_tWjKIoCozmS`xX(9Sz|dn}K9i?V#CciToxJ~j37qOZWIS)?RXkoezA2%$CXmVyubO?AnBd%1W{7`GR9GCxD z%b*#DB{5Q)Bf9lMm+$Op@Z{Hrs`;kvN>l1Dv;c=m=bTQEG(O@(DMY1SN*dh4Y8nO1 zu*9tuahV;~iR_Xu*%Fh8X`P+Km%wt`PJ_>T^C@@NTLoTStOOVtm8vX@7IAbL012S= zOX1GOa$IL1p`#wUuL}FaDv4-+NT;6skG_h(mfuiZaMI>`a2D((G~xX=j;VeY{OehV zvzhu7ptx%ITgiIDIc-@pB?9oNt%tjB{8$s-uL9=jpBGY}*^s`R~ zC!tGzn8t$ZCD#y(uReRZl*VVvS6QR|J#pcC zsbGd0nTty|!c=SL7@D_iW4rXf#ZlO&P=c>@j$9|Tb+%#L$*x}42I`9e_U_sWmH5*38Qs4SfA`6Ja@)0*{QmnGb72Pn`xZ5$uw&me^MS2NKtWv&45EiR3s0M8`smYji~Gr+^LIx6 z;4;lCdd+=(Vc&_a+V)iMF3VLh?TJ){tw%zbpZ=!sFz>}M^A^TS$E7~Ur91DAyD4u} z=914Kyo6Lk*BA3vg(J_N|0S316lSK$s@im%DFQB%RjDr_vIF{hwbY>$;5`z)vp)pl z_e*n8C;a{3op%ZH$joS>dEC_!m5~XIZYt{P+>xJqVXxtW+%IRid6BLt>v2bxe40;! ze0plMxjylTM?1BBpTvWI%&ihLvmL$8MqmFaU2en|O2Z z_|LslN2}ZUkCjFp)gv4H9narsUkXuif&?cT6?+0}v9Q?qjnxK3k+4_o!ruq~D0P)j_cZu*+SM)| zKDiy&?BYn-5!BBnr+4BZAlOe(-C-T&DDy`Wu~$m3DLhWwC1E?$%rj<;u{i_&HTTNy zoq*v+w5Z>Xb@JM|_2loCPaO9u{B-B-XT&Zn(!1_j>9Om-{|t=B6V0bZ*)b2K)4;92 z0k^gublm&u!H%s29^r}Q`KaB$?)dC{lHrXz_^r4K+cjs3CI*R&=V`aTeLK-Apylhe?;qS0 zbMXl3iO7Gmgvq6)8+&N~dw2Zb&1SM$&u`IX0$Ac0!v7`KeR9118h@3w?f)axJCC*O z9bPGhL}y@JZ3aib;0ugaX)ofQn-m*4*wEp#Ei=qwK5((8# zZZ5$oYw=di@1W?`ndtOG!qUl2g8`h))ofop4B5cN=Q3SWvtK|L*4EbMrW-HR;R42~ z_8|-Gva7!Asll)K%3{U9ajLk8XIf+LStMKj-I^}O#^*p4R$a=?Wo&Q4D9cNzTfCHH zN?%oP^$b(|J&3736*=g=dx((ffMLm(UY z&1nS(dUf$P*-8h!0&GXFWd1-5t41>u^PGQ#R`A2Yq*i(xfWt_g=6&3dh@>gfUEHj#7O*LK%7 z?&BzKBa%_gmByi*u%0oIqvg`JEz#(C8vPEnXTdN;7t`|d`Svc`hpl27ZFq%`S~ApRu8?g-v+)#62Xycn45U{TiwEjfp4KZSaUjh*5}Om7FedLUruszvQ=1E zJ>oECI!wI*Q)b1*xK&sWxP4TjgZmJC6 zyZAHx|L^1P*Q;}q&V|syp>nE3Iai91hQKcIqTn;#B(4*$2P)~iWN z>+SWtA}!8@q`G1Cu*Oibx1v>zS}5>P_HBx~mZo+3KrEX$fy5VLEoGfTM|&sOOleJ3 zdjoLwF{$lrnk3u*NFL<6Gs33j4p#?tcA=K`8x8G8pjhyuD_h?^tC$W69NN&ie=km%6Xnj z>R9|ZCh1VoN2p;&+E|bmXZDSVI?&&K@4X;@rg_E2ObqAI)h#&{+|f}b>_&I+3%&PM zsj>HD=?H#yT`V z>7lk{DVmcV=IY5h=NeROcsUXouxV#zHsK6BD|Lj`uo=O1Nx=Jij zhpOV|(UZk4^FELg?ESvc>KH18X=j1Hvc~>#F}eD;cb0^XxE*STnss;nzvT(zBlMx? zhvn{$GaTkZ20Eu(QSJ@!Cw860=N{<#;$UoORUWvR^``n9r7C!TyI=fpcOz~Aht4>} z1}q#ye^7k1@hSloGH9XT?(vu}O7?yp?IFRDNZ?p~dCx77yqlsYR{}OI)`+NmA{C_Qy#TWuotVjR;p~e<)?1 zW2QW`Wr-&_n%55LkKLfeI=~XObvJuVh4;L(WZ{1|;g3tsvimUr#-JEu-Q#ZT-ggb~ zmCIo-j0|vS(E?Mb2;WDsY9_AWImoo5X*AHUXD#I9i1*{|_9O{rzwI@bws(iAxIG1< zdxTDHXoL7hZ+wM0W1)`9OTW!B$xsw?_|!B^u;z^e?&@%TQ<`!ZXsEbLD z+ugivS&Y6=>aw>VHeHsG@S7O>Vey|ui-ljF2s-|USj$Yi5S>b@9douCi@+x0XJqSV zId}HfSsRB^y2(995N}l|ie&Fl+ zmUOkq@5g(RzshOFF9Lb_nhG#@_;X=wG^X3infiP8%e;G3MVYYSd}Zy%iDy7{eZ}sy zMvQI^)e;cccHBu1C&{10l<1N32|};YEm6NM5@FZiy!9My%|*UjN17Svq{l8lBirbs zgv}m5Uho#+w7tr;E9}M;J>~%u!$V+VFsT%Cbv|OH{3wi!A*@K&yRS#H%0u0QfyQR9 zn^!-|1vI;pMe7^qrmB`eNt}94F$_vE5VFsi?F1{HE`PcTuU;xNZS22u&`TXQZbXY3 zrzzDQj}DLBo}Q*r{D?cK?d?@F&ZxXaWB1jZ1TuBAs)exiJF%|>;JmXJ^bjzcfn!PoIJYl)@fI%txQ zm#b?Jf&JfE26o|41%G2>Wv8pZDrF2}FgPxE!)KA^R>@W<^s>>}1oQl34z(-TfBi`&4Gs=}^`G*Y{_%ms zV^&0aU2=dFPOkyBm_FSZ*P)9s@EyJZ)IL{_3b1cl zQyZ#j0A>P-&JzXpV@vWC4OMz|b<=s0# z9`74N(&u9m)8fuXd}Y5O)Te?{Dauj7Mpq_-a_$vKmvZ=Xh2db^{;r5rZna+YhMuzN zOvL7~9ARKOi*?sr6QOT3z2GGU&AjpU0YUBd&0AY|gT))810px3iUas-uhUy+25aq1 zs@Gnhh5+8qOJ}2Nuk=ZsniqL;rGMMlNFK^&6wgIa3Yj*3DCrk6H)WETpbF`AyqfQE zZCM7BJR+}bTJ0AZs4FkO1l3H_pGoId<(-rnPtdCXemny|#m?FvGEs-=&v=y>T#6z&gvJ9`*@_JX}5 zUa5{&mWuYSq!g=V*FLSx_2)=!!;lzb7TCPMz0iY57{a1!*_T)@H^+8pa~dvnNX}Xa zdh8e1umWT%l3TB&9@;;Pbb+iBdZ`3$0*&2+!&Rz!m86xLeBX}OO;Ek!XzbQ0%f|GD z)ZFJ~CX%Dg8@rihyu(te8arnV_~lQnrIt!?s~R$_tjO`;q-Tm=R%6N4J|qh=NSru@ zVy>*oLB(EG`&o(Nbu~P4dou?X>$n%|to{jgsr|PC6W*VNt!G-JukhBH-n&A_TK67$ z>rw&P_Q{5m*htNSpGxe9uz zdcZN?6~oxhX-)9im)`i9v-IJjPjFY8&(N3XTWSx69{YwwS#^*`ng4x@W^G3!vuJH^mrDPoVnO%I}$8o7~nKh6nXf|=2I6fz+V(`5uY@U@Xc>Eia~~%WR4C>44$y#z_s6>Jix z)2Bbrb2>YX>H%x4#)gF&FF~^I&M@@>pQ-V^?uNlasaav9;Ab)2fcWECmA2#Uni<}ZyQf($ojPI{0cl9VWy_Gy-hf zV*R=#Pg8COi;xLH^X4?ULAiy~oH3VV?S_Z1DT3;ON5)+}l7-|eR?Zq<%7`<#Pyg;7 zaP`WpqdLNsK|y`h5xFMeuwq0iC%}IMTO9}DV7{mef!my~jzoi45q8J#Lm>4mZuzeA zBJ|*l3p<5`Yru^SN$c$@=Y-)JBO$8cNG%@s zYlVb=%K;@2M2uVu&|#zH3nXQzU58pakzlF-*3P=wl5(SC1Ej9>{P+sE5H{_smPWqg zZlqIO^@v3}BBY1-D8bmUc&R+<*8w@eXbV>J-NNFizu?aPXU7s&-|DD6DS@yIbTxXD zG#5|sIkh5zm3S9JzxsI`f0Cb?W8n56jj>~lao2TG^0P3@+Q0F%$#X!bl9n5sCp_%- z#U$J>wjASEqiRyQwazrJJJyC5q?95(uKRf@=&6{4-1PaqWws3N-D^J6?8>i0oY_Qk zE2vbu2SKOWxO|9duRasOH^wm|zW*GI()J%2@8xyN8Nc4`bENtAQ+sY&_-}OAi)?Vj z^$wbINuKR^MeEk0nH{PBiM$vCk`Y^H4gX3j-pH^Az=n(9;L)tP9ny`qNfp=l2+=-` z1HD&iUT2bqU!$a-UibOJvD)kXz8HH-=>d~l@u7JN@0!G|vn!0?Z5}_*bXKeDeJ{~< z) zblNker0&zgd`UhBdI#f~)C+h6j}Qxk8zb08Oi_T05FWUE5d-=@!0-U6FVRToQeBshzZ3xC8cnqr4|NSRUX`V6ZLrhTK zsYQTN`P?j+R78K5(1kCU%V`N8OB%dCyG6Lq(`M#q$A<&n2cgDjK7`C zYS49bZa*6ukaeJp2N+G#9#&b4qLj2g21d+&p0 zRa7=Kgnd{1`lKyvq2u({*IO}aN0)>B-B&tmm>RBU`B5FgupwWThT!4&HmzN2mo+_` zw|M9@tCN@dYh8g{1hq7c4?&c#y;66;>RIv|aAtNrWwr~)ApSFNn8)U&XpXs5CP=)t zj9liFdl^OQLM(6tTeuepa^i6hlCaq>qDwEwiPTy40RBCRmDv&u=#+6|*wqui>7~}B z|IkZgr)?^cVZY)H3WF<)4v7{!JsmqI8)HpcHm-99T?a4KBwsgf4hL|Suht|DN3W*2 z8V;F|QbbVPPr3L0in~%q{##PCgX#qYmO`UPCUy#4zxhLlWRr$s|DaJ*ZIWgov8|$Kbkc<3|QHk?To2p-MqI|KqIWm%t?8! zkv{F1Hal0RSa<5x#KmN-l!K}yvu5Jnx{_7ZR4DlCWf#zOUtm6q-4?!JJ(Cwco9P)_ z6x)utEldS*&c~f~?n+ZMZq1`y`~j*lzD-8q`uyt`suDzIwez;$wFlHylf6#~ z*P=t1577);M>rd#`RnlMm30b*5wG@i8YOggEyR&Pbou`c-zsIwwMtkql ze1DK2rb$ZZw(tkBsy~A5S$knSoAG%NjY@YulV!X%O%6F)4a>&*|q%8$rU}$IGlx z6Yf;VZ0y64XD+up{Ey{ix<7GfSEEUs0+XJL_#aCbf`xT3M-|_e6B%zr#^>x2zwRI zMVv&Y_ny30mrs0MIbCxdI)!g1e<(Ld2^M38avPJa;}f_>**DlR)<#}U_xAG}R|{aO zEzbs{yzLrI2BZAzI_d`0Jh63aX%XO>yX2!)jsx9Rjo`-PE0?cfd>~`V5xIU)1$bk_ zG+_fCcvP}BW-)@VLdwNEN1f(@~rUDt^RDk;e_L| z!|&7seS-C36zvJ4lA?lYh(^#GNOU`kLrLvWGLW%rzDm%@zYb^(kc`lI(8Y#_y7^Vk zOvXa<2$uDl?69^*P+_|0`ue-%mizdzBR*E*+5ZIp3*LyvlO#fh&m&OU5$g|LJDuUF zx*myT+X9fVAd01N)i#Qw<|4J4Xy`*!}w z)jYHZz3_u5KPC+hF2~P~h0@08{wF6kxU^_m#r=W+RV%Mk9S_~!devi$=dz?_K?Wp) zzz?&Z!rc)NWJm)L%rL;#;QU5hsl38uomF-n8C$mLO$(b6)_v5@Y_P3(9OCyi^mlMV zoXakH&hgnlM~86*qeDYOhxHD|K@S=_)Z}A613%3nhGylu07^OFg5}zOvT@t@(w;=< zTo#6ADBcWxIuA80`2CJDfp>HPm>)HATaDa2^aZIp8$T_;=Htq|P;9OC){o;MeXZ%d zXO-GUuo0#oVd{Gqc9+UXf-1@u`uC&yVXKJs>c*x`mNi~R?~ge3JIPv;)>2;F443H1 z0L#ph=*Bto)1FrhMBmx!tZt|b6?$kj&6j%nuCLAGch*bR${X(9#q~=@Zd6(a0C_R3 zDcdt@$C$8vJ{u*6x zw0dwj=_^B)@0A>oj%?aui~5hS@k_^Wm4B~K_>*z^Gne&LI@&D78I+I2VFT>!Nb^)4 z?gM%JiJ)Q9sD3+dAGDvU(*F=Fg};Rqz@$JU#>ZtxKBo(>pgrIsjlS!OoPLzw@S1Yd z$r|Dn8d(O{ncMiYO^f#`@oi`0g05Lz=$MDa_~uWyV$2=UAO5&?4SrF?2cZafTI}H_ zjctY9nV41Sx2V&YY7fWAW=#D&m`|RuqCf=j;o^szKfxH!_$81m^@i*I_SI^V&c+km zShS!pWOJr)VUaI|6Vb_IGD}SJT>xL=5!r?n-z$iLki#F2q9p&1(t(BF|K|<9hCH}k zYbB%nDVojJY4J5ykcNDTb%N{ap`&h8PzbhWi`S+ZsQ&XXRa1sw4LuwnuHjrX7D65a z520#*Lp23ZjoDElfqNIzYy0cDyB^B6Pc>5!_Z+>97xJx=0eNL`UO7^MR68Et9>JA* zqJ8CxPrCzx^Y_AqV4%j@tZ$XaG2yJiwgl3PcQvC_g1hRryDd}d|i6yuI%sNi;N*X zrd}gS7Vl7FajODzpE>v_rFKH><-5VkU3(8Ld)F<07aCZLuXJfI4o=BA%xtb4@gDGw z@2Cr;M+}t|dA=_MVo9V1wxhW9v`r0C2?m4F%X~q0fGHv-YiUvZl0~~CsJB4+(Fl-u z0tFzbQi;IsYf(9v&v-nAD?G!%7Okf4$+dG>3_gI+3}#poBf)hA5_jut0=lTDf;LXb z_*MTX*%A8E2#84RWWg>kGV68jDYOiY_{-%?26`VBtF~JAdt7Lex0>fsVf18sQ~7lL z4R)Q1>L5x4A@5oHEo!A}z&^SsaVBz7JM@8X#$0Ute}bQ?sw$}LH|^C{N* zV9Iqq;GI)5R7)A-YR{R2{J+tZa1`12`ot9Ts!+BqHlhRH{S()jV+k&cWq~q}p?HtT zs8Gj>irxaBE|^r|xlwP;bmNuZO^_qz(dUV@(mE=SeBS88z@k0E<5z;(LX2zj-wn&>lDg+HT(P}E! zk)r2B8&)@dkAo{ICJHW@{AtTxG8-rB(;}LdH0!c6(&ay>>1xL9pUS8!9wVrh1lW zPp-*Ai$nB(Uo`~X`?ti3gh@Vkmd*Rn{rGREBv8Jos!Eer#x4*)I?Xy)Jvxt7iZ1Om z5|8L1MZN3XxMi#S!qyh-v`-@P4Z_mWw&{f?wNh8J%f$b#q1k>vyKJ=J5}k@n;dsf{cG4Bp6)hy|hwK*B=6@3oWp7vy>%5W+}* z-l@@(g7CZh^6|*4J81V>WAxsYGIj7SoA6>MCmE_V7~AgUYhrlA(Efvb?|Rhl>?>_Q z!shH@D$xIjj4882;T8`fVtEeXGYGXk8m?uzc#vYw+(ig+>k=z z43p@`I)~0yYE*CrzI4?+-S67rQDfkHD)6Zu;zOg~F;pEi{Up-4@6&yy7pkrS@4pv} zE)#ygSXQSGsBgj|70?XO{>=cyTbrmH2_(y%FH`egJeL0Cm|OJLzpC+?q1RsTcGsGQY89ccpFsqpCLEaGpH&@$#Xq;S@y>Foi(K41 zy~)MC?Hu>ti!o-fLj)P^7I$v>_XkNufuXYg_(+o+fBwM7XsEUkfA|-ilT4RnALAd?*Zo-Oeun@tN>uAfhF@&cjP!7FZcAF9@9^* zZ%^_+k8)I75R5JHuHBiAX!9~Ol}}7+@@aXd>qzRC!7Y8;SN9lHXGM#&(4rx)yEf;1 z!4q6DAYm!so&FC{bmC>khy5R|g)cCHU>#oSVE0=yMb#}7hZahMYc8y{95*fcWa6x9 z_33~c+AIHkr5Mf_ZCJVu1XCH4Q?*Y+!PWzz(Ao`=fpiu_gOb|PGi|e4^&@dbY z^w7{tdTS+KK)hrwot&Tjq8^C8gJs^CbZ6$D%(>$g`;PQw8{kpa&nL$Wf@e|JgG@D9 zH_{jG%z>?}N@flU??G>C9!K4siS35u+KqHxu@c1`8@hAV3IqzirBrKzJ1L=ar$bzE zwM_7*&ys6B=~O&PU)Y~IWVRmk{tinmXgU~d-bcmWcZ>+GO3;zI8e<(S4`m-=|GF<87Lf1yb&Ed*+!6?(rp|3ry!fOS$yq)!Nia|lEY2YxAZ15}g zr8>2!>-RuuLX=xBpu#_2zXSQvD4ciJinU}K@9)=tE8dA^zND9)4U``t!b~n}GzGd-q^%lxJ9Q-FFXD{U5T7mjCuda5^Nk(79 z_om${wElkGE3RI^&p=$p))>^8WNP%xU|SkmY8JOG1TYfda&=XiOfN!cB~muo@gEEG z4I?(Oeiao`N1{(1^VIejKbGEtk8U=w|2Ph-`IgJoIw2w_0Z$_cnx`V)I2lF==c3gl z)fRZSKHl=;>WS!H&odojQryYO1jUmj24~?fLCV&P?WK83(YM+;x>wUMchHQCXO4Qm zNz8w!{Z|CQeI>%B4n9pzmdf8Tw|W`ez8dypT*Am`K%svZ0rp^BMok9;JYsXKr%N01 zd)%LohKLSd&p9~Sy6@V{a>mko0t=APyj(HGrs(MpQ4WZ*;7W9PZKe>a(ShfTbb6Iu zVGE4|g8ww8mz!cm)&gFR_B1|nX72f4M0M8v&Gi9OKHenWuw29$B#N?0trW`>{wQl= z#1(n)CimIPD0PY%`d|+k$?n8-r)WDgx9pwI$aMknAD|gw!ejh!fM_`dy!m)A5HuZ6(ef0BAKB(df_^QImeU^N`*j^6uRk z6U?d${=;9KJagH$Zf;DKXTNUKlq$uBJ{!8h`r1Z~OEUm@Ks%cMpy6wWo`s&VL+eEu98m7?w6}+AQC-4Qm;v4HLm++}9f) zf2G_&{7Tusy#g4-rd*a#vT7%{jug|j7e(MwZaA+?U{XmjrZr>?=18>O9te*PuB$;8 z_wqR6bNoC`JvK7KHrII4^7_N?OhR1#GB1`le>P&9lI=&8;zCWaiK3{yGRhf`zl?U*YoQb4B*)>)QA!n>s!1!LUNCAYH1Mcz_OM|g@FlKitdCE5^HcDBagg3T0^1sGpqN%zd8 zY7+0&-kpRjAxx+)2EuGA6?|`xVE_`)IW0AK1E`h5AtAGTKZ;WF|Vp(B9cuWOhM6rsGjl* zV0OYQ$D9z`oHFE8in?_TQ6?RdwsW#J_Ha_%1>ZMGxpYScTZmd-@mXXi>Z=3C`xW#Ihi-#$uMhnM zC=g9Z#f+g`?sWTQZ?k^I_hSZngRu(_I+`9!DXO+J(%ypH(0ju}Ngyi#W@00(NjgA= zYXn~D$h}IOu|{lg=hs0EHy3vqS2c@jm1+lP)#?dAn)uVnu`w-d2zM@^8NkHVjheBW zKzf!@vL+CA2hXjt!9wM~uVi7>&Wab+;DGQ$ficxyv}0uI#A!&7*%en55*cZCrt0RQ zJzHIlJJ-h?OEE-kZjPVD(?=u)$`X(Ux)Kn)_}F)N2o!Y7;(rT5Wt>&t03TQivSYO9 znD#L}K|z_R~=;CM_Vd#0k$bCzdeN-i@0 zAgXhTJ2}WXkZ6Khg?ByB)Nc`yhnzul%k1#_`K(VGI}53fFc_Y&}vF)t~97QfS$XLfAxDb{?}4ex5t=3+kL~_9%)Ig|puH z8xHR6IWI1fezuFm#%UpwtH9pUe~RDOU2)Gd+hdtA4kkLfRWxvt0DRGf(<~+7(s+d* z4(rXZ$u)A#o$3|n%vq=|F5{Q>*C2 zKWEvH|7r-ve;nQuNriK*Nvo}#j+9ol(D`H8jjkVhAcR4rcN%}WfgD&+IC||nbmpns3_|HCuvggUTYv(< z?EYs>6IlU!u8*MY06{&V-UQ~=X}}0H9pXHH`ZJIM`Y`U0VLbY(f=?LGY7KZhfTZ9H zAV$1`c-aK-lA32r$j6?ZrSf?Qv5RWHjp=#@eYrk6wUuDGFTe&MRm@!w?P z_Hbus=V2fr5g~6?JTsp&;(cHNh`n;Cc&7d0OMcPkqUtK8T|l;DOzn##arFS#l9|0n zuiO<+01|v}^mLNG{5>)S1b@=${UO2#@g(7E&Mo_OA03^wCh@wI>J^~ktm8xbhA

)Q+4Tk6@Mu zH8MU2sOD644(jmi<9p)g%T402(*GL~x&YQT?tks2aCZRO7jd9fcWY(K`cJSWrPR9u z+n#sU&s1wWM|)=-Yo_Jbq&u|`v@u@LToH2ZwDEWAzYMh+uR9?VF0*b1&5?8#mfjz= zvLn77|IMMyDkE18-9Lcx#x#v^2X=I{e4yUzXGdvRP?Mm!(4_g4@otzJ6drjv{cBc< z<)2C#f={2lS-OHc3Xdb|I@9OhnK2sLw998Et5{k*P)`A*qDMgr+bbDR0q4|vrwJ*6 zU7z*_t&Ds*-!mNP!e|efZ_=mC=wWbOJKBSfHs(9}n!fqZ&a~8y2banAg^X{mNL@EX z^K!lz!nCe(42jl?s7tl+A1B|Do=Mps(QnTS>a4K83dl%m(cfl1FB)dUna^V&r}*~O zDRpMzjGarVo|#I8_^ox1k_gDl2Bt!r?%G`~N6(;Wf&a+Kj?usTA?(bbIupSPwl~AT zf#4kGSuARnI^VS2C_EZ0A_Wec$|*F1gj@=qxKuZl za!#VVt0@zOHfsx^9(gXiLdAVF=O~|;iS)pmWeAdHTPSK&y$LSW^UdBsXGVrzYG zo6c&gM*))6rhRQZ?&J{N<29wt4PyhZw#WN?P5_bkKBlo}J1Fk**JM!DLoyDSVre&Gnips|6mq}Yi zPD%aB7s0CN<(Z2H8ydsI_N;XP+N2h>xwkv<^e%lMtKCLRL?DfR-6mPIrdzF59Q z=A}g2eB+Rtc1X#>f#@j78z;W>%J^q!wtHQ&`)(F|7+@-Vx}VQlgWy$qf8114D?VmD zWBdr9PRnT>&sx0v|NXnSc##tWjhJWbJlM3*ltiwd91?F=F4+EMogd~UmXh>^=}_(2w#h& z0@<69Po1Db$LfGeqy6Yo_D$D%qhS;VPz(_qmol!o^y9tqMoe`!Taq(Xz0YVEUc!|Y>6L}2e``bMx3#v}@c7b2*dWSv&?tgTsz>I@tFQhjqYEP;3t`QAD> z(~3H)cb1RESJhSst$hV{b4eL(?PgM3oiIz;OtfYqUQWvhc z;~0bw$d=8`BDd%2(WbQAEs-EL92+Wl!O4v|jQtoLo+SbAUK!uX;#MKsjT)ibQi573w=BLERFZ&o;#Ynh zkY7GV-I>i^N~%!w;#xW8PtVkyLA*-R5fVH272_YXy~y?8hs%J2E4J1uwyU6dqayJU z|GZLF`OlSV2xR)8eq+=PAWvRG90h3mmumo>|7cI%tE|{r#KNFP z#~={8`-4pHtwdgz-WQ`KY;IBoS^{oO4C(<+#9|^15q&To#46mp-OTH$jNA-RHVziH zsVZn#j!4$#EUy^@B_UG=TmuPgaqqLQLBf7?4hDN#JJrOEdpEXtaUwsd^EU7E%N!1} zDrU5~qf>lOrQvhZpyJ*F4v7h9=hU?6nI3ubGp{w0CAIN-RImJJIYVVoy*96qB#>LO z*T(E)kJd+;tkiKA!@hGbfg|FVn2b19v*N| zRMMGF@}{XaOGN1=j4Sp;#RF95lR7PSeT!#pX*g|&Ce>2LXW|UbY+}vn~vE#m`_S@qM8q zRxk$4LRoXFTUXx=H~dyLQR!Q1-8}uV-TewgU)Z*AEx=;Sj639?0L&V;=((Y0TT4-! zZSOz_oZQexmRtpvlQMw=R)Bn*$N=fO=}HD}nQ9xGi=u+fa=-ulN>ksc2Rrn2#ogR- z?TyUd>@ZJGrYlq?yDQ^t0@F8Pil|d)N|B}T1@n<+_`Zd-aaw3`&zoK;4?$agOjy_? z3BOpBYe}ZeT4yO&cqZk2s1GZittG(UYYY1Cp=^#zq0m2piFh`%W@cuawLMQ7_s!cD zQ(VGxugNP9s5EqsXYri2)Kh)l;S?d`XS7Q~*NEMF0bOFul$bcZ@_RQ>C()&B_eqX9P z8~m3>NwEKRrUglK5^Iey)p74Uh8R4=uX_wNsj{*oWLW0{9+`RuiOW^0yT9Cn^x#jW z7v3C=@YXo7tSJ?p3S{vKp{5Gn8XNii0QBGsUCMTDf*lTGt})HDapPtFMJ^>^3X40M zul#RudSv>YtjpLN?|Y6N`txyt(^IxFX+bN0#K{&>Cg$pOq@eTG_FQ4I{bR4jS=u4K zM;BQuFU7AwM+0n=btQ0RvbC(J0Z_SZ-6{w)K4jF%{>^hbhgrN*2PB1TvLo%z&}(u^ zG=&aCw#PIPp~vGv(DbXPIwghYB2?wPjwGC?f-6*)<^WN9KYZ7fwHmTRV4WBTwCQC1 zBvf-<98Q?E0o0CS zoANuIS5!8EI9mS+@x?XZ+-!h8djUgzUhsOT{M zOK;}4FahV1d`z$t&}Kuge{38VNecI(vc~sG1{=zDgzSVzC*B9uy`mV*6O>K^T5+3b z9x9>Ip^-N{Uy?h~?^|*w23|UR%O$wc<4i=cXKw6X@TnPI*c*ziaF>|zZIa7or|oIN zF$bFF)+I;!byD0;B6m&QR_3Mnq)(^msiEP&v^hy}q{Ji`fRd)8{D(ZXDp0+J5APqS z$@Z%o02u8-xJzfXwU$0PaIr2Lo7cz1cFt+sZA13n+!-M^lpA#!FAyq|OR5wfJdN)e z+q*OattLD0>#W%iiKXv8KH7ZU^^LH-mx3(4eL6$9@=)@O_gSSoGUA$b^s*lVQGIpF zJMAYsj(k6<6a@FBy2pFB;)O6oI^43~|NJ-m!2UvrTPyu6@y0G0cR z1&FA1C(kDs^M+wdM3b2U$}*>3s#RxJnL6-CIiKyN5=d94m%f;hTk>a~Zo{OyS9A|P&feMlZrq^Nly|Qh z#__M-qG6TWns?H;SUW$dsVeqoD>=S~+Zh3KRUSTWmHz7(5$RWL+5@cn+Pv;L0!RF- zaojsn5=)X>5%k+>;(t3o)GG(HGT_O&3u{?MUS5};bPyMg)W7X*@p_+?eH^9H{_|)v zJh$_pp0F``JglqSp_p*BNpo$=b#X9)f9oiU_Zl60)ZsW>r%h+)D}uAg3aBAJ|AW&h z?l;_-(+x8G8@*qe5cy9H-#y^gR7Uf-wQYL6<2iYf^r;fynqnkAmv!8Mp7uHT82qE0 zDaXmDnKQZ_@Zk#+O~}3&8b-Sfo}5X`u2SO^EBaPtc}BU6Yv4VSh< zelb4$*YFz;t6HX8;*Y~=3!GlY`R@@m<+G~7%|afv@x9jrL5^-kK*kXYrRmcJnI#Nq z4{o8c9z+hej$4UDcK+r>Z%nC%!hGxMsI@00Y_EkE-q+y*f}lh|5Ol1P@zUT=%$-1w zw!4=6CjL(MExVec{v_=ky3()7@XJ(}0_%ziQ|)n1Ec}IGoKsL?MXA7Y?`2t zDOZQAvzD1SQA^VvatPmMy?bI3%SL=bLb3i+`Rs+qJ)RMirCuXUvsV2z*W3vG;-rtm zCX9gg==P$(@I^fJ2_U=^*nh%*d3arE>S-<0w_%Kl3T)sFX?C3bBEFCls1#I5sW)x9 z)#VRW*eo^?w9p|en^lk|yISR#(^=ZZnx@%ZB&p`ucOYQ{l)7s$=BSn9)G@2I^urPX zhrNI!2 z@k{;ucx?`aOs#GR)y?Kij9T~a6@`eyab(;G5RgM(t@*^g&3`s~m!=locZGOtDCeuD z2&I`8=i@*EL2s-w#J31@{I4xuKi+HyW&54@e9|x6Ak$+vW-#)BZN_(IUlTr_0D)+dV;7esvB}w%=6RknmWH*S^Sz- z?|Z1v1G+0o`2tT^%+-&;_06giE4mU8%={nh^q5L^ivI;4YR*r}Xvhb|v%0$&Y;d-7 zAXQh{&b*gC7HYwN&CWFh^=01fO(Tu)MnC`PLH6KK8(K3yi()$75Zu;i;7pW$025co zenF(*YUZBB0!Bic!@zZt)$i-1RxhnXzS#*~*86LIlTiw}6>9*?_SF)!_tS2K@g<4$ z{rVl92}*UtO{ReA9;Yc4^kL4X^w@{^_xx`nNtMUk{!TN>N~73#kWHnmTd%#l@)rL2z(Z_P&kBc#ZZ~9a-`>(tPRpf>6QXQm17216#j+ zfo2+EOO0rBL!5`1QMVFg%vpX2N7YN-Ch|l^FYZX5%C!na56Ovsx#Hs4V?=c^b;E*f z-CF;_Z$D`@@^Xglf65PvGKql2ZZoQ}tmn;1>l{jsHM_gNbfGdi$#}MpBsIyY(&q4D zg~h`gwbggrvUObU1T{*HT%2-P-}Hd7sXc)5r6L-$NTBan=je0ShNq%@26VHg?%J`U z=R}g&x(rh2W7gVE9JZhM2{Y4bB`3$N?N8F3KasuC@$%BCk@mjTaef=m22Mo6{R`trr8r5dq`0uQZr2BRh`X2^##dOu3)rFF2Vf>WJ)t~RG-t#S)J2i+A) zrM4x(bi9bgbHgF9Xr2}ThdaWCD(|hh>FF1b{7uVXe&9w#jlQRU? zTa^AL?i{$r!GE*#!~C_0TZnKLv^GM*^0e@tg_Ii<9gmd@x{sIw?SNNOC~F5tkJFAj ze*EaZ^k#`qQz7Ht&m!zYF1h>yi~jlbSMa@Xf;q%Ln_0iaR-{%|=Y!TqT2Cq1 zd^D1Ei}BKROEB``jTQtm?dRAEQomAwL>-NB>?z1|{i7oF6kma4Kuvnt_6-AOql;t$ z7Q}R^V~V0`paj8$frOuSC!ZH(+gs)^>$bQ}~$aX{C^cx>73)Yee?4767=eGt_uV5QKC9@f0v^bnZ1DOnWFf$}; zb9dY=9Z(75l;RMnPn!kp?RGXxxebs>pPvV*!NkV$>Z)W%)a)7UU%#ZnB)+RR2hWVd zh#1n5PP0v7!*Z7JxCbrh(%7>^okPCY2P+wJbzxmBuKrBz$kzQX$oGh}(hK;sJM{}k zJ>Xytr{Hz4NXR?PTBC@E-I-1sZp2Dw@myf>`Sh-S-8m97+qZrUd0J_8;(fEF+S@;3 zLJYZ@tHV&R*TE z%|H19BV%-A$sf34)x3+Kfc3&~lBbQ8Luk3s&y&JzHCW$WM`iyj6JoEB7ak0G`!=?6 zH=C-=VbQ${ht0vvYUo@IxrlzyEkA@~!#Np0@sy8o6l8&v9Zz-rn&SnPkNDV4cKyZ{ z{{LUs7UXWv#Mow&-6gI+*OW~|&X;bqRtSE5vPkKZK^T*S=g)qwEPdBH|03?}M88oL zvpI8ZfPG%}Q3S1)uQMaig@sPab=c6tcan!|J`7f=@A-3A+ur1(ZJc+N+A(CT{#o1~ z)ZsSl8Egfa7nTS;pZ!fpzD%U&(7D|Y6b~r8dcXU@gL8jNzUb>SH!kNhe9KTv36o_QI}j$*+_^tyTGpc27q1@}k2QdIKRy0N2gr^q9#>MUp2Z;gv&rC107hu>_1NmMUEa8b*oLv6wcVzmnAFe-i+JBGYaM^_LvqqcKbfuP4Osht&rhlbohzfGCI zFo@#eZCH8V1!~g%ZO`Lt5@qwO8td(Ck)5(FDjc>jH+k|cD0;uUKt&^DwR%f8dOLJg z=yW3_ic#X|cWnmyKA*h@g`6h%gHQ=KB)MsVK7Qe56iaQhKiCoX|FQR;aZP61{y3wfI5x%xgESEp z5FtttAwaC4QU&Qvk$`lB049{!89_m$*N6xRp%|*vs7MVEDFLYwA%qY@gb+d?<@W?U z=gv9z%)PJvZ~mYAyzv<_&)$3Owf1+dwbxpEUkX{}>DAHyR37VNh4VlB#t21Y!C)S_ zayooq5K=(Gz=_~R8l7i09?(;oMt(y~_uS4_CgZ4mgUiiS-wF~hLrK?hscKbtD{@I5 zsY2%VD|OQEHG)8)>@5^Jan1GHw^Qsf;@ZQd%Er!THwIe!P$49)N+jS~d@-v<02dsX z0hfL^#7ajDW(3PwRNi17EmZ96#Z@|JL{bTRO6FfQATnjzXY;j7Wn^bXWF=#QnG1}W z>-3SZNOYaiBGu(J5Pl|f-P+7Rdggr?oeMB%b3v!d7S z+p)O$XC?}6_f~JQx2%F3iU!P!(ykSjyy;(|Q!ARkynf0y=_`m)Oox3gL=j{CcNzRT z8*+bjFD?v~SJxfezr8yp5QS*m`_4+*TAW_XhY4Bs0=JIjJYQp- z1?OE7KbU!HaAa{PrmAZPbgiCQ#RIDIdEt`c>3~lkY8A0E8A)neIAzf>yGEnq^!(Oz z)K?j&==b&bOA#@dI!7Vn%y*+qF8^q(UZr)6NNhTbAc~n^sIAvSy}MBb4M2dUxLRW` z_ciS+^J47_7#w4@Z5Gj00ZB_55g%@`_&Uot&I=n^><<(+_F%)b2`JpuvvB9Rw@_C3 zAnQ(tbO8KwJ>ExL+Wv*G3|ybbG!RK&?8TC`u`!&m*%}};I|fm;fPDYj2w~Gxk;o{B zCRjI@Udd~$9ej(;S(T>V8mp$>VT{XYtGeypK@)lIos|<|T^fqwaH%)%o900nz=ppA zu$8e#+>j2_Y53T`^nml-9yzq1MqlHKKuc>l5!UB{va;I>jt#(NFK>Z+qQ{$K$CoT1 z0R*wEwDb2jj}m=ETiO1`?H)75st;_z?^-6P1tIb!uSaE~cf1XV3gL(^lGE_A^<^hDHR zMemkKE?KNa%=-8!zCA@}!gvRl^cMGO%4lU^oe6`MCM%5o-o6;`pvUhTrNc)L@!SjW zI7sV{npG4U&K>&GcCF6eQwKS}I#xHm7BWWN?p)5*ZO0iQu{o2g!%?vmB#W~q$%|o= zU6CR_FsD$IHbRT5m5aaPOs4!6UhL9g%ozKK;C~FAL~dQJb8Gaw`x=T}JKex&FMz{m zDGg;%oKn}@F)XhX)s_^wxaRm9a1-#WY#0-`1ML-XOLnLP#7;w^QX7RNdRqCbbh_nO z2KDE+pjgQic2ANNp^&v_TtDB=bG3D$dZC&XOEvhEgv!A}kPX=98VY{pRD=<0E(X0b4k}-FCOzy+Si)!CGTq&@dDD$ zIR`0e4;vX-VGTi$f&7m*(~V(EXo=>7<00&zm6gHtOEP^r5xhSd>vtNO@)_HZRyVbL zB6d&Lj2tE;e3)miu*P*X)ezQ1c`$+cX^?8_&ZuksjhOsHcOh1$(TxvwR5a)cui1n5 z?JreSYo#$No9no`3@mnam7@uycDg%7p8D;ek{Iw%XplSB|`3VJvx`5{CiUX0dLU=g2ZOYNP|S*<)x_O z!I`WIsZ;3W$C0wj)m?yT-uu|Xc{P=R2nE1%wv$4NwsaPK0<~X}W4* zyp$k3aFEvH-kvUM{<0noj>?^8kZD|UDK_IOCaoK39(~EWWOY;DvtgKpg-+feO2@|; zWzy8!i^WdJCNYEeeGFV??4&Z%g6ZKuGBj30zxmPgc|niqb$Ssi!_2ZDAx->Fjt8Ij zWJ`(6{bGC%vbuzqm52D--@!(7{e!j)qNClU=%_65>6k|p)UE`>^{+y37=1Wq-$>` z!dFI;OYz%nGd*Ps8SkkvrlxH+?x&ROxRVK}bbUW}eN^pvLaYh1nz^L}TMRM9)Smix z^fE76eUy9C=Jo2zBHH1P)xO3v<<y%R^eu7g4@rJx`uwv^MSG8CR~B7E|` ziYm`^rbCcnU_>9pox$P!Y0fRN2powC_g1b-QUstaGM~%-p}P%d&kx5e2dp(dkuwip zaCb@;^$%oI#Rr;Nw+>Qo-L|DU$JQ8|?O$c##BzA2I!bf)vTC~~?vAa^$|!|_xSYl} zOAt_a)^zYZtmXu(`Zth>DXPCoi3%LQB4S#x9g_c}9!{HQ&&*t7jK3dAg+D+=BX`-k zlCgO9+I~Gveg=p8D8KxIGkzvyvWkQaN2GH;!y!J4@%FtLEQ@0xEjnp2p4`}(P)KKw ziL~?X?|U72Ynf>2a2H?q_^KY8d*-tBPcasSOsDquKF1|wBGi^5{*M<=CGf_YW{r%m_h^PVV6b@Aq% zzk`yTU-3UR&l*J$8ljI}0^5IWwkB~r%^R^1xt#iev_)>QMxU=MRExweW7C*JvI#gv zymPJNQ4$-G(IamE&f3ZJ4Bqu+Vh!T`H;aAI0Y7Hm?tI>z(o+DRn*hw|A4xK3uhB5z z3b1>5Jp_s7|2~3-wfREb+&J2_PNV}v&GknUOtaW zifCMslMwq?LJO}n7b?E#7%V<)=fXbhHIxwjA)teSA_3ZZxt1yo`Yi zjYg3iy|E~QN9bkX43)@);6aM)!>;W3!Iis-??=plLZdW&YhYth$n&{NqBl&z3U8ud z)#JKf?QFeG#C_flF{LO%;swm`NCpM((PcZi_RbY!*NCEg3%w)%klw3|A0(YhdS16n zpqf4YshWR#Dz_hfTQy4jkkbk5S^nc_%j>||@fL~)ty49`&?Fp}`#u3AkA8;q!^7;l z4@P%i&9-|V%lRG4GyCv(q-2`qp8KABSzwAC{D&!KdH(yhe_axnFtq`n0P>5>z7#X> zTu~$)h85dVcpZbu=IoNMz1$U4sZ4R<;80(%T_aYbRrlEcGDgCa{b;WfZ&wq6A@VBU z7sz02Oh`a!JeybEs)%jXMU+}|wXN6cJFuWJ^Tw{BwynB(GCex;CR8Zk3#<|74vu)P z`yJM~|8?BMM@fT|<|^9Oj%P|vk=E#TMaw_9P76UvDEw7CT8C(^Nc7DqKNm}mCj5BO zqkTTey9J^@l85sd+|QC!1k&1#bI_WAtW@e_u3nb>H&j0T#^&gw46cB+2e1B{BQQ#e-n}e&BtZq9{nqN{g<&yZ~@?UKYji~=58donL$p*r| z>Io&|0FnI%fu`>OP5;%fdkw7Nm;X0S|2IwlCr$s~EV{^%*WH>$;Fl39$>1lu+qw)` z?H{X@Eh9yvE){0Us3%+Eq|aaWQ&rW3zQ*Mw6&UN6lv+QcLy?Qap7}e2<5H(QH{A`GbR1kqQ?sCUox%1tHM5 z;qQ^KPa@-nV(8%O8B*fzxQu|F3)?7;VHnROg9dM36(y^n&xIl z#3Ewr4xpRO3HW&WZI_M&4}RY1{pho?H6#VjWg7)6$2dD_MNRF@loS{FKeSpC6#Jx2 z94pLrMH7UVw>B5rmlTJE+uNWv3H>haSS{Fdc<%pjxU6|LQ+{`wvO$GSW=JHDc`3%t zMBm-rsN<4|?3<8w{+Ev^RD4shImT|~#zcrq^w9lX7nN<`L-CC^UD4vs!z+DZzFR$C zuS}?c>u!zgDho%0tcJvQW*qRn&%k9KJ7HD%W}*4{)2!W58Sh4AA1$5c{lL1xVK{f2 zqFL0D$JD`352T%T3!Qa^X$V~H(}k!~>*H2rKp7NVf!Uq_p#Xa#Oe^1}`Pg?WCwAlE zwYr8N1Dm2J)b}Rv$HK3i7q@6a(r;r{6QNO-RiXEHU9&|#A!5=1#lP~pG7R8DNRNx< zK9$%!n=#FX08EGHexct_IVK9okNxqmiY?qGiVoQ&{lZ>^-0XQ;wR^u7!}g<{3SyhA z?lUDTVKFzOC(~rtS85<`l9G zW3&}Rx4MmG-{TJ6IP(!9Ds<0=;`vM+tqo*7l=kNb9L*}ff^ zFD=LCg833*#hSdIylPN0b_#aZSFhG|tG3;%?zS(V5|jg{y|ZVdf6DOUg#go3()DOd zDawwaI>4O%4?fuH**y$A8$-;pQP1wkPRa8C128b8wLJ})T5kV+vz(ppG5rT zi6e@BShY-+w=Yr|-h=K7hK&|b>TNV07e+%k3hM)9BEOx~&V2vp&aOX4>yYNbG8r1g zY+fdlv$A&V85G|2#yu`O1Dwo7VK6^+bnE$VOh<8MX7kGYSM0~mYa4SGt3=qFdPEb=gkw? zXR{{XyxBp?q^Rm&-boWk1r9~SasEEynI7q?tzuxkvF$Q>MyXlc0=`^ioVY~8s3^Lx zZ0#EW0KsEZRNnnRgzhP%__P2hc-C0@z8(SXktO-U^t!QS}pUTm|tT?p;Yk_*yiH@;!RYK8;f z9%a#ts~LGdwrZ;J@BdV^=c=CwNl}FkuNAa_Vi7a**)RmZ5e~n5NUGY+P1RAbf< zfMl0beOd)nNKqj26s(zD(fA$Cz0u&gEn59)i4#K};%Vs#`xx;Qm^K)qt~sB_(-Ck_ zODZe##D4Uufrvr;76!Ksq43Sav82x1rrh~p`uX0|*tW*&Ozen#HL^z(Jzo#<;mG!U z>f@ajEE$@*k_9+;IACajSQ_U8ZoECM#z<0zZ@EXn?GWW|KAH>dloKYCc;V$^(S$>G z^S_^8-Or666x{gii9npm)O&|KCk?;TQ)C!Tz2Z1N)_6;8i_jsH4JS`xLP5q9o4jBz zzjZ2U5|pwBjd*(v0A_>Mp*`rwdx`>={Xedc0p{waZLe49N?_D>j>BN3wol}b5QTk` z?u2{_`=eb9e#l(;igxhlbUxR&I_UGP37i$mi}of48>fNNnxbYyPOmy(#-_`p1dR<| z=L>!B;NNLro4D#Cn`2_Wn{$TjVC8?{3S`V*)4aVs(C{0|6PmM0~ZN9ugxNm%f`0Ks|ddRFe7PgWb%| zE_^CIpg$p68W8AZUxcUc_{HR|9=5HTZp9gSXWhm6Q9am6C9CotctmPQ@-Np6MK3{w z*SH3ae|iML`U}_bh^Yeq2rL}aC6PlabsR&t)OwdSW?9#>>)e+Pjm&7bvw%0}ADjy9 zd~QX68XDB-M%+C&AMSe?WaU`Vsig~zCM*;*UtQOYCZ;f>8f|UDDRDznk$j2px-8Vo zR;P^ek-Lu%ymj;~9auT&mP6h@#DV{EwHPcH;eC&XM|HQbKskJCzz z4`fi-aB&!h$UM^oD9H$;e0S(wHI5MFaK+{k7cn*3q&8t>VbdCX8?6SKvz@igfq;o8_&N8aRgKFSgem|3E4M=6lt<3==n9YkBCpzwT*8^L_T3?+K?gxV#I;U@h#{Aw} zS}`1O1^^)DS@&yBMxfBswh8tz^@oUB5nm%(J>R{p<~l2t4+0CUm2=08Jvfx0_OW&H z6Z1DL@2u2J?PWtLtj_RP#>C=|8>TlaYw8 zCVQ_yP&ihsOdMVpW8a8?m5#!ev}#B{0s}^}8v=0I++*5@=e-PItD9usnD#UU5yP(M zOl_qUU5G+XClY)%BtvdrdsjD z*mLJLc^ZXqhnX*Q4|-J7hG(HC20rNbl5i@~fs_Z8WdFm}bm>IlW-WOg2 z`bBM-80T+5a7eqaUcE~uPQr9on{|s*Lc0Fesp*LeDv=~~PNcVf%)axh2@*hjiNl!r zg+2ULwq{`}I)FN}hu!itCqTjR37}`w9>24JUjF^{;8kSun9P=^yyAOfT1A9HM^p_l zC{};#+dbArwWMAUmR(g0adfc8{IOZ$ko^l-jaMqfs8`kPOjo$@C27gauB`hPgL5Sw zi9FJ}yp0mt3tixCUS4__xz{3ZUT-umw2vX@!Xmk%9zJBAP%;xbJGX9h0p&igK7RUs z;%i=eFo(hpyM9aM>$TfoT|OvyUrwCg2mCA~bnz+RDg)=-Hv!>xHD#;h<7Pu9z_~HN zeeP7n?am4|r-5gC2h_91h8AplquC|s@yKSu z-|pD~92ERpvPermg0a{4ed#z25-G*0^j(qvZTmn#cE|Dt<-kVt#{c-qMFXp@Q><<1 zvx$Aa(raT>LjM?^;}9*~U#Rd{!x65t{MZXm3X*~Ar!N!1U7nKX1 z58a);F8tc<4&$rOTT5SwVxNaQ1OpRViW|~=A=oJp61rV9KXR(^2(98$l~bcd$>^Pi z_5mL-bYz)2wWw4QN29QPteq04J!`_K(iZ|YHI3@T|8*2Y=4B%1hWtizwQc=bfEZv=1|dda=fILBbIfzv^qDw6^;eB?Bpzedk4Eh?2d-qAy1n*YJD z9v=jNpup31&}`#H$SikcX+sWQXuR5Z5qfpCnj+ZW0+fc=*^L*Wvx&Za78_DEaL;`G zMd;q~vAp~Z87TTCcl||3$gRuMju1M50D%{(-s6=eU zdHs{6m%hXeDZO3v{j{lBW37ILSy7?6$87%NxZ=l~2JvJ=%b$|(a&cWcU-c};q4i2u zjOMn+bLacbF|)^TV(8_F;UPh&LGXMAJWk-Bvr_}rQs&B5j5Wi(%2uWExZt@WZwa=G zr5~E)amUpw0@CbKd6z#{9Hh7Y*_Ry~-yg1Sq`lyTTH*Q#qUi5Y1ZwSm^l~kyDwk~6 zXwif(@1@#N5bIyOi`!Vy7w3(tFVv2zj0#-^POqNaTI0ROObqCcg>^*}B&MQhgiS!E zOVT`6wH!12b2TmP-2k<=OiJ)&N-*iQg=t9G&Dt8J+y!ISV_@_Iud#nv(byU z=arYPn8jYzD=?h`?3n^lblyqA1x{_;+i3_e#qNXYvth~3cZ@MtfrbczfE}RrM;% z5bt=}WmcsIm%S`Mn(6w{$jAGBZBT1vK=bmABc5gn$Cb}B=c-_O%2xXt&__U5yO3PY zQ_{X`_O8P{Z>nqvFZ;#N zw&!N|eJQsqbiy$0C@JUiB0hT@t__F!ewQ(m&y1;H%8rJosty1almZYH0PJeudBumR zZQ$F7Jd)w{v2J#YCOx4qpKR1#Pzl8ub9HPV^yDQwi|BlWOaS9a6G#X{=_C#SVx zO&plW)~j(D5p%`g0=aIXo*(f&bl~d1#b|;+4K$u)TL0?!b{ttN4r8BQb1Yg8p3drh zUK~?{v>2bwGe;}{2Ouj6G*tux(&+3II59%fD2-^C3+LUFqRh^3@S;D~m$tE>zW&Nt z{_{CVFZ^0_Vj`sXm zAYqv72yZHDE*wAaSI))eGknH8d`IjmSlL=ZwaV2Ilg=cN9^$FPN}6*5ah1kOr{5ZD zAm0mHW;iFCB0ar=-KT$i*qG-e7$b9C;6EJu(qN+@p=^yg++nncXa$mlF%O7zg1$8jWb>26$e4lju|$8yad|e+-Maj9QP~B9FQKi*8hiE>*;hFnx`M#yzr3Uwnn`&q!AR>WM|kBNd`U61&O~oN0TN4u z<0GfWg_qruHw-mFS{=~;yY?a4{;?{LN}mFDjtnhHq@FKVa>Tf_@A)BJQlmRTrX>Uh zbMjrA7MWwctEDbMmP5^$kgAg!M|}`%&BldraVf9V=O&Pj`%-ifC&zc98i{_RfJehF1 zlBVBKyGAf{Y%jbMUYuGz=A=H@)g4v8cmU!jVmw#6b$!+<^GCQFxN#a2bHwi6tfKY3 zG>^3E2HGI>+lM>$&(e;#d80vB>RSAcuy;rziF!k~h9-)iWbw;I_|L=0C2M8y^@h$$ z{&V>qn~t&^{PPKE_QQu)4od^X3ZP1?uYI>zQQ1g>e&rM3?4D7l9^a8uZzsasa?rQv zlXXNcCTzu&-|Jvnuukn+htAUxf=~ot!1o>Z&7WD00a*prGKNQ{WcYWyZhkIM*0rZl zE(O{u0-EHJ~7z?>d`=UVp2m|AaF;x2fAN zT`%R))tGD7Dn+>sEWQ}KM}(Dv^Vw54q5&XKPl=Xw(i4_tt%{d&*XscK%^@Q>U zrP&|WAW6^dXfW@Bv{;iazez*eK*!)zr*F5&8kc<1H2EBCa?Zp2uF0=HC;p*SH++&; zH$S;l%F)z!NGV^r#GB})1<)!r$&uSt!i7EW>OuDV8`u?FSrt}^iZVW_XNIj>pyKi2 z%cyd5OR{E;{TOGR#zVV*s+A63Y{~`pY{BOkm4=yO{*~SJXEGAAcx$#7B0d*3bcYM0 z$H!5yCMQ(r^efJ6DF@{?RXDa{cUi{onxA5(C~?CIJ!|xn%7n3q9-pj}n}to5LDG^U}pq)4junFvZn#rwH>s5)-Qs!mw#_)JnNI|UhNHk zu6R|W(LqD=4l&zP=95!lb771P@Czt{D1!?Z(edzgpC2cle7_Ia@u;E!w_gL#7t>m) z4LknEJ_WHt^r&8sZ3W$E`1Ep-;stt*dv!hkp-lS@^AP0p4=mz-U9o81Dz^Z!sH7Qg z$eYDR^zJW0$mE6WdO5Q-*`%)=v3I@I&mX;U@;|w1S`Dp`3?jxCUg0wq46ATKU@U3P zLXP9vpHxA6JFqkz7SOiR<6~DfQFoSnSEQtx21HEEdxN z!g8x~CwEw6UJwz^=*}m07K#zWS=e?IcgQ)-oDl1+QSSQHYMoVw+%`sUW~OYV0o3xM z9aa~lre}k3F;)b%>60HkoR-T5aZ42ugN1d*ck6Gg_xZc-#_F)SFJ6%5I~(?m#dDYk zrcnIpte0iOydQ5hKA_LI+C8_h*q+f-I;I>Vwf@~(iJ#wXVT=`ebnCJ>9j#QyGs1yYGf#(tDorO$hibtyvy@q^_{Xv*l zI`{hrKg1;1b>?%1q8;s*N8I3Rz|m8{^E-H>x09n)Mg=xq*yDs4J;EktuOTQIH29GEo{T{^+bF4<{E>>4(F*sgmUHt4hYX3uV4ddg;QUNV*~7GbP{81V~9b=>il zwEGuHn~|+z>)p)AZFIBEGTr>j%ZtN4;MM*JWZ~G0UZ4Egxmx2l&4v3$O`)M4J`xV= zAH0g?@PvL2E$md9-w15ygqV3^E|=l2xa2FGFEE;Xx=;7=HLx! zP5h|x8&+ND<%XK;p}M9+{D&)gPyC>ZnC*b?nFw0EWFw47gg9J} z+IGn4O09nHuVN`Mu12ZK38B^OC3tzg&1@m=nMHRJ*$NSXsjc5^Qy!+Zy^3^|YLK}? zI$0k@kT(;FfbUH1`LWosxC?M~IuW4tKoLpVHAGud(Ok!`FZh=)1|Q43#7N%*4n`Q+ zjF5L6I+p9abfaN_$!e@0-~1(XDByN7ajpn_O}7*U4i^XftD;C{E5o&yon=YR)5#zG z6Wa^-y}f_q+zC+5rCgGP(7iUwtRS4neUA+5gqW)gVGSJlH;>P66YST!3ENcdGfFn7 z>$45fqsr*5usd)t;8Hx72^qdCwjXirozbHO_}ORT>L}=ZgdL!LrD_T?XxOrmH9k z3*9sR6{Mg)W}Duii)$yD^>?C7uyNy<$ap#_`bN? zd;2sC>>N4NbR#*+GBEFYkz2AFKmn+4Q{7H6-aSzJvD8Jl5cP)WF~?=jHK|(8FPnEr znpsSg+4Bl_=)P&JJ5=|81`#?t+UZVM-|K!P)1t9ln#d*lVIL0r^Upqxl-%;gU`$Va z2FjZ}SG6b88Jcq8L>b2=zAg9on0tv>5;249 zN7PD=iBQJVj(`e<+FdiP#ZR1N<63>?kJZjul^y|Pf#^nS*p3+As~4{UF*J;LQcAo2Ri&W>eVRmY)}JRl|uZA`y& zysJRH=`^iRMNS2w?fCGh+@lu}xoRnH3XY_@N3ZOcU-Y!08IaW)G<~NJdT^i%C6oI zQ|MxC6`m^D2WpeL0BWOM5bl#q0-H*{sN1R`Y&Aa}``)OO;U zC_0EVMT@-f6zK}wWX@*B;@w)EdH5+uClOFv)8-v+;sel{+AeZ3yo&!bY5=mUsu(*Y z#KJ#~30wR4?Hqe_Qlzc8b-81(FO0Sl9l*edk*eg@Na-Fh#7LYNX&?t+1}P~ApH5=r z<>1=~V{}29QV~{y0DK8zbx#dr=A7CC0*LHpi#G{S@bli92^%Bq;Y#}GL8wU4Na$*6 z`qZ}&P+V%Ho?N*|go&GU{qZh>n>O={rE1OE@zk%y`49tO+FL@9R(aCAFoa!l7aF?w z5#P}%hl;;NTzkZB+IGbx-P|%{i+kj0b*c)s{ErW*fsx;nIOW;AQ&~xBA-b^h(xYHy z5na@D%@+6%cG4&|He?s<+^AI_y6*$|D6sYIu1?Ro)?UWuVq3V1ZfPz$Mq#jNQ?db*{Z)!^_H)OsXkL(+yUj7wsA9Yb zUnI^#Bo}TqwZu3)8QM3a-u1cf^sUg}WVs!8YrRe8%JU5X>F8IZz> zHp#nSv(D$c*$xthhEFH`0=wBLud#CBaj=U)ksiX|B$$>S|8Z&aR6FdUzpHAd$~dFx4l0b-c{e;&K4I>O}}C~l97Qr{AfwBsl4yl!*UBZxTI5x z2J*)bX44ueW>VMzW2Xyy%|5v|L-=+7l|^Uo%-~+)?7)@4KYN}=LgZBQa%b;&7-zg_ zzLGxh`3}WMQsat=$Q~dxwfwaQCN7n*T<{U5CAC*vMjbczxd|0dkuMQj%S7{Lw&%J8 z`ABpM9iP)Xf;~t8Hqd}%71I6izJLI$seV}PWs{+2c`^zH%rcmEUJrGfY66`mlMU}^`}ANZNF@cj@FpwE`Nf53PD=zz6v zvLOmLz=vf4r+wL!s&MF#@3Po1_sNKEQ}NBzP~eJ&h#9rR`K+t%VK$=jkw!Jcsb^H$ zR~D8Q_f@r>vN(U)`Sqp3mJHM6Rt!vgF=J+;mKkcD8?LQn#-QE8(r~LgY)~$k!c%fq zBA|}&(EO>XXUg6!@;=`1_%=#Gl`%qG_6-Ay#%KAV^gK$aZ*Br~)AOSzA5M!%cPa!x0)oOyjUWM)ojMpb$7{Q4pgP|q z1}c3{Zhq^ZGt>*c68pNe;u(DETm2~O{Hj);R`pS0okgPKb)spaYTeY>X26gX0eQf^Z8*NWQi=zpU@D-L? z*aNxuh4OK^K8*XriM_mVVczURM}*anwcH6d{+a|6UKoay2*TP_b}OcahUK%f^tHJ4 z;JLRE)rBC~lkJg5jy~#_TmJC@kDHpu0|`2rd&)114cPuh#XK+Xp%yH1efqiIy|*XJ zgI$8I%@Z|2a&GDw%9g*$YDBL+6b3TF3|>$}ipJ}ZR?^Zo?;VjC-0O#~nySzYk<@ zoPWKu`S=$g@croGAAk&;y&#yBpRFD@I}ha1Cjdx*hy}9Cw+Z${ZYu~R#b6`9e#_OL zeVwHXn!G7+`gQ?Z0k4(?e+Nn(&H`Sq>@^^^JtJuHFwhlHs3`E9q&M=;HLSuSz+OKa zG5)w(3KUuV_Vgs+jNN-8NLvt$3n*tH*etMVMcDum9cD$Rw@|U#g7^PUkY*a+AyNLy z@JF`8%da8k`D_nxDFjY6&!D{U}(|O-x+&-ZD|CCWrcy2XTFzVyJ&? z#>|kWul;3w+?LhSr=Lhq|0dCXR$p=6tfXaLVVUlX_Ay0QWpxK19$_=C=^0KiMrmvz z2Covh(i9VphmjY7bDYh_7h?F{Ak#ozlJ&4u$43b?QOn&A2Obj>P$M2iK-}aG?wnY> z2u#|4s#mBK?$s2|V_6K4{)G z=Wdd8=cKc#mW`ms0$4z^c9|2avZ0%8`PqNZRmWEHtEJ~NGcVGYmNR3B6fZh%Zs&k) zFF$PNf=qno;JAxPdP;g;b=;*uI|IfHiJZYGoI?brhHqmpxIbCY`hToZL6ab$L_M_{&5I!%H*jK~ z?XhFp0TW&k*RtTN{#6z=KFf!x-B55x>->qT;VXtqx!7BL{p>3aYiGINYn2mF1nCPIGDM074RgGc5lmv$Mf!RBtg{jq#z$4}src+Qlu z68)>}r_!PYem0PbybiS1=3}Xni}c9($uS^Avlxk7>G5`dH%6NyRTHE>jfLzcGiE}1 zD1T-ulN(I5Y2WUA4ku@HmzT&~WLAFo346X3i?y*#nZo}8fUa`LZ{|^novaVJY(Laj z+2jaLRM;cq-n~)T-Hq$iT-1{Q<>{*C5DKY!T24pgNi_Ikn6cU7nkTA{XUl;=M{6Fz+V^=QNgC zQhNG7Y;cWnzIq{%_TA+~EioNfFys5SLv@=&NliyectM(ioMr&Tf{LxM(l@w~xsMy}-x|{gj1j_#d2eC?wS?rN&Jxs2j+N4IP z7@gRIUZj`XN01jgO6xE`8RS*Dh`0yvXfF>4WB+HgQ{ij9EejmL{^H>jI*~rro}Z=Z z-X*)I35~0^pSYu%t9eG^{wfM4!Mwov4uJO5KupFKYSczx=+AnK)DQhvYBV0 zx}rr>HvEq}kkWsL%+@@~sgU8>-46V%83!{HQ1CIlJKf;@t(xM{2&Il><%Rc&nqPrk zZ(H}>l_^dk^EJU!Cc(E`wgZLz0r_=DrDX(aIV0P~5uG@=xrqPn<&RcD!P`%u9CH(x z?ze!>1)5VM!0(>)-i2&onMB}*!Pw5-PC$X8u8&B%jrco@Y=(ywPcjMVoasuu1@u#@ zE*HYd7O$JfwWR(OyDVrC|I4NS0oAhR`zgjpy^&4aQ@^*N z)Sny>(tJ%Tcq#@yvjidg!F`r>mb3sc<1JXTbj;3eXvHxbfQ*BGkM}`Gd4VrOxkD!Q zb#m4r947>Y6wtJ*+BMnIuX>bRL?SEEC70>gi21%&J!G_0%|70ZPFV@&32+h~usPPn zE$6_XF+hs(UrB*h$qN!#fohKRazV!{cUt}6?!{TaDZ`R`7w!vZsr$ubULuQ#1ikMk zDAZbyu$I)*Jp%!1!agsTgGENyMKmM|v8%*)a>b!i-1Yk4@@8 zt>?N4P;J9M$7b36mk9x_z9ulN{s4Xl3i^V=%QCAeBY}%k3H20XzvSlT#cs!NIxIin z+4wzqaUjkiP;8!lY*rl8x%_`n zpv>Pw#y#h;{8Mmzs9pl?8v%BNm|X-88bQom3_Mo1@{CPSX2b@~YZ2nW#U@}0O#E2M zniCW-BR4vDOcmcn#kwuhEB~(UuvTx?=~1L^KeuwwF{GR`M&!neH}7k7o@jI^pM-CQ zL{KI~ndPs>ONVMQ_M@yd=Nz0i{bQDvbX~cTKf?3=rlhe}kH`IS3LMf$Ihq`HwUI-z z@A4S?hz6%#kf<@$96dKDLyR3=Aa}7kp@E8ed}V>{CorAti}tv+1FGnq_?+2fC}{x2#F_@-wic%3#+Q zPU9HxdrO{Fpu~?~PXqf(f1Q0cs_wt$ro&rCV66c~Hyf%^Z*|OPOurL2&1sid`;Q;q zjZHN^<=3LXkqkV{tu-hOP$}IyPAeImA>9-m>gqY)@A%lCYIod)EBrqgkb}60m|m&+ zV+fUYqWo9^XLS&(fh2Z+O3w$!f#+?am*F#6Uf`GcV`bpvAbub`pS7CRag4{gys@Tf z8Usj$M%3Sd;Gkj1w_vv9fw_k8Z#><_aTX6VR5Nn?dDgXv{CC!RNVn1O_rk*gS={a# z;Jk~;wao9!74r#I;Ev?#rx1ww2I7VSnm39r4{-ZGTJuahm)5e5<%Kx$aw)R>QUniGti$1!T#o}b^tbB_fVtr8@i}Rko z@{%dD6|=ax43EyNm<#F59n1_Zk*X<}qr|C&^L?FAOI6p7x6N>dlUw#4~U9Kr;rTURZT&xE3arON|t0ZU$~t1UA| z@0ngTiT>L5Cj>G-t(J=a%i8$Y=Jlz(n!?PfNI4yNz*Q&K6QsN{OFu~jZZqCqUk`1Z zG=a4SPh^PKOrl!Z_D<>Cvq8e;#G3l5R^p?dJ^ZhJ7z2e~#9P-s9KiZ%pVbfKm*h{4 z(YgM?;cY1Av739)v0{az)zJ~Y{dJI^%7O_dch(z^VT^u?ZuRWypMZBI_E)y2((eD& zrUTq7sfMB5j>*kS5O8UoqaJS|)}4{U9?-FCfyBtvSa6N4O*Yw$5Hqoplo{_4a>~SSeJhM_fJw4F^frzkr z;+|hV!e%F3WiNCniUp4OT1Tj)&6UB|9J;t~@E-ob9T}%f<#lDcWyX_H`JAw}VRIgv zqF(4ifMf?8?*NBU7uaMvfB9<7#45Nfvtc5n=fHI*6fw14kB6zs2+0<&v55GCkIJq=Mv;{+3wbI<5Ar)J>A>$*xY=1{oV{!+dm9t)O1S++gGl{ zVQx`iCmS*wd;$<{Eu-{>rP`dKz;00WAVob)>gF72xz=TlL}KF zjl(SdEuAn<_spTkHO3ZO7<8_c((B-{ty_1{OT$f?J~DKDebJ1Okp!YlgX13|HR;tq zR(Rz7Z#a5xrFE((J$YjmwcM5Neu_N_I>6V#Vd={VKBL1TtCSNKjddnb2h&>)ph?K0 zQnJhl90?$v#GR65YI2@(#gXfc(m?f((`l)IKPQf#ZIMC+qQ#QdMmA6()qYR?VOc0kgkaqkK0Rjj{q4msDaEa zw^DW?8-xDMsS6cTHmJO(Gtoil8K}=l$$S~ttYz_P_3&c)lRT(&A*|KGBS^X^~Q@sKw$equ<;H6|%}Hf1nW_(`dtFiRpo&2+T;AN-WHMzL~x% zH&jVI6?pg4QB(wp>C?#=tn;jFo>R~xXIQ6t1gF-G>!COcNyHk52;?ANGPEQTIB#R< z*c>;g;<~Y1pIq<3z%A6TX0_*~x;oTS|43T?x=?OaG>umReGmI+o0V_h(L8o_%WZZ9 zP&bImH*YKU$D*IZWhxyqzOdFFRmk_z-ad_|tmTttPc6^~&6_KF}%$r zzI2V%$_+bC4}muxBC(S}R_JQ{1&rN}2}o}<@||G^|JdLp(GA#Ixv&!G<%03t-gW}M zC;#!=q@=tPg!4hFAfw86{gk$gt@|_bNBdm|L<<_t2s6-I2smQsouU%6p|U^+WqGWg z`!*?ORmW|K=kwGvy0`v@)D?>p&W=NAiIyDouF6^SLsh!fmeL!r*ba$2##8LxT86MfLc~`c8&pE4evqYjrWO`86H5pa-TNZ zI5OQ+5E0`G;CSMXC(*8V?+#=P>``?~FoI8)e?Ve0_%&If-H#LhYD1B_vLl;4!3`o( znB(#CUBKx-o_Qr_n2$3QE&AZFLNfGl1GgZD9eyB6KhvPpzSf4B9C_)c)8S!i*hi1N z$0Ov&->+Uui$#@c^F6QDds-b=x0bMYRZGrcRi7!5yT!$8hMCrip$Wg1ni7e950`Z% zO{?)H6ueb5gr#=6+1o5?m3+O%RO0{9^_Ed>wokt(Qlv5oF)e+>})!T)E_h0Sqp z_>+XB#K(7~HX_!X=km6xq>(kwzW$<-#^iLYo=txyiVPFOWfx+keGN_V|z zHTF&98n!=FH9MUimsq$^z>xXBejrc685ms7ZnK)0sd@6oGX`7#sOq~jd90z}jfs)tYPMBdgKZy#jz2x7D+loo7i#ANIY%AhlFX)@ zxoN)a;5WDA*Br~$F&}2GJ%l8MU+InfX%L%`t*3EmFK0G)_XQRq~-{?>b*a5Tn|)mFvCVE0NM9TbXTz&}Oe%$EmYFQTA@~ zg_+aU6GNyGe{4Jt_m6Ee)U9fM>8q#ikYbzrhdac1(!zM{QT>tE-Q8(Vp@^?z(^tdt z6T=c>T~=w`jr`b)q1S<>0{0{8QbwytWJ80BTf~)%!b0z_4(9`w~qp!WhK}lDsx+TEA1r( zyVgRoZZ>^1ijce>z3mI0IQZ=3_>af7kxJcRMJX*@awS=*fSa=xVVZKoZTbdZuR4%h zMF3psh9W+2)3i?9;pTCgO3ZsQAwjFVT9}93#q3D*+&768@*sXIuC!U0f=DPJWg6DT zcQsKT$d`cE&>c11`{Ipq_}(+VE+WE-$&z~K-`Zy4-J0c4BKP)gL#q2;a^QU9JYglK zK=VO_)2)3!vT5ZV5N4TO`B+>Y=S}&a47YCOT$M^|Gi^w2Y+o_4VD-fsPy=IlyOiTz z_eny+vVt%CLEzZHd+F^$Mt#z*mxP4iE`WY2V>m3i)@B|!N}SQJ^oaVI(ZL1HQP zgUREb7l&_=C@)WY@0v%!nrp%SC^w8lSx3yo6$K~!zvjiD!f3O^-=UP{rDfr;M@e+k zvDVZvr$G30Mp~koi&h~N^=WFs`Z)~CqKO?TU)UaCVik6tQT8)$<<~Ky8d;O|lda;h z#p}pBC4eAess^Qjd7>z(pRQPz9wOAV>K5gnULcY z5D|O287CfQ@wZE&AaF-n>;YM>Iw0YeSC>N%Zd}J(If;YCHc(We!)#=(s_r9JggLo~ zmP$Cu6B1Y2!;Lo8p_!zW9WP@Ea^GtPuUdl!(8d3P|KW>6l!<{ajiyFKH z%Ms5`^y1T0&$pF}gtmsV5BGIGjM4r$IT?c1?X?`*<M5V8ru4Q;@SuD-B1OD9V`3pP+T|BE{Lx_| zZX2ukvF$>w)f9K@QRi;BP0j5&+$J;+2uL>R2Am|D1g7V$8D{enu;ULh3u(VLX3O7; zJ5L%AFNG5W-L!aRFBa&-8v)LVHpK%E%zvpwGF_Z5N`Tcgw+-si<0k5_5cI6uI==PY z5D|*>ni4I1Ru~cHz2&3j-}uplr(@pjzX8DH$0_H!5kg%7)S_J;Gq=g>JHG;SSZ@?1 ztka@nPimVggicW<#|xi)Y{zMR=Lm-mN`HTi>+OE_=GK1Q8xmmS#i_B} zeWcS`3SM|0!;KufSaH=&*sfW7m@qTW?{VNFq^RtT3GXaACAXgndGgyD)lZnxJ z;FmI2h+2$@g!5jOTH0C8#-&1^-spzPO@W3A>Q?+!_==SC6>x}h{XJG>g6~$;))gdF z>06L^n{*3G<^;Mm8@9{~16X)5&%kIw?!p$d4ams#f==ZrRefMVF*`N-4ts)CYB9rp zrXU5@VwZFUYaf2FQ~|se{UhOSX>(bW^WDwFospc#g?Ik5b!o#4$AuP)e8;Ot)H)`T{T^GauMhDr!J zg?gKMqoy>v4~GN6|DLS)pvbKAsKBgu(E-tGX@Vb_Dc7B~;0p{Ut;Uy;i<|>^SR9}f zV^F-J&QutdFbqBCXW+%?qGWyF=+EeL0?&j!_usQT;_gL4(ZGgF9;(XQE6)VW`s5o5{_#!1{L}#;-!{>K9rN}wx_2H)uxb}263D&=A{o0>MAm7$! z$zDc>4N(dyhwv5FI1y^QU4MN3VUU?Y=)GDLodSSo8XLgX-etDrbO2Bj?_m_k^{U{i zD$(D(=J6vtf!DX5HtuwvZaCZh0q2Yi<+N zP0LS8+0lkD&~Qqxc6rt8BxV{}_XXZrmD?QOEQmgHS;>2IxR5B>AUoyNjs(@uuRIp{U zO|!k6aXmOecxI8Vt$DF~w|E0?6Bg<_FIe|W_H=)wUomN2Mz&O0Va|M|qL8y{DXqU~ z);s(Q+#i}D-iptiSKM^JY5u#@16KxZ9dlJK`sZhwkfPn)SQaVc!Y40f-?3I!X=QBy-Qknbk8mX8@7{_N;9;`y= zdTbMq$p}4nH1_5X-|1J`kOre1@_%_l1QJHf0waKb>(E|>_2X^Y5-qTu*A4t={duG9 z<|A5r+-p+LbMoD(S1)riAIoUv4zf30L*Yddd*eYRKAoWKv2;hA@qW-O0S0gzdKC)3 z!{zxr+7PjhYZq0udusa<25g$XeeJ+DuQ4{2$}}qH-T1kY)<@58`H0HYOvU9h0-3ry zHVC<4z4P_h36&eh9*CX>7LhtS2RA2`LLyEjg*YbWTUz>}&Z z>~j`WsikBb5GtuJob`%HJBC0zr*KG)I?w~}>&>STji{>M*wmC!H;m6XdT!L~pFsmn z35Gx0eHASbDIs{F$MZZHcPxeJBM4JK7%h=7bC)L9yIk~h6?O{2^jk5;NA|tRBhv^E z!?#3B9v+~-z>VTd^AVj)dZ>xI><2=B#`n_?=)8kY3o~;7HFOfaIao(NRvJ+GZ59dC zLTjK?f^0kbRRs)vIE_aSefcDF-*3j2AZZ_s_`3j3{_lA_g|r*ApR8oux1CfQn`1i) z;jwIm=#0HgecWSKI>UM~D!~><3@;O^nIoMPui<+v+(XDoe|?|QK>rWrA)DeBwvT#e zw^MGEL#4KXIWUZQ83FRvn0i}16U+m_O8uj_^+?xNBGF^G@t~Er5e{sA=z&Wgf^!_O z4ZS=pwuOS*Sd)uBzIKie8z@<*6XF(bEp-vXdluPu%vA`jOusx3kqxG!8))}1HC>u) zwR9-Bj~^KK-QWizgB4{4yp-_2Ril&Qzxn)(VB%!VgbdIfsY57?WXooiMcP^c6tPX6qgO^DL1G*WAseu>xXk}e5%>drwRO$MwF;n6ikZ$RVFFH$tNx)6@ zu$k;WY;`BYR-5ZmIdV++-f3pNeZ8L`6zKl(q1F7~>a=&SeD%_zqZDGQu%@ zraZ~5NguG+v)xZvS2`hM;(yMuNv5$VueYdYWhow0EF?$T9vT zY_`6=RYwaCOL!e1Sekn4hq(MfNn}3v?Yv?2=4>?0ef-c5=d9asWVbN-Y1egh>b2mk z=HC;a*GV-WfSUkiNzdu5N%7Bnd3$ArN8y#sL14m$19e4AM9>Ap2C`B3-T}1+UPHs5 zGXcvbpNjNEU09m=Z=rjKq^BMyR44j#r z_|tKBoHGCwO|m4$pAD~X;wk|2%-`5$gZ5)#lhJsFU0z0%^ZgI#frUTxmBuR!@tZiQ z&Q|#vN^UWAkzbvfjCd~Q;U)0-sXJXDTyctBPOVrdfuy1OrNZSVCRBTWJ&Bw8Vc=F0 z8}^)zsR!Z3{=VN2TBa>UBD>}IOB^sRDCdS>%nG*hFD-bCKmWP+aT%XbCD=1YAhwcv zQy_hFU%7Q#(78Ys3=2dqha%owr6K~|x&?wIv{-Gt2;Ud*NsAg*q{WE1puFm6AKuDn zOHHGl=v9bidu(JSogl6 zRAjtpaOg?td9Jg)Hee8?GqwpVNvQN5Li>i6N#&}TV8QsEip^iEIj(>8eYV&!rL=Vl zy1i03^9VBO^qQw($w%#vd6LKG4SHAkjdfgt0~jWWF5-2d9>@AV;<3&Y98TOjJLaA# z%r%uu;=HFrGjWoH$h+;}k>I=$5f6Co#S`GAoXA!+jg6?oOGfng=$20t?t9%D8y-=7vkKB1CEZRvl`d}*AM?_T(w&-UEWh6Daw?az$wx$T3{@`gOt&WC19 z@{Wb<{$8R%PPrN-GKDFERSt2aSNN-_C}%_PA|jJ$za5FniJS< zif5lBr#WmPf9R>FiOJFmBul0za{^E4&}{c_r!LM@fv%so$-|7S%xa3S&gTF~;>03myEX1E^{7xTV zi#9&!;A@-up$(Oeh^47+7LIMz!K~syPv#U2FFii{HMb!IE(T-RybJ?A>vH69c7Xnt z5tYs<4@fwDoi4i#Th5e9vQVW`%WAn3JiNYPdVZ%X&jxNj!27|w$ktboyXZQ9wJ>@y z6|S^X(7(R>eNM}ISSU;5vIr#-_^##j zu3|Shv|^KJ*AkhUJukhpa09cmHg8s!e93aKj}k1uCs|I z3zhL``n{?CuT>%Mn>xmeu8pQOkE4cK+Z89gbXH58uL;JlrddM$uXlAa8^TIFpZ>5p zoU)BPH2SlZJ$e1YvJ%jqQiM$2Sg#)&lG}Q|^R)TTof<{S2Z0eb2w_6LqfU9lfm1tP zZaFUKis&3nZM@+?7$7OmzU>e)z=EqA0j8a%kS22kZ;NB)i~n$rJumNKYUNK$^lMFD zqY~Vx>;i;GK4H}@kSb&}8><+&AJ)!43+%ZQTGo?Is<&0+Y)IPsCTP00E}X=+XR>(7 zd+UFw&so%s#K;BEi^R&2GX(6l$%XH(v2kvDQ+O@%LwATFd6JitS_F_B-A&(0{S0cn zKi%D78WKfuoX)?B6|&NHJ^k&0$nsN{bA2L6NfdprO#y!T6lcO})g#wa1BbJCK>{!Y zQ8hovy18G>6Ni$kt2GhOi4Weqc3r!f&k7)lC#p23^lw!5Y+U@2AO-}b3kP0NM9IH^ z+a#BEm*`jW#1B{We|Wy})lS;jlECizBUVDVGnh`ffLq%}IV`c$usO$8LD6+G?~SFO zqC~9@%d5@nY{6&33pM5pJz|eBB89s6{WMzXTD7!f!YyhomN7uZ@6>CG^K$Tm6QBxb&Dhs&ro2|o z(3U{UkBY`}d~Y*L;|`*7bWU-ZTED-hpLK6RpF>p;m}f?6h8(Zdr$7^x!}0)Gg_R(|ZTT zb;FY^+AdNFP9xo(K{<-^;fur)XMO8r>#F(K<kw z2IIoyeN%y*K24)5q5qE0ViaX}H;D4My+{)IRF$cvDjfs{Y)3Ygc**MdfM+r{E+US3 zGCf7zd6igQLOpgSgK3W5G(TiMuGVSnhxpGpEDtN{NNjZ0?R6{ib2EU6@U zw4^Rb_kAoXdNKNZZ@GWMLQB=S52F_?H%8)=paOG*UOZOOZr-hWdUI9b-PqKU!{4i& z(PrI|H&ID7`bZRuCbN!-zubv$_@o#&Im}? z!uWHbOa2$cP3!A@!0iGMDwDJz$>*DWDCAU(k~%5(*q9^Lo5LqGDMLiXjhipGmN!&X z|8X!SaohLjSn|Kim~8P>m^-r~OKp3jpX zSyH?*orCy{?NT8}!L@S}7agql{JKwrY~Qhp3i;&`v*gPH$7(sq=jm09ZcZRar9K%1 zAt_|eD`$WL7Ml%c;sECPdN=wEGXvMCsVz9LKgi%u4}Z1T6l=W|#H4WQTK8_B68D`L zEDhX<;&1#yFAVM^-=_&4tGneQp?X_&VsT*b=KD=RjM4q*zO}u|GooZ<_hvg?W_6f8 zS*Hzfvk(IY*?~BHKImg&?mUDx&c0!ZUiL;ZvJPn7hzg`@BhghTMza`RC9@@xx7=3a z0r5AphuoeE3!V3DgLcqOkCtxn!OzZ_*LTp%aehpnr*g(`e9f8eKXtvL^()@4)8S(e zW%8u0y~t=FgF)arNB96QSk_BqteZUtee zW}xSN$DL!lkC&YrXrVUPr{|$xUeyw17)NI)^Sh^q0!3^UBqenx%hc#B{gy{4a7RU|;ff z-H70j)|lkGmATZWC;9FR|1MN_QkUM{rzqS*7YhZlj$o@287-}go`q{$tw1Shg=1A4 zRs^Isb&+FdK-;qPeBBBLL=Cf`V5Z^Gvwv}V?D9GF>NSCiuc9V?WsMXFEjZIR66%Kh zeTSCs_B(!JUZknIC&bd#8WWLa;$<|>6wn!LaDg}3st4Qd_zSb1W1;)7$RgC7m#?RZ z1C=ezgJVL5MA_hxx~N_WsnETiW)S?b(Y}VjjE1$yUB~rf@#58pKbdTsRzMR05xhK{ zu>ELt<)`|uw$tf5l491z+u;Fv7Ri$5J;FW51`Z@ku7eq_0Z$6W^816E_btyahOQ|Z zIcPPnPk)Y{rt@6g-4U(1o0reNq2l)x9yY1`jZsmvHDp;C;pA989Z2vbb`)B&5oO!A zS$`;Oh2o65?I<5(#m-lfP(&4<#gMKY&*Cz{zZ{nCcR@@@-m!jDY4@!S);>;_uX1LH zCkpO{k9cwsQl5eiMcgOx)?tG##uHr_qRGyhkJ)aGpvwARXNvQ+Zm{c4BWW-BB4$RR z@FP0m4$V5rn*%;y8g<~npMCp7IfhI+B?UgdMkDF*TKH$G`#$xEZ*H*+oYCaICz%Hg zx}~3TbdkM`+`4YUOZ*mH=dR*nk(XJ*Bc-~vUwBoWRD;2j0XK&Yx^oetIImz)!!=VlwcHJeSleB_#@&eQ{!8JZ^8d8ZF50Yvb}JLpFD$JVFoFd>X$WCXnPS#*Cte>FhJu|$-twNdtpSwk6CmtlFHaeAi^P2Om^Dd6| zT6hrOLZTd>T)dPrDjUQy{F7MT{)i1}biRhBD_AW`1Ke6_8Dal8H03Zgpl2ugU;MyG z)%5ZlEfdn3u3|`UAa#@Uw|s#qgH}stB&K2B$29s)b zH6LVzLhKg~4g3?cXDQ_xkb9l#REVl&-6XYTuVB^9{3<-Jm3ou@iN8-g*{z$+bc=#8 ztDdH=^tg)~G5q3ZrItV{uepVaNQrBrtDBwfhmwvY8AelpIhNN2vYIH~`SvvxpD#C< zY~y@cg7-wy*nrt0;ChSg2=gxDVd<9}=|i=Rh!eo1xcbFfbE*{I_PQG(JoRqn=0Ex7 zARrT!Z#G+s29P}Xa1?TQ=O*O%DQn{$;qLO??YP91r1Z>X%a@jKggp*aTt@SS5*!bo zL@?EAf8z8p?a_(HavsTj*l_n zkLJ~knlGWkc-(1=jR^!YLruEojSDo)8em*QI<8*Uk(IGK)$*htv&p{d!C{J&g3cfl z0fi1D+IJu~bPW?9v#RSuN;M>{s($T`99}3ZGl;!FHKUmi)Vja?D2}~hF?FMj?!l<* z_B;dtC-eByL4oVnRy{M?Qm>+?%Ivso)o_UD63rul?By0^QCS%??vTtJvq@XL$c^BHqvLU;buRJ+H#g`K%{O z9Y7uEA^z7Rua*38Df%#!pL@~?0HP_nQ2tx)^?M7&V?U`zo-_r!Lg{NvWz zLI@+L?(G@>vLyDKCo3eXucsvb2d_h6kO9q>Kxii`o4x!US{A`9-dSDu_sUw3O`!`AB5(WhI*>NRo^{n9VVQk!!;)f^k8Q`}PV!+uF&&O1fN?XeyD z*Bee|*Yh1BCD-7wtj>&n7W6Jf??v0LcufSAMC?QXAL@h zrKdK8HJJDk6oZ}buW`_LX`HZ+8L#plY2+vNss4pi&IMeaBE#mc@;JXt(F7_=54>+= z-6G(gS-&+VV>|lAOA>Vbzl_lCwm~UAt=Ty+QhFWHShT<{l03t zaGV@2Cft3;QJ{PL*D>|WRZiKX(tY#?`^D;{fQsDZt{!@4gFu+eHpiy|mOa&Ee!WQ+ z(@MNej;(691BcEFhZtXWM$#{55H$dC-zDh(H zug@j@YU|obZ|?E2|LpIaZ%+38K7$~tK{ps4-Q547`Fd^Y{u?SQF~bU`#n9b0nA#TE zw$XTnIk{WWqTJ76a1x`+EG4lPZk@!b6{EM5C-7l;gAgzTw5=+wO;fzto%7hGFe@Ot9MCW>{PO@ z`i93)q9m&OzYJ@Ak*r6g>kV@2h_@-&_Do)DEwxVT?-W&4|4CwAfrFLi-5t5i$#BZX zdUta(N2qP@7h6$<3aFRQ?b3kXe7fsoc2b%FLGAC*_Vl*f)34`A{D>UBIKFS^;ZIz4 zB?t4onAlCV3Lw>;Vg-vCPDv9EbS@bP{gG%``n37dP0WZu&m`@V&N>Xf1X&Co8e z&CAv&ZCF-LH}TsxAFR5 zW4^r*yU4loY#EMg-JrM zOLHKpV^BZkA@POJl0atNobE4OskFINdGOT!89}3uH z*%eKm z@3x~O$JH&ypbt0`l@6$A4DT6NX;@Tky6K@wq&PYK`vM4le|@3HWq=4+Y07$%0Y0Md z|Na)oq67QjB>4uIgBB#=urusNEohh1VPg*DE|%f$V4-oWm*204PlwoOu_>dgGX^q3 zN-S{fh$%n%1C0XfYjz<4KJdM!|HxMmW#^#w-4_h{<>w<1{AfKX0B`<&D-F*kC z6z_tuKbHBkbKLz5J+gd1rkJyKvR=81anj5rH0|1-MjBm#Kq!#dZ8-?*ew{}8;+LD$ zDG!>)MUCgK!emb5^=f*uG*a9iz!DWu7ZvcGuyOvlov5Zd9$23X=(=|{aB`}>n$=Kp zi)h;MEs2Sy>#0Pufs=FS|L9)Lwir$wPqb^-O2p^}QTLWkRsOU!zdu++>ASYgbk#Nx z-RGu7H80u+%o3*O6^yy=9Vu~69h?3^;U1;5oHp5|E@gWz`gOYLOxm;i8DeMMotZjD{UX}q?>@S-3aCU|_6c2n`}j78=K0at zAC~i-kM=m~c6*WQjLBM1aLmWf#gvWCm337n+WD#snZGe?lIH5qL$@uJq@YOi6e4cm zOF-$Q_H*_rLt;sg%W8T~U?^TC25z2ORO`g1T|ZDO1{>fR2rg@pM5i3MT2Pa;!1||S zFM>nTO^cp&vNg#JJyN}`fa`v{^Wi%=eb{zyUimR#jpOtK=h-0^^)Co9a9Q8gz-5s zUMlCi-(Ec>I0^|S4jL`3I_vX00D|3&pSlc3S|e}71rA;SZVo=Hn;7DAHaD*`)>=KX zvyMo$c*MRszzIm z%W&)O(76v`Uc+1nu&lhjwB-|VWtZM5%$nu~Vddusex{C49w{1F4VciS#rs;k-G3g8 zr`ahV5^^|uq#4#uwCF=rwFZtyw~|X<>8ThtbHu4^_D43r;R!b2jg62svi0HI*8%ld zKXDd4k_K*4RqPA|x48Q}u69wClG6F6)3Q)4=2wMGJ*A5krBJt3TNlu?UuK-SCjRTbs2oe=}?@%0!$PT2G0ir_E_e)X_`ca1B)();YGd|t2X zKL7dA$qnU`C1l7yYX%OMu&Z?`7weurZkEKAc&&fZQ|+7(9G0I;SaF`+Im^ykP8Ld< zA%B&&st&?j#z@4tc5d99jfA&U;L3bI>1d3;bXb!1Sen#If0BA0Mk$UMBbCE(I?{!+ zRb<`r5v!myyzZ5$USTU~cP9J?LI2@TD{4faLl|DBJ%c4o9?qu;w;W8(7!H;yK{PM| zBep!>EkNj@+AYnh;;rlq zl+paI`+S|di$=0BNbUeY&l7uxQm|U@Py5AhS3qay+T<)VyCoj5nHf1WqO(5v&!}US zJ-OlZMEKD7o3%Y$Dri=*_WnfKb&6^_9b{^$@xc{rAHv>U{&RA_AGr>2;9Z*C9$}3D zOM{uA509EkRRint*Jk5PvYh8sy`netKkRPoq$*|<2HQ-z52BQEO&Sij7OSaPv+76P zW5i7s{=Eh|D8*1!Y>Q?TE42w?gPe9dcg3?nmNg-*iR6Ra?K#BX(@rp+0xOOk|77MVTwh;XYl$4Q&ik$Q zd~p3c%Y$8TBE8CDGY`j_*?sDR9>uuSM|i76(+;BWbR!K&I7WRok9HX@1$!^n)TCD4 zyJ~0P$1fG_yYB{0?sV%0#v0z%Q#V0+N0(8Z3OONL1v`T;MUR}hec%G~@(HWTKCyXOscn~a;e@%dfEnu2OZ4Z5k_qERQ+#5oMp=eGk^|s$0 z2eX-F9ibsFBT3k}FF4{bcQu&RN(@X_ z*^-F$B{8?Zj&++k;$bk}HxFJFP{sGIBeCHqIQVZ zZNGpb2V<5Cu9Kg*$BZBQao_{-`!1+8apka1-gz26U)(0My$_XCKV~rZR5DKpD)`X0 zE+~4n=2!h_^smXB%OmdmCjR;U+joio(~oB`5Oq1F@e`EOJWKpt#Yl#Vj^Xl15DYcj zAID=*yxMdbw}7B>S4nDQHF}}WmaUsaPG;Y6tjfQ1Vmq-ivenmK_yQr)#tV zn)%TgjlC~M{|Zv*Om4r>#@9a)vG%K&J4Oj?-F$7Uhuz@g2b2Q z9=2bl+IN&5LIuwGuiD!YoW=jDr5;52px`g8JjX{8K76{Ncv`_=;mLv%^~$gF4f}o4 zH7FLr9Avbr5M87<9z={cOTlazu>GJ$_WNRN#w38Pdoh=bnM)(dgsphA%p+pu<7of@ z0yaty*hDty8AVFs5Ks--Xfx}h%jf9UjjFI2#n3C|^?ui{tiD)Nn%A(m=K-23Ka%Wu zf<3Tva9xISqU1>6&QG~RSNru5D;~Vt2f6Y}dR2HY7fO^o=(@o=CUW61>G|E^TA@V~ zoxvrwPh%fM9j~9!l06(rB+DvjRfB}LZw4vh7|Cn&SGfVkN<8*(4S*~aCN{p5_v^G4 zrw{OW+NUmpqw{hf9A4Cxuchzt6tU-m(5e55W%t$pAyh9L8{%LqCwZihP8cK$1M%C8 z;dKn)sh{o(a*SGr#x<9FI7o(_h5ylH>16=tToHlZ--PB4Ljz@)2havEV%?9&Z4+zO+E`#c z7Faheo9@sF5fnUmxY))KJmtT3>{{Z96+dh-wVPSiu1OM}S*9@h-9J4dXW5)NV_2QI z!piK>1Gm#4YtXN9hmekKi+`i@?#9zOJx!B{o@}3am9>LWmLpGcV0jV71k=jFH7_Wz z65Bs9PFa`+%2kYH-WEaYc%{;&%F$qGo!%{s$!apuLumtJ`lX)egFFb07(n6MnBdso zAoE7(^Yn`2<%F3w7X-k05j~55gS|XB-YmR7vT_W-MCf-d(`K*BLL+Up9?xp4`y)+$ zy*Db~vnXz6Z7M~J9TzYQyTaHE&W*8r7ZZC&N3B#bn%RHjdv$+_5A)IpxUBuad&SoM ziE^4U=PSAu?4Y8|=*Cih3$T2CN8X$-uZ7)PM#ZSXY*2;|brA@#7(o!~qJ*dxnJ7`b z-7mEND1rBf0>N0NzgsGHoft$UzD*X`60x$wq2>h;O$`36t!oxZ&XiT#JAhFWfaZj_ zl4Ng4&&4=l)urcjO3L2UoH={f$Y3mz7Zs8VB1W6)`)h2V{g*EDq=lrG+jKLH#jsJm}oV zU)kjDAo%!%qnmJNFnlw=^&dwGN_`+|^WSv{hl=1vf2pTn50STAY^pCtuHv=pct3N9 z%4d~bCw4NE!TP)g)T@hsTxN;O5H6|$8lH_cZN^A3)7wCp!J(Lm{-;Qx>GtHdr%s$E zzrf+-YvIyk=kei&Ma3NTS8|%z1*QajhovrX!1ZweV=+f;2S5ZePTotx9E+wCj)@JL zh616})JZS>|6(z7FhzPV1yy$2L;9DwwtlRRZu8i1|K|q&*No>b3DM9fTqzm7f0IJb z>LXEBh78Xw3Obog=~^YVoe$0Dh=5KVQN6K7mLAU$6n1RG@Ak zxI-Q=4%@=Kz8axE|J0{5KGJ!(qZIY(?pdAf_mk$T_zk!5PNR0E+3?EhN5khDa)Dnp zfA%|at(5=4ii-cM%O?2C(4R7RD*ABY!{+DrU78ya9Q}iZ0E!0F0`u6J{)he_ZPlDm zhJnRpgRr4cs#uqNY;>8L-NPG~Cc2l)THRslRV#=lsTV=I-CKk!9NRb0!`^7mkc4Wq za!MSJ^TtZ-a|`7yfm^;xy!W%^Zd%3$62@iv=9{d64$N8GI6Ejg4jPOgU|oWwFtUnb z%fb=Wtd1TEtqiSt@~CYXNia^k%;!_|(c<@1rG-6t{Ll`^{!_v(6V<*0qcX#Vg&vADOxInvcPt_091hWzo0XU zjsL>U^eSzFVN~$T#v_XngZU<?eEh#>!uAAtKVo<)^Ulh>=(%J1Vj~uO4r1jYJrQ^VZuJS7U?Ab2VMX~z*8_Mu;elGBWX zVWxOM(%^r_?%CbH4d?$hqS{;-!n6dk-)3q9kOrD>AHoPMuD#USajaH<9sjv4GuW}R zz_y!fjkFjI_)b-C)4zSn^`l)MDPc~5%1|9PQ3`a3o5+Q_Jg)Yq=r5b6aAj|gUCjtJ z`@U_%=g%(|GM5N!(4`DamNU;w*A(t>=zZUcd_gn6_D%1e`k4$x3OVi&<6mL~+En7z z*=Abdn~lFzV}oRY_`>7}7z`aVlRb<-yrNZ;Wes8?-sJF3J=&q_+E6P=DgugBKEXxA9L_3pCI#y^*u1MLRLuvQ z_$e}7KFXAYyH7M9>x@01tgM`tm=N{L-KDq)Js@ks7lKRQ-O08Ga{^o=p1uG|Ifn3u zBy@&sW;R8={^XujD0)BIn1xElZ}tC0*joog*)IRXgn%MpfYP9dl!A1Jq98~~cZhTf z(kzRjpeWtlxeH4z-6-9#G%N^9Elc+T`&-cGoaZ_3Ip25xA@;tnnwihcHP_5sVLrFo z-YILmK2JV|Jo$qEK)ep$MBv-}T>EB2{OYR z%ancTktfaRuUU?IH4iO&hWm~8@Bn2GtvoFY&BLBYH#!<{${d`ytWKFT?&!Q;iiqi5 z`KA&RA;lW0j6e*1pr5j8O0FzR;Kk<0TBvv_<7uZF+-_<>))@V?{SemD%0MICez-Zm z;P=_Mt~N}MwVm&W-9-6K1f5iei`-0)jCYkvQlgYM4J4Bg6h*%Ocy>WFYz%J=htBuW zB3OqSl_o&y>U-cQW(#!P$JB?OL9}G`dsALOv?SA)Lbp`F-Hsf(A`Uv z0=QqJe4G7MBzK6GTgB1}1pCoZQm{K-bhIv0b3DoQYf@L1K%|sjdZ6=mirM~SoE@+{ z_Y*tA?M|ZiE{Am5M(e672=}xmy`|D8anUBk(rKa~0 zxeU=Jhuo*fBKm(e-3e&p_o&}n?q$oSf+W7Es2T6s4wIYgYLaLl5y0uH_TIT%>hK7) z%Sr-6eaHNFvtr^W)%MHxI4QQ$Gp$M9;&QR7B+5KxlZbxivKY5*Kr~y@EwHycilPu^ zNk>Mlx^ylmy)pUX)O-v4%vDc9hrlUHL@!(IPTEe0H<1(EDb@;^KiD7i4yXiN7kw#J zG^OZRSMteg#_8LU!~HIgh>paOC@t_Pj#kKRm%x+TE^=5e;tV7gHFfx!P2(w1Xx0$m zj4`T_Xoib0@A_R}#}f)wK6MiCF-z>0qsE@p`Dg2>80xuQk#qIEZbz|vOj`O#K61YR zqM%ia2kugB+n$zcAum&a-oRFUC)UXG9@I*7OYZy%yb7M%$P5k1GLYKoDu6cEm;h8a z%g;#tJM>WhL+$rNMJWfaGuUDscRR0$X1KUq^&eCrg%2xa3#A*yc`+|Ponxc^$$^`Qg&lCw5w4Op}u z2}`#;OmxUPk`300K53f6XyN9DhR}4{i$>mh3@y+~+NleAOFi<_!}%MIk<20|SDF+V zD)V8iQHvAhu9wYp*+rrQH#X7X!bUf4^*0kHF}*plow9X9_cA-#NBh!5??T`e)M?$} z829v9h9oW1m_+TSwvA-IwS4rm9dF}jvGd3RSS7Z?LDt*N&SL-mGVBvCh3Px&$@$4% zK3L}kZC)8*K}CTl?y+W)UvUN;Z00SJ#ceXR-g*1$=~aS9%NekoO=;9>gr`U->NL1@ z!MS5hS}>SO{7XAQ+%=V33pUY7FTe6&9BMM$u?@V@-99L(e9(=M1K(7 z4;JCmF}xnO?Aqy5V4o;-YvFU(o%C?N)d|+XZQ)|Vjd7|z+{x$DiK>C2-+a+<04?QG zQu%>@J8efgDsfC58zWdncY;f27rwak?oFM1$v)K<0FAITNNK(L4f^R#boaOII&l=i zb)&BOA_lO+z5|!UK?f6IpkFDt;^ztf_hkW@#89lqtU&c=4Rbrkftc>J9Cg??9-Py@ zpO#TjcjpA8RkN6d-)~TCFBf{eB~o8&KZcT^elSxOBJ_=}~q=r)qi)&cVdCoVcvc zUqq+nNrWsL_Gl0%u2pbe(HN}Pa?1DxSFG@+d>Wyu^+Q3#jum7f+Hkd^!TlYOG_jn- zAJ)MWNWowJxsxETr;|I!vRy^w^l{!8Z&5r(zY+0_h@gsi zL;5D^+DL}0m@_N?;xXQek!!E(sMpiplcUH5&+mREB>I#TFqiVBV5gQh4XSN5Gdu2J zeegn!Yoqh5jwCQhSaFBC*=FF+_S*t1NDEYf@c+(_9mnR!HrU?4R^T$iWsb!h;qPxu z#kL+lL*(D%Y+{qPhDdMQ8x0XCnVniQ%H#AXsQAwL;dHY#}4>X8iNX!*BX(Qmvug?PRcZ$D?0Jy?DaOe+>pWHlV{f!Z(XlYw6% z&q&QXUM1i=x@9@@rd647ooiWCdkFKP`BoxIo#1p^NP|K^$_uWZIuhT{EtIajA?p@b z37Z5Qm-re;zb zY=)y5JlKTL@TcMy2H$Ec9kPfvR$6aFHgoL^2aUZ|A(7bbkXDc={)RoQv;bpWNJrDi?<@4&0iK=m|O)A3!seMl9ESS4Px==-y_E@9X5A0Xf@ggzNIVzoF zNeKOO`@MZ`$gKlW^AN4NP3IT1dKDfY3~UKB`w{*9saV^!!XMVdcRD+b$*{lMb9p!> zfUku}!qxINaKGL9l-@`izbmJZD;?>5n7(YBq|Cm0OrnVUFnsE^D>;2Pg=<9K2hI-` zy@n=Ga=))`Dp@CaaKC$tdrVTCI99wrS)pnAj`e{d^eUMR?lawU6=%_g!YUk5c~(M1 zre8D}=^`qnmzZES%6a0@wY|hmLDfykLsjPmEpZDsRud<2sNK73y+D+bLMPYUI|TDI zmj;A*Xic@ln>;BC=};AIS}+{1$fYe~Gg(e5A!W*IKT%^CKGgtq2x z>*~xMB_cwIRqRu6UZfFK)}y+Vh$j^G4Bb6x&_uz(AgGjD!OC@%nRpUL z1K|TpQy%LiCIBtU{;cEGNF?$7De~!B8)s_S=cl8pR0~6Cal4&bWfpCi8th^IBbqzZ zw@Nb9JU^?UsuFd{V_x4WNTI1c2Qup*yacQCXnL0E(mdJQt+2U)Rw$uvYh(>H%J+zg z^4R&Uk#clsgo~N;7sG6wee*%l*Q1E2C-#Gl`jm+VQGN<}F`c!~^D~9;b`DAi)Y1lm ztI51u^vjE_PmbzW(wF^X*}ZzD036v6oyG9|Zr4ZTh;7M^;$C6ash^&ox@Jy>$;(Y) zm2W1LyL_q@dO@&`q);J7Pjx&Ce{-h}HD;zHkVSDR=9 zcSTSWrQy$3+ciTpNH-Cuy4&B8ieT~+obTu7iQLu;ldvea_l~3#GhzA*8AJ z)Yj#`Fl?x|1!b~uRpOXFgq5?1x;UWOwH$kHGu3RfWW50dh8n6KHo81PyJC~g@ibM{ z@NN+*Uj(%7__Kf+%;mpAb-LS?LKkacV?6eJbniARFY? zcN?q;Zp5b4-2D73!*I~J9aLm$;)zT??BdDuRZVTDW{X8R#~EDH8suiYu8}v;%4^Kw z47yM9WGC6#<4ta?Ub&r8Bc=iCBTB*-e9+7?l49jAvyl-T_9+f2XqVfof1u`&V&HFL zD)L$RtJnE0f|p;7)B6z9`RTU;TBxg^>Q!3PJ~l3isX&QJ=j$H70d)N|=<;B2CQ+#7 z2uW8iic=RRamuFLp)&W}$T-uq_06G@dxRlNIb6eIU|ljb-sCWWRlxt))I4U;seja` zNWo{YfwJMS8sXl@eQ@kkawXO;*e_w7<2@u>?K203&9FLe`&x>boNh_bH1C3?;=&Hy z8=Yp=5!@;m0H=W0Pu=t@^$uQ2fkCd*cO`s{?=LE|+k-@D$jl^T^fZHC;cEos;`r34 zRN=Kw6HY>(!aMePeciAJ6(zf6#QR}7Gz<)73*U8&7Q=U|^v6T;kuN_hg2NAl{IJ1v zaY)=Z)@hx`N@aPEv~ksOn#KL(@*Nh>aeiuGXpG*CPoEml?5P3+3a$92o_R-hn z*YgzarGAWOV;#G;b;HHNB!BdqW}}xR4VYuBor(tZ&RX@`A&ug%vcCB)^fyr($y~h%U7d|16C?re{J(US6M#LL9NA_M!YMWqlxHxQ>`K{ z38XTR>;CVaoe8VPQR<1H>-VabZxdL5!9 zc6gT~ceQMZN9RR(Mw?RNz>rD|r;b6N`2&(sm)JC|<)@$3C{VvB0msB|8AfnUWIUx6 z2@Z=1okBn3CXirNlOMu0_#%dJ&1#lbMp7pi#+{xA8>?1XN_RFO|F`xM!>F&dEIjTfUQyM z*tsJPXi~rcapLDhVt~8+;K$J1>sARI9&cg8CSf)LT(BU-@WodK)<8?ZQs>P(Tct7v zgn`6)x*Uqq>H!9(Tb*H=&0JcbUu7Tt3{u-UWzKl7zSYdsFb>d+<-lqII%(FfwSJoh zFA@^+YK(@6l$g_k%dP2&bj~Kr7Y+pmQUux{Kg}K`gfw&sjAO?>O1^)iK}hIv9XZYY zr2t%ZNH|{ls8N2sBq^A1h^lDEl1+SHBzpiMg7f7~uP5L|qokj*2oa7UVI+xFiPq|= z?z3gP>l}PXvh=gG>x^?{Y|CO?V>nJQvOE`+u99Q*@sd8Uk%a)y65-W^44hg6p{T(i z10*A^6|CfAB~6(>gFd=3))mY6;JxKehsMhtqtUUa13@O#dLbbu>~J$ z+@8*2`kL3r;&DNymPdrefrqM!1=_yGO!LhCJ?}~IrCGM^4`ZrW9C)_-n5M-hY__0;oHV(p(cl?R0ax6 z#Lxm7mzQ{Ic`uFQ_>p6cm^QRO_&VR%ar#tUk)-4azt+{wjuy?T)7EF)a;HTZ9 zY*K?+eztmjkN%}dr>82tvQYgK?}zxBg%4o>T~|#BZ_L4s20i+3lCGXJcbY(i?o?%3 z@ixNU3Yz+V_wY)Qu6iuS@d{`%M%3iyJpy(j!tT8`kuTv(G7_3ZyIP_2E@_m5$G(FG18jbXKZSi5rgV~w{8@I{{ z0ILkhn^q{?2sb$7iW)}20-EUL?Vf=!x{RaAF-cG*&ynYKgCnrt+BrZMMAe;wd~ zx%^=-Z&;F-Tz?slVtpm?b(dyf)d_mwlcx8Wn8^x3ObNODus412q5aCLW$w_9qxRrI zGTLgU&3F~%KAY=CFB;yR_~TfWNSr?^EHoOHtOv_)yYeMqdu2gXii1&t>uMR^sJd?K{1Um#3Se(>rf{NBfb*-B-JMM`dalorCBNN0G;$~n0Tt0 zzK|igYD{yHO}A)K*BXx=y2(VZK+pW>Z3e0~WT*g|xRQJ*m2*HZ>Y}qywfOM;mJUrI zV}-xkhH>@X-yqk9&*WCmv*Fg zxL-Q1^>i(%$VNrO*jf3i0Qm6f20_t_S#>SlOvYJx4OFga6mx9>os;aJA%ywsW}LcjAExL<9sV(0KrRg4lxb0+YD`YR>*hLgo`e6 zH?h%~&7GF1maaiH%@*AT0?9JL%^b>zgY1tVkt^iC%4sJbsR7WWjZriF) z8jX&EKw(~_`^V7j1YQ%01+(a^Ptcl_6xz_&pI2c)nt(nvGg&;HMO)ZHs{sYu2(n4R zge{;X*RN3Foee7TW4Kx}2P`Q`2$lfks5B3jYt}{8x#4`#qICv52qS#1T-nCiejnFH zn7<5fv##=1l%Dh`cXUp+xWlz})K~&xY>izNV>RJ)D0!D&H4zTBf|LfwA+v#_f(NMx ze~CtGSBVK@`F%}JhgQ=J>Z(i?10o!V!Nbq!urwo7ynS-U|KeSn@psCgN2sP-rcNRx zkkq$q(|FWtv9P48`ua?v-8bqu>g=D2$h6UWhYYV+KD?;n#1dP{^HVsogMRGGWI6JO z^nPn-;{)J5ia10}jXFL*>63Xcakm@!rff-+X)AZ7Z*yq%+{3dvx&BJHoQoorZlh8f zp$v0bK3Jq#sW}3t=aOK}{#<|SsNpVMQaR@6axXB6?_h|%T^{#ty@Irabr&Z|^3C(g zVNLDf9f)?&yU$~73?buqxt1SueYmEL3th2%a+wfKFYu3q=w&*t%JL4p@BaCK->W@UB+F+56jd7{{^iMLqCJslX9KWlCkA?UwQrVPpBwLaSChd$hg6~ zrKL8|=LN=v@1iwGS$Y7Ie))uLY^|O~uL|&tGxa2ZSc7`}SZC26(UIirGu?piWxVwh zno|VC2LBVjwEPG{;2KkXUz&Qh%lk!r1OGwyP4nIS@y*PnsD;H>QVQnx{K^dm<%%>_ z0s#d4Fq7zAXF?aMuy95u(m8cJ{%ib{d&B8Yl#4&8aBM`@L3s9~b#{nqNzPl*HsPje z3AI{^dRkK+gvTW~SLdk&@|wPc;&N|W?qgp?wU?2yzWI*!PuTYeEwO+}99n-xUeTI6)A zKW7y35>rHd9lOu=I)g{uUq*X~;Gv6e7 zVPyBa-M4#%MXdm&C&y0?f3y5?f(~e`=}NZ?8K(`(zO$rTB-u!1bRWMosD^d8lwD#< zu37aa!N=T3#d%0wSC}Za;s@Hl_kR`ZvzhilZDBQrF2Q|5ewP!U1QGiW`%pr1Ay8-q ztIVM;9I*H4oQ1sc=Ylq>VxdBQ%bT+zZD*JN*yIQTxVxBca@BwW!x`B{2OVWlX|?$z zT!U&H#TM$bl@T+E%v86zRCk^c%xX>EzBu)iYChxAua4shV?Z(81m#Hp zGO)|ReaAYTJR8|2R3_R0N#|_m!tyN^oh&q_eun;eJA*}|M)Vm|AkTH_A-iEQo3> z3Y((aGvPS2E^^(v4NOo$)IWH1ey~gqVAhm5rN&G!a7-~SxkKnJ+|ra0vD&wma+~>UPv4@3QRiNz73+l$Ix*QnOITmD1rt=*o2$U?g>Pzov}L z?&4eQGmuCBWOq!`SX)7Ud^K`F#PesdHZE%Fd<)S8rWQnA8V;$fSm2_CuwXqr*y4`Z zn2*eWDQrRDi{A!+uVkyOn9fgTL}_URhl*gK(}x3{kCt0DWH8}nC;t}5P4tbkx_3m; zu{Sy2o*NCr$x-CdppZ_awGqqdy;^&N!K<(%NZj8S1S5~e@_|F0mY3lb(21|b25_)+i)!sM{HKBbKZ)`8sPW&N>RZgw zKryd>lZ0HPuz7V;=%m?x5#PYv>lr2{wRvxK$2o)Lm1H)LnNgU+1Id8EC1C7lMvke% zTjcSa+kVvK(TSfJIkfLD8$@S7G3lj$R>Av%Ei!DE&fU2z;L88G{B4bV9GASSLy9e^ zfh7&S5!N~SnI@Rwb|^WycwzsJw=!((i^w=9rWFFCiwJRP!PZPss+^gW@|jL(hnvHI z+|Pj?Ve8xcg&8A*7VKVJp8~u7kRK~6Sn^lIqv6$Yvy9mF$E4PaZI@*h>|!w7rxx#- zRzK0}UZGBY(M~`=5fu;5`_kQW%HT_c`?_(Bjf;to_sfFx5WVV+2M_K&HP^l>uQT$_ z)cDrbk4*QxnXc)IzS^8|N1v{_#1yzV!Z)&pmEfX=kX+w9Z{%p9H+ohwzhHMm{l)8W zY+^c2TH5!2)816KIy?hfsi} z{H#|?r&`hE@d;l{quRqMMTp8-xpr=}9&Gx6zHKtz z=hwB(X!>OT9Q83bw%B1rugkvF?H}$Ofrq_$_mZO(r*ar=mHlAfgt7B$RiuymhpDZ4vKW{#?>7>gQiFIS&<&x>4;UzN-)W%j1ACBB#p8_24e zW2$l63@zk4KlZd(;jbm_kpu*rP)di~1gmM#Q38q+8t%bM)Li=VI6@9PNAh{mWZiAwkz z4b54+QdsS{>&+n-p6J-{imQ4Nm}St%aM)GkIB`|`34;UuxyBb?=w!T|Xmq5QYp#@{ zllRFPn|Oy0-HXTRA@R5)LN>@KBh3=))^RTfAgkK)Rx6Rgh;r$*hrd)mYE*Tv!_ zq*_**5FVIFljw{L@H0Zt5|Js|ooW1B>H8e=9(MJ0ezQ`ICo`6^wwIJusyBh5hDp3L zdk@?{n(On!&V=e~&Ey)BoLXwvl*RJQ-t_Pvt9W!h+tnD(7e5a{Z1D|{*#XWfDl&(a z5p87VyG1ncp17mU^ohgp+m|bS6U$Guv=zYH%|Oc_>`?Brf5hHJ{&aQeg1|+W1QvjP zgqfsTjMLgr!{5=D`BvYFj(>m=+5Qx1u5J04_8v1Os-;(1hRO)j`@JYWFT}i(43Heu z@x5YUO?K_56tb!Ur1EJqyW^`OGK0Ov;exNmFZiki%U3tC7w7&D;tmpl9^YM;2_C|o z7YhIc+GkrpmUF)OoSLPEvoZij-8N1nr^=Z<_s;me?P~^Ow|h7uOD!cvFj4lb&M-+q zv$of(= zquO3q3Uq(}7|UItif`#k(ffc+oxT*l6W_^mw?{)yMnz`{o_zCZ8L|T0W)^~!q|ww;0V{YoaC;~ za<$vw1G-8J4bwhnY2&$mOV1a36MUQ@|j5sMlg15YTM3w3Y2tP*nu4@|x9W7~ZBn_DbW8~+VF_jLefnjkOIN;|1nQnk)67FTmIY>ZM! z_z;;9rnv@JGNZpUrB(`U-5U#9{k(AMZYdf!gL<1m-YfniWMmr-(#=?*XaB<{OV_6w z-(3CQGjdHDBLA_eBCw&pdR9uEpE(vitUT4%XC$)y!(HJ8+Q+sc()}oK($0{>C)py@ zd%v4MCFr$Sv1Old!^U1<&d0->A$#l;mmqgli&gjJX-K)!8?^S7e@yT1!f+EDqJCof zePqIRdDV4Jk;p-fHED}$q9juY^`mgbD49$1P($K##(}sHv2pG;v~K+!Ft~P4&%J1J zGDHxvDHaVND$`z3{*9}bUEcfu?@~ExDY?GEU#m7|e^l2JAj<$;|6Hyvw0^YT9n8QNSLci+f%K#P)sepWCluLK`|s^f7%X~(1(6zL3@UPa-;Zhto?XD#on{-!lC#9gmV@X2 zM(V!<*L@v&$jKR-_eslKLp-bVZbmKQ*SCgLSGHPFr`ALGUwRJKK!nmbpzoSMqW4LR zjZH5wAnHZ2=*0MW`Z@YSe*f^=e-J}ONg={laLGR(4jGc1ZD#};^_?G8SNM;K%f;rw z42SL4*NodQ6q{>ILr#Oq)Sg+dceY?1jY?+EZnp zh{NXl=>2ypZo?hLJfY9sf0gTd)}@+NV3v$VXGY1)>r;~JbCM*n;&l$~Mb^$pik0S; zNMje}-JRyOw`S?6u?bS zGe52mj&FGgr@`L1E@xX|(I;MXp5V5@Q>hV&ccG6iME@I#QDZ#iZ;&RUm5)W5e<2L3 z?2E&mTZjTTs&A`3XN#;-oGRiaLvD+s>*CSL6;5~ujnlF7W2=pffXBo6V2J*8(U{@b z5%Yci5F(q8vp|MF)R?5P!vsCH`x}xe2w`3FIL^V?_4~#45w^XZ%TVrK+P^#nKW}(s z;GsRUjJR8t%T(S^&kfg2E*jR{E)GFK+`qGyDc*l`^9-&Uz( zVnnZ4mUpx&LwR4q8!@`fpvD94?(b}!hDku$XW%i;GjfPI+p{Gjd>auDbA@<8+(NmKliYneZjd{k0 z4aWFnfAox*zyS(xN{XNO|3KyU4Yzf3qZL@6=5bS|%nCLBU!-oTR^ zT4k?Qr`TfUWmAJptZZL`4S3QzI2!!$f`z;8>m=*gVTz?%Ac_B$%XqT@toikCiGjED zA&Mi%`feYGVvbTC>AkO|=VA8pdVZ*TOm5pKS-gZc6R3AGsF5@buPwx?JQ+Un5@y$y z*dcWM67nb8Umba1JK~-nyP(s5vD~qAN_JZc7>g_0wOb`B*;=Ya=))W8R+opyuSp2{ zVjit)kcoyyPViaM8M~jzs5id~+1$VW{g0yj`tyJs&3t-(?Z4ZRS3sH9#Vn!z6Ygxb z68npMJA-BCp>aF{0`Pf#_)Q;^Qf2eThWGWCnq!j%VCX(Y3rF6i`$LA?SeZWv#28_j z>%S{|SL1S*%B0y{`tGSpRK#r+kHCq0oMY2{@l(kK8VXCEIkHDhvT=`1bDdwRK&(VV zL6iADilg}k!^3=|j}>xvJpO{bZmbm%S#>a|#M#))g!)G|_{~mwBI_oj{ zy?k|Lu~!?wt->8mzSrmxH(x;zbx zt#$z>1TV>t_sti(RHwLcXxmqQuHO`mZbj$d>shi4SFE-t3^NUth+q4QkF1m`IPEY4 zO#cTs0<23@DfZSUk#%1UyjJb1*GogncZCf3sq$|Z$ah9n&lK0Om;7s6l(=j@R=S%! z_WqH|lfUc}y+~9LX3NG7WAYD`6>k>wPjbjks5mFrPvRPIed|dD)+V|1VFZ$!H0ksB zGQiALCF&Eyt~R|%o=L0VgVb7}1Xh!@RQ-$EvJ*wcXa2V2m6PXv1RA=D#+AV^*#9rPjhkTQ9^bGDkyl#k`Csbxh8~J+R5gZgY^aU0g8`kZMB34MFS^~B5=Axw zFg{p>Dq#jr?_!NAp&=;p0RH22TK(3)fQn}dw*vU{&flkE9%5*7p|LJx^g?5u&5-YO z!?;k?%h(ig>fge~<@w=r6kw1`9EQk0GT})nFUoWVbcl zojL!|$-emnF3yeBPGCac0+ic#Jx-|h+c-t>Xo9AX9?tEx# z!BOk}Ak>xYx*JElxliqCgR#iA#S6RkGJ^w!;a+~}>ThS5F2yjy#rKZ!sz zzv4^;*hzKG4=WZ+=v)06f-)XLIlLBuSwON zdW-sE%`e5uxQRn^AUVGdz{WmRyd;PJkq&H%2E4|mg%wyEd)Jx+>jDr*vi#I^{Isvn zz8_d;DwPm4+16^J3B!ny;z%3xR1{uVJghX3k}NSeP1|%OAbEOw?c0V%lb{Z2eQb>{ zI~R9WaOC!tB8nGEZYZj?7<>HshpIELIXWw{1V;iELxQb(NIFALI_#h(9J@xRCe{8NU0f}4WZm~ zLv5NU0U4Rn3f>2xh}Kmxp`r-15JVt%zsPpEC~|V9S@Nj=6$ZpO=1+cZq%4MFUVWJy zA2=Q^+OY7f+aQ$dr|%?f{sx6*14r`Ee$V|?z=>D5A!n}ut$F{eBCYs2SvS^7_f-Jk zdq&>q$hZ66n5gofQbFrJ_u?NOxYyJUxsPWZdz_6v%N{1rfT{S(bq z(%3JFw%{UUIP2J6w;-uzecsLe z#q$MQFU4Y;q7s@47tCe}rX4FbFXIizE+8%GrP$L5YfVgDi3Oy@3vP>CJh>awO`oXM zolHCSU|7I85u|V2yH=sCooF7%Yc6^`Tszu$-`!D<)4icaYeQiY+}CKnz;Qgtm=Zx$ z=@2D3+~NPGF}>Q9p7ew|IughTwY?u-^G>~{@nGl`a&g@3TsB8Vtmty*?)u3Rzlrv_ z)UUnis3oaRRPxYa`_v+oh%0&{7Z^3-I&?Tfo`vaTd@yXAwVcoRnj+eN&FdG3TzVejkNq_#7?{+`(hbdb7rKZG+OQMpLk1JC= zcv_h4P@-Nl#Q^qLg(aJLj=>nl9a>1kyWhd-U!ldiYqV>j%wu}x`zND&R#YB#N~x6YL0wDj zc4c*I%rhTNl*-?6oOy6qml8c|T<0qB;`L*WVL>6su%8h7uxxp6gNSM4I?qD!DwO8L zBe%V$HK4Eq-$_BKinCyLt|Nm-a9mTdSliB#(4?Ws+R8U4EG>%Mbony5I}dmm8^$3D zjrhZ_r&-kWSS0%4_h*1rx+f|6Z2tYduhwMw=wB9Wl3!OO6_$xI`%rTG1`o#_)v zb&*HTU zr_v3cGPK@>5AwDEI}0W#N-_6qy>c>2dGIx;#;i!4em_V3alJ)mqta^)+gD;uErR)# z(1S1Zo+bzEHWTdXfOfVz07?ey*XmTZjo#ZGmU=$e?P){gY{tl&+!dK?!Cjbeu7<4C z94?x-8KP0BjaW@di$IIwae+Brye$X0CSZPU}XvL zU|$654)0PlDWZM7n?30{S;HSTC{+y0CoD7-=o?1FU$XkF-YCPXjfxlt5kQ`eby&h| z3C$)x&W)Qh-!BYLQn~5TX{x$48C3klwOBdHl zUP75ggZAlZ8oaq>|xC_w)!tF7$@bdL#1zu|fpOlgN8+ zer8?_g`=09bgD8{gjyUDTvwS`Cady4uoumP-Xv~DTy ziUnPz0{v|U~5W7d89M8Zc4@KBm_T{B)7=4ZnhJ|4QAaP=kKt~Y~^oNvl; z&8d4^6#KAzs-RuqsjGOI=VSxQNpkx+c;|<$e#T9&0peB0x!?h^VO?ha zy0!_dvNVE0XY_>O5~BkZNJb(YCZigO3npehfQQ%6Z-l%3I7==$lOdQReUp$eQRbB0 zTP-i9gbAAHg^2yUPqS_d=kWO+f*9?3b{BoU2>$emUah?ToHw3xOQ;SkajCJ$eiW)J z*57Ay>VHkK4q)Gn)q9lIe3RPVGzY?Pp#d*17R^fXN2YxHz_7)`&xNMFR9LiLJ-9G( ze6M&<24l{T0(*SULNTm?^)5_|KD8rJBaAYEq?Myt^w`5HIiZ-4UrrNex(v5FQ%)wQ zuljZ(Rs0Pxalg=bJIxjSU{66?$i&P%^B~W5+s-EQRAXFruaz;XQ+)Ku5UaH(cbaWl zB^|M4^*Svmar~^}7g{d#L>-l>iubs3Rik?0$4oPtAZ z1=mK-o~k}!Bw{;a0JouLt)}wBmo`HON56Fnu8%d@`j}*)DPBO+)+$5ePk@cIsWFf7 zaio*}@W|R>xq7dAi!1AF^n@;BL#y(|s9+vAsE?VV8%)#UU(bj#H)Ke}m}!;;_=Zl; zcqY9{;M0fN@HtOpnXo*NCucW!5GxLBaIH6{*< zx5cb^A@a3ZYy||_N@ZG7oaP(j2OuHjxktWkEg=rmeTylr3l7l^7Tefh%x+6|Ys@4| zb3JW#zH`U?khDH>n4GTLf8$4x^4_# z-1O#MVDH;T!{NYQXIB-Qs$hBkS=61WMuT^#4np1{o&p1|T3eqz}O2YV-9i!IgTXhO>a;ar;{q{HTzjiB@J6A47D3VFuIH0%S| z&+}01JJ`?nfQ9%cY_3GIP(8O#8ik~t#p>~61fVWR zXWITE^E1dyY!2_DWt_*)FQ=0>J~Zc&Wk&6tl=%T0VZQon^K?o*@8GifHJ!hp1P;y@tOFbS!|x;QnQ&X^^L<=oW~d}*>d_Aoi>v~|0?X8W?5zUf zcZbY*(Zs(eFRl1z^4A#iv}^>qU<+9`YI1bG5hSGMq2#1r=-gJOhSl-V$YN)oD;Hff zjXe8JBBi4_{=!A6prY$P-y{Md zu)8uQfqXt?DSaak)57XLo0oEhCnlPV-si{Ki{JSq9a3|!P&G5eFjLT!ggC0I9c24agvXr}Zy(8nZ zaHj=;V*qoID)r(< zEG2gaU+1pVaHx0vN~+=Sq;aeYD^P&yDR0IjA~4f#Q}sLh5McZC=yER7>8V||nnEi6 zWotYAd$c3J>8eOgLQ%M3C4BK*DQ$9;PU9Q`0cyI{E zpzVIx*5Easj)(l=d1T!Ahrht2q@DZW7KNCeCa6+wF^?Ie(1l=n;M)HrQ)Qp7c@==R zLs+8yMniohcD^)5efV%SkvxsYK=6*$ywRcytSfQ zJy3Qo4w+oSiTcS&@d(3P^D!I1O1kBuda9_9K4YNbWfRZ8{%)fAXyJ~&t`i6EVZan( zryHW6GGmK36fnH6#o-mKL8?|}mg+Mo^Lrwn ze~%)4B$Ugw*;040e)E_}x3-M|f8qj15(gPM?ex=^aN{?M4gJ72 z@B|?srU|JxYlUt4RvIq#))vld)KjXr1|zCi^|i(%+&3OOA0y~jk?WiJ0l!C#{Ea$s zLmbf&EAN8HIeMR}3S3uZ#4g@WzDjZ=LBZ<1;8`}b*oU@-l<<%b>(0Tsgy(y?eL@gx zV{Y%FYx*;|I%eaS*o~w28||AYEJM|Oiq|0CHZ8Elws5)AY!Z^AV%TG|hs^YF{yxWw z?_Dw1E+ALdrU<+6lvdu1vAZi#q! zumJH$6{c|RGG+e?fxK%JLs%sE9)>fTB3hLpr?7E+5Y=QRKDy1lv{^HC&i}*s>;%}= z?_j8Q7&qp@csZ)_6o#Td5+oWxkl;p9K(4y)ky}<~JnQwVHd|x+gAE&9iA@TEYX43IIkjDB2sqnvA+4|jzbh)BI z{?lCA_ymH4iG4MJ5&mu#_vQD~4^U-}@4ro?4NtofNlmzFq3YhdHlGZQ4f?a@=H$`} z;yP6H6;wMFkiIQ5<3yh-^roa#Nq9=pYxlk~eihpj(;IWR$&P?iT`oc#MGj{g)~Lt+ z!K5bng?~Q(6|3#TEin0m{q$%~?e0LSy;Y_sg*HFp=_n@5yUg#F+N^>X1KR!{d*2z? z^t!Bz6mPABBY7`6s zLK2Y@S`r`)Qg6_8&e>=0v+u|I<(_;BKh3|)%scNhGtbQXcJB67i;inbuVi=Vl4xOJ z54`>uiR-FuWgqlq-(6(M<(@*Ub|AkwRq_lHyX+FV#Ul0(1AV8|qt^9P>W`jpKTZm@ z0nO`V`_DQ7HkpCRMDfZkNx7e(Onre;3?92$ICndF2cREjT=(|TNep*)xbgMyrR&QF zf)&cVyUy9>=Jq+aO+_Bb0NB>vcq`{C{A8bNAI5a2d^vnHht+sZ*3DgAnfkXa_p)UE z_Hcjo=FSsJ-^!Su%M?)G@6dcn40cUR+EzbpIqE#L`?c=oQ-_BRbA@&qtK)h_LA>oN zBDtJXwca?Z4kzeqcf{C@+MgNzB1IpPR2Iq98&W&!LnR!8EJ!d zn=j)i6&Dq{rVH$Vh<3f=Al~vyGtT{x8C{qw;~UGpG5ngFY9PKJvhLRZm=5EJ=GQzm zl3ITnK_Eu$-cL_egq(pCB_6b6x4bnd4LLVeb&wn^8>90{8QJuuuv=JchB!)v(X+kp z%uxrhpB|vR_2vD?Jirylje{!n?+#M4*a3$WT~hJJlIF>ZI(K8u9ib@QCfAz914=aw zDhJ)0zA1&_p1YTxue4;}=}_!cy}e5G3f=FVf8o^+J)J|1ilpxU*sh-;FJFE?xhwoN z*~#(5HQmDVINp^0%91DIMSG3RX$JJ$ZGC&w5=&^dOQ;CZ^PCU&vJ@|Dx~6uAx4V!Q zK9jvt=+ZyAw=uBB5;0G{QhcjK@x9yVZIcM}_1q(X+5A-V^iylOE=v!ePJi_f45%IBeyslK$;-Wy79@jIZhS zBHV@#L{j)wG4vhsz2}F>e^A|*PHCU9)l7E6R{$(f79$WG8k^p$ zfpwa}VM$35U>wz9$NHn(9j<{%{d}rFUT#Iwtv@JF9cm<0P!5=?vKzxfiNq-Z4&j`3 zl8@e(Ilsv&{FGg4_}G438DG~mB2DG#tqA^mo$)d`gAP7?E!(^bS?CoogXP={)yI}a zSDrtGW*pELw}pJ_+ZU2rn99Y#Ndw#w1;Wy9rlp+o7K}fO^N;FZ{NaYAmbPkl zEl3KK>-VS*lRgDN`yzLk^uNLF&nRov_2;~R(C;l)KVM_yB&`j8(QycB^K=ijY~6~| zld#KLrteQ45sL);6ZU4`t*jQQcyt1Sz9hYVF{yV-m*hIO{|H1rn3-dw!{tzsp}A^V|;`uP}=pOd8cI?Yao=y5t>=P@am#=NGm6Lr!?K z`gWqzavdWIst{JAyhVX^Xy0Bz&X1dXODv85{{T?npdr~ z?GyL|&cbh>FU`Gm-VYitbA{wz(={HhV^q&vB!JBjL=XBu`KwwOD#?R3X}{K?^6fhE0}k2QQu%f&60$Yq($7bg^7GA98S>45Oxx;p?+yR`R@cSg=981k7(rRmmLrYR ztJ~J4clT}V4`(N}&K+6*C~W!pMben?L)^$dM9Bxteerf zpI}Ss?o|onQ1bWPbw2C*r|>6!lUlEauYWH71VB>j$=CHy!cTzv|N8@YbWXHOq$YLw zuk+U>Qu%+aj#nd!8}W+@Z~W+F5}#6>5JBsrg)c{cb5OxUmc)8c;7*lBoF!=XIWr0}!@ejh}| z&RyB?hX>oW=ckQs;cnfE0`Qw$JEf6_^`=L*uiMw%jGqR1tcf?7F>!v*q8(so0(>Ns zbnl{!op52-PdvTr;NO>Mtn>h5`sOnTyUKXf5f6ZjeeC{-DGxvtx>_al$KCG!{?oLr zL-8$lYon{O5*=jB4&tB4*$3!ez)uq^o%Y*uH)8{97f?yGs2_WiRNZsczmeXts{Ib& z^KY=U*ul^bNISqbbAW?IWnY&0SY_T(u{IylTJZ$-0f_9n0=zCm{SlGgRmgzf{X3L= z9e&vdvU7@#!}wS9Wog^@bO%YYDWeey$cxCm$B#Dlw;^Gnc6^yLt&?#rY`ltnqRvl* zKb7^LPjEVN=bp`Z6T&35zI1hOT#&@QhM$`*nQ89M9aUl88o)D$0?K%{#rFwsvwsHI zVg0O4e*?zf>>~PY)S`Gq1AY%@Eo^wy{e@#Kbv=)e<)05ubup$MJ;U$WGDxnzR59P= zXyDzSo;12+da}EwmMEW*f-h^(IuB?hfgSbXKeem)w{|NI6>dR6e)%>U{`+NUm&G6u zis{P(;5NTitQhcfKco^KoaSiXm!8Dg62`WDYW~J4;vRClz0%)qXV>As^;de(?y!oZ ze}`qRy)KtMoU+4#^atnMmGn!&J8f&CZCx?mYPDS#+kG80VJhj>KOW)ku?*3F$oGBE zMby|kn+|K0j|RsS6J#^RV>Qy&27}i-hLGwYTpC$(4g8fg+WyGlZ+~{~g~b0&UEzcz zCG9$X*#ehes@<_cZ6-g{zokkt616K!%6G5KcK-%rUe&7-3w)!qUhVTbJGF}mL7;Wz z+Fa>RWLX7gz%G$8b@(qtf8b?c=5v-M>Qd#CV+DRsFcw*pKpJizyi-;z?W^OuUU+a` zFFfegAi`On4U|YXl2Ag+7S-tNcl9@e7pfx-o)ndEk3aFh+`O>O|CUol+Pqv9Y}9lp z6cEJpBu+@~`R+GsJm|f4SNPsNP1nJ3s*GmQVeHe9v;CY?8|tz+2eZ8c=!b0aqAZD} zCa+n)EZHITC^xuw{XI?lRUNWp{pHK%Wn0vq`7>=v`y#I*m42a$S9YJ#h$#X0#f0_cb zOupGk#!^46fmtQoD78zo*8OKy?#OlguQGPDB|Tw?)iu}4G%Z{0dcC)KMG%swB;nq` zOPvSMYMzf*W^l%6S1PY%fbaUv`p4RvsCjTDtMyfsOV9H>o_&ZtO%2b%F9N50i|wmR zjv)bn{UU|WG4l<>kl-eFQ)|&kC`xYG zD(Fr3N@6SrD9N@Th4H@f;Nj4ZB?lZa@N;%$W@Gbl8=@m@x&;&Eai|h60$76xs;l=g z*eJE%QnHf5y4)nk625$rXL-Fy^R2X-vnBx~t{Y#QR}bR{RDLSYt{?ssR9XIv= z8RVpn9ccna$H+)t0ZG&y@<5Yu4>p{Z}A<7GpduwPUUgef# zNxizHa|s-4SNxhP^}AsYEwdiB@*xX8I<8p*cuPb7VF-PGrG&`;tAXvsl&lg9@R6V} z)AXS1+-{)?=`zv0gx4j9D`V&t;!J@vg<=JXDXu@>_lTANp4_vM+kw#3x#zTn=_Gsc zt@~L)v9Ox(cf*|aG8P6{(kQ8Fj@_R-SGlODn4%&iU6z~%s#3hc(SA-ZD6)IF{)hv> zP&L?K<-D#$kWbz*uYOhprI}U7QE3%WUupG4B73-HIw)_4|aE===wW}jo*D6 zzg}i+M$^~nW*AJni>IyCmatzmg7O>f4oAJQ#nzIegOJu^~FD%8ALBf#j1U5z^g_{yXhHKG5pblkGW( z;xF8n5R_u7kX+b16A9a(M;a-CNfuuaJEb12H`nB#vWDWl+x_#m2kLy#nwUSAzTMuU z%4smA9+#t-@q-$Kk9dQaeRZ%~=({5Ga>LrkIDD8m;C$8m4+Qj^^QvCGNaS&?SMSBt z#C`|bol-^iJ)|+s_jK})UxW8igow=J*#+3nTQWa(X+okZAC$98c(#YgNH4@ZAD>p z5~BS*kE>K84_Nx-wqw191PFunwsbPZ4JlsZ>2>%Cq-ufXU367_kru3yHUocmvgm#Wrp zc3eAa{H0oROTn3Ak%3N!%^CuAf<|ljzbIRI`d63MbFc(FQexu<9td12h3{0AZ@x)t zrO@tI;zf4EFg&3CiSs)(>ag0vV8Xq}63VTNoqgc3ZqLe{-UscnAFQwECc&|=QwlpQN%C8){?fE}^jGvIsDlfJS;ZYrad z+$bWLXQ>cm^G$2F9{IuJcUWJq_a%Ti1-ecpHQinIdIUKm?_`2s;yT6JD%ClGvD{$C zu??S{Amt2y0Ka1!QKNSEtQ0xw26Vk?<$vtPkzC`*GYQ?Uwvg5%#*}iw&0|Yl5^sJ3 z9|--rj7kjJ^G-=U@JCnS&qlca-acaco3l{jJws1Kn6|~&C^%k z3;+Bt4)Zi0x{j27mym-$O3sx#))2O`!?hb;33znbKcdS>e_@hpX3B3Xn#tlz5OEOz z)iJy`TtZqKLrPfEN)q&~mr1m5kc_4>3{DxfCmmBWTrgsG?b?ibr=&Tq`EMJPUVrxA zt5kAC(nu#LfH--muytEKgLUr%-YjAKb_DN}&{!<#UmH8Z`fbBn4;hTPh!wtXt<7<4 z`0_|jN&!`+|8E)VSAFT}qhdFHKzReM`Ab3^6T2X~?F^asaF&s?{tfWh_T}^A=qQhT%UPE>ztZ~T< zQwgmjt0qPXF9E99V7C&HzEr1XpwGJ1{N2o`tUbIgS%v?QtZu8Ca^9GI04FEAul{<6 z2VP-VO3`GU;#&TmGduRSVU#0FfXBw461zz4IsA{8`e$>AD)DQ71yD}Ko3 zDp5CO#EfmN-yToX%KQ<2)XMAnRWzW7Vt%Zj|5EpV?;Sevz~x@A^hA z6aeDRXgCBhw#C}!`cP{rJEtynZ0)9$gZ7>-II(o`m+S-ouJK{-!xHSUjEX6w5j}bI zOsmkFn<8gd8+s%_8m^Wl*%8V!#|Al$TDIc&-GJlLz50Ql?rz_s`)}#`Cw>VU2@b9G zs1MwjecGj5&Q7u*z1A8oOI{ziF0U$y@}#y%!lwP)bsv}{v4dxLp+>8rLGubSd+Le) zjy|^HsYa~CbA`oncGS6zl&$~2|DiEjvZ~J9-?!hb&ixhpz4wKGE1GA!`=pj%3pbX_ z>#F~IjU{+&eI!ql_eEDZ{`*C6YqR#bHec+yg@^tBrjK~tF&@m=@cjaRi<4%c-GABb zKO5V=_MN*l$231fZq)w668Z1#hOa+u^S&GP1%7XMNA})-tG^2uPjCA0^nXS5zoN4J z?SHN6|KS4cwW(?edSxH6lU>SXHw+NTgoCYdTW>|KkRv$FUgK>lLNq73c*7o}IZuPi zTCA0UG{1kPbt})y9_xxlsN`bu%-Q;e?B-xHXKKwko_DL)BMf`ScVijBAo5C=9Q+v7 zZ7F(f6(p{Wp*|67EeL6a$yfMth-0;t=+3sZiVkf25*32Rc1cE0AOhSYAw(;7bY(iv zi)mdM4cqg&IUfWIk(o~al5e8!nvN~UMfN`I<=wSRAL5qhC&S%%wf@EVshHBK1~^dU z^BLy~au;e05`F=Ku(CFX`9j~hfFefMDHJ6iVeHztwk8ef3lsT#?GiHXk%zepAU$Cc zt9FnYL~F$6^P1K~q%aZnG^G~m&Le{}Sz3+3O#MR?R2o^Afu4Z+dr3gwfAeSaQ}}^f z(rE3q1dDSefRsKnAOmf@caGjrUy3@>Dyi-aV4gS*08MX2+MglWq^Y?;V_m@P`jyX~ zxE?oHS=Ym$Zb)7<-Kr+qIsFcMlrTba1yKC#SE3@*%U!zzW|^$|J0dcAuQuMRKm2)@ zL2JNZIm0BK;Fk{*qFY=DukNtZydrVdHL6$*zM?US(|}QP#U%wBvJ3VDB>U3($dR9& zO-^JjIHL*8ZoswK8&uu@Jd>&4Kv*DB@K5Avx(~>&gl8iBQ#IeKWyXy(K)PRtt%W$2@0!Ioh@>Hs+q>Dd-p>f&1 z5{M0aKS~nn79DoT{ZYC4PXoxiZ>#q8;Y8kBjA}FAUM|aniSw!J#-(ZK_GUcG*Elw` zfAM}L%$BSZ9;bX~q+n&D7I-8Z$|>29zkPbEYn1$hTfke)!hOO=rrIaMU%3NUmTSFu ztbJ!_ujW?hHo#+)HkRSa#}HVD5q#pKe6PECNNTK{{)f$BbI{Af6Prv<$w)<}(Pr)U zq_ph9CFCrE@8tnYbv*v;8R$uoaU$`D&m07jpcL0rnMA+P(OVzh$N|B5a7%En@0)!O z;zY14cI%l9fh$I8=JULmy9h{iUy!CA(y_xqyT#ER)J)stB(BXO+__f`S0h4OENE9S z(n6phOa!Y#ZnL}k->?#R<%(%Aoz~bG8DSrIcg!YDE@$OHrmk?;o;1;d&w%9IR%^VM z&Wpu62eOhBTwCGsZP8;pYXxQcgzV4P38(|J(w0JmMHR%Fcy;AnrP@msS1-dXoi$4< zW8q2t*>7_*au38eJ9pXy#j8H+8v?7Gh*tyW2qwFL2jqYtmC{m*WMAqX<;qDJHG`Vp`$9D}pNQl}t@LukF(lWX7;a|MJd!QzI&5wL02-A(sKo zFN4BRGK25ga^8gpVcV#&pIXjcS2G)KzR~+TGfS}y&Px>scG$@L>AHWFK@pTh$Bgxv zJm2IEHE>lE`JJYAXv(Pczp>=Dh>YWld;Le=fKuhmVUOlaU-ED!r4yG1PkUGrDkn<$ z%|l;GHU$@zIAh(d6kA$wX0cabSh!GBd&RV+g}s^|6dq!4(1MR~*RaMvYn*w}(|v}g zD&%t|ldIY~7|!m_?xmw|6Nf&JphLv1-x++Ip5Wx2mQtfz_pifTOhNp2_+&eF>29~> z!KYzld(27Gyr}y|?DtHL@dA|4MS}YM4<7#I)*|2`{9Uk5c>-!b*gPrFzsO&|_&IU`jx>QlsVhWK0jbQ=y~ z5kcRlgCj78q)|g-vS|&OL(>_Js+y+C(6T-w!7+&)WJ+4`!p|7{b{2$%*3fZ?wSb!7 z$p!DJw|S6}oSsIjSUfW3+Y4>4PIDD4LmM(D3Hxl)il?6Wtjv^>D%591Ma7+jE)i-< zDG$4jD9>>H$N#Z=dtK>3mmQ`j{6k9)WF$G@H(9A1;MIPNBqW}8ku=nv5vG_-ziQaG zGMp;9=H^s;u|?f$k8D;zXvmO6Q#KUz+R|0dsM)L%B zT!C-!+=2^vpT z1X_r`e>Rz#*xNglUw+%Q{KWl#0-L#QD}|WJ2kR`jV;78T!2IAjT|#s{0fGnT zizX0f4!VPuxTS}2487!qmORAm#2Ai1GwYpwa=MlNr1|Iw%Z9(!bbh|L=kW?5giI+|@1X|+T%v8QF^X|aI=3Y1xyFwi*$=uXmj1?5` zt42x=+BVHEBsj1S+C_55tcTojw(80xBu=aFysK|UTpnHL9r#}1jx%6ZiQH~BTw&?q z!g8UGaDhkQ+(60g>9GD11Ju|r8G&dph~JVTGF+l0YV3i-JBAk!aJ>_{Dl+pF><+jP zy2Q|o5yM?)BG2Vdnhf9j=Jb{xerc0~WKLO0nOt=TMV+bN${krH%?chij^D(X2{CFg z?+ZM)zF9lHn@unO1xlydLr7QE9?T#6tkTMxp7pdk(?0Km6SlR{im+)7`{H|{mOu*# z>qWa?innPleID5}CPVD|^jY#x-FsCC8M{$4*IQlp&dm?maOVTA_XT7w)E;31Zs9%H z9(x|;(JSsbz0`oYLr-|OG`1c==>2EdX8zvEUVX)Lf^S*)*p4mL|(E4O{#hE z>P)6KF+jF0sRW|u#+xwGt=v}rG>s8ex(|)4hcRAlkF$6)=4Cp3ZE~H)@}Kj*rcXHl z4qi256QuCg3sgCBNF_MyD~+q6dw|+m?mVr6;9Ejl$R@HcKXNG{@2_cZyk>$v05x#h z^Z7_Gwh5j)-mt1V*EFNpEa*}9BEP?|!V1R~AGEMIOCX%m40h3!0}4_EOI@RL53v-o z1~z(Xw+$rghRkvWRS9@^TJU2D~vHVUL=C~qQnCVaLvM)PLNtAPA3$tbI^qq0hpCqe`CCqCgExL|f*3MM=rT>%%TH{#ON z`q1r?UbxoC(bM|d?7?+4ayRSkDkKvENR8$g1YX%s{G~oj%z&#aHH2srJn^_o6hb6c z3<_JS)7>-Zj+iv64Hk-6!uf4eofLex#=knBUp8G-sfTpo(zBi+6<>&AmajF>f;0e) z?5}OfMxq22njdD5;h1eeKabFV1`>4LcyHAJv(U*sDmqs%!?pSpq6!&%@D`G2tJGsR z^qe|-;4eY+I}ZMk#H&$a=~>29TP&+8wHmeZEgzXpdaX!h}k zxt~d{&xb6<(wPxU=K5=}(x!96u1^8Yq|~n z^Phu^Q=pAn=%m?m?ygmkE@SQ?=~UOS84&D)z>AhGn|CB!UPI?z-qr+fm00@qx^7mo z(r`C650NMMcq6_i+k4>j!CwJJL*O}$u+aB9ZZDNdnlB1=G4ca>)k8YnnG)N^^ z-+8=UP58)G#tm?bRZ7BhL+R<*Lm>vYN(9k0dD=kU?&$LH(lti`RPu(J2w-7noV({D+@w?gx?^%WcF8MUORCPFo^QJL312dCdK346s_r7T$Q^3k3FlJUya zlb7AgMQ#D?UCm3`=^6ViD$9}WC0h%4F`%2a$!3)~;Xyo@L?P~GNv6c4lzs7Pp7(8G zi7hr8tU%Yv;(J4Vv)9FLI^Ppq#z?<$^(3&In>)`$rMM-%@GWGB8gyX5? zcBEaDqP<#~<ys8EXb#B2R>lv42<8}0YnZ=ndG`kryW zrU-$%_+<<$A9M&*cYUQck=n2m}{GF6LgQ--m$<^oiszdBESnoLUVNCsKL z;bPd4H-a8L%neHH4m*oCSzix}g~m*Xq9TToFyuuVJ{562u!jB-%kxtFLzSD<*y||f zX|9lmECF0Gf&o2!$*r#DGeg#14F zi|_qMPu~1>;ljaW>~vsh42yk$tyvb4PJRtPKn@KRj+2>9G_c&tCiBj%;nH zKtkn6Fl@NxbV@^{`4^jp&GpUFA>yWalf80rbXqw#uP{k@PXdr88RJ>GN+gu}pgpk4 z6`18*hQgk(&b2Om(2bcs9iMoWuoPYmq0N9+%_Xm$y(wxiq-SU}twzS?RV=scpf?rH z7Z44pRC=pB{cY=zDzqpa%ftI&H_4UV?-qsjYn27lqls7zy>Z8fHMi0Ejbf4hr-XpD z8f>%$T&R48oAwc~I>W{4&7RL;aL1s^UCQTul)0+t{0Sv8Z}`qPGq)^v?fahmRY)9NXgtZNCsnWh+O>!8i-Ve zCIcpZHdinmF!+5hYN$4^HYCf~CwR<@aCC-TSnV3P=e*Nq|1X{ZdbJ`EM&Hquyxoq% zZ%>}G4(E*LhG!_Yd(TxXw$JyGxh+jy{%v;6A^!Wk&oWBIMszf<4?BkCk|JAKAC4Mp z$0Kj|nWb0sw)?YUNZd-fPbQl{b$7s!(9wDFZPH6OW|rQ}>4?rn+MpF8g1moQ!Y;ie zbhPo;gC`|)Rh7LVA@^-Pit3L$BtpwBeqMW?u6k|DYc!J9Xl+ zVLWDjx+VYPsD9DY8?K%Xqfa%Ob>biF-UVFtvr=L0Ocb7StUi+q<7f_+?X&&N8LEZb zkvze+VE4}s)L6B(OTakCaBA^Kbre6VGo_(eo@$fF&r#-l)IIDW=+^fsEj4B|&n4uG z+Y}qlZp4vm+}*-%ldU$VTfuolC~~8gAOuSYZu8zg+`}B9&Y^`b&#^D#VlEmm8s%l^ z)h$u|lEV(wD=)b}_E?%g5~%iAVZQ#YgeBIybo5Cf91bW7k%dEvF}w`VmD_*4i+yB=sL zt}sO9UkE|lz*&WD6bDX@kfRi+1pXe0$?f>E&6~o@i!8nHRIR70zKGkpHmNNK<5d;-^(~ zo2aI%R`xFxsNRq;vS;5!K_v}22g_~S=MzT#W>`pRzwJ69O|;?h9opRBMQP;jsgG^5 z1(}<=i_HdKAAX8*%NBmtD%stKUY*vgg!m0)<_0?p&cV^+%(_N84kyePY~gD9m|KZq z1EaRHV`_PNs;0^K)tkx-CtVx<$c-3w^qXmm)~}2C4&+Zk!)H9-<`P`ZRIecBRjf8g z?_PQ7`PkEN^|MKMl>++AWR#iJm$Q{44LR=P?(Bkk6te*e;yda) zC#)leNtVLtQLMs`t?GVnW)qwFPJEZkg zyu>NJ*7MmZipb~X2yiBt*{~y;)WRW+DNirXvI+k0~$*% zWAMMluRx}$=_cKxs{U#2A5g{XZdcl{vecwLTvBSxZ(ief+vIb~jEaKCDLNHGwjSvjw5F5$%EWCgt1b8eO6$RYpx>1fK5r?>3OARFO>4t{6hzSOycev!^ zQ~P3hmDRzv{LwYLlgr7ss3k$({r%iqB=61!Kk+8F9FcIfvPe3Og~6V3PfaVE&*eS` ztO>FRe;U94wQS?(tIM1t4qGldW#7k-4tW=kqBX;nB9L~{3O5k#`pKo z-;0@=B3Fa8uk#K)(LzR|-V_AsAuVD+@phpOzg}zfLq~% zL3#IZF!&}mggGOI z%O6@0(`jfS_GEv~p2lmVK`|CSF!>jI8TYK`7SmtYATneP zrc&mC1>xSMdTo=m*t;5(^g7Cp8rFw|cgwsjI|hkl(LmioFm!5{2IRuUPz85u&qotc z%3R_}W5t6_hJ3*`JD(!IaPQ(FM1-BJ?!o>28|)ciGUCHCKh9zhbC+2w0|U*{huV$Z zCb$lI*$YGPCSe!$j(_AuR#retFXBEa$)A+G389rKyexj*C{{2y`~K6my}i{_^ zS@^Xe03#o?s84qjKe}0{?Lk_Ksk?ZO9aP&1KY3JVK4M z;@=+g9!sdBEjIfR_k|B8_-Ti)?wbf!HE3$YHU%%AO;Z1Je%n<~g-HiSCJsvORBaPV zHIK(Oj#oAal5+3#L`tL6HCtAm7*%ebWCwuCek~jHy1&m?lO`WmNiN%rva$cNptJF7 zB$~Pj&Csb`0r3Riv1<92;<-^Y??Q$4fI%sjw1l>;G6*NK=5VkZOWK*o98T?R_aB!B zzxDuNKw>uYpaV_KO4e>jGMtT@I-6 ze~wiJxdG8c=dcFTF-N#Au=jPNbF-6{Wt`Ti@`Af+Pyf;Npzh-x!^3iUsrgd{wwf*`b$qk zH6V{?PM0IZgZV<7CdB>b!KR$dfkiZYv-9Bx39mHn<3}Qf%N+Gxa&qPmVGLwl2Lxa@RqdKC;Zuz_Dnz^?VrO~+&YQ1g6{ zZ|}zKI-x-4RwRV0w>!<<<1yU%{y?!p9##$NnAP;_a*lWRF!)yu}j;>b}#n~ zjHjKKs_Iaq5rS>@0Nn5Ij>Tts#^T@n8r5n2PM9yweX45CY#{3Q&#Uwn#G{WGH`VcS zA9;W&Lb;wU&Nvml|J0nC&GpTtavE;w!OdP*8Bjt*^rLS=Spyx(W=Py&rxyEqEPdOh>@(NoJ_a`bIU;V(mdE|$woD5 zoPjH*%D4Zjr{_4%t)p7f+B>1G93Ehs{CV39J!a{ijt@0AtZHA^3Wb`V9zT@mndG@P zXpeE)@)XT%=5;Eq1bl2*b}H`jm$fL2r_S5Xfe=ytK!#Y)^P+4v6K-cm&`=A&HU_?B z$P*5ysFHArI~AbPK6N#J^)I|qQ>5yS;8xh_>vXINwC|Ov z&-J-|%I*HjHmncm?{N!TjqirE@CO9ikC=QGHF?1Jez?_V0A8yTvb$_1uN2KZet>%% zoe$nz6Ydk7jS=SB41$@|QV==9tcBC?Fd9NCN4I$0Xu;^ABPIlb+QfjhGAam!9j?ve zxA~8JducnOeBLj|jJ~BU!gg;?=Q6iJe~w5QAOHun=+#*af}%0R-jc=;{q-++s6y4H z8VF+=md+fAE~PSR!l4wyMysKcewuN9y2+OeOU9{u{Q$T60ha#UB2%M}7#vh(;M2*f zV=%dla%UAQZHIDBy#|^OQ|q}G64NLoSiyCT>u#CIuB4rGClYaHotsf+c13q6&*w!| z)iKM@U(UV7#n_2d8;ACAyU`x$Q=h>@{|uciBq3Zxf@WZ8Ui3x!a?_pZj=8%0oU1Zw zhPCq>AT#2e?8e8tPCyj%cob0odB>Q`M$@Js#qj`b?sb=HBiOeNHOuvImGF4N2W1`R z*Mh*Wu$at&a!A=g&a+s2n0YnhYzbH8_Fr(?HoNqb&RMdE4$2Iwffx*fe3;NXKDKu(htDRq?Cq6x*V~q7cfU|=j_%^ zn+T;+AkMPRlY%Q%PU3wRs(8hfu%OQSO|JCa(sC0*57! zISQGK5a<_M`xy`L04tIBa{pPm{&7Zsw#hX7krIK;4&k*b%i@T^@iExprcSnJ;t9fW zX@a^0ZOlwO^ub-mHnRp#6fyEWh1}Z?|7sTBfyJ7@slCxJ3Zsd^>A`JtX?GZCv!P9L zEFg~37-%k>s;forhC|p%+%`}Me`E{F*4n_W3NnmCjPf;%4CQPDFG`J0oob=pQ+}l} zP=5sd?V%8pQ=w9qJzqb`t@GFMPyc;EaRMWze?k z&Q~RQcY_mTvV^vJNvYnWo^?W*)v|~aPv}dMp!aC7f&|I2k_ro&e3m@cnq8E$k9(GM z$~A(G$DeLQ^GdB?C*r-TN%~lkt?Z$LIZu~P(f741?BVtiKcYz{NU$AC{hFR?cTUEfKpo9*smlOF@}U@=`#i3LeaE#em7Hs zk+}EWOAf2sm3-{ApPGA5WLz?2U`&C-+dyJ&+yY9qQn*x`6S95v`{JP8A*dM;0xT#hsU9C+-|n?(ZcSxsW^oe@>Fu zU^640OyB}R8-7IVR=DcLsNu3Z{cHl&vt2lgOjw-~^KM}D0&^&O zSTVXi8h|j3xBan6F-IJ+@Ja4t8T1Z$TfI@HHO165oWx_UGyy}}gb9u084h%=IK5<# z<=F$dQQsfOdQH*_E86HS;DxQso1~fgw*ea2zH4c1a6&@!FT$f*U*I?9ze+-Vc6Ft~ zwMLPk0A4u&5-fNA#)Ey9pcDlS{}0VfKM-6u&?yWe_lT?{L5%9ZFA)-%5tZ zyAG0`(tAzj$uYgge%X%18OEi2@pnl%QGKS1Wvq7g4_rPVBwo3s6BE9T0L_(Gjtb4v zg+=fdU~-qt(v5S{PP%56_NG<#fV*d5Ix9&Gzqe+OJLhqTnd4A8E)-p>O|YG=Mj{* zUObluJur~*QZ^co3iH^<)I@W#?z}s%Ie_-?%-$DHEkXU!m!O)mdlfH1Sqw*?{MyUA zR>^W|FHM^Km@_kX&%DQP6Xh+BQ)S(IrD_6{W^c*Gdin3 zpiMj->9lB)TKH+jO53EjtP}kmqnzZ)B1Z0+Z3iRAJd?-O_2O1s+aR{|w`;h3u?vLK z1$-Xcah9Q$a~mRivoK;b9A|WB$(RfJ%&%!Ma%IPZitN#JaP+7+Q_DC+X~OG4?QM)6 zB`g5I3JqB+{jQ1D5jZUUb@+T!NXeGXF9EZD+%6 zpuKyf>)u`58PiLO_k7wo?zGmBmT#}0t^g8uMTuM^C1xhF8=LgAvY;3X>-^o(u+Ck* zRtOO@EJcODP7HyBx2$lnIKFe;t=k3*0_NaTx7)s|z$O<;6DVNrixStnzhegut1lF# zZSjP($)Yx+?qqt*RNKmJ=s?<{2KU;#Ji`#7`Mi-XV49nlM<{K?3Tbm4YNAjcnTtuM zH%%tB^CllY+_UAem+%Z$)v|MPM%3wC`N+#Fo7Z+QIf0*xyTqL7SL#y=aZ=TDFO1&F zCX9@0rA>qTUo~KY(-Z$`5umnNd(IN0l+XJ=D7vbpSt90jZ)_s^r^mK4@d-y<9V3 z#V_^~N)S7D$B0O+WXFdnp``g5?Gn!DB{TKVeAj5u?oYrbZ(Ck`VbUk&!0 zdYu_yXGrBAMX>&Cat8EX=fX^0E_zNt48e6N^Tl7~u_^KTN$;vR7v46KM2iFZ!;hiE zGby!^ZC^a=AN23;=>%)^t^}}>7)|7c!w%?q-ogstsaxyr2bgau=`;E6%I<=)1`<5J zZov&IlXP^R?~{`UEx!0ou^@1BRV#yC6a&{1*afzdEDfoYXAWXb2o(Wl_`D=Af+D#> zZmBVse3XIRd0DxAaXW=KbBX1$cRh^FEGO)=>78T_7;4hcT(Y0Hz~sd~?G>F9am|{^ z`QuQv@(@EekEZ67>SM%m*Vc$W=IgbXHqVNuYrD%jwStLIwcJZE{>qpa+){LiGIie?1{pp>dro($ zI~!lxtmg|#IYsPjo2`b7z9dYigbg$p7Rjp#R+O#DbxT$fXCEF8qmS(2p)aGVXe0*c zghjToWH0dx<)(~G~m`KQpj_uNl30*wOY3|*_y$mwu3u4?yW?}r# zJv!UA;O$;`?7f@xq*1T^#Oc9{#~sQkT{S@SfUn8$W#;)u5j|Fxs9{>A+6LFnp#)OR zY1+!vQJMGMSQwueHp@eqXSp8%)zSyS{`6e!dCS9YINXTjA2ginLLtHzx zL@Upq4%NCuOO&>5_{NT;Xw4qy zk4}mTTeyLzjFRVWQ;d$gSB~iZIl3)z(y6oMs+c#5E6J4%-s9~XK)#>L=wv^w;tWeB ziz6a-e@eRxc>k@5;y)e$;QHp-s&ueuGsPVgUfP}Be0ujnnuk$b4B3-`X0nf=tt^uQ{C>D?r3sM9fhC%6wfRuoW%8WD> z=~Wqe2^|tjR1`!hj!HnPsDO|_=sg5PiPAy~p+|`jAV7c+0trd(jyf}*`>k`&ulwV! zJA1iUaVNXH&-?T`wRT51>bPmHK1iUrGLq|;1O*JO)cpoq%ORic2oy|ANpsiWM$0fNwXXn)|^ptw4JC8A+nt_YM<*zJ2k8~j16?xj%RX$W||zhFe4LX0h8yY%mxzhdoahry3O*@uKG5I~ zD|;X;^N*gM4-89@LLB(Howi#==F`Kg>Nr@0D++%ax$pw(#h+b0nSVh4axa4v9HYN` zYgM%dh{*7J$X-mu%KniQpjji$pCt@#qI&JG``LF9bPSF zK6MVJ%<;>B;U;2pA<^uc&sX+L?|3x)5#|AOv!<*WnX zq5);HJjKBf_UIXWN`AeW9!i=tliY}CcPQsPP4~p8hVoqA%wbD3!E3I_4q7^ct&g7{ zjY@KTFtNL1-f(ByZ4CpQ<;2*T8vBYSad_=s5JKLNu3z_BkkDAeXtFbiujy=8j&y?MwqXBN|&8W7eq=8GT{PkZ%c6o-4-I=$@^DNS{%iv3j1e_OV00c-(JBgo%$HIdoq%=3i( zHXfvcho%e%fv#-WlwQ&%m=00nk<&-#@A%v1#7^zjV2!0du8Q%G^_ev2VQMI1Q&aM4 zog7%J>Iw$0f|6r%J+>5O5|eXU*nB!O%1%J1~Q?hq+dUPnr^X7Q=`k2Z@ z6s_A8PKdlWL>4`GKh7+*>bQ=U%E=zu^Az>!^xJHZNYUA7+WcdKW=G?cQR~ zr}T0vSnwO2GEH5E+Zcu)B%9YGtF0KPCXSX`9p2^@s)FA=)`7M{$lvik95w?{Km=5c zYiX>b1KIMbNiNm*+5!dXBn8V0q=>~~nApR*CM#`J>w39Aw|?~D53a(|lRFXDWPee( zV*~mvaXLTEeGhzBp6i;j`1ywT1$i85wOk@g7nbX^$=!4aCe-o)M6_ zwBv7*ut}6_m>Ah%C5-NL23)m)pGD8Cc3aJfw{{Jk0(3x_8h^mJy2kdNqfADI zPpKG^AShtyOORx07W9$U0DHhsXZo2%zZzEZX)+K{4q2)v7r1r8g=&)6mmI@J+~%Gb zEC0?~s5IXn;=J}!G~ZYSeFleF?NAV4B@q&s6Vw>3VQLx>#8`Xey!CU5Lw!&guEKw# zH(vi4LmNEVwAD0uuqLKSYcj%BhrKp1Alkfqe=QXiCCgq-!h>Kb9H(% zqM3c}lH@_Q6bhWI zR`Rw#X`7p%fCIE*hzj(~OU3rdRPm?N38yHjZghe?_Uzx8qxV+B^Jzr zK@gm&v0I2m5xOw8dy0v=Rtkoc1c5sAJDO(bD1nO2YH!Q!xC#^$cPHs}!?HW;Q`u>Loz)M;D^a7Qo2dXlKG@}p7Fmtv zvWPg4-mBVqtp0krDO^ihi~Rn~1>9*Qn7>LbDY~6hXuzJEn|h=og~WLFy=<*uLZ^?e zj|bygV@SidST;xU!2AVn6NUQpBJEE|Y2&l=Z(gnHs2y64z#c;fIh;G$tUyGcy>F&o>CfKopuFP4Rj)nt6{^QP;_zL+O(Pc+o0-jH_ zSAClI9(&iZAk0XU-vWkf<9F$YWCp*%yZCO)J@6~bmT&XZi3?tK&V~HpD|LdGbZ4ZH zNiC=w(Y8pUgFarPFK}o_4^)EyL^{QKlfpsmW8uNDgO>oUo`Sz1mS z3~T9%y81~~#AfiLBA3+CpLpP1W77<{z?kSQz=rO%@wr9sU6ECR{JvjICOzu&KIinw zjdra9pbd85PkNVegR^sDFXI=M8mT#9V}7;g--w^^)FXxh!IjD!I@Cqgi640=rc2efUu0>kxO=i_}YeVmo$Gd+gmx; zv++EAN3Cfwu`=Z8W=d!(!OJQbR2?d7v-8EPsYTPWy@^$tyAaptWePK3FJNM)2E#Vj zS;{~P7)J@$z}0Vxll;_eH5h8wOv9SGp}$~3PD!=?(K^@g1PONyoIwHLWxJqS89I+X zDI=Mf^WuPbsS_GH?vz6+LWyph#Tsx{Br}d2c->3}QN`{h8EkzPrforS-LpyJD2emF zl|~+V245Z<_8S>6`>I0b+PN=N+3otII&%cs&HCW^+J!5 zkfkBgen^F(Tk`0*`cl0e6cV#und?`2dUkPhMhgS!+NvNPR{_t1QtS)<3#fS9wdu)Y>UwsiqF_x~YN1={!&OWQh|0*)d{K*rwe zzShYMJ+KaU6+!yvtnX{H<|*+-h&?;{e}_WN>KubvO4;1qJ}+CWO#^?Rqmsqe`H0WJ ze6Hv*_j6V=_{fIj+NdVCS2yUS@o_b+__}OQ)@cmVy2fV-Y>e21rqs;0dedxPbu{%y z)+2#{P)b1tRvm9?01Q%glt~|f${rneO#uZx-dON3&@~`F38X*SYg{H3@(d-KGSGk~Y*!W?!cE-QOo#z_aAD1K#e5y&1<{k>&=Pd=N9- zmgZDMCq#~oO~PvvPCUf!yW6QhWpj%)`&@jAPTcGX-#?>dTk*$Jny{!-#OD`epx+(s zpA{TX1=B_0iKM|=v!}7{+f9LS%_MWm6jYApf^xT$A0mYG)G&{y}b0OztfqsDtQt9 zIfyYCNevmk89yCuq=K9MxGBtC7Is}fmO{!YERUHhw{r&K%i+`P_8Qr;agW^S{=9gp zj7ctsL6$07$tWi>YXDD7gRG4bjWOzjYC;j3W;F2oUdtcy&jGK+B`uOP0H_+wK>83` zHK48KHf&ZT+GxSAb^_iuclOsG$lDJ{cWeC7`1A286UPe<&7Ys0xyu8=WUj7_pOi;s z&03ut;-eF}&uRor+sHs^jX=(jwQ6%c(e9~>8F&X&Jtu-DwYc8)W~TZtS&$6{ccY@F z6=JVet;t9KlB_+GF4=K8nnTMqhcO3o+ZBn?j0qZt$X*Ih5!0#TN0cupfg#5GPf_yR z7HZe7u*b>zX*R`e@192w0iIJ32vRgNJ}%7 zLYnD(DHHeqk-~jiz14qvMCV6jqAG|pD;mz=3YupR*x#TJ+Fi^Qr0|v-LxWUijA;*T&cDc%IZ;hm)wH5(!6&pYYR0#i4%~BD0_=zy?Jc{G}bechg!JLlV zA?Z%8B&&5A^1}~g4N=SO-o?0%1n08G)o2t}()5Ahq0~rZPquD&*H)dm_q}|;!EDSJ z4>r#s2wKaGg8=@&x^5mlZy`Xrs6wdI41^5<3Te_v!2;>sL32sLNJX0oy?^jM;VwR5 zhGASby843-4Jv6K0iK=%?^32}IYweS*z z*4V9G7kEEfqd9C`c5u{^b!zWZ81p^aCNvoQH zLZk=)8MsY2ARc(=e5mKH?i+tGV0rwLA9gCk#j+@j=T11-r5AQ(LzHol&9fb1TSdy? zkE@*y9M|9cXfEZ;kOr84>||EC9*_{DBz}U_Dj=N&m?vp^u%^7*t{db1q|nhSug#U5 z$ff5yhKIqnN2&qyl>CqY{b=lT`e<_m@H3Llt%;YClxl)grr~Gi=KGa)%Ee1(C4pRt z@nD)JJ@W14s}4(f4c(e!U$EtV+KiOw1=rZ)yesb`CH_(xEQ@7-(3Z}$7GGcdmXZhY zedbh{ckh9yL%?~RH6We!xcKLrd-ltOB=#%SqE9fY2faS30Oqy|#{`s_J|HAHy#d2mW3({Yi%B`fCz8oebGQCjf{2e{bpi$B zrDGx{sZVp(UN>7-IA}Sj}jpR5trIVW%@pLX_#Dq@0pIng$f9 zzudATcJqA%cHGjn^aA&HlPc$Q9iAfnUpIaPdT{5x>6OuHA8eP;uGneFg2&62GJT9x zyC%cJ*(uin3tC9ElWD9zlxiJm!@6yDSw`Y-{WLXfF$##AU?nAvS6 zto`Nj8k-6;5Z=+`(t0;!5<<>TiTWb5If)u5G{+6>cIk|1SMS?W77O0VG{5zd{dTYj z5GBRel}`O&7w@r3^)}jLHFvsI5N6iBtiH6^uvjxz;5AS%eOn`LD=D(5os`}Vcq=C~ z;!91;zRRQkr<=&ylnASDp{db8h`1VM+PF|*y#L0lu<<}F&tIJ{lYI7PCA%umt{M}Q z|9Q8zH30E?fq4gyrL|%QXM#2V%ESM!s1j#T7?g`oSnAf9jP}T6xVzo@D-k=@`}ZG& zrfFI64c#kWYe_bOB~(rVJaI9dOut=SuY7Redme7_4dWjT2SEOe0tq5OJwRKo1?U&Q zUC3@SomCbWN*NM@tLgtGzy2%x->Od%%}Ex`NUoL2Ec{<6A6HaSUl#H*vaAB~KNs*% zG0cBu#OEUaC$sw}|M(=!*A=a_lLsnmA4y+Iyp7ks# z**j=ZMzQx-TgU>V=HCjee!5ioQ1}5_^Dyku!+2rFe8zp@f?$4fs$2&NMGY0uz;*x>pDPSkGVz#z z_%KI~)y~Y#g%dMkqHrbk)>+rR&xSIO?HdpapM27@fVT}NKIyR;jErJzt}nbAG)Op{Kw5TY=(BPvaFdcp`2qfF39RAR&N0zf<_Nf~c~owPM6sMogn`>!?GT za-v7jLwM1{VQ%F4|~W(&HR!JXOx5tM0o?YweU|4mvk&iaZ_;Y%a$sy4SG z?=&@)9E0>?gJdDj=ZlLeLz{U9S7~QeXRPbib(Pca^Z`OiH5DWCzP5Qj(*?Hta&0+I zK0V=KS@n$skC*eb)aw@Z`Nv8}dTowyZGOfR<+W;PpZpHpu6e{z`}!&~T>Gce)7%^+ zBJSbHPu!bVBn)n+aW(O|*0c^-tcj^qo1&oD(Qh{Ti9_wp(1!sU#SGt{D)#HEz`bH5{mQx7YU&yq*{gF*AxoJ> zkdn++J5r7M=QPC6Nl$X?l|7$l-j-7h&{#V0!EAYhvZeED(mpzqegIj(ft7T?T#Hr4 z@bN3F!!Vf3@D-pA#x@N+jOI>bQ_lx45k3XC%dlFu$#`_30su9U|J&>2%m+`kQy~ES9m!1d{kE2e&aQCAbQg<~vq8kYXo>C(O@t zbSOjgw)Cyn>qbbS6^#7tV+X%%P=y8YE=Q@JX`9lQkcd&AO#u-v6LqOWdQyp-j-~A8 zeeN!6fV_5ad-k-I{9)4LLp6o)#og*chG)ElPuq)*p!~s~iif=PMg!T>h8Km-xchEl zSNM9(R%&>tZN#>S*nI5NP>tu%PA|pNk(~VT!NCykLcDZChe7huD%^OCR-#|sL9JV! z`D&OBAm{NwSIaC<>-pJU>E>KJM8d#T(?HidPIWs|sQk>BNcv14R&(G2B6LbvpZ6GL zV4lxqV~&bNF(>%Cv{-RX#4R_s{J|U6QXfkQ=O|*{M;>)Pt$5+I%tNkvAozDz>}WKg z#^9Kv=o8-0<$I5y!|d5I4|W?9ULWm=tgzRZE8lx=>^;_sheuOH#pHVG)Wr_zstnk$ zcaQZIQ%Cxa!PTnzCe9g&F2?(eb%`w}Le9I3syklsJR~Dk+Nl_YQ~7q+BsxeNK4wJ~ ziW(AU`Sg_EeNA-~n|6($(HOtNmc6YUA^zUpI@JR)sqqK9!O91bs}ZUj=u%KZg3GSU zxqwP?URQ zB}x^g2Ov9QsY-qFv}dClru`pyrnhgF65bSlv>Z4(@y4lT`B`ln^gUMr{O7@DOQ`hD z7t^ccej(nypK;@SnfIYrGdG1TXgivUislAQwKyw2>`&6aC$eAJlz4di-ca$Z|IfQK zbA_-b6peM6S1yMy8`URSpr^Qgr%_&Sck=b_-wd7uJhrOxQwBOKxqm9LUw|g%!hm-^ zVXC$3%>m>P;$E>Upx^sx3jW&b?HUu$GQainR~~RdB2mcxQNLvB^|m6@MBRgYQqEJ0 zz;K?#mzZ$gchcmG%B#F1~9bWdp19%M5y zyx-X33_z~Txr>WISNWFD9{vi;5R>I?Q6vvqd;EHPkjsNBw87(ZFVrxrKR~W#4g6a* z^-gz~C$L}7ydZUb8!*vix~8Kj->AYQ0Z7pPpdkG&li{m4?kaHkoY#YVh`(=L^Y64e ztr_d4e^71!&N)`J0xH|owm79RZ7Jn{CMI$+1jOfA8{y0wV%K8zpmC2Dce2Xg@d`F( z<#ovHO|YqvbZ`)=Ia?pr70wLpNM|mT6R%wnD!QVp$DExc;b6MQ1~S9D%DK;hT0DCw z4FiwGL&qjRFr0zr=%u`71}#>_OD7zbmB1az{2I*vOQynUqOhioy;#N%~%W^pjKtt11wrJBX-af(cw~D#FKmK=el8rlVxh^*~{P# z;5F>N4U%hlHg?AO6wKPr96q0y(!20t%QwU)pIDxLm2wNLQnm0-EUH&%W^)}FylWTGP zaPKonHAW>h2h4t79p6&6CUMKFMpjjcaWUt&YCUMp_H(*`^9i`~(T7Rfyy;r`LJ~vy zfL3>=axf52xOT-f5D(458a%YA&D(p!7|`fuzbZsy8DE*s?Ne5qb{ zg&RBYtyKQtvm21Ia;&!{g@qH?C6UNF#AhcriT!K8isQWn_ciEWQo+mLa2$w~%Kp6T z*i?sf;jV-(=H!_;krGCj2Cz&0US`)=_(;h+^XcnAMZlB6E$nQP`L-9KtC1ss2Ep4l zi(;nk^~0s5()WrEJTEQA&)dDPxK7^$m~M%N1amPU+I56EYVhzw2k{XCBc>H(#pikK z`;{u3*+VC~8iY1)3i3;~cD11!+&{<872Ct}6+}y~q!BBAb!-lBEba79uzUK=3bG!r zsUT=dn^})H=r}^`QI5(E*~+9cvzz9d9{bf*Q>b0lu9rQ-E7@oGD#>hAcLa4Q;T(5F&#vg$O$DXB(p zUAU9uFW5r*Ho5S2-T}Z1|E>GNwNU2C}WfTTKrC~OZBzxb8CIOUfDNB6N+I;Ji_@lxS2;d`W4WsJM*Mw{+AZ^ z00rs*u6}@|T6%X|Nds+Lb4E4|;vDiA~2S;%f%DXc&X>&tXBQS`Q6qb{TUG-F0*O>Gf%*FZu3DwPJY=_P*B~T43 ztwj1>EIk;3dg@$@af*&k#~}r>!_cYDj+$=Q2?x+oO3J|tX+CMVa_KkvRi~%L z!R(&@8+29>_wGHC`Ng3^*_;G@1`U{FIsjGJ4sPrUzY%k>)3vtU1RtwCCygIy&vw^G zwYiF@!ar5%CG*jm^C_$KgE;l=-8NP|C2KuNWbLd)u-4Jh+BM~4WDPCr1N(t&Z|Orb z4++j%Zz7U{CQ~c}CV&zOpmELLtWzf@)kVGZ{Jmm^iqlwTDazZHzl7q`N{o}-(ug1G zW{Jt&n0Nvtenr7I>9+%YiWh=a69WwnDc$kT3g*YsBlF;X?Q1TJ0w~$5*>B|LBP=mi z2lmg*gPIww_&H(<_cHTsptHHCLj9+_#@vHnlA!J3&&hy#S@Maj(B_{d-5&9{L-5t&tVu$H0-2>+%O3@J4 zuJAhiQ?H9t_>wYjk08;Glb#X)f#S z3a{Q4N^tFVVFO_Qn*Z|;wMyWnnVQ@CS)wR20m+Epm_Uv0K-Kf$0p-S}5rkX;(3?cI zJ7*wMD>^Jg-_?w|vhKx1E)I+l>8(}EHIV~T<2x#gnO7T9Uc-@B<|bu*2GY#7{6`rr zZT-Y%IvQdS%NLU6C*^hNH@gXo>U$YJ15c)XXxi9KJ@TeNK|WB6`bKyt?z3aRw8$nI zQ>Q9`JMa$fQr#f!XW30qw2Y%Wjz>x|tdzzS=PVuCywqzuTj_=R-1}=S{g(A|eo{{$ zUY6bL(7*Y^Z7v4QT7FWcOxwAH383ybDcmFY;@+jpLGHiB@*ln?dg$UMH04*q1YOGU z{ku@cLui0dzrIh@#C= zt%3nUtWHG(>D$wR_tB}U8L7uvZngG-J-7| zIIlQ$EBVoKn3;Ek9(^^S=f!p1r73b++Hqat0E{_xzuZ0G&6bjt=6dWz`-;w2h<#_k zN&|DP-ZK1>jed$YB^R1WW_sEU4TbLy7)d+dkE}lf2K}LU(*^c!U?2RwE{;h{AFcE; zSUT@{4NEq%1R+xutTO8c1Wd{1;AUQ0sK91`7YqaXd92=W}g*nn|$L80=lz4LK8_AplCZfVHE*ggK1?l2Xvs6?Z zrQ@XnFYK`jUz$r&fCYp`?+uBPKaCerxN4` z@~T}q?$dD_o5vW~?(lI5R#Qw^NI0W?1&dbm$1LX5(T6qzjTW0ld}r2XnSzE|?F0Lq zA5fjy&Vio^DhaqO*!d_{qcwJmmGh?Hst?v*QM#!q%+VHT@6KbVJ=P zE$a9XLOK^2*tcY>Zfq{Sm1>-4l?f5Zc zYorpv@&Y9aCV-fJT3atnp=xVy8GY3azN_2t14vFxtGC9s=N1K)b*WF^(5NuQg7x+H zJ)nt|_3)fyK3d6em(y*S!Wi@~lhI28uY8zm!$q}PVe?BxAb-0MM%RO!$>%Ft(wumk za%lAPw+>|ywkC5O(e@q}tru+^9d(Vu!^8L31V0Hup*_-6m7HnMFR8TKQnhQLzKdrq z3f5Cymj*RTBpE848JTKQse(nh+s{=}I%W>VwGQTNG+-f2XQb%W=%Gw<;2<-E+nr$P zLBZc>>^e?d=Bq{p)GL+JbjBb=mBp-8=4!gFQIEEo6y#Q) zO$*zek#iV1w2yBT=rO=kJib9}zPR;I5oP$aALS`rf#CZvR@mTY0BrAamuj7nk~Yq7 zKM1l*(JMpg{S(#~@Mw2j@&HaxJz3x1UwggBBJx!&*gb06NA!KmCr>@b+xg*W9Q)SL zQskxZMH?}s8BjGDHWr#PToN`0a3)75^&LIjhR>q1FS>gJb4LWQbJVlC$skP}1gagj z>zOl)0B-NV8@km`2x@hPc(9=~yPCsD_XoO#Vcqo$OBoK{;V!Pod7{l-;wr1*vq;Tk z07KwsTjoTV;+%gXGb9Z27 zL#FFDrBobNuP=^AzA7w~Gt=4Z?whDJ4lx~*oyiKg-ZZ`F(Y_S9U~%^g(iSxabaVQ{ ztNM?Xj71Z~(gR*Pn&CGq&{M2+9^N|tu-rvM?t&4iu6=Fho=Y{E5w^oeeXyGI4ex1so_&+jCKASfLPPiikAd4@+>I?(PAmK~6DqC5} ziEwUghtD&&Sb|4-z5f2QUp}ReP8{~Yu%g|Z``Sg`i5;IIl{PQMon3q5O3IM%M*vBd z@9~C#EJu7`uz1M&op`HS0;+r=O(zWfG#1|-_;&nh=e-3WQ}`e`o3Uxf4&EX`av#z3 zX`35g3b{pF9H5<7?#8=9eQ9zt7x$`AuJ8|5$MqJmWKB3of3{cKjf$_$UMZ~*^=-bK z6K*RXG4b1YG`CGQqI>$O3Y$T-MmaYc$XThsDM{9q*{{Elp3ZF{;WgI4$)e74DV04j z&U;dVgXpfg8{-=lozv6`5|?TPl=T&U)8Kng-^G7y^-jERMQWX26G6_L@gRY0tDB*| zGAFyddIdoS>mO+^5h$$!v&JohzY1PTu zk39vBFJFcd>%8F`4mKI_^4?>GxT=BkJGoyw%4`I@-c?CFh8Xf6y_IoBI-G${xoA#JtjMKk zVKh+V@Vu3mWO(X}FmkpJS=G;KoYg{_UT>OdcXf(!b6Y4UQ#!S~)9DQ&NGn$3kRhqP zAEVp9iaE%0LB#(9Ge3R=Jy&iQ!pL;5m9=|sTIY5`p-k}Rn4pfqavny$bIko}#vK07^FJbz=dpq}hWw-YN(e)p1`hAj!#KT1T;#|N&S0vFwVR}{VS&O+sx zYz(5s_vE_4M%iifW)36fMN^EeyH?vk*0XNc(lLi$ZE5M78(Hf>hP=R3N6`2{M{qEa zfX#8RAyPHk29!|Y_O0G=`9SBW2d>H5r;>HN{Uu`t0|UQg9h~}Lm#a=r3Vf#~2QS-v z9Gh~|cpG^VReva;`Eq^FDC8J0&!d`=r_ghjvLJ$lPXlc^v|@>)kZ}!en%hSr5*$qy zn1-H?a$0@+aGpJzAx=F@o0x7W?yZ74uWAel`9DmF-fCLaCs$GLUQ$UVr%^g72atQM zwhf>7hiYMRHrX|nWp?m^ZrNA?#9XIH0Dj3}{oYgKUM<(5obuvwJwMPwq<8D1l~&9K zYnUEd=~}pCt5)ric6v_+R;67h;sEw}TPjNf-i6ft-7}+qh z=Cw#aeIpv$N?W;O!2PW$3^yXn-?OK;w5Zy3raZF2_}u33(|yGW1ihu#lnEUhP<|HB zghJXH?YZkUnU>_B%& zRI`y75>6=2=0-C>*Y6Sq165cIi0hrzx3b2rSFFR=*648A2f@UVcG_u|ZW2QnnF4E^ z?H3XEsAE2|r=Fq`hKa$B%42%XA+-Hp&8B}Os=tAaW3~I)WP_E#(@D=;z0J(vj6W{e zqye8^&G63!tCT8s_RomagLM`YZR*`EY~fml@4yZrwhI+wIn&MAjUJbYT_D$pRTFT6@uJKURs9JFS zP`FgbG6t1!()gVHCLQ_^FQl9@`#7^s18w4)nqMjJ!vEd>DQc;~st zs&lo?@~YkaS8cK_4jN~#7hDC`t-wWglbzlGh>mrodMNLVTE1zNc69?c8ESR-o%@Gz zXJSgtdio{}X6Ymsi#NtZTh=QPJKy6}RE&3%AarMDH#^nY5fZR# zL~A0-8kFAs@$;6%9;=-QzJ&*XVGkTti7uh6R_|R8~(B|+qSM~`+T;6GOw zc$-~ye}PZG+}*p1(%&kdozJ!{xN5-IR7w54Q;_B4RHGR-5}lIXkGFr@o=Xf0SG2Fr zQzSOJFSc(mwUkR$c8*J==r_OKYe^et8r{M`%a}ctMC%JZ@>^Z3c-Ogyt?A`sQY&)0 zjP^XEte=2aG)LN?^>=w=yO;zR%l)}k_v6(+v7*oNAJXcy%QMK?Hv^V_&mc7E6<#G+ z-dP+!Xs(2U4F8FRlW!PP_G2ruoF=YI|Ci% zTIcP}xs^8M?iTebX=*(LoKNHCOd}%yHlPcRrNJ;w#7#Rfq`ON7^&?pyc3$ID*0umn z^w5pY?H*;!Iowum`lv3NAA#Fq-Vf>$bXVG`XtUD=`^{Cuvw_Bh0m0JAz8qXM=6&i@ zx9o96+{dE6td_1%fr|lK1e||z5lFq`qIX=L3c8}fZf{)}F@-~F5Xo@2*<@3>zYLWg zJN4ODN-kxyF9jClySmyZCu+^AhRenoP0?A>H~e8_ZRwMFw{|6p&ClGM!xP32abGp6 zG!)_g%@x z^;H-*lu{I^#)}-G1D)Q4u?V&v(1NB>n+Xj+W&9J2{7`zx5I;X^rbG908)fviCCjIj ztUH(dYDEiGMy~;d(K8io#H}HqRdF|sOP{j7;*!Cbz6jpcA;w3pw<4pp0S1?>jqOr8 z5oWfB>^DswHGI8gQAhkF@P+}X=7==v4aEZAm z^StAt(CY5*m1>C=ce^VD14_ti-qyrKk<*hQ1IPM^&Pqj@ozu$DC9QI74l8jdLKwsh z*<;zsDXGRvh5*b*xQ{KY)3lZI^d*8|eP^i<$GvV+G*N4zd_R&hF-^BZ7QY<_s3!@; z7_HUQlCy)+hhLUciZ6`K9~Haco=Q=oURpXB>0J+ROV(>PsaCdEw_AVKeKpi_S9vU6 zZFL2xV2%VZ_1q?4Oc-gKnZ2G_G1$n%D6!Bpp#~ctEa)VHkBfz5;M$6;-UP%u{5b$G z=VoZIA|%jFo7#j5XO^z?wwdE&2mV~mnXY(iP(S10w)t5PSJ|`aqKy;u?<7{~Pc#_v z+YqiUmE6TSn-hGdH_uK7zJ&*z8_VWJYK*L+;t4N86(EKqQ(M$_3=B-Ju8Z8A zx!x)!;{Iu5w+@DxWL!*G>LHt@EO`yMq6k#`^_z|re_RS3&siE>RPQ7?u^P~wE0HYs z_Do_3bWgDQFWoNH?+CH556XK@&z`A%Y8_f3}d^V!7-S!UP>D6E4rJ0?5HOjhsQ zoMxvva?q=Gh+RgvKGRUo7-xtGwy-`@Pc!!^uAB6SW#mXtq6)WD7XPFjB2*v0uwGyp zUgdTNkRu^~`Az`mbxc`95dA8E-ykb>hi(@-Y-vz*V!%lv{+Pa84~U^cQYXVs`ER|S zppz2e37x5wIBfFM^Uh`glYxxbz%Odpw16{)ou?cd-z~6H9zbX@B54jmkTpPkJ!Mgd zeT8dM4U>ZGS5dK-pmLpI7L8!{5XPb%oSq&1DbMoUsDViMB+ErIbYahmoFiIxysLMG z&XW2m%^4Ev0G+dxjp`!nZ!Q><2sX}~@C7L4M1Wb;j8ugf(8{#PND@>#Dq7uK=q+=j z`y=6Sy<}iXDfj&`*)Ls$T*fP(_Q^((kW#Va!d%-!g;`L$=gLZVJc)_Jeko|u(_A*W zj@pXt2=!29Wh-zi`5VQKDnnx++El@Nn2qOL1lc3f{&5YS9E%`SbDO)Rdakp7g*P0DrR(jpcc@grC%EiGN~a*1 z_jFr-THntT0%-hauaPATYI0!C(R) zM0)eSYcVE;0zgpHjTcS1`Zo}ZZ%FW>3qLT;E7e~o9sK)~f8oLlg8z#QyljCViR2Sr zOYK@or|JwxHa3t@xkJ{*QC(GBfgohPt>+y2O1eydWpQftt%(g>67I`t>x0JJxfGUY zNlMb@ihoqIL@qsHC!gQj%Z>Rq%|f1pM_-_C-L zbCt2_9)l0*S8Dsk5T}If1rYYjY&E_`z~umN&lZILtrw#kptf|DC^&hH0!YI5CFzI0 ze8Jg|7f_f8A_Z^rQWw_aLQnP}B<)1lN7Z3BI7e^uhF1WWzt|~Glu3f+-cZ`v6zF>^ z)VIE`x}NB5nRKpjuVd=C}152~K>+lp!0&~L&D^>@032kji$NUk!sfeB32!EfH zqRm(JJ^u;SP)^ON5kr=y5CtzQf=#f%3N0U~A6Yud1D#9tfl@O;E2@2ya$Ki}ec9fY zWTWpuQ;toWe^j(WeWHnv*_wWm6i6QMF)nMU(yz9IOE~JsjSobRRPiC>;;#mI{hgv z3Zl$?bJQEE5j&Li&}Y2#nYZaQY`F}HX(oFws@E&V;_Wc35uugdDIt3PZA$fF9&DSx z!*+0a*>iKhd@SBsdGsg3BqOtqGIFPm_1q4GZnlyz#>w8LMz~vya~2BfNF($sMMv-; zobMa&)|om=QEOeH6)kt-C}A9PX34%zUvowv65*nO(UK~ju)1Z<=V|Hh z7HchcXs5ATBn#^KFo1z9RM~ORekNy{Foq z3wm8PGNKe2)7jT2GisB*bia}fRzhz*-nQ`OdU2>FSCYWI8q+wwF*}nn=~hH!3N*!7 zZziU2_wxBEa4{y|@kJrF|HCm+%0ck+!DlKjPygX((q!8Eo7-#kUg&1vu~OI}iQJE? zP`2-H~e# z#&O*xT8lp&`nn#z*KTCR;P}%&VP82z{@}f zju!)oiKa;RvIfu3#&p1`Hyu-?8N=g49S<7Pkrn=P!T?*t8e506AD zJFQ;VD7Rx39w^-}42)Uj|Mt_{$3|yo13sE+30rG>??T8J0=H!EKU-jeoY`^Ns6E#5 z;9vJF{0KC|csIW-*!AO7nz66luU5i(8yg$+w`A;H{2ue?8~eW9v1LE_RvGgxt#dvS znS8N{%9c(Lb+XZ~ITSGBqydwZcI3%q;F#V0{E2%Rb26B#47{Rk^sVZ!wo_oJHPG(F z)!qcT*YX=zxo!6Bcdz|#)=i1Jntqaivr642n#kg60^O9xIwp(lm3IQU{k)@BJBSZh zvYUeDdnXvwGqnPqf7=Hb0DX`v33RAr6wBULcTjS>^_?%x>eXZCUCs5RgOj6;Kbi$A zeIf`FMln(Bin`^uV7P?QYFq;`r^J1)Pt*4+h6A6S7_`Go*Theu$s9lS5EPhgI)O<9 z8tb7gAKM{5J4n~@D(Jr+Wu%W)Yj;%~C7qj>`r>ci$r08Ymn-AoFGiLGN|wM>Uyt5>=f?-%~XKwYp;>U4X)8?P^XuZ}AP(%gpMd2`ftbi`$2Af~UrOud3s=Iq%g;YFu6ni6o$lgSKQRv`>oZ?RFz3kFHe9*rM?TFqe1l(hujaR4 z2_Pw~e;+s=)HzpU?;jbK8W-^Gof@b6x{NUN z66tq@=CF|?wqCF14zow!2!qxWKCOeo|rMn2;5$Q$lqU^ndMAY3y}$w72;8 z|9oXbnVIEgMdxeo9y~blM(Oq1MHAe*E#c``C0A zxwUZ#D)A|AJZ^IatN+q{f9Q$17ly&H*T0^7Tff8eW~$tdz^`GgtCdp^m~6gqb2C3E zH%EqjpSy>pz;AonUbXpmZcHeB)NOJsir@Onk{yq~xqD5Bx;rg+^;MSao7JU> zORwdN)v8~2b?iqA=1ZB@oe>3lNY@;KWnD{TG^+6KI`}R z&;9;1VoSl-8@CRndOkQIc(aN_`s>-)?KT%wqT_y;-Pgv9lGy9ko6L7R9{ZcpuXsd@N6PGtG~x^q7gGJVciZkK)X4|u79{FKI*Kc2key?(ymna@u1-^a)C z9bcu3?mqlDS?YHCR=Hhyxt}DNnh>eu_G+l1aPu`B#~*R3qFZvEAx;_;Gu zZL5!}il5is-_O1wG%pg^-zi-&E6(_5+uHi|J#!t_>g~>ndt;SdR`W?X9+=pErte*~t9{=8K2I_MG!Ggr3X*>32`ONT}VOaX<=`;B4 zKKKBSK)l}(>~Cvq_xq;)u@4`XBwwBWM^^5h%+$5cSGUYf*jjfx<7?7a`R!|NIj`IE zXxFCyug*yRKE@8b&)zWqp8w-ruk(7_<*Jn8YBFzbbI*|uwf`2jT^l8n0{v{Y-A(=K zlHbzI3wtxKxW(^VWG`3unUb*sawTu7#drLn(e|K;1 zqbcTdWvky8yPT11-D&%U9fhl{w{7c9FwK=* zmzTf!x=+1v#91k+wOKpjX16cuxtO>4Q{`7Fq!J1e;VU_(&U}>rcI&Mvl_`%cx862? zUb|IFavht?cGI^}StUE3%s(ud)fd)(-p)|??zI2xQM*^Ce&l@x+z7|O;OXk;vd$@? F2>>X&@6P}L literal 0 HcmV?d00001 diff --git a/rfc/rfc-55/rfc-55.md b/rfc/rfc-55/rfc-55.md new file mode 100644 index 0000000000000..3e8d130216647 --- /dev/null +++ b/rfc/rfc-55/rfc-55.md @@ -0,0 +1,157 @@ + +# RFC-55: Improve hudi-sync classes design and simplify configs + +## Proposers + +- @ +- @ + +## Approvers + + - @ + - @ + +## Status + +JIRA: [HUDI-3730](https://issues.apache.org/jira/browse/HUDI-3730) + +## Abstract + +![hudi-sync-flows.png](hudi-sync-flows.png) + +Hudi support sync to various metastores via different processing framework like Spark, Flink, and Kafka connect. + +There are some room for improvement + +* The way to generate Sync configs are inconsistent in different framework +* The abstraction of SyncClasses was designed for HiveSync, hence there are duplicated code, unused method, and parameters. + +We need a standard way to run hudi sync. We also need a unified abstraction of XXXSyncTool , XXXSyncClient and XXXSyncConfig to handle supported metastores, like hive metastore, bigquery, datahub, etc. + +## Classes design + +![hudi-sync-class-diagram.png](hudi-sync-class-diagram.png) + +Below are the proposed key classes to handle the main sync logic. They are extensible for different metastores. + +### `HoodieSyncTool` + +*Renamed from `AbstractSyncTool`.* + +```java +public abstract class HoodieSyncTool implements AutoCloseable { + + protected HoodieSyncClient syncClient; + + /** + * Sync tool class is the entrypoint to run meta sync. + * + * @param props A bag of properties passed by users. It can contain all hoodie.* and any other config. + * @param hadoopConf Hadoop specific configs. + */ + public HoodieSyncTool(Properties props, Configuration hadoopConf); + + public abstract void syncHoodieTable(); + + public static void main(String[] args) { + // instantiate HoodieSyncConfig and concrete sync tool, and run sync. + } +} +``` + +### `HoodieSyncConfig` + +```java +public class HoodieSyncConfig extends HoodieConfig { + + public static class HoodieSyncConfigParams { + // POJO class to take command line parameters + @Parameter() + private String basePath; // common essential parameters + + public Properties toProps(); + } + + /** + * XXXSyncConfig is meant to be created and used by XXXSyncTool exclusively and internally. + * + * @param props passed from XXXSyncTool. + * @param hadoopConf passed from XXXSyncTool. + */ + public HoodieSyncConfig(Properties props, Configuration hadoopConf); +} + +public class HiveSyncConfig extends HoodieSyncConfig { + + public static class HiveSyncConfigParams { + + @Parameter() + private String syncMode; + + // delegate common parameters to other XXXParams class + // this overcomes single-inheritance's inconvenience + // see https://jcommander.org/#_parameter_delegates + @ParametersDelegate() + private HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams(); + + public Properties toProps(); + } + + public HoodieSyncConfig(Properties props); +} +``` + +### `HoodieSyncClient` + +*Renamed from `AbstractSyncHoodieClient`.* + +```java +public abstract class HoodieSyncClient implements AutoCloseable { + // metastore-agnostic APIs +} +``` + +## Config simplification + +- rename all sync related configs to suffix as `hoodie.sync.*` + - no more `hoodie.meta.sync.*` or `hoodie.meta_sync.*` + - no more variable name or class name like `metaSyncEnabled` or `metaSyncTool`; standardize as `hoodieSync*` to align with module name `hudi-sync` +- remove all sync related option constants from `DataSourceOptions` +- `database` and `table` should not be required by sync tool; they should be inferred from table properties +- users should not need to set PartitionValueExtractor; partition values should be inferred automatically +- remove `USE_JDBC` and fully adopt `SYNC_MODE` +- remove `HIVE_SYNC_ENDABLED` and related arguments from sync tools and delta streamers. Use `SYNC_ENABLED` +- migrate repeated sync config to original config + - `META_SYNC_BASE_FILE_FORMAT` -> `org.apache.hudi.common.table.HoodieTableConfig.BASE_FILE_FORMAT` + - `META_SYNC_PARTITION_FIELDS` -> `org.apache.hudi.common.table.HoodieTableConfig.PARTITION_FIELDS` + - `META_SYNC_ASSUME_DATE_PARTITION` -> `org.apache.hudi.common.config.HoodieMetadataConfig.ASSUME_DATE_PARTITIONING` + - `META_SYNC_DECODE_PARTITION` -> `org.apache.hudi.common.table.HoodieTableConfig.URL_ENCODE_PARTITIONING` + - `META_SYNC_USE_FILE_LISTING_FROM_METADATA` -> `org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE` + +## Rollout/Adoption Plan + +- Users who set `USE_JDBC` will need to change to set `SYNC_MODE=jdbc` +- Users who set `--enable-hive-sync` or `HIVE_SYNC_ENABLED` will need to drop the argument or config and change to `--enable-sync` or `SYNC_ENABLED`. +- Users who import from `DataSourceOptions` for meta sync constants will need to import relevant configs from `HoodieSyncConfig` and subclasses. +- Users who set `AwsGlueCatalogSyncTool` as sync tool class need to update the class name to `AWSGlueCatalogSyncTool` + +## Test Plan + +- CI covers most operations for Hive sync with HMS +- end-to-end testing with setup for Glue Catalog, BigQuery, DataHub instance +- manual testing with partitions added and removed