From baca3e93acff6240f079d20656835c6a0a16fe71 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Wed, 14 Jan 2026 10:33:30 -0800 Subject: [PATCH 01/32] Fix timestamp_millis issue --- .github/workflows/bot.yml | 180 +++- azure-pipelines-20230430.yml | 6 +- hudi-client/hudi-client-common/pom.xml | 46 + .../hudi/io/HoodieMergedReadHandle.java | 6 +- .../TimestampBasedAvroKeyGenerator.java | 6 +- .../org/apache/hudi/table/HoodieTable.java | 1 + .../action/commit/HoodieMergeHelper.java | 4 +- hudi-client/hudi-spark-client/pom.xml | 24 + .../utils/SparkInternalSchemaConverter.java | 22 +- .../io/storage/HoodieSparkParquetReader.java | 83 +- .../row/HoodieRowParquetWriteSupport.java | 14 +- .../ParquetBootstrapMetadataHandler.java | 4 +- .../org/apache/hudi/AvroConversionUtils.scala | 2 +- .../org/apache/hudi/HoodieSparkUtils.scala | 4 +- .../spark/HoodieSparkKryoRegistrar.scala | 4 +- .../HoodieParquetFileFormatHelper.scala | 38 +- .../apache/spark/sql/hudi/SparkAdapter.scala | 43 +- .../HoodieLegacyParquetFileFormatHelper.scala | 121 +++ .../parquet/HoodieParquetReadSupport.scala | 115 ++ .../parquet/SparkBasicSchemaEvolution.scala | 40 + .../hudi/client/TestTableSchemaEvolution.java | 4 +- .../hudi/testutils/HoodieClientTestUtils.java | 19 +- .../SparkClientFunctionalTestHarness.java | 11 +- hudi-common/pom.xml | 33 + .../parquet/schema/AvroSchemaRepair.java | 259 +++++ .../parquet/schema/TestAvroSchemaRepair.java | 984 ++++++++++++++++++ .../apache/hudi/BaseHoodieTableFileIndex.java | 5 + .../org/apache/hudi/avro/AvroSchemaCache.java | 47 + .../org/apache/hudi/avro/AvroSchemaUtils.java | 38 +- .../hudi/avro/ConvertingGenericData.java | 100 +- .../org/apache/hudi/avro/HoodieAvroUtils.java | 154 ++- .../hudi/avro/MercifulJsonConverter.java | 23 +- .../config/TimestampKeyGeneratorConfig.java | 3 +- .../table/log/block/HoodieAvroDataBlock.java | 20 +- .../view/AbstractTableFileSystemView.java | 13 + .../hudi/common/util/DateTimeUtils.java | 45 + .../apache/hudi/common/util/JsonUtils.java | 2 + .../apache/hudi/common/util/ParquetUtils.java | 4 +- .../hudi/common/util/ReflectionUtils.java | 11 + .../org/apache/hudi/internal/schema/Type.java | 7 +- .../apache/hudi/internal/schema/Types.java | 72 ++ .../convert/AvroInternalSchemaConverter.java | 93 +- .../internal/schema/utils/SerDeHelper.java | 8 + .../storage/HoodieAvroFileWriterFactory.java | 5 +- .../io/storage/HoodieAvroParquetReader.java | 31 +- .../metadata/HoodieTableMetadataUtil.java | 7 +- .../avro/HoodieAvroParquetReaderBuilder.java | 11 +- .../HoodieAvroParquetSchemaConverter.java | 56 + .../parquet/avro/HoodieAvroReadSupport.java | 24 +- .../avro/NativeAvroSchemaConverter.java | 46 + .../testutils/HoodieTestDataGenerator.java | 499 ++++++++- .../common/testutils/HoodieTestUtils.java | 50 + hudi-hadoop-common/pom.xml | 174 ++++ .../AvroSchemaConverterWithTimestampNTZ.java | 598 +++++++++++ .../table/ParquetTableSchemaResolver.java | 64 ++ .../apache/parquet/schema/SchemaRepair.java | 162 +++ .../parquet/avro/TestAvroSchemaConverter.java | 954 +++++++++++++++++ .../parquet/schema/TestSchemaRepair.java | 600 +++++++++++ .../schema/TestSchemaRepairEquivalence.java | 481 +++++++++ .../src/test/resources/parquet-java/all.avsc | 110 ++ .../allFromParquetNewBehavior.avsc | 108 ++ .../allFromParquetOldBehavior.avsc | 117 +++ .../resources/parquet-java/fixedToInt96.avsc | 97 ++ .../hudi/hadoop/HoodieParquetInputFormat.java | 3 +- .../hudi/hadoop/SchemaEvolutionContext.java | 4 + .../hadoop/avro/HoodieAvroParquetReader.java | 38 +- ...oodieTimestampAwareParquetInputFormat.java | 12 + .../hudi/hadoop/utils/HiveAvroSerializer.java | 12 +- .../apache/hudi/ColumnStatsIndexSupport.scala | 60 +- .../org/apache/hudi/HoodieBaseRelation.scala | 6 +- .../apache/hudi/HoodieDataSourceHelper.scala | 5 +- .../org/apache/hudi/HoodieFileIndex.scala | 4 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 4 +- .../hudi/SparkHoodieTableFileIndex.scala | 42 +- .../org/apache/hudi/cdc/CDCRelation.scala | 3 +- .../spark/sql/avro/SchemaConverters.scala | 77 +- .../LegacyHoodieParquetFileFormat.scala | 10 +- hudi-spark-datasource/hudi-spark/pom.xml | 78 ++ .../apache/spark/sql/hudi/SparkHelpers.scala | 1 + .../sql/hudi/analysis/HoodieAnalysis.scala | 10 +- .../InsertIntoHoodieTableCommand.scala | 32 +- .../command/payload/ExpressionPayload.scala | 8 +- .../parser/HoodieSqlCommonAstBuilder.scala | 34 +- .../TestSparkSortAndSizeClustering.java | 68 +- .../TestTimestampBasedKeyGenerator.java | 27 +- .../trips_logical_types_json_cow_read_v6.zip | Bin 0 -> 103597 bytes .../trips_logical_types_json_cow_read_v8.zip | Bin 0 -> 139172 bytes .../trips_logical_types_json_cow_read_v9.zip | Bin 0 -> 200295 bytes .../trips_logical_types_json_mor_read_v6.zip | Bin 0 -> 48290 bytes ...cal_types_json_mor_read_v6_parquet_log.zip | Bin 0 -> 55640 bytes .../trips_logical_types_json_mor_read_v8.zip | Bin 0 -> 68870 bytes ...cal_types_json_mor_read_v8_parquet_log.zip | Bin 0 -> 76217 bytes .../trips_logical_types_json_mor_read_v9.zip | Bin 0 -> 68838 bytes ...cal_types_json_mor_read_v9_parquet_log.zip | Bin 0 -> 76199 bytes .../TestAvroSchemaResolutionSupport.scala | 87 ++ .../hudi/TestHoodieSparkSqlWriter.scala | 5 +- .../functional/ColumnStatIndexTestBase.scala | 3 + .../hudi/functional/TestCOWDataSource.scala | 87 +- .../functional/TestColumnStatsIndex.scala | 9 +- .../TestColumnStatsIndexWithSQL.scala | 19 +- .../hudi/functional/TestMORDataSource.scala | 158 ++- .../spark/sql/adapter/Spark2Adapter.scala | 37 +- .../spark/sql/adapter/BaseSpark3Adapter.scala | 26 +- .../spark/sql/adapter/Spark3_0Adapter.scala | 6 +- .../spark/sql/adapter/Spark3_1Adapter.scala | 6 +- .../spark/sql/adapter/Spark3_2Adapter.scala | 6 +- ...Spark32LegacyHoodieParquetFileFormat.scala | 3 +- .../analysis/HoodieSpark32PlusAnalysis.scala | 9 +- .../spark/sql/adapter/Spark3_3Adapter.scala | 8 +- ...Spark33LegacyHoodieParquetFileFormat.scala | 3 +- .../spark/sql/adapter/Spark3_4Adapter.scala | 66 +- ...Spark34LegacyHoodieParquetFileFormat.scala | 186 ++-- .../hudi/hive/testutils/HiveTestService.java | 130 ++- hudi-timeline-service/pom.xml | 46 +- hudi-utilities/pom.xml | 27 + .../ChainedSchemaPostProcessor.java | 2 +- .../TestHoodieMetadataTableValidator.java | 330 ++++++ .../HoodieDeltaStreamerTestBase.java | 1 + .../TestHoodieDeltaStreamer.java | 515 ++++++++- ...odieDeltaStreamerSchemaEvolutionQuick.java | 44 +- .../sources/AbstractBaseTestSource.java | 20 +- .../test/resources/logical-repair/README.md | 88 ++ .../cow_write_updates/2/data.json | 6 + .../cow_write_updates/3/data.json | 3 + .../mor_write_updates/5/data.json | 3 + .../test/resources/logical-repair/schema.avsc | 112 ++ .../trips_logical_types_json_cow_write.zip | Bin 0 -> 61182 bytes ..._logical_types_json_mor_write_avro_log.zip | Bin 0 -> 54841 bytes ...gical_types_json_mor_write_parquet_log.zip | Bin 0 -> 59817 bytes .../source-timestamp-millis.avsc | 162 +++ pom.xml | 144 +++ scripts/release/validate_source_copyright.sh | 2 +- 132 files changed, 9380 insertions(+), 429 deletions(-) rename {hudi-spark-datasource/hudi-spark-common => hudi-client/hudi-spark-client}/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala (78%) create mode 100644 hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala create mode 100644 hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala create mode 100644 hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala create mode 100644 hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java create mode 100644 hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.java create mode 100644 hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java create mode 100644 hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java create mode 100644 hudi-hadoop-common/pom.xml create mode 100644 hudi-hadoop-common/src/avro/java/org/apache/parquet.avro/AvroSchemaConverterWithTimestampNTZ.java create mode 100644 hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java create mode 100644 hudi-hadoop-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java create mode 100644 hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java create mode 100644 hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepair.java create mode 100644 hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java create mode 100644 hudi-hadoop-common/src/test/resources/parquet-java/all.avsc create mode 100644 hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc create mode 100644 hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc create mode 100644 hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v6.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v8.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v9.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6_parquet_log.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8_parquet_log.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9.zip create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9_parquet_log.zip create mode 100644 hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java create mode 100644 hudi-utilities/src/test/resources/logical-repair/README.md create mode 100644 hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json create mode 100644 hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json create mode 100644 hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json create mode 100644 hudi-utilities/src/test/resources/logical-repair/schema.avsc create mode 100644 hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_cow_write.zip create mode 100644 hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_avro_log.zip create mode 100644 hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_parquet_log.zip create mode 100644 hudi-utilities/src/test/resources/streamer-config/source-timestamp-millis.avsc diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index fd3cc67976a16..5c851b8041c34 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -177,29 +177,197 @@ jobs: java-version: '17' distribution: 'adopt' architecture: x64 + cache: maven + - name: Verify Java 17 version + run: | + echo "JAVA_HOME: $JAVA_HOME" + java -version + which java - name: Quickstart Test + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + run: | + export PATH="$JAVA_HOME/bin:$PATH" + mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl hudi-examples/hudi-examples-spark $MVN_ARGS + - name: Java UT - Common & Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + run: | + export PATH="$JAVA_HOME/bin:$PATH" + mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + - name: Java FT - Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + if: ${{ !endsWith(env.SPARK_PROFILE, '3.4') }} # skip test spark 3.4 as it's covered by Azure CI + run: | + export PATH="$JAVA_HOME/bin:$PATH" + mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + + test-spark-java17-scala-tests: + runs-on: ubuntu-latest + strategy: + matrix: + include: + - scalaProfile: "scala-2.12" + sparkProfile: "spark3.3" + sparkModules: "hudi-spark-datasource/hudi-spark3.3.x" + - scalaProfile: "scala-2.12" + sparkProfile: "spark3.4" + sparkModules: "hudi-spark-datasource/hudi-spark3.4.x" + + steps: + - uses: actions/checkout@v3 + - name: Set up JDK 8 + uses: actions/setup-java@v3 + with: + java-version: '8' + distribution: 'temurin' + architecture: x64 + cache: maven + - name: Build Project env: SCALA_PROFILE: ${{ matrix.scalaProfile }} SPARK_PROFILE: ${{ matrix.sparkProfile }} run: - mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl hudi-examples/hudi-examples-spark $MVN_ARGS - - name: UT - Common & Spark + mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" + - name: Set up JDK 17 + uses: actions/setup-java@v3 + with: + java-version: '17' + distribution: 'temurin' + architecture: x64 + cache: maven + - name: Verify Java 17 version + run: | + echo "JAVA_HOME: $JAVA_HOME" + java -version + which java + - name: Scala UT - Common & Spark env: SCALA_PROFILE: ${{ matrix.scalaProfile }} SPARK_PROFILE: ${{ matrix.sparkProfile }} SPARK_MODULES: ${{ matrix.sparkModules }} - if: ${{ !endsWith(env.SPARK_PROFILE, '3.2') }} # skip test spark 3.2 as it's covered by Azure CI + run: | + export PATH="$JAVA_HOME/bin:$PATH" + mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + - name: Scala FT - Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + run: | + export PATH="$JAVA_HOME/bin:$PATH" + mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + + test-spark-java11-17-java-tests: + runs-on: ubuntu-latest + strategy: + matrix: + include: + - scalaProfile: "scala-2.12" + sparkProfile: "spark3.5" + sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" + - scalaProfile: "scala-2.13" + sparkProfile: "spark3.5" + sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" + + steps: + - uses: actions/checkout@v3 + - name: Set up JDK 11 + uses: actions/setup-java@v3 + with: + java-version: '11' + distribution: 'temurin' + architecture: x64 + cache: maven + - name: Build Project + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} run: - mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - - name: FT - Spark + mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" + - name: Set up JDK 17 + uses: actions/setup-java@v3 + with: + java-version: '17' + distribution: 'temurin' + architecture: x64 + cache: maven + - name: Quickstart Test + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + run: + mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl hudi-examples/hudi-examples-spark $MVN_ARGS + - name: Java UT - Common & Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + run: + mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + - name: Java FT - Spark env: SCALA_PROFILE: ${{ matrix.scalaProfile }} SPARK_PROFILE: ${{ matrix.sparkProfile }} SPARK_MODULES: ${{ matrix.sparkModules }} - if: ${{ !endsWith(env.SPARK_PROFILE, '3.2') }} # skip test spark 3.2 as it's covered by Azure CI run: mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + test-spark-java11-17-scala-tests: + runs-on: ubuntu-latest + strategy: + matrix: + include: + - scalaProfile: "scala-2.12" + sparkProfile: "spark3.5" + sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" + - scalaProfile: "scala-2.13" + sparkProfile: "spark3.5" + sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" + + steps: + - uses: actions/checkout@v3 + - name: Set up JDK 11 + uses: actions/setup-java@v3 + with: + java-version: '11' + distribution: 'temurin' + architecture: x64 + cache: maven + - name: Build Project + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + run: + mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" + - name: Set up JDK 17 + uses: actions/setup-java@v3 + with: + java-version: '17' + distribution: 'temurin' + architecture: x64 + cache: maven + - name: Scala UT - Common & Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + run: + mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + - name: Scala FT - Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + run: + mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + test-flink: runs-on: ubuntu-latest strategy: diff --git a/azure-pipelines-20230430.yml b/azure-pipelines-20230430.yml index 85d185fbc2c5c..59d48de4271e7 100644 --- a/azure-pipelines-20230430.yml +++ b/azure-pipelines-20230430.yml @@ -47,7 +47,7 @@ parameters: default: - 'hudi-spark-datasource' - 'hudi-spark-datasource/hudi-spark' - - 'hudi-spark-datasource/hudi-spark3.2.x' + - 'hudi-spark-datasource/hudi-spark3.4.x' - 'hudi-spark-datasource/hudi-spark3.2plus-common' - 'hudi-spark-datasource/hudi-spark3-common' - 'hudi-spark-datasource/hudi-spark-common' @@ -73,7 +73,7 @@ parameters: - '!hudi-flink-datasource/hudi-flink1.18.x' - '!hudi-spark-datasource' - '!hudi-spark-datasource/hudi-spark' - - '!hudi-spark-datasource/hudi-spark3.2.x' + - '!hudi-spark-datasource/hudi-spark3.4.x' - '!hudi-spark-datasource/hudi-spark3.2plus-common' - '!hudi-spark-datasource/hudi-spark3-common' - '!hudi-spark-datasource/hudi-spark-common' @@ -97,7 +97,7 @@ parameters: - '!hudi-flink-datasource/hudi-flink1.18.x' variables: - BUILD_PROFILES: '-Dscala-2.12 -Dspark3.2 -Dflink1.18' + BUILD_PROFILES: '-Dscala-2.12 -Dspark3.4 -Dflink1.18' PLUGIN_OPTS: '-Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn' MVN_OPTS_INSTALL: '-Phudi-platform-service -DskipTests $(BUILD_PROFILES) $(PLUGIN_OPTS) -Dmaven.wagon.httpconnectionManager.ttlSeconds=25 -Dmaven.wagon.http.retryHandler.count=5' MVN_OPTS_TEST: '-fae -Pwarn-log $(BUILD_PROFILES) $(PLUGIN_OPTS)' diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 7a10ae12c35dd..e38327283df9f 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -53,6 +53,48 @@ org.apache.hudi hudi-timeline-service ${project.version} + + + + org.eclipse.jetty + * + + + + + + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-http + + + org.eclipse.jetty + jetty-io + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-webapp + + + org.eclipse.jetty + jetty-xml + + + org.eclipse.jetty + jetty-security @@ -188,6 +230,10 @@ org.pentaho * + + org.codehaus.janino + janino + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java index 738688c62193a..15ef4367d2d0d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java @@ -19,6 +19,7 @@ package org.apache.hudi.io; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieLogFile; @@ -59,9 +60,12 @@ public HoodieMergedReadHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathFileIDPair) { super(config, instantTime, hoodieTable, partitionPathFileIDPair); - readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField()); + Schema orignalReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField()); // config.getSchema is not canonicalized, while config.getWriteSchema is canonicalized. So, we have to use the canonicalized schema to read the existing data. baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField()); + // Repair reader schema. + // Assume writer schema should be correct. If not, no repair happens. + readerSchema = AvroSchemaUtils.getRepairedSchema(orignalReaderSchema, baseFileReaderSchema); } public List> getMergedRecords() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java index 1990b2dab44ef..ea2e0911d3010 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java @@ -41,6 +41,7 @@ import java.util.TimeZone; import java.util.concurrent.TimeUnit; +import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.DATE_TIME_PARSER; @@ -54,7 +55,7 @@ */ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator { public enum TimestampType implements Serializable { - UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS, SCALAR + UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS, EPOCHMICROSECONDS, SCALAR } private final TimeUnit timeUnit; @@ -93,6 +94,9 @@ public TimestampBasedAvroKeyGenerator(TypedProperties config) throws IOException case EPOCHMILLISECONDS: timeUnit = MILLISECONDS; break; + case EPOCHMICROSECONDS: + timeUnit = MICROSECONDS; + break; case UNIX_TIMESTAMP: timeUnit = SECONDS; break; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index dfa464d8af8b5..080479fc417d6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -45,6 +45,7 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; 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 c1523d564e480..8b8848fa95d6e 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 @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.commit; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; @@ -86,7 +87,8 @@ public void runMerge(HoodieTable table, HoodieFileReader bootstrapFileReader = null; Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields(); - Schema readerSchema = baseFileReader.getSchema(); + Schema readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema); + // In case Advanced Schema Evolution is enabled we might need to rewrite currently // persisted records to adhere to an evolved schema diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index fa437494fd9f5..10b98bfb1dbba 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -169,6 +169,10 @@ org.pentaho * + + org.codehaus.janino + janino + @@ -253,6 +257,26 @@ org.apache.rat apache-rat-plugin + + org.codehaus.mojo + build-helper-maven-plugin + 3.5.0 + + + add-spark32plus-parquet-sources + generate-sources + + add-source + + + ${spark31orEarlier} + + src/parquet/scala + + + + + diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java index 2b14bb3a0665b..7b7c18dd95c66 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java @@ -61,6 +61,7 @@ import org.apache.spark.sql.types.UserDefinedType; import org.apache.spark.sql.types.VarcharType; +import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; import java.sql.Date; import java.util.ArrayList; @@ -80,6 +81,21 @@ private SparkInternalSchemaConverter() { public static final String HOODIE_TABLE_PATH = "hoodie.tablePath"; public static final String HOODIE_VALID_COMMITS_LIST = "hoodie.valid.commits.list"; + /** + * Get TimestampNTZType$ using reflection, as it's only available in Spark 3.3+. + * Falls back to TimestampType$ if TimestampNTZType is not available. + */ + private static DataType getTimestampNTZType() { + try { + Class timestampNTZTypeClass = Class.forName("org.apache.spark.sql.types.TimestampNTZType$"); + Field moduleField = timestampNTZTypeClass.getField("MODULE$"); + return (DataType) moduleField.get(null); + } catch (ClassNotFoundException | NoSuchFieldException | IllegalAccessException e) { + // TimestampNTZType is not available in this Spark version, fall back to TimestampType + return TimestampType$.MODULE$; + } + } + public static Type buildTypeFromStructType(DataType sparkType, Boolean firstVisitRoot, AtomicInteger nextId) { if (sparkType instanceof StructType) { StructField[] fields = ((StructType) sparkType).fields(); @@ -265,10 +281,14 @@ private static DataType constructSparkSchemaFromType(Type type) { case DATE: return DateType$.MODULE$; case TIME: + case TIME_MILLIS: throw new UnsupportedOperationException(String.format("cannot convert %s type to Spark", type)); case TIMESTAMP: - // todo support TimeStampNTZ + case TIMESTAMP_MILLIS: return TimestampType$.MODULE$; + case LOCAL_TIMESTAMP_MILLIS: + case LOCAL_TIMESTAMP_MICROS: + return getTimestampNTZType(); case STRING: return StringType$.MODULE$; case UUID: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 2a22eacea8c5a..bc41d6e972d60 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -18,22 +18,26 @@ package org.apache.hudi.io.storage; -import org.apache.avro.Schema; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; + import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.util.BaseFileUtils; -import org.apache.hudi.common.util.collection.ClosableIterator; -import org.apache.hudi.common.util.collection.CloseableMappingIterator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetReaderIterator; import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.CloseableMappingIterator; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.api.ReadSupport; import org.apache.parquet.schema.MessageType; @@ -53,6 +57,7 @@ import static org.apache.hudi.common.util.TypeUtils.unsafeCast; import static org.apache.parquet.avro.AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; public class HoodieSparkParquetReader implements HoodieSparkFileReader { @@ -60,6 +65,10 @@ public class HoodieSparkParquetReader implements HoodieSparkFileReader { private final Configuration conf; private final BaseFileUtils parquetUtils; private List readerIterators = new ArrayList<>(); + public static final String ENABLE_LOGICAL_TIMESTAMP_REPAIR = "spark.hudi.logicalTimestampField.repair.enable"; + private Option fileSchemaOption = Option.empty(); + private Option structTypeOption = Option.empty(); + private Option schemaOption = Option.empty(); public HoodieSparkParquetReader(Configuration conf, Path path) { this.path = path; @@ -118,35 +127,63 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema if (requestedSchema == null) { requestedSchema = readerSchema; } - StructType readerStructType = HoodieInternalRowUtils.getCachedSchema(readerSchema); - StructType requestedStructType = HoodieInternalRowUtils.getCachedSchema(requestedSchema); - conf.set(ParquetReadSupport.PARQUET_READ_SCHEMA, readerStructType.json()); - conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), requestedStructType.json()); - conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING())); - conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP())); - ParquetReader reader = ParquetReader.builder((ReadSupport) new ParquetReadSupport(), path) - .withConf(conf) + MessageType fileSchema = getFileSchema(); + Schema nonNullSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema); + Option messageSchema = Option.of(getAvroSchemaConverter(conf).convert(nonNullSchema)); + Pair readerSchemas = + SparkAdapterSupport$.MODULE$.sparkAdapter().getReaderSchemas(conf, readerSchema, requestedSchema, fileSchema); + conf.set(ParquetReadSupport.PARQUET_READ_SCHEMA, readerSchemas.getLeft().json()); + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), readerSchemas.getRight().json()); + conf.set(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING()).toString()); + conf.set(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP()).toString()); + ParquetReader reader = ParquetReader.builder( + (ReadSupport) SparkAdapterSupport$.MODULE$.sparkAdapter().getParquetReadSupport(messageSchema), + new Path(path.toUri())).withConf(conf) .build(); ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); readerIterators.add(parquetReaderIterator); return parquetReaderIterator; } + private MessageType getFileSchema() { + if (fileSchemaOption.isEmpty()) { + MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(storage, path); + fileSchemaOption = Option.of(messageType); + } + return fileSchemaOption.get(); + } + @Override public Schema getSchema() { - // Some types in avro are not compatible with parquet. - // Avro only supports representing Decimals as fixed byte array - // and therefore if we convert to Avro directly we'll lose logical type-info. - MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path); - StructType structType = new ParquetToSparkSchemaConverter(conf).convert(messageType); - return SparkAdapterSupport$.MODULE$.sparkAdapter() - .getAvroSchemaConverters() - .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING); + if (schemaOption.isPresent()) { + // Some types in avro are not compatible with parquet. + // Avro only supports representing Decimals as fixed byte array + // and therefore if we convert to Avro directly we'll lose logical type-info. + MessageType messageType = getFileSchema(); + StructType structType = getStructSchema(); + schemaOption = Option.of(SparkAdapterSupport$.MODULE$.sparkAdapter() + .getAvroSchemaConverters() + .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING)); + } + return schemaOption.get(); + } + + protected StructType getStructSchema() { + if (structTypeOption.isPresent()) { + MessageType messageType = getFileSchema(); + structTypeOption = Option.of(convertToStruct(messageType)); + } + return structTypeOption.get(); + } + + private StructType convertToStruct(MessageType messageType) { + return new ParquetToSparkSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(messageType); +>>>>>>> 086b6aad2e27 (Fix timestamp_millis issue) } @Override public void close() { - readerIterators.forEach(ParquetReaderIterator::close); + readerIterators.forEach(it -> it.close()); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 3a1b6d000becc..d89f397f703d5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -18,11 +18,14 @@ package org.apache.hudi.io.storage.row; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.HoodieBloomFilterWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; + +import org.apache.hadoop.conf.Configuration; import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; import org.apache.spark.sql.types.StructType; @@ -89,4 +92,13 @@ protected UTF8String dereference(UTF8String key) { } } + public static HoodieRowParquetWriteSupport getHoodieRowParquetWriteSupport(Configuration conf, + StructType structType, + Option bloomFilterOpt, + HoodieConfig config) { + return (HoodieRowParquetWriteSupport) ReflectionUtils.loadClass( + config.getStringOrDefault(HoodieStorageConfig.HOODIE_PARQUET_SPARK_ROW_WRITE_SUPPORT_CLASS), + new Class[] {Configuration.class, StructType.class, Option.class, HoodieConfig.class}, + conf, structType, bloomFilterOpt, config); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index 2c3ddfdcda2ce..f6874e28d5d95 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -40,7 +40,6 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -56,6 +55,7 @@ import java.util.function.Function; import static org.apache.hudi.io.HoodieBootstrapHandle.METADATA_BOOTSTRAP_RECORD_SCHEMA; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { @@ -68,7 +68,7 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException { ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath, ParquetMetadataConverter.NO_FILTER); MessageType parquetSchema = readFooter.getFileMetaData().getSchema(); - return new AvroSchemaConverter().convert(parquetSchema); + return getAvroSchemaConverter(table.getHadoopConf()).convert(parquetSchema); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index d84679eaf923a..a107d00466e62 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -84,7 +84,7 @@ object AvroConversionUtils { recordNamespace: String): Row => GenericRecord = { val serde = sparkAdapter.createSparkRowSerDe(sourceSqlType) val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(sourceSqlType, structName, recordNamespace) - val nullable = AvroSchemaUtils.resolveNullableSchema(avroSchema) != avroSchema + val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(avroSchema) != avroSchema val converter = AvroConversionUtils.createInternalRowToAvroConverter(sourceSqlType, avroSchema, nullable) 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 a0fe879b3dbea..efd32bae76487 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 @@ -93,7 +93,7 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi // making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion // (and back) val sameSchema = writerAvroSchema.equals(readerAvroSchema) - val nullable = AvroSchemaUtils.resolveNullableSchema(writerAvroSchema) != writerAvroSchema + val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(writerAvroSchema) != writerAvroSchema // NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark // serializer is not able to digest it @@ -152,7 +152,7 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi // making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion // (and back) val sameSchema = writerAvroSchema.equals(readerAvroSchema) - val nullable = AvroSchemaUtils.resolveNullableSchema(writerAvroSchema) != writerAvroSchema + val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(writerAvroSchema) != writerAvroSchema // NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark // serializer is not able to digest it diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala index dd98227d4407c..b22a0f070364f 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala @@ -89,6 +89,8 @@ object HoodieSparkKryoRegistrar { private val KRYO_USER_REGISTRATORS = "spark.kryo.registrator" def register(conf: SparkConf): SparkConf = { - conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[HoodieSparkKryoRegistrar].getName).mkString(",")) + // Use class name directly to avoid Scala collection binary compatibility issues + // when compiled with Scala 2.13 but running with Spark 3.5 (Scala 2.12) + conf.set(KRYO_USER_REGISTRATORS, classOf[HoodieSparkKryoRegistrar].getName) } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala similarity index 78% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala index 599bbebe4f6c4..3e13e3bf7f9de 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala @@ -1,34 +1,41 @@ /* - * 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 + * 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 + * 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. + * 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.execution.datasources.parquet import org.apache.hadoop.conf.Configuration import org.apache.parquet.hadoop.metadata.FileMetaData -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} +import org.apache.spark.sql.types._ object HoodieParquetFileFormatHelper { - def buildImplicitSchemaChangeInfo(hadoopConf: Configuration, parquetFileMetaData: FileMetaData, requiredSchema: StructType): (java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]], StructType) = { - val implicitTypeChangeInfo: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]] = new java.util.HashMap() val convert = new ParquetToSparkSchemaConverter(hadoopConf) val fileStruct = convert.convert(parquetFileMetaData.getSchema) + buildImplicitSchemaChangeInfo(fileStruct, requiredSchema) + } + + def buildImplicitSchemaChangeInfo(fileStruct: StructType, + requiredSchema: StructType): (java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]], StructType) = { + val implicitTypeChangeInfo: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]] = new java.util.HashMap() + val fileStructMap = fileStruct.fields.map(f => (f.name, f.dataType)).toMap // if there are missing fields or if field's data type needs to be changed while reading, we handle it here. val sparkRequestStructFields = requiredSchema.map(f => { @@ -45,6 +52,7 @@ object HoodieParquetFileFormatHelper { } def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = (requiredType, fileType) match { + case (requiredType, fileType) if requiredType == fileType => true case (ArrayType(rt, _), ArrayType(ft, _)) => diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 1c6111afe47f3..e8058aa1f2248 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -19,14 +19,18 @@ package org.apache.spark.sql.hudi import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient + +import org.apache.parquet.schema.MessageType import org.apache.spark.sql._ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConverters, HoodieAvroSerializer} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, InterpretedPredicate} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} @@ -36,6 +40,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, Metadata, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} import org.apache.spark.storage.StorageLevel @@ -48,10 +53,21 @@ import java.util.{Locale, TimeZone} trait SparkAdapter extends Serializable { /** - * Checks whether provided instance of [[InternalRow]] is actually an instance of [[ColumnarBatchRow]] + * Checks whether provided instance of [[InternalRow]] is actually an instance of [[org.apache.spark.sql.vectorized.ColumnarBatchRow]] */ def isColumnarBatchRow(r: InternalRow): Boolean + def isTimestampNTZType(dataType: DataType): Boolean + + def getParquetReadSupport(messageSchema: org.apache.hudi.common.util.Option[MessageType]): org.apache.parquet.hadoop.api.ReadSupport[_] + + def repairSchemaIfSpecified(shouldRepair: Boolean, + fileSchema: MessageType, + tableSchemaOpt: org.apache.hudi.common.util.Option[MessageType]): MessageType + + def getReaderSchemas(conf: Configuration, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): + org.apache.hudi.common.util.collection.Pair[StructType, StructType] + /** * Creates Catalyst [[Metadata]] for Hudi's meta-fields (designating these w/ * [[METADATA_COL_ATTR_KEY]] if available (available in Spark >= 3.2) @@ -65,7 +81,7 @@ trait SparkAdapter extends Serializable { /** * Returns an instance of [[HoodieCatalogUtils]] providing for common utils operating on Spark's - * [[TableCatalog]]s + * [[org.apache.spark.sql.connector.catalog.TableCatalog]]s */ def getCatalogUtils: HoodieCatalogUtils @@ -169,7 +185,7 @@ trait SparkAdapter extends Serializable { /** * Create instance of [[ParquetFileFormat]] */ - def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] + def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] def makeColumnarBatch(vectors: Array[ColumnVector], numRows: Int): ColumnarBatch @@ -200,7 +216,7 @@ trait SparkAdapter extends Serializable { metadataColumns: Seq[AttributeReference] = Seq.empty): FileScanRDD /** - * Extract condition in [[DeleteFromTable]] + * Extract condition in [[org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable]] * SPARK-38626 condition is no longer Option in Spark 3.3 */ def extractDeleteCondition(deleteFromTable: Command): Expression @@ -214,4 +230,23 @@ trait SparkAdapter extends Serializable { * Tries to translate a Catalyst Expression into data source Filter */ def translateFilter(predicate: Expression, supportNestedPredicatePushdown: Boolean = false): Option[Filter] + + /** + * @param sparkSession Spark session (required for Spark 3.5 to access Analyzer) + * @param tableName table name + * @param expected expected attributes + * @param query query logical plan + * @param byName whether to match by name + * @param conf SQL configuration + * @return resolved logical plan + */ + def resolveOutputColumns(sparkSession: SparkSession, + tableName: String, + expected: Seq[Attribute], + query: LogicalPlan, + byName: Boolean, + conf: SQLConf): LogicalPlan = { + // Default implementation delegates to CatalystPlanUtils (for Spark < 3.5) + getCatalystPlanUtils.resolveOutputColumns(tableName, expected, query, byName, conf) + } } diff --git a/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala new file mode 100644 index 0000000000000..5b303e3119dbf --- /dev/null +++ b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala @@ -0,0 +1,121 @@ +/* + * 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.execution.datasources.parquet + +import org.apache.spark.sql.HoodieSchemaUtils +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, Cast, CreateNamedStruct, CreateStruct, Expression, GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, NamedLambdaVariable, UnsafeProjection} +import org.apache.spark.sql.types._ + +object HoodieLegacyParquetFileFormatHelper { + def generateUnsafeProjection(fullSchema: Seq[Attribute], + timeZoneId: Option[String], + typeChangeInfos: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]], + requiredSchema: StructType, + partitionSchema: StructType, + schemaUtils: HoodieSchemaUtils): UnsafeProjection = { + val addedCastCache = scala.collection.mutable.HashMap.empty[(DataType, DataType), Boolean] + + def hasUnsupportedConversion(src: DataType, dst: DataType): Boolean = { + addedCastCache.getOrElseUpdate((src, dst), { + (src, dst) match { + case (FloatType, DoubleType) => true + case (IntegerType, DecimalType()) => true + case (LongType, DecimalType()) => true + case (FloatType, DecimalType()) => true + case (DoubleType, DecimalType()) => true + case (StringType, DecimalType()) => true + case (StringType, DateType) => true + case (StructType(srcFields), StructType(dstFields)) => + srcFields.zip(dstFields).exists { case (sf, df) => hasUnsupportedConversion(sf.dataType, df.dataType) } + case (ArrayType(sElem, _), ArrayType(dElem, _)) => + hasUnsupportedConversion(sElem, dElem) + case (MapType(sKey, sVal, _), MapType(dKey, dVal, _)) => + hasUnsupportedConversion(sKey, dKey) || hasUnsupportedConversion(sVal, dVal) + case _ => false + } + }) + } + + def recursivelyCastExpressions(expr: Expression, srcType: DataType, dstType: DataType): Expression = { + lazy val needTimeZone = Cast.needsTimeZone(srcType, dstType) + (srcType, dstType) match { + case (FloatType, DoubleType) => + val toStr = Cast(expr, StringType, if (needTimeZone) timeZoneId else None) + Cast(toStr, dstType, if (needTimeZone) timeZoneId else None) + case (IntegerType | LongType | FloatType | DoubleType, dec: DecimalType) => + val toStr = Cast(expr, StringType, if (needTimeZone) timeZoneId else None) + Cast(toStr, dec, if (needTimeZone) timeZoneId else None) + case (StringType, dec: DecimalType) => + Cast(expr, dec, if (needTimeZone) timeZoneId else None) + case (StringType, DateType) => + Cast(expr, DateType, if (needTimeZone) timeZoneId else None) + case (s: StructType, d: StructType) if hasUnsupportedConversion(s, d) => + val structFields = s.fields.zip(d.fields).zipWithIndex.map { + case ((srcField, dstField), i) => + val child = GetStructField(expr, i, Some(dstField.name)) + recursivelyCastExpressions(child, srcField.dataType, dstField.dataType) + } + CreateNamedStruct(d.fields.zip(structFields).flatMap { + case (f, c) => Seq(Literal(f.name), c) + }) + case (ArrayType(sElementType, containsNull), ArrayType(dElementType, _)) if hasUnsupportedConversion(sElementType, dElementType) => + val lambdaVar = NamedLambdaVariable("element", sElementType, containsNull) + val body = recursivelyCastExpressions(lambdaVar, sElementType, dElementType) + val func = LambdaFunction(body, Seq(lambdaVar)) + ArrayTransform(expr, func) + case (MapType(sKeyType, sValType, vnull), MapType(dKeyType, dValType, _)) + if hasUnsupportedConversion(sKeyType, dKeyType) || hasUnsupportedConversion(sValType, dValType) => + val kv = NamedLambdaVariable("kv", new StructType() + .add("key", sKeyType, nullable = false) + .add("value", sValType, nullable = vnull), nullable = false) + val newKey = recursivelyCastExpressions(GetStructField(kv, 0), sKeyType, dKeyType) + val newVal = recursivelyCastExpressions(GetStructField(kv, 1), sValType, dValType) + val entry = CreateStruct(Seq(newKey, newVal)) + val func = LambdaFunction(entry, Seq(kv)) + val transformed = ArrayTransform(MapEntries(expr), func) + MapFromEntries(transformed) + case _ => + // most cases should be covered here we only need to do the recursive work for float to double + Cast(expr, dstType, if (needTimeZone) timeZoneId else None) + } + } + + if (typeChangeInfos.isEmpty) { + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + } else { + // find type changed. + val newSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => + if (typeChangeInfos.containsKey(i)) { + StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) + } else f + }) + val newFullSchema = schemaUtils.toAttributes(newSchema) ++ schemaUtils.toAttributes(partitionSchema) + val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => + if (typeChangeInfos.containsKey(i)) { + val srcType = typeChangeInfos.get(i).getRight + val dstType = typeChangeInfos.get(i).getLeft + recursivelyCastExpressions(attr, srcType, dstType) + } else attr + } + GenerateUnsafeProjection.generate(castSchema, newFullSchema) + } + } +} diff --git a/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala new file mode 100644 index 0000000000000..f19a2952a6637 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala @@ -0,0 +1,115 @@ +/* + * 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.execution.datasources.parquet + +import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.common.util.{Option => HOption} +import org.apache.hudi.common.util.ValidationUtils +import org.apache.parquet.hadoop.api.InitContext +import org.apache.parquet.hadoop.api.ReadSupport.ReadContext +import org.apache.parquet.schema.{GroupType, MessageType, Type, Types} +import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec + +import java.time.ZoneId +import scala.collection.JavaConverters._ + +class HoodieParquetReadSupport(convertTz: Option[ZoneId], + enableVectorizedReader: Boolean, + val enableTimestampFieldRepair: Boolean, + datetimeRebaseSpec: RebaseSpec, + int96RebaseSpec: RebaseSpec, + tableSchemaOpt: HOption[MessageType] = HOption.empty()) + extends ParquetReadSupport(convertTz, enableVectorizedReader, datetimeRebaseSpec, int96RebaseSpec) with SparkAdapterSupport { + + override def init(context: InitContext): ReadContext = { + val readContext = super.init(context) + // Repairing is needed here because this is the schema that is used by the reader to decide what + // conversions are necessary + val requestedParquetSchema = if (enableTimestampFieldRepair) { + HoodieParquetReadSupport.getRepairedSchema(readContext.getRequestedSchema, tableSchemaOpt) + } else { + readContext.getRequestedSchema + } + val trimmedParquetSchema = HoodieParquetReadSupport.trimParquetSchema(requestedParquetSchema, context.getFileSchema) + new ReadContext(trimmedParquetSchema, readContext.getReadSupportMetadata) + } +} + +object HoodieParquetReadSupport { + /** + * Removes any fields from the parquet schema that do not have any child fields in the actual file schema after the + * schema is trimmed down to the requested fields. This can happen when the table schema evolves and only a subset of + * the nested fields are required by the query. + * + * @param requestedSchema the initial parquet schema requested by Spark + * @param fileSchema the actual parquet schema of the file + * @return a potentially updated schema with empty struct fields removed + */ + def trimParquetSchema(requestedSchema: MessageType, fileSchema: MessageType): MessageType = { + val trimmedFields = requestedSchema.getFields.asScala.map(field => { + if (fileSchema.containsField(field.getName)) { + trimParquetType(field, fileSchema.asGroupType().getType(field.getName)) + } else { + Some(field) + } + }).filter(_.isDefined).map(_.get).toArray[Type] + Types.buildMessage().addFields(trimmedFields: _*).named(requestedSchema.getName) + } + + private def trimParquetType(requestedType: Type, fileType: Type): Option[Type] = { + if (requestedType.equals(fileType)) { + Some(requestedType) + } else { + requestedType match { + case groupType: GroupType => + ValidationUtils.checkState(!fileType.isPrimitive, + "Group type provided by requested schema but existing type in the file is a primitive") + val fileTypeGroup = fileType.asGroupType() + var hasMatchingField = false + val fields = groupType.getFields.asScala.map(field => { + if (fileTypeGroup.containsField(field.getName)) { + hasMatchingField = true + trimParquetType(field, fileType.asGroupType().getType(field.getName)) + } else { + Some(field) + } + }).filter(_.isDefined).map(_.get).asJava + if (hasMatchingField && !fields.isEmpty) { + Some(groupType.withNewFields(fields)) + } else { + None + } + case _ => Some(requestedType) + } + } + } + + def getRepairedSchema(fileSchema: MessageType, + tableSchema: org.apache.hudi.common.util.Option[MessageType]): MessageType = { + try { + val schemaRepairClass = Class.forName("org.apache.parquet.schema.SchemaRepair") + val repairMethod = schemaRepairClass.getMethod( + "repairLogicalTypes", classOf[MessageType], classOf[org.apache.hudi.common.util.Option[MessageType]]) + repairMethod.invoke(null, fileSchema, tableSchema).asInstanceOf[MessageType] + } catch { + case _: Exception => fileSchema + } + } +} diff --git a/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala new file mode 100644 index 0000000000000..6a6a094bbac3c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala @@ -0,0 +1,40 @@ +/* + * 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.execution.datasources.parquet + +import org.apache.spark.sql.types.StructType + +/** + * Intended to be used just with HoodieSparkParquetReader to avoid any java/scala issues + */ +class SparkBasicSchemaEvolution(fileSchema: StructType, + requiredSchema: StructType, + sessionLocalTimeZone: String) { + + val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(fileSchema, requiredSchema) + + def getRequestSchema: StructType = { + if (implicitTypeChangeInfo.isEmpty) { + requiredSchema + } else { + sparkRequestSchema + } + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 1a0d5a95f9a0f..856d8a3c9e88e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -72,7 +72,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { public static final String EXTRA_FIELD_WITHOUT_DEFAULT_SCHEMA = "{\"name\": \"new_field_without_default\", \"type\": \"boolean\"},"; public static final String EXTRA_FIELD_NULLABLE_SCHEMA = - ",{\"name\": \"new_field_without_default\", \"type\": [\"boolean\", \"null\"]}"; + "{\"name\": \"new_field_without_default\", \"type\": [\"null\", \"boolean\"], \"default\": null},"; // TRIP_EXAMPLE_SCHEMA with a new_field added public static final String TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA @@ -152,7 +152,7 @@ public void testSchemaCompatibilityBasic() { "Added field without default and not nullable is not compatible (Evolved Schema)"); assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA - + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX + EXTRA_FIELD_NULLABLE_SCHEMA, false), + + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_NULLABLE_SCHEMA + TRIP_SCHEMA_SUFFIX, false), "Added nullable field is compatible (Evolved Schema)"); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index a7808ea938248..b47a763b6b245 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -107,9 +107,16 @@ public static SparkConf getSparkConfForTest(String appName) { sparkConf.set("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension"); } - if (canLoadClass("org.apache.spark.sql.hudi.catalog.HoodieCatalog") && HoodieSparkUtils.gteqSpark3_2()) { - sparkConf.set("spark.sql.catalog.spark_catalog", - "org.apache.spark.sql.hudi.catalog.HoodieCatalog"); + try { + if (canLoadClass("org.apache.spark.sql.hudi.catalog.HoodieCatalog") && HoodieSparkUtils.gteqSpark3_2()) { + sparkConf.set("spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.hudi.catalog.HoodieCatalog"); + } + } catch (LinkageError e) { + // Handle Scala version compatibility issues (e.g., Scala 2.12 vs 2.13) + // If we can't determine the Spark version, skip setting the catalog + // This can happen when Scala code compiled with 2.12 references types removed in 2.13 + // LinkageError catches both NoClassDefFoundError and other linkage errors } String evlogDir = System.getProperty("SPARK_EVLOG_DIR"); @@ -342,6 +349,12 @@ private static boolean canLoadClass(String className) { return ReflectionUtils.getClass(className) != null; } catch (Exception e) { return false; + } catch (NoClassDefFoundError e) { + // Handle cases where class exists but dependencies are missing (e.g., Scala 2.12 vs 2.13 compatibility) + return false; + } catch (Throwable e) { + // Catch any other errors (LinkageError, etc.) that might occur during class loading + return false; } } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index 511613d904438..9546cb5349530 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -112,8 +112,15 @@ public static Map getSparkSqlConf() { Map sqlConf = new HashMap<>(); sqlConf.put("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension"); - if (HoodieSparkUtils.gteqSpark3_2()) { - sqlConf.put("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.hudi.catalog.HoodieCatalog"); + try { + if (HoodieSparkUtils.gteqSpark3_2()) { + sqlConf.put("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.hudi.catalog.HoodieCatalog"); + } + } catch (LinkageError e) { + // Handle Scala version compatibility issues (e.g., Scala 2.12 vs 2.13) + // If we can't determine the Spark version, skip setting the catalog + // This can happen when Scala code compiled with 2.12 references types removed in 2.13 + // LinkageError catches both NoClassDefFoundError and other linkage errors } return sqlConf; diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 1d5cda0f75e82..48074821c1857 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -54,6 +54,39 @@ false + + org.codehaus.mojo + build-helper-maven-plugin + 3.5.0 + + + add-spark34plus-avro-sources + generate-sources + + add-source + + + ${spark33orEarlier} + + src/avro/java + + + + + add-spark34plus-avro-test-sources + generate-test-sources + + add-test-source + + + ${spark33orEarlier} + + src/avro/test/java + + + + + org.apache.rat apache-rat-plugin diff --git a/hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java b/hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java new file mode 100644 index 0000000000000..75f880ae3f955 --- /dev/null +++ b/hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java @@ -0,0 +1,259 @@ +/* + * 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.parquet.schema; + +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.avro.AvroSchemaUtils; +import org.apache.hudi.avro.HoodieAvroUtils; + +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class AvroSchemaRepair { + public static boolean isLocalTimestampSupported = isLocalTimestampMillisSupported(); + + public static Schema repairLogicalTypes(Schema fileSchema, Schema tableSchema) { + Schema repairedSchema = repairAvroSchema(fileSchema, tableSchema); + if (repairedSchema != fileSchema) { + return AvroSchemaCache.intern(repairedSchema); + } + return fileSchema; + } + + /** + * Performs schema repair on a schema, handling nullable unions. + */ + private static Schema repairAvroSchema(Schema fileSchema, Schema tableSchema) { + // Always resolve nullable schemas first (returns unchanged if not a union) + Schema nonNullFileSchema = AvroSchemaUtils.getNonNullTypeFromUnion(fileSchema); + Schema nonNullTableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(tableSchema); + + // Perform repair on the non-null types + Schema nonNullRepairedSchema = repairAvroSchemaNonNull(nonNullFileSchema, nonNullTableSchema); + + // If nothing changed, return the original schema + if (nonNullRepairedSchema == nonNullFileSchema) { + return fileSchema; + } + + // If the original was a union, wrap the repaired schema back in a nullable union + if (fileSchema.getType() == Schema.Type.UNION) { + return AvroSchemaUtils.createNullableSchema(nonNullRepairedSchema); + } + + return nonNullRepairedSchema; + } + + /** + * Repairs non-nullable schemas (after unions have been resolved). + */ + private static Schema repairAvroSchemaNonNull(Schema fileSchema, Schema tableSchema) { + // If schemas are already equal, nothing to repair + if (fileSchema.equals(tableSchema)) { + return fileSchema; + } + + // If types are different, no repair can be done + if (fileSchema.getType() != tableSchema.getType()) { + return fileSchema; + } + + // Handle record types (nested structs) + if (fileSchema.getType() == Schema.Type.RECORD) { + return repairRecord(fileSchema, tableSchema); + } + + // Handle array types + if (fileSchema.getType() == Schema.Type.ARRAY) { + Schema repairedElementSchema = repairAvroSchema(fileSchema.getElementType(), tableSchema.getElementType()); + // If element didn't change, return original array schema + if (repairedElementSchema == fileSchema.getElementType()) { + return fileSchema; + } + return Schema.createArray(repairedElementSchema); + } + + // Handle map types + if (fileSchema.getType() == Schema.Type.MAP) { + Schema repairedValueSchema = repairAvroSchema(fileSchema.getValueType(), tableSchema.getValueType()); + // If value didn't change, return original map schema + if (repairedValueSchema == fileSchema.getValueType()) { + return fileSchema; + } + return Schema.createMap(repairedValueSchema); + } + + // Check primitive if we need to repair + if (needsLogicalTypeRepair(fileSchema, tableSchema)) { + // If we need to repair, return the table schema + return tableSchema; + } + + // Default: return file schema + return fileSchema; + } + + /** + * Quick check if a logical type repair is needed (no allocations). + */ + private static boolean needsLogicalTypeRepair(Schema fileSchema, Schema tableSchema) { + if (fileSchema.getType() != Schema.Type.LONG || tableSchema.getType() != Schema.Type.LONG) { + return false; + } + + LogicalType fileSchemaLogicalType = fileSchema.getLogicalType(); + LogicalType tableSchemaLogicalType = tableSchema.getLogicalType(); + + // if file scheam has no logical type, and the table has a local timestamp, then we need to repair + if (fileSchemaLogicalType == null) { + try { + return tableSchemaLogicalType instanceof LogicalTypes.LocalTimestampMillis + || tableSchemaLogicalType instanceof LogicalTypes.LocalTimestampMicros; + } catch (Exception e) { + return false; + } + } + + // if file schema is timestamp-micros, and the table is timestamp-millis, then we need to repair + return fileSchemaLogicalType instanceof LogicalTypes.TimestampMicros + && tableSchemaLogicalType instanceof LogicalTypes.TimestampMillis; + } + + /** + * Performs record repair, returning the original schema if nothing changed. + */ + private static Schema repairRecord(Schema fileSchema, Schema tableSchema) { + List fields = fileSchema.getFields(); + + // First pass: find the first field that changes + int firstChangedIndex = -1; + Schema firstRepairedSchema = null; + + for (int i = 0; i < fields.size(); i++) { + Schema.Field requestedField = fields.get(i); + Schema.Field tableField = tableSchema.getField(requestedField.name()); + if (tableField != null) { + Schema repairedSchema = repairAvroSchema(requestedField.schema(), tableField.schema()); + if (repairedSchema != requestedField.schema()) { + firstChangedIndex = i; + firstRepairedSchema = repairedSchema; + break; + } + } + } + + // If nothing changed, return the original schema + if (firstChangedIndex == -1) { + return fileSchema; + } + + // Second pass: build the new schema with repaired fields + List repairedFields = new ArrayList<>(fields.size()); + + // Copy all fields before the first changed field + for (int i = 0; i < firstChangedIndex; i++) { + Schema.Field field = fields.get(i); + // Must create new Field since they cannot be reused + repairedFields.add(HoodieAvroUtils.createNewSchemaField(field)); + } + + // Add the first changed field (using cached repaired schema) + Schema.Field firstChangedField = fields.get(firstChangedIndex); + repairedFields.add(HoodieAvroUtils.createNewSchemaField( + firstChangedField.name(), + firstRepairedSchema, + firstChangedField.doc(), + firstChangedField.defaultVal() + )); + + // Process remaining fields + for (int i = firstChangedIndex + 1; i < fields.size(); i++) { + Schema.Field requestedField = fields.get(i); + Schema.Field tableField = tableSchema.getField(requestedField.name()); + Schema repairedSchema; + + if (tableField != null) { + repairedSchema = repairAvroSchema(requestedField.schema(), tableField.schema()); + } else { + repairedSchema = requestedField.schema(); + } + + // Must create new Field since they cannot be reused + repairedFields.add(HoodieAvroUtils.createNewSchemaField( + requestedField.name(), + repairedSchema, + requestedField.doc(), + requestedField.defaultVal() + )); + } + + return Schema.createRecord( + fileSchema.getName(), + fileSchema.getDoc(), + fileSchema.getNamespace(), + fileSchema.isError(), + repairedFields + ); + } + + public static boolean hasTimestampMillisField(Schema tableSchema) { + switch (tableSchema.getType()) { + case RECORD: + for (Schema.Field field : tableSchema.getFields()) { + if (hasTimestampMillisField(field.schema())) { + return true; + } + } + return false; + + case ARRAY: + return hasTimestampMillisField(tableSchema.getElementType()); + + case MAP: + return hasTimestampMillisField(tableSchema.getValueType()); + + case UNION: + return hasTimestampMillisField(AvroSchemaUtils.getNonNullTypeFromUnion(tableSchema)); + + default: + return tableSchema.getType() == Schema.Type.LONG + && (tableSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis || tableSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMillis); + } + } + + /** + * Check if LogicalTypes.LocalTimestampMillis is supported in the current Avro version + * + * @return true if LocalTimestampMillis is available, false otherwise + */ + public static boolean isLocalTimestampMillisSupported() { + try { + return Arrays.stream(LogicalTypes.class.getDeclaredClasses()) + .anyMatch(c -> c.getSimpleName().equals("LocalTimestampMillis")); + } catch (Exception e) { + return false; + } + } +} diff --git a/hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java b/hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java new file mode 100644 index 0000000000000..fb3d7e375b2c4 --- /dev/null +++ b/hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java @@ -0,0 +1,984 @@ +/* + * 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.parquet.schema; + +import org.apache.hudi.avro.AvroSchemaUtils; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests {@link AvroSchemaRepair}. + */ +public class TestAvroSchemaRepair { + + @Test + public void testNoRepairNeededIdenticalSchemas() { + Schema requestedSchema = Schema.create(Schema.Type.LONG); + Schema tableSchema = Schema.create(Schema.Type.LONG); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When schemas are identical, should return same instance"); + + } + + @Test + public void testNoRepairNeededDifferentPrimitiveTypes() { + Schema requestedSchema = Schema.create(Schema.Type.STRING); + Schema tableSchema = Schema.create(Schema.Type.INT); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When types differ, should return original schema"); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMillis() { + Schema requestedSchema = Schema.create(Schema.Type.LONG); + Schema tableSchema = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + assertEquals(Schema.Type.LONG, result.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), result.getLogicalType()); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMicros() { + Schema requestedSchema = Schema.create(Schema.Type.LONG); + Schema tableSchema = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + assertEquals(Schema.Type.LONG, result.getType()); + assertEquals(LogicalTypes.localTimestampMicros(), result.getLogicalType()); + } + + @Test + public void testRepairTimestampMicrosToTimestampMillis() { + Schema requestedSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + Schema tableSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with timestamp-millis"); + assertEquals(Schema.Type.LONG, result.getType()); + assertEquals(LogicalTypes.timestampMillis(), result.getLogicalType()); + } + + @Test + public void testNoRepairNeededTimestampMillisToTimestampMicros() { + // This direction should NOT trigger repair + Schema requestedSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + Schema tableSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair timestamp-millis to timestamp-micros"); + } + + @Test + public void testNoRepairNeededNonLongTypes() { + Schema requestedSchema = Schema.create(Schema.Type.INT); + Schema tableSchema = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair non-LONG types"); + } + + @Test + public void testRepairNullableSchemaLongToLocalTimestampMillis() { + Schema requestedSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.LONG) + ); + Schema tableSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new nullable schema with repaired type"); + assertEquals(Schema.Type.UNION, result.getType()); + assertEquals(2, result.getTypes().size()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(result); + assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairNullableSchemaTimestampMicrosToMillis() { + Schema requestedSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ); + Schema tableSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new nullable schema"); + assertEquals(Schema.Type.UNION, result.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(result); + assertEquals(LogicalTypes.timestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairRecordSingleField() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(Schema.Type.RECORD, result.getType()); + assertEquals("TestRecord", result.getName()); + assertEquals(1, result.getFields().size()); + + Schema.Field field = result.getField("timestamp"); + assertEquals(LogicalTypes.localTimestampMillis(), field.schema().getLogicalType()); + } + + @Test + public void testRepairRecordMultipleFieldsOnlyOneNeedsRepair() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(3, result.getFields().size()); + + // Verify id field unchanged - should be same schema instance + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify timestamp field repaired + assertEquals(LogicalTypes.localTimestampMicros(), result.getField("timestamp").schema().getLogicalType()); + + // Verify name field unchanged - should be same schema instance + assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema()); + } + + @Test + public void testRepairRecordNestedRecord() { + Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedTableSchema).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema for nested record"); + + // Verify id field unchanged - should be same schema instance + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify nested record was repaired + Schema nestedResult = result.getField("nested").schema(); + assertEquals(Schema.Type.RECORD, nestedResult.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), + nestedResult.getField("timestamp").schema().getLogicalType()); + } + + @Test + public void testRepairRecordNullableNestedField() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type().optional().longType() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type().optional().type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema"); + + Schema fieldSchema = result.getField("timestamp").schema(); + assertEquals(Schema.Type.UNION, fieldSchema.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(fieldSchema); + assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairArrayElementNeedsRepair() { + Schema requestedSchema = Schema.createArray(Schema.create(Schema.Type.LONG)); + Schema tableSchema = Schema.createArray( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new array schema"); + assertEquals(Schema.Type.ARRAY, result.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), result.getElementType().getLogicalType()); + } + + @Test + public void testRepairArrayNoRepairNeeded() { + Schema elementSchema = Schema.create(Schema.Type.STRING); + Schema requestedSchema = Schema.createArray(elementSchema); + Schema tableSchema = Schema.createArray(elementSchema); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should return same array when no repair needed"); + } + + @Test + public void testRepairArrayNullableElements() { + Schema requestedSchema = Schema.createArray( + Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG)) + ); + Schema tableSchema = Schema.createArray( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new array schema"); + Schema elementSchema = result.getElementType(); + assertEquals(Schema.Type.UNION, elementSchema.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(elementSchema); + assertEquals(LogicalTypes.localTimestampMicros(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairMapValueNeedsRepair() { + Schema requestedSchema = Schema.createMap(Schema.create(Schema.Type.LONG)); + Schema tableSchema = Schema.createMap( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new map schema"); + assertEquals(Schema.Type.MAP, result.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), result.getValueType().getLogicalType()); + } + + @Test + public void testRepairMapNoRepairNeeded() { + Schema valueSchema = Schema.create(Schema.Type.STRING); + Schema requestedSchema = Schema.createMap(valueSchema); + Schema tableSchema = Schema.createMap(valueSchema); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should return same map when no repair needed"); + } + + @Test + public void testRepairMapNullableValues() { + Schema requestedSchema = Schema.createMap( + Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG)) + ); + Schema tableSchema = Schema.createMap( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new map schema"); + Schema valueSchema = result.getValueType(); + assertEquals(Schema.Type.UNION, valueSchema.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(valueSchema); + assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testComplexSchemaMultiLevelNesting() { + // Create a complex schema with nested records, arrays, and maps + Schema innerRecordRequested = SchemaBuilder.record("Inner") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema innerRecordTable = SchemaBuilder.record("Inner") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("Outer") + .fields() + .name("id").type().intType().noDefault() + .name("records").type().array().items(innerRecordRequested).noDefault() + .name("mapping").type().map().values(Schema.create(Schema.Type.LONG)).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("Outer") + .fields() + .name("id").type().intType().noDefault() + .name("records").type().array().items(innerRecordTable).noDefault() + .name("mapping").type().map().values( + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new complex schema"); + + // Verify id field unchanged - should be same schema instance + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify array of records was repaired + Schema arrayElementSchema = result.getField("records").schema().getElementType(); + assertEquals(LogicalTypes.localTimestampMillis(), + arrayElementSchema.getField("timestamp").schema().getLogicalType()); + + // Verify map values were repaired + Schema mapValueSchema = result.getField("mapping").schema().getValueType(); + assertEquals(LogicalTypes.localTimestampMicros(), mapValueSchema.getLogicalType()); + } + + @Test + public void testRepairRecordMissingFieldInTableSchema() { + // Requested schema has a field not present in table schema + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("newField").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newField doesn't exist in table schema + assertSame(requestedSchema, result, "Should return original when field missing in table schema"); + } + + @Test + public void testRepairRecordMultipleFieldsMissingInTableSchema() { + // Requested schema has multiple fields not present in table schema + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("newField1").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .name("newField2").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since new fields don't exist in table schema + assertSame(requestedSchema, result, "Should return original when multiple fields missing in table schema"); + } + + @Test + public void testRepairRecordMixedMissingAndRepairableFields() { + // Requested schema has some fields missing in table, some needing repair, some unchanged + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("newField").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should create new schema with timestamp repaired, but newField preserved from requested + assertNotSame(requestedSchema, result, "Should create new schema"); + assertEquals(4, result.getFields().size()); + + // Verify id field unchanged + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify timestamp field repaired + assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp").schema().getLogicalType()); + + // Verify newField preserved from requested schema (not in table) + assertSame(requestedSchema.getField("newField").schema(), result.getField("newField").schema()); + + // Verify name field unchanged + assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema()); + } + + @Test + public void testRepairNestedRecordFieldMissingInTableSchema() { + // Requested nested record has a field not present in table's nested record + Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .name("extraField").type().stringType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedTableSchema).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema"); + + // Verify id field unchanged + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify nested record was repaired but still has extraField + Schema nestedResult = result.getField("nested").schema(); + assertEquals(Schema.Type.RECORD, nestedResult.getType()); + assertEquals(2, nestedResult.getFields().size()); + + // Timestamp should be repaired + assertEquals(LogicalTypes.localTimestampMillis(), + nestedResult.getField("timestamp").schema().getLogicalType()); + + // extraField should be preserved from requested schema + assertSame(nestedRequestedSchema.getField("extraField").schema(), + nestedResult.getField("extraField").schema()); + } + + @Test + public void testRepairRecordWholeNestedRecordMissingInTableSchema() { + // Requested schema has a nested record field that doesn't exist in table schema + Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("newNested").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newNested field doesn't exist in table + assertSame(requestedSchema, result, "Should return original when nested field missing in table schema"); + } + + @Test + public void testRepairRecordPreservesFieldMetadata() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .doc("Test documentation") + .fields() + .name("timestamp").doc("Timestamp field").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + assertEquals("TestRecord", result.getName()); + assertEquals("Test documentation", result.getDoc()); + assertEquals("Timestamp field", result.getField("timestamp").doc()); + } + + @Test + public void testEdgeCaseEmptyRecord() { + Schema requestedSchema = SchemaBuilder.record("EmptyRecord").fields().endRecord(); + Schema tableSchema = SchemaBuilder.record("EmptyRecord").fields().endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Empty records should return same instance"); + } + + @Test + public void testRepairRecordFirstFieldChanged() { + // Test the optimization path where the first field needs repair + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp1").type().longType().noDefault() + .name("timestamp2").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp1").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("timestamp2").type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp1").schema().getLogicalType()); + assertEquals(LogicalTypes.localTimestampMicros(), result.getField("timestamp2").schema().getLogicalType()); + } + + @Test + public void testRepairRecordLastFieldChanged() { + // Test the optimization path where only the last field needs repair + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + // Verify id and name fields unchanged - should be same schema instances + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema()); + // Verify timestamp field repaired + assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp").schema().getLogicalType()); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithTimestampMillis() { + Schema schema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for LONG with timestamp-millis logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithoutLogicalType() { + Schema schema = Schema.create(Schema.Type.LONG); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for LONG without logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithTimestampMicros() { + Schema schema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for LONG with timestamp-micros logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithLocalTimestampMillis() { + Schema schema = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for LONG with local-timestamp-millis logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithLocalTimestampMicros() { + Schema schema = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for LONG with local-timestamp-micros logical type"); + } + + @Test + public void testHasTimestampMillisFieldOtherPrimitiveTypes() { + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.STRING)), + "Should return false for STRING type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.INT)), + "Should return false for INT type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.FLOAT)), + "Should return false for FLOAT type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.DOUBLE)), + "Should return false for DOUBLE type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.BOOLEAN)), + "Should return false for BOOLEAN type"); + } + + @Test + public void testHasTimestampMillisFieldRecordWithTimestampMillis() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for record containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldRecordWithoutTimestampMillis() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for record without timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldRecordEmpty() { + Schema schema = SchemaBuilder.record("EmptyRecord").fields().endRecord(); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for empty record"); + } + + @Test + public void testHasTimestampMillisFieldNestedRecord() { + Schema innerSchema = SchemaBuilder.record("InnerRecord") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema outerSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("inner").type(innerSchema).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(outerSchema), + "Should return true for nested record containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldDeeplyNestedRecord() { + Schema level3 = SchemaBuilder.record("Level3") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema level2 = SchemaBuilder.record("Level2") + .fields() + .name("data").type(level3).noDefault() + .endRecord(); + + Schema level1 = SchemaBuilder.record("Level1") + .fields() + .name("nested").type(level2).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(level1), + "Should return true for deeply nested record containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldArrayWithTimestampMillis() { + Schema schema = Schema.createArray( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for array with timestamp-millis elements"); + } + + @Test + public void testHasTimestampMillisFieldArrayWithoutTimestampMillis() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.STRING)); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for array without timestamp-millis elements"); + } + + @Test + public void testHasTimestampMillisFieldArrayOfRecordsWithTimestampMillis() { + Schema elementSchema = SchemaBuilder.record("Element") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema schema = Schema.createArray(elementSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for array of records containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldMapWithTimestampMillis() { + Schema schema = Schema.createMap( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for map with timestamp-millis values"); + } + + @Test + public void testHasTimestampMillisFieldMapWithoutTimestampMillis() { + Schema schema = Schema.createMap(Schema.create(Schema.Type.STRING)); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for map without timestamp-millis values"); + } + + @Test + public void testHasTimestampMillisFieldMapOfRecordsWithTimestampMillis() { + Schema valueSchema = SchemaBuilder.record("Value") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema schema = Schema.createMap(valueSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for map of records containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldUnionWithTimestampMillis() { + Schema schema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for nullable union with timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldUnionWithoutTimestampMillis() { + Schema schema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.LONG) + ); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for nullable union without timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldUnionWithRecordContainingTimestampMillis() { + Schema recordSchema = SchemaBuilder.record("Record") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema schema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + recordSchema + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for nullable union with record containing timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldComplexNestedStructure() { + // Create a complex schema with arrays, maps, and nested records + Schema innerRecordSchema = SchemaBuilder.record("InnerRecord") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema complexSchema = SchemaBuilder.record("ComplexRecord") + .fields() + .name("id").type().intType().noDefault() + .name("arrayOfRecords").type().array().items(innerRecordSchema).noDefault() + .name("mapOfStrings").type().map().values().stringType().noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(complexSchema), + "Should return true for complex nested structure containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldComplexStructureWithoutTimestampMillis() { + Schema innerRecordSchema = SchemaBuilder.record("InnerRecord") + .fields() + .name("value").type().longType().noDefault() + .endRecord(); + + Schema complexSchema = SchemaBuilder.record("ComplexRecord") + .fields() + .name("id").type().intType().noDefault() + .name("arrayOfRecords").type().array().items(innerRecordSchema).noDefault() + .name("mapOfLongs").type().map().values( + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ).noDefault() + .endRecord(); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(complexSchema), + "Should return false for complex structure without timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldFirstFieldMatches() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true when first field is timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldLastFieldMatches() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true when last field is timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldMultipleTimestampMillisFields() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("createdAt").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("id").type().intType().noDefault() + .name("updatedAt").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true when multiple timestamp-millis fields exist"); + } + + @Test + public void testHasTimestampMillisFieldNullableFieldWithTimestampMillis() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().optional().type( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ) + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for nullable field with timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldArrayOfNullableTimestampMillis() { + Schema elementSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema schema = Schema.createArray(elementSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for array of nullable timestamp-millis elements"); + } + + @Test + public void testHasTimestampMillisFieldMapOfNullableTimestampMillis() { + Schema valueSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema schema = Schema.createMap(valueSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for map of nullable timestamp-millis values"); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index 824a94abab4bd..6808ae1528279 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -479,6 +479,11 @@ protected boolean shouldReadAsPartitionedTable() { return (partitionColumns.length > 0 && canParsePartitionValues()) || HoodieTableMetadata.isMetadataTable(basePath.toString()); } + protected PartitionPath convertToPartitionPath(String partitionPath) { + Object[] partitionColumnValues = parsePartitionColumnValues(partitionColumns, partitionPath); + return new PartitionPath(partitionPath, partitionColumnValues); + } + private static long fileSliceSize(FileSlice fileSlice) { long logFileSize = fileSlice.getLogFiles().map(HoodieLogFile::getFileSize) .filter(s -> s > 0) diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.java new file mode 100644 index 0000000000000..b679ea3c8d508 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.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.avro; + +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.LoadingCache; +import org.apache.avro.Schema; + +/** + * An avro schema cache implementation for reusing avro schema instantces in JVM/process scope. + * This is a global cache which works for a JVM lifecycle. + * A collection of schema instants are maintained. + * + *

NOTE: The schema which be used frequently should be cached through this cache. + */ +public class AvroSchemaCache { + + + // Ensure that there is only one variable instance of the same schema within an entire JVM lifetime + private static final LoadingCache SCHEMA_CACHE = Caffeine.newBuilder().weakValues().maximumSize(1024).build(k -> k); + + /** + * Get schema variable from global cache. If not found, put it into the cache and then return it. + * @param schema schema to get + * @return if found, return the exist schema variable, otherwise return the param itself. + */ + public static Schema intern(Schema schema) { + return SCHEMA_CACHE.get(schema); + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java index 3c5486c47c742..427ea568cf9a2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java @@ -18,12 +18,16 @@ package org.apache.hudi.avro; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.SchemaCompatibilityException; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; import org.apache.avro.SchemaCompatibility; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -218,6 +222,25 @@ private static boolean isProjectionOfInternal(Schema sourceSchema, return atomicTypeEqualityPredicate.apply(sourceSchema, targetSchema); } + public static Option findNestedFieldSchema(Schema schema, String fieldName) { + if (StringUtils.isNullOrEmpty(fieldName)) { + return Option.empty(); + } + String[] parts = fieldName.split("\\."); + for (String part : parts) { + Schema.Field foundField = getNonNullTypeFromUnion(schema).getField(part); + if (foundField == null) { + throw new HoodieAvroSchemaException(fieldName + " not a field in " + schema); + } + schema = foundField.schema(); + } + return Option.of(getNonNullTypeFromUnion(schema)); + } + + public static Option findNestedFieldType(Schema schema, String fieldName) { + return findNestedFieldSchema(schema, fieldName).map(Schema::getType); + } + /** * Appends provided new fields at the end of the given schema * @@ -251,7 +274,7 @@ public static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullNam List innerTypes = schema.getTypes(); if (innerTypes.size() == 2 && isNullable(schema)) { // this is a basic nullable field so handle it more efficiently - return resolveNullableSchema(schema); + return getNonNullTypeFromUnion(schema); } Schema nonNullType = @@ -285,7 +308,7 @@ public static boolean isNullable(Schema schema) { * Resolves typical Avro's nullable schema definition: {@code Union(Schema.Type.NULL, )}, * decomposing union and returning the target non-null type */ - public static Schema resolveNullableSchema(Schema schema) { + public static Schema getNonNullTypeFromUnion(Schema schema) { if (schema.getType() != Schema.Type.UNION) { return schema; } @@ -373,4 +396,15 @@ public static void checkSchemaCompatible( throw new SchemaCompatibilityException(errorDetails); } } + + public static Schema getRepairedSchema(Schema writerSchema, Schema readerSchema) { + try { + Class avroSchemaRepairClass = Class.forName("org.apache.parquet.schema.AvroSchemaRepair"); + Method repairMethod = avroSchemaRepairClass.getMethod("repairLogicalTypes", Schema.class, Schema.class); + return (Schema) repairMethod.invoke(null, writerSchema, readerSchema); + } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + // Fallback if class/method not available + return writerSchema; + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java b/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java index 9d36e214fb852..70a653726f4e8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java @@ -25,6 +25,7 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; +import java.lang.reflect.Constructor; import java.util.Map; /** @@ -42,13 +43,12 @@ public class ConvertingGenericData extends GenericData { private static final TimeConversions.TimeMicrosConversion TIME_MICROS_CONVERSION = new TimeConversions.TimeMicrosConversion(); private static final TimeConversions.TimestampMicrosConversion TIMESTAMP_MICROS_CONVERSION = new TimeConversions.TimestampMicrosConversion(); - // NOTE: Those are not supported in Avro 1.8.2 - // TODO re-enable upon upgrading to 1.10 - // private static final TimeConversions.TimestampMillisConversion TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.TimestampMillisConversion(); - // private static final TimeConversions.TimeMillisConversion TIME_MILLIS_CONVERSION = new TimeConversions.TimeMillisConversion(); - // private static final TimeConversions.LocalTimestampMillisConversion LOCAL_TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.LocalTimestampMillisConversion(); - // private static final TimeConversions.LocalTimestampMicrosConversion LOCAL_TIMESTAMP_MICROS_CONVERSION = new TimeConversions.LocalTimestampMicrosConversion(); - + // NOTE: Those are not supported in Avro 1.8.2 (used by Spark 2) + // Use reflection to conditionally initialize them only if available + private static final Object TIMESTAMP_MILLIS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$TimestampMillisConversion"); + private static final Object TIME_MILLIS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$TimeMillisConversion"); + private static final Object LOCAL_TIMESTAMP_MILLIS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$LocalTimestampMillisConversion"); + private static final Object LOCAL_TIMESTAMP_MICROS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$LocalTimestampMicrosConversion"); public static final GenericData INSTANCE = new ConvertingGenericData(); private ConvertingGenericData() { @@ -57,12 +57,20 @@ private ConvertingGenericData() { addLogicalTypeConversion(DATE_CONVERSION); addLogicalTypeConversion(TIME_MICROS_CONVERSION); addLogicalTypeConversion(TIMESTAMP_MICROS_CONVERSION); - // NOTE: Those are not supported in Avro 1.8.2 - // TODO re-enable upon upgrading to 1.10 - // addLogicalTypeConversion(TIME_MILLIS_CONVERSION); - // addLogicalTypeConversion(TIMESTAMP_MILLIS_CONVERSION); - // addLogicalTypeConversion(LOCAL_TIMESTAMP_MILLIS_CONVERSION); - // addLogicalTypeConversion(LOCAL_TIMESTAMP_MICROS_CONVERSION); + // NOTE: Those are not supported in Avro 1.8.2 (used by Spark 2) + // Only add conversions if they're available + if (TIME_MILLIS_CONVERSION != null) { + addLogicalTypeConversionReflectively(TIME_MILLIS_CONVERSION); + } + if (TIMESTAMP_MILLIS_CONVERSION != null) { + addLogicalTypeConversionReflectively(TIMESTAMP_MILLIS_CONVERSION); + } + if (LOCAL_TIMESTAMP_MILLIS_CONVERSION != null) { + addLogicalTypeConversionReflectively(LOCAL_TIMESTAMP_MILLIS_CONVERSION); + } + if (LOCAL_TIMESTAMP_MICROS_CONVERSION != null) { + addLogicalTypeConversionReflectively(LOCAL_TIMESTAMP_MICROS_CONVERSION); + } } @Override @@ -125,9 +133,31 @@ public boolean validate(Schema schema, Object datum) { return isInteger(datum) || DATE_CONVERSION.getConvertedType().isInstance(datum); case LONG: - return isLong(datum) - || TIME_MICROS_CONVERSION.getConvertedType().isInstance(datum) - || TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum); + if (isLong(datum)) { + return true; + } + if (TIME_MICROS_CONVERSION.getConvertedType().isInstance(datum) + || TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum)) { + return true; + } + // Check optional conversions that may not be available in Avro 1.8.2 + Class convertedType; + if (TIMESTAMP_MILLIS_CONVERSION != null + && (convertedType = getConvertedType(TIMESTAMP_MILLIS_CONVERSION)) != null + && convertedType.isInstance(datum)) { + return true; + } + if (LOCAL_TIMESTAMP_MICROS_CONVERSION != null + && (convertedType = getConvertedType(LOCAL_TIMESTAMP_MICROS_CONVERSION)) != null + && convertedType.isInstance(datum)) { + return true; + } + if (LOCAL_TIMESTAMP_MILLIS_CONVERSION != null + && (convertedType = getConvertedType(LOCAL_TIMESTAMP_MILLIS_CONVERSION)) != null + && convertedType.isInstance(datum)) { + return true; + } + return false; case FLOAT: return isFloat(datum); case DOUBLE: @@ -140,5 +170,43 @@ public boolean validate(Schema schema, Object datum) { return false; } } + + /** + * Creates a conversion instance using reflection if the class is available. + * Returns null if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static Object createConversionIfAvailable(String className) { + try { + Class clazz = Class.forName(className); + Constructor constructor = clazz.getConstructor(); + return constructor.newInstance(); + } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException + | IllegalAccessException | java.lang.reflect.InvocationTargetException e) { + // Class doesn't exist or can't be instantiated (e.g., Avro 1.8.2) + return null; + } + } + + /** + * Gets the converted type from a conversion object using reflection. + */ + private static Class getConvertedType(Object conversion) { + try { + return (Class) conversion.getClass().getMethod("getConvertedType").invoke(conversion); + } catch (Exception e) { + // Should not happen if conversion is valid, but handle gracefully + return null; + } + } + + /** + * Adds a logical type conversion using unchecked cast to avoid compile-time dependency + * on classes that may not exist in older Avro versions. + */ + private void addLogicalTypeConversionReflectively(Object conversion) { + // Cast to Conversion since we know it's a Conversion if not null + // This avoids compile-time dependency on specific Conversion subclasses + addLogicalTypeConversion((org.apache.avro.Conversion) conversion); + } } 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 d04e986487b5e..f12beaa058322 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 @@ -33,6 +33,7 @@ import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.DateTimeUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.StringUtils; @@ -47,10 +48,12 @@ import org.apache.avro.Conversions; import org.apache.avro.Conversions.DecimalConversion; import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.LogicalTypes.Decimal; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Field.Order; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.generic.GenericDatumReader; @@ -104,10 +107,10 @@ import static org.apache.avro.Schema.Type.UNION; import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; import static org.apache.hudi.avro.AvroSchemaUtils.isNullable; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema; import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros; import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.util.ValidationUtils.checkState; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.tryUpcastDecimal; @@ -653,7 +656,7 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String Object val = valueNode.get(part); if (i == parts.length - 1) { - return resolveNullableSchema(valueNode.getSchema().getField(part).schema()); + return getNonNullTypeFromUnion(valueNode.getSchema().getField(part).schema()); } else { if (!(val instanceof GenericRecord)) { throw new HoodieException("Cannot find a record at part value :" + part); @@ -674,13 +677,20 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String */ public static Schema getNestedFieldSchemaFromWriteSchema(Schema writeSchema, String fieldName) { String[] parts = fieldName.split("\\."); + Schema currentSchema = writeSchema; int i = 0; for (; i < parts.length; i++) { String part = parts[i]; - Schema schema = writeSchema.getField(part).schema(); + try { + // Resolve nullable/union schema to the actual schema + currentSchema = getNonNullTypeFromUnion(currentSchema.getField(part).schema()); - if (i == parts.length - 1) { - return resolveNullableSchema(schema); + if (i == parts.length - 1) { + // Return the schema for the final part + return getNonNullTypeFromUnion(currentSchema); + } + } catch (Exception e) { + throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName); } } throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName); @@ -718,7 +728,7 @@ public static Object convertValueForSpecificDataTypes(Schema fieldSchema, return null; } - return convertValueForAvroLogicalTypes(resolveNullableSchema(fieldSchema), fieldValue, consistentLogicalTimestampEnabled); + return convertValueForAvroLogicalTypes(getNonNullTypeFromUnion(fieldSchema), fieldValue, consistentLogicalTimestampEnabled); } /** @@ -968,12 +978,35 @@ private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Sche case NULL: case BOOLEAN: case INT: - case LONG: case FLOAT: case DOUBLE: case BYTES: case STRING: return oldValue; + case LONG: + if (oldSchema.getLogicalType() != newSchema.getLogicalType()) { + if (oldSchema.getLogicalType() == null || newSchema.getLogicalType() == null) { + return oldValue; + } else if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { + return DateTimeUtils.millisToMicros((Long) oldValue); + } + } else if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { + if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + return DateTimeUtils.microsToMillis((Long) oldValue); + } + } else if (isLocalTimestampMillis(oldSchema.getLogicalType())) { + if (isLocalTimestampMicros(newSchema.getLogicalType())) { + return DateTimeUtils.millisToMicros((Long) oldValue); + } + } else if (isLocalTimestampMicros(oldSchema.getLogicalType())) { + if (isLocalTimestampMillis(newSchema.getLogicalType())) { + return DateTimeUtils.microsToMillis((Long) oldValue); + } + } + throw new HoodieAvroSchemaException("Long type logical change from " + oldSchema.getLogicalType() + " to " + newSchema.getLogicalType() + " is not supported"); + } + return oldValue; case FIXED: if (oldSchema.getFixedSize() != newSchema.getFixedSize()) { // Check whether this is a [[Decimal]]'s precision change @@ -1271,6 +1304,10 @@ public static boolean gteqAvro1_10() { return VersionUtil.compareVersions(AVRO_VERSION, "1.10") >= 0; } + public static boolean gteqAvro1_12() { + return StringUtils.compareVersions(AVRO_VERSION, "1.12") >= 0; + } + /** * Wraps a value into Avro type wrapper. * @@ -1364,4 +1401,107 @@ public static Comparable unwrapAvroValueWrapper(Object avroValueWrapper) { } } + /** + * Checks if a logical type is an instance of LocalTimestampMillis using reflection. + * Returns false if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static boolean isLocalTimestampMillis(LogicalType logicalType) { + if (logicalType == null) { + return false; + } + try { + Class localTimestampMillisClass = Class.forName("org.apache.avro.LogicalTypes$LocalTimestampMillis"); + return localTimestampMillisClass.isInstance(logicalType); + } catch (ClassNotFoundException e) { + // Class doesn't exist (e.g., Avro 1.8.2) + return false; + } + } + + /** + * Checks if a logical type is an instance of LocalTimestampMicros using reflection. + * Returns false if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static boolean isLocalTimestampMicros(LogicalType logicalType) { + if (logicalType == null) { + return false; + } + try { + Class localTimestampMicrosClass = Class.forName("org.apache.avro.LogicalTypes$LocalTimestampMicros"); + return localTimestampMicrosClass.isInstance(logicalType); + } catch (ClassNotFoundException e) { + // Class doesn't exist (e.g., Avro 1.8.2) + return false; + } + } + + private static Object convertDefaultValueForAvroCompatibility(Object defaultValue) { + if (gteqAvro1_12() && defaultValue instanceof byte[]) { + // For Avro 1.12.0 compatibility, we need to convert the default value in byte array + // to String so that correct JsonNode is used for the default value for validation, + // instead of directly relying on Avro's JacksonUtils.toJsonNode which is called + // by `Schema.Field` constructor + // The logic of getting the String value is copied from JacksonUtils.toJsonNode in Avro 1.11.4 + return new String((byte[]) defaultValue, StandardCharsets.ISO_8859_1); + } + return defaultValue; + } + + /** + * Creates a new Avro Schema.Field from an existing field, with special handling for + * default values to ensure compatibility with Avro 1.12.0 and later versions. + * + * @param field the original Schema.Field to create a new field from + * @return a new Schema.Field with the same properties but properly formatted default value + */ + public static Schema.Field createNewSchemaField(Schema.Field field) { + return createNewSchemaField(field.name(), field.schema(), field.doc(), field.defaultVal()); + } + + /** + * Creates a new Avro Schema.Field with special handling for default values to ensure + * compatibility with Avro 1.12.0 and later versions. + * + *

In Avro 1.12.0+, the validation of default values for bytes fields is stricter. + * When the default value is a byte array, it needs to be converted to a String using + * ISO-8859-1 encoding so that the correct JsonNode type (TextNode) is used for validation, + * rather than BinaryNode which would fail validation. Changes in Avro 1.12.0 that + * lead to this behavior: [AVRO-3876] https://github.com/apache/avro/pull/2529 + * + *

This conversion ensures that schemas with bytes fields having default values + * can be properly constructed without AvroTypeException in Avro 1.12.0+. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation for the field (can be null) + * @param defaultValue the default value for the field (can be null) + * @return a new Schema.Field with properly formatted default value for Avro 1.12.0+ compatibility + */ + public static Schema.Field createNewSchemaField(String name, Schema schema, String doc, Object defaultValue) { + return new Schema.Field(name, schema, doc, convertDefaultValueForAvroCompatibility(defaultValue)); + } + + /** + * Creates a new Avro Schema.Field with special handling for default values to ensure + * compatibility with Avro 1.12.0 and later versions. + * + *

In Avro 1.12.0+, the validation of default values for bytes fields is stricter. + * When the default value is a byte array, it needs to be converted to a String using + * ISO-8859-1 encoding so that the correct JsonNode type (TextNode) is used for validation, + * rather than BinaryNode which would fail validation. Changes in Avro 1.12.0 that + * lead to this behavior: [AVRO-3876] https://github.com/apache/avro/pull/2529 + * + *

This conversion ensures that schemas with bytes fields having default values + * can be properly constructed without AvroTypeException in Avro 1.12.0+. + * + * @param name the name of the field + * @param schema the schema of the field + * @param doc the documentation for the field (can be null) + * @param defaultValue the default value for the field (can be null) + * @param order the sort order for this field (can be null, defaults to ascending) + * @return a new Schema.Field with properly formatted default value for Avro 1.12.0+ compatibility + */ + public static Schema.Field createNewSchemaField(String name, Schema schema, String doc, Object defaultValue, Order order) { + return new Schema.Field(name, schema, doc, convertDefaultValueForAvroCompatibility(defaultValue), order); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java b/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java index cdf0f15d80deb..cce3016aed859 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java @@ -299,12 +299,27 @@ private static JsonToAvroFieldProcessor generateFixedTypeHandler() { return new JsonToAvroFieldProcessor() { @Override public Pair convert(Object value, String name, Schema schema, boolean shouldSanitize, String invalidCharMask) { + byte[] src; // The ObjectMapper use List to represent FixedType // eg: "decimal_val": [0, 0, 14, -63, -52] will convert to ArrayList - List converval = (List) value; - byte[] src = new byte[converval.size()]; - for (int i = 0; i < converval.size(); i++) { - src[i] = converval.get(i).byteValue(); + if (value instanceof List) { + List converval = (List) value; + src = new byte[converval.size()]; + for (int i = 0; i < converval.size(); i++) { + src[i] = converval.get(i).byteValue(); + } + } else if (value instanceof ByteBuffer) { + // Handle ByteBuffer when reading from existing records + ByteBuffer buffer = (ByteBuffer) value; + int start = buffer.position(); + int length = buffer.limit() - start; + src = new byte[length]; + buffer.get(src, 0, length); + buffer.position(start); + } else if (value instanceof byte[]) { + src = (byte[]) value; + } else { + return Pair.of(false, null); } byte[] dst = new byte[schema.getFixedSize()]; System.arraycopy(src, 0, dst, 0, Math.min(schema.getFixedSize(), src.length)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java index 7098c076279b0..aa97b10c2c83c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java @@ -42,7 +42,8 @@ public class TimestampKeyGeneratorConfig { .withAlternatives(OLD_TIMESTAMP_KEYGEN_CONFIG_PREFIX + "timestamp.type") .markAdvanced() .withDocumentation("Timestamp type of the field, which should be one of the timestamp types " - + "supported: `UNIX_TIMESTAMP`, `DATE_STRING`, `MIXED`, `EPOCHMILLISECONDS`, `SCALAR`."); + + "supported: `UNIX_TIMESTAMP`, `DATE_STRING`, `MIXED`, `EPOCHMILLISECONDS`," + + " `EPOCHMICROSECONDS`, `SCALAR`."); public static final ConfigProperty INPUT_TIME_UNIT = ConfigProperty .key(TIMESTAMP_KEYGEN_CONFIG_PREFIX + "timestamp.scalar.time.unit") diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java index bdcd0ac690fd2..6f04583c0f191 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java @@ -62,6 +62,7 @@ import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; +import static org.apache.hudi.avro.AvroSchemaUtils.getRepairedSchema; import static org.apache.hudi.avro.HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -142,7 +143,7 @@ protected ClosableIterator> deserializeRecords(byte[] conten checkState(this.readerSchema != null, "Reader's schema has to be non-null"); checkArgument(type != HoodieRecordType.SPARK, "Not support read avro to spark record"); // TODO AvroSparkReader need - RecordIterator iterator = RecordIterator.getInstance(this, content); + RecordIterator iterator = RecordIterator.getInstance(this, content, true); return new CloseableMappingIterator<>(iterator, data -> (HoodieRecord) new HoodieAvroIndexedRecord(data)); } @@ -155,7 +156,7 @@ private static class RecordIterator implements ClosableIterator { private int totalRecords = 0; private int readRecords = 0; - private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) throws IOException { + private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, boolean enableLogicalTimestampFieldRepair) throws IOException { this.content = content; this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content))); @@ -166,16 +167,21 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) this.totalRecords = this.dis.readInt(); } - if (recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema, readerSchema)) { - this.reader = new GenericDatumReader<>(writerSchema, writerSchema); + // writer schema could refer to table schema. + // avoid this for MDT for sure. + // and for tables having no logical ts column. + Schema repairedWriterSchema = enableLogicalTimestampFieldRepair + ? getRepairedSchema(writerSchema, readerSchema) : writerSchema; + if (recordNeedsRewriteForExtendedAvroTypePromotion(repairedWriterSchema, readerSchema)) { + this.reader = new GenericDatumReader<>(repairedWriterSchema, repairedWriterSchema); this.promotedSchema = Option.of(readerSchema); } else { - this.reader = new GenericDatumReader<>(writerSchema, readerSchema); + this.reader = new GenericDatumReader<>(repairedWriterSchema, readerSchema); } } - public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException { - return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content); + public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, boolean enableLogicalTimestampFieldRepair) throws IOException { + return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content, enableLogicalTimestampFieldRepair); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index c6e524e8dd78a..c5df6b4d8b063 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -638,6 +638,19 @@ public final List getPartitionPaths() { } } + public final List getPartitionNames() { + try { + readLock.lock(); + return fetchAllStoredFileGroups() + .filter(fg -> !isFileGroupReplaced(fg)) + .map(HoodieFileGroup::getPartitionPath) + .distinct() + .collect(Collectors.toList()); + } finally { + readLock.unlock(); + } + } + @Override public final Stream> getPendingLogCompactionOperations() { try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java index 9dde7727806c2..99efa89fa0542 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java @@ -52,6 +52,12 @@ public static Instant microsToInstant(long microsFromEpoch) { return Instant.ofEpochSecond(epochSeconds, nanoAdjustment); } + public static Instant nanosToInstant(long nanosFromEpoch) { + long epochSeconds = nanosFromEpoch / (1_000_000_000L); + long nanoAdjustment = nanosFromEpoch % (1_000_000_000L); + return Instant.ofEpochSecond(epochSeconds, nanoAdjustment); + } + /** * Converts provided {@link Instant} to microseconds (from epoch) */ @@ -71,6 +77,45 @@ public static long instantToMicros(Instant instant) { } } + /** + * This is based off instantToMicros above. + * */ + public static long instantToNanos(Instant instant) { + long seconds = instant.getEpochSecond(); + int nanos = instant.getNano(); + + if (seconds < 0 && nanos > 0) { + // Shift seconds by +1, then subtract a full second in nanos + long totalNanos = Math.multiplyExact(seconds + 1, 1_000_000_000L); + long adjustment = nanos - 1_000_000_000L; + return Math.addExact(totalNanos, adjustment); + } else { + long totalNanos = Math.multiplyExact(seconds, 1_000_000_000L); + return Math.addExact(totalNanos, nanos); + } + } + + public static final long MICROS_PER_MILLIS = 1000L; + + /** + * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have microseconds + * precision, so this conversion is lossy. + */ + public static Long microsToMillis(Long micros) { + // When the timestamp is negative i.e before 1970, we need to adjust the milliseconds portion. + // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. + // In millis precision the above needs to be represented as (-157700927877). + return Math.floorDiv(micros, MICROS_PER_MILLIS); + } + + /** + * Converts milliseconds since the epoch to microseconds. + */ + public static Long millisToMicros(Long millis) { + return Math.multiplyExact(millis, MICROS_PER_MILLIS); + } + + /** * Parse input String to a {@link java.time.Instant}. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java index b35f8a1c18ccb..820f564f1cb1f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java @@ -44,6 +44,8 @@ public class JsonUtils { MAPPER.setVisibility(PropertyAccessor.IS_GETTER, JsonAutoDetect.Visibility.NONE); MAPPER.setVisibility(PropertyAccessor.SETTER, JsonAutoDetect.Visibility.NONE); MAPPER.setVisibility(PropertyAccessor.CREATOR, JsonAutoDetect.Visibility.NONE); + // NOTE: Registering [[JavaTimeModule]] is required for Jackson >= 2.11 (Spark >= 3.3) + MAPPER.registerModule(new JavaTimeModule()); } public static ObjectMapper getObjectMapper() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index de5572523c1eb..85f9ea5a59e96 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -65,6 +65,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; + /** * Utility functions involving with parquet. */ @@ -228,7 +230,7 @@ public Map readFooter(Configuration configuration, boolean requi @Override public Schema readAvroSchema(Configuration conf, Path parquetFilePath) { MessageType parquetSchema = readSchema(conf, parquetFilePath); - return new AvroSchemaConverter(conf).convert(parquetSchema); + return getAvroSchemaConverter(conf).convert(parquetSchema); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index 21d91a8a3344f..babe35c6e6028 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -53,6 +53,8 @@ public static Class getClass(String clazzName) { return Class.forName(c); } catch (ClassNotFoundException e) { throw new HoodieException("Unable to load class", e); + } catch (NoClassDefFoundError e) { + throw new HoodieException("Unable to load class due to missing dependency", e); } }); } @@ -112,6 +114,15 @@ public static boolean hasConstructor(String clazz, Class[] constructorArgType LOG.warn(message, e); } return false; + } catch (HoodieException e) { + // Class cannot be loaded (e.g., ClassNotFoundException or NoClassDefFoundError) + String message = "Unable to load class " + clazz; + if (silenceWarning) { + LOG.debug(message, e); + } else { + LOG.warn(message, e); + } + return false; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java index bc8b89004d695..95c6504446b95 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java @@ -64,7 +64,12 @@ enum TypeID { TIME(Long.class), TIMESTAMP(Long.class), DECIMAL(BigDecimal.class), - UUID(UUID.class); + UUID(UUID.class), + TIME_MILLIS(Integer.class), + TIMESTAMP_MILLIS(Long.class), + LOCAL_TIMESTAMP_MILLIS(Long.class), + LOCAL_TIMESTAMP_MICROS(Long.class); + private final String name; private final Class classTag; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java index ed03a7349cb72..86e39959c5178 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -383,6 +383,78 @@ public int hashCode() { } } + public static class TimeMillisType extends PrimitiveType { + private static final TimeMillisType INSTANCE = new TimeMillisType(); + + public static TimeMillisType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.TIME_MILLIS; + } + + @Override + public String toString() { + return "time-millis"; + } + } + + public static class TimestampMillisType extends PrimitiveType { + private static final TimestampMillisType INSTANCE = new TimestampMillisType(); + + public static TimestampMillisType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.TIMESTAMP_MILLIS; + } + + @Override + public String toString() { + return "timestamp-millis"; + } + } + + public static class LocalTimestampMillisType extends PrimitiveType { + private static final LocalTimestampMillisType INSTANCE = new LocalTimestampMillisType(); + + public static LocalTimestampMillisType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.LOCAL_TIMESTAMP_MILLIS; + } + + @Override + public String toString() { + return "local-timestamp-millis"; + } + } + + public static class LocalTimestampMicrosType extends PrimitiveType { + private static final LocalTimestampMicrosType INSTANCE = new LocalTimestampMicrosType(); + + public static LocalTimestampMicrosType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.LOCAL_TIMESTAMP_MICROS; + } + + @Override + public String toString() { + return "local-timestamp-micros"; + } + } + /** * UUID primitive type. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index 786ac538271a2..ac14ea9e5c6e8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -231,15 +231,18 @@ private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) { } else if (logical instanceof LogicalTypes.Date) { return Types.DateType.get(); - } else if ( - logical instanceof LogicalTypes.TimeMillis - || logical instanceof LogicalTypes.TimeMicros) { + } else if (logical instanceof LogicalTypes.TimeMillis) { + return Types.TimeMillisType.get(); + } else if (logical instanceof LogicalTypes.TimeMicros) { return Types.TimeType.get(); - - } else if ( - logical instanceof LogicalTypes.TimestampMillis - || logical instanceof LogicalTypes.TimestampMicros) { + } else if (logical instanceof LogicalTypes.TimestampMillis) { + return Types.TimestampMillisType.get(); + } else if (logical instanceof LogicalTypes.TimestampMicros) { return Types.TimestampType.get(); + } else if (isLocalTimestampMillis(logical)) { + return Types.LocalTimestampMillisType.get(); + } else if (isLocalTimestampMicros(logical)) { + return Types.LocalTimestampMicrosType.get(); } else if (LogicalTypes.uuid().getName().equals(name)) { return Types.UUIDType.get(); } @@ -428,9 +431,21 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit case TIME: return LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case TIME_MILLIS: + return LogicalTypes.timeMillis().addToSchema(Schema.create(Schema.Type.INT)); + case TIMESTAMP: return LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case TIMESTAMP_MILLIS: + return LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + + case LOCAL_TIMESTAMP_MICROS: + return createLocalTimestampMicrosSchema(); + + case LOCAL_TIMESTAMP_MILLIS: + return createLocalTimestampMillisSchema(); + case STRING: return Schema.create(Schema.Type.STRING); @@ -481,4 +496,68 @@ private static int computeMinBytesForPrecision(int precision) { } return numBytes; } + + /** + * Checks if a logical type is an instance of LocalTimestampMillis using reflection. + * Returns false if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static boolean isLocalTimestampMillis(LogicalType logicalType) { + if (logicalType == null) { + return false; + } + try { + Class localTimestampMillisClass = Class.forName("org.apache.avro.LogicalTypes$LocalTimestampMillis"); + return localTimestampMillisClass.isInstance(logicalType); + } catch (ClassNotFoundException e) { + // Class doesn't exist (e.g., Avro 1.8.2) + return false; + } + } + + /** + * Checks if a logical type is an instance of LocalTimestampMicros using reflection. + * Returns false if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static boolean isLocalTimestampMicros(LogicalType logicalType) { + if (logicalType == null) { + return false; + } + try { + Class localTimestampMicrosClass = Class.forName("org.apache.avro.LogicalTypes$LocalTimestampMicros"); + return localTimestampMicrosClass.isInstance(logicalType); + } catch (ClassNotFoundException e) { + // Class doesn't exist (e.g., Avro 1.8.2) + return false; + } + } + + /** + * Creates a LocalTimestampMicros schema using reflection. + * Returns null if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static Schema createLocalTimestampMicrosSchema() { + try { + java.lang.reflect.Method method = LogicalTypes.class.getMethod("localTimestampMicros"); + LogicalType logicalType = (LogicalType) method.invoke(null); + return logicalType.addToSchema(Schema.create(Schema.Type.LONG)); + } catch (Exception e) { + // Method doesn't exist (e.g., Avro 1.8.2) + throw new UnsupportedOperationException("LocalTimestampMicros is not supported in this Avro version", e); + } + } + + /** + * Creates a LocalTimestampMillis schema using reflection. + * Returns null if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private static Schema createLocalTimestampMillisSchema() { + try { + java.lang.reflect.Method method = LogicalTypes.class.getMethod("localTimestampMillis"); + LogicalType logicalType = (LogicalType) method.invoke(null); + return logicalType.addToSchema(Schema.create(Schema.Type.LONG)); + } catch (Exception e) { + // Method doesn't exist (e.g., Avro 1.8.2) + throw new UnsupportedOperationException("LocalTimestampMillis is not supported in this Avro version", e); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java index f47d7f8da517b..6d009fbe55aac 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java @@ -217,8 +217,16 @@ private static Type parseTypeFromJson(JsonNode jsonNode) { return Types.DateType.get(); case TIME: return Types.TimeType.get(); + case TIME_MILLIS: + return Types.TimeMillisType.get(); case TIMESTAMP: return Types.TimestampType.get(); + case TIMESTAMP_MILLIS: + return Types.TimestampMillisType.get(); + case LOCAL_TIMESTAMP_MICROS: + return Types.LocalTimestampMicrosType.get(); + case LOCAL_TIMESTAMP_MILLIS: + return Types.LocalTimestampMillisType.get(); case STRING: return Types.StringType.get(); case UUID: diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java index 8ed597ed920df..380a7527a42f9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java @@ -33,7 +33,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.orc.CompressionKind; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.MessageType; @@ -45,6 +44,8 @@ import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR; import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; + public class HoodieAvroFileWriterFactory extends HoodieFileWriterFactory { protected HoodieFileWriter newParquetFileWriter( @@ -113,6 +114,6 @@ private HoodieAvroWriteSupport getHoodieAvroWriteSupport(Configuration conf, Sch return (HoodieAvroWriteSupport) ReflectionUtils.loadClass( config.getStringOrDefault(HoodieStorageConfig.HOODIE_AVRO_WRITE_SUPPORT_CLASS), new Class[] {MessageType.class, Schema.class, Option.class, Properties.class}, - new AvroSchemaConverter(conf).convert(schema), schema, filter, config.getProps()); + getAvroSchemaConverter(conf).convert(schema), schema, filter, config.getProps()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index ad4d1f16a60ce..8a5d9b45ee36f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -45,7 +45,9 @@ import java.util.List; import java.util.Set; +import static org.apache.hudi.avro.AvroSchemaUtils.getRepairedSchema; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; /** * {@link HoodieFileReader} implementation for parquet format. @@ -95,8 +97,8 @@ protected ClosableIterator getIndexedRecordIterator(Schema schema } @Override - protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { - return getIndexedRecordIteratorInternal(readerSchema, Option.of(requestedSchema)); + public ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + return getIndexedRecordIteratorInternal(requestedSchema, Option.empty()); } @Override @@ -154,19 +156,30 @@ private static Configuration tryOverrideDefaultConfigs(Configuration conf) { return conf; } - private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Option requestedSchema) throws IOException { + private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Option renamedColumns) throws IOException { // NOTE: We have to set both Avro read-schema and projection schema to make // sure that in case the file-schema is not equal to read-schema we'd still // be able to read that file (in case projection is a proper one) - if (!requestedSchema.isPresent()) { + Schema repairedFileSchema = getRepairedSchema(getSchema(), schema); + Option promotedSchema = Option.empty(); + if (!renamedColumns.isPresent() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema)) { + AvroReadSupport.setAvroReadSchema(conf, repairedFileSchema); + AvroReadSupport.setRequestedProjection(conf, repairedFileSchema); + promotedSchema = Option.of(schema); + } else { AvroReadSupport.setAvroReadSchema(conf, schema); AvroReadSupport.setRequestedProjection(conf, schema); - } else { - AvroReadSupport.setAvroReadSchema(conf, requestedSchema.get()); - AvroReadSupport.setRequestedProjection(conf, requestedSchema.get()); } - ParquetReader reader = new HoodieAvroParquetReaderBuilder(path).withConf(conf).build(); - ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader); + ParquetReader reader = + new HoodieAvroParquetReaderBuilder(path) + .withTableSchema(getAvroSchemaConverter(conf).convert(schema)) + .withConf(conf) + .set(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS, conf.get(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS)) + .set(ParquetInputFormat.STRICT_TYPE_CHECKING, conf.get(ParquetInputFormat.STRICT_TYPE_CHECKING)) + .build(); + ParquetReaderIterator parquetReaderIterator = promotedSchema.isPresent() + ? new HoodieAvroParquetReaderIterator(reader, promotedSchema.get()) + : new ParquetReaderIterator<>(reader); readerIterators.add(parquetReaderIterator); return parquetReaderIterator; } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 62b0232583293..e1bb7d08397d5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -109,7 +109,7 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; import static org.apache.hudi.avro.HoodieAvroUtils.convertValueForSpecificDataTypes; import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema; @@ -215,9 +215,8 @@ class ColumnStats { ColumnStats colStats = allColumnStats.computeIfAbsent(field.name(), (ignored) -> new ColumnStats()); GenericRecord genericRecord = (GenericRecord) record; - final Object fieldVal = convertValueForSpecificDataTypes(field.schema(), genericRecord.get(field.name()), false); - final Schema fieldSchema = getNestedFieldSchemaFromWriteSchema(genericRecord.getSchema(), field.name()); + final Schema fieldSchema = getNonNullTypeFromUnion(getNestedFieldSchemaFromWriteSchema(genericRecord.getSchema(), field.name())); colStats.valueCount++; @@ -1214,7 +1213,7 @@ private static Comparable coerceToComparable(Schema schema, Object val) { switch (schema.getType()) { case UNION: // TODO we need to handle unions in general case as well - return coerceToComparable(resolveNullableSchema(schema), val); + return coerceToComparable(getNonNullTypeFromUnion(schema), val); case FIXED: case BYTES: diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java index d6179ea1aacd2..61f526fa710d1 100644 --- a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java +++ b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java @@ -18,12 +18,15 @@ package org.apache.parquet.avro; +import org.apache.hudi.common.util.Option; + import org.apache.avro.generic.GenericData; import org.apache.avro.specific.SpecificData; import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.api.ReadSupport; import org.apache.parquet.io.InputFile; +import org.apache.parquet.schema.MessageType; /** * Copy from org.apache.parquet.avro.AvroParquetReader.Builder. @@ -35,6 +38,7 @@ public class HoodieAvroParquetReaderBuilder extends ParquetReader.Builder private GenericData model = null; private boolean enableCompatibility = true; private boolean isReflect = true; + private Option tableSchema = Option.empty(); @Deprecated public HoodieAvroParquetReaderBuilder(Path path) { @@ -67,6 +71,11 @@ public HoodieAvroParquetReaderBuilder withCompatibility(boolean enableCompati return this; } + public HoodieAvroParquetReaderBuilder withTableSchema(MessageType tableSchema) { + this.tableSchema = Option.of(tableSchema); + return this; + } + @Override protected ReadSupport getReadSupport() { if (isReflect) { @@ -74,6 +83,6 @@ protected ReadSupport getReadSupport() { } else { conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, enableCompatibility); } - return new HoodieAvroReadSupport<>(model); + return new HoodieAvroReadSupport<>(model, tableSchema); } } diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java new file mode 100644 index 0000000000000..2a748eb247f77 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java @@ -0,0 +1,56 @@ +/* + * 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.parquet.avro; + +import org.apache.hudi.common.util.ReflectionUtils; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Parquet-Java AvroSchemaConverter doesn't support local timestamp types until version 1.14 + * for this reason we use a modified version of the AvroSchemaConverter that adds support for local timestamp types + * Parquet-Java still supports local timestamp types from version 1.11.0, just that the AvroSchemaConverter + * doesn't work. + *

+ * However, for versions < 1.11.0, local timestamp is not supported at all. Therefore, we just use the + * library AvroSchemaConverter in this case. + * + */ +public abstract class HoodieAvroParquetSchemaConverter { + private static final Logger LOG = LoggerFactory.getLogger(HoodieAvroParquetSchemaConverter.class); + public static HoodieAvroParquetSchemaConverter getAvroSchemaConverter(Configuration configuration) { + try { + return (HoodieAvroParquetSchemaConverter) ReflectionUtils.loadClass("org.apache.parquet.avro.AvroSchemaConverterWithTimestampNTZ", + new Class[] {Configuration.class}, configuration); + } catch (Throwable t) { + LOG.debug("Failed to load AvroSchemaConverterWithTimestampNTZ, using NativeAvroSchemaConverter instead", t); + return (HoodieAvroParquetSchemaConverter) ReflectionUtils.loadClass("org.apache.parquet.avro.NativeAvroSchemaConverter", + new Class[] {Configuration.class}, configuration); + } + } + + public abstract MessageType convert(Schema schema); + + public abstract Schema convert(MessageType schema); +} diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java index 326accb66b2c2..b2c90553464fb 100644 --- a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java +++ b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java @@ -18,6 +18,8 @@ package org.apache.parquet.avro; +import org.apache.hudi.common.util.Option; + import org.apache.avro.generic.GenericData; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.schema.GroupType; @@ -25,6 +27,8 @@ import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.Type; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -36,11 +40,15 @@ */ public class HoodieAvroReadSupport extends AvroReadSupport { - public HoodieAvroReadSupport(GenericData model) { + private Option tableSchema; + + public HoodieAvroReadSupport(GenericData model, Option tableSchema) { super(model); + this.tableSchema = tableSchema; } public HoodieAvroReadSupport() { + tableSchema = Option.empty(); } @Override @@ -52,7 +60,7 @@ public ReadContext init(Configuration configuration, Map keyValu "false", "support reading avro from non-legacy map/list in parquet file"); } ReadContext readContext = super.init(configuration, keyValueMetaData, fileSchema); - MessageType requestedSchema = readContext.getRequestedSchema(); + MessageType requestedSchema = repairLogicalTypes(readContext.getRequestedSchema(), tableSchema); // support non-legacy map. Convert non-legacy map to legacy map // Because there is no AvroWriteSupport.WRITE_OLD_MAP_STRUCTURE // according to AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE @@ -130,4 +138,16 @@ private List convertLegacyMap(List oldTypes) { } return newTypes; } + + private MessageType repairLogicalTypes(MessageType fileSchema, Option tableSchemaOpt) { + try { + Class repairClass = Class.forName("org.apache.parquet.schema.SchemaRepair"); + Method repairMethod = repairClass.getDeclaredMethod( + "repairLogicalTypes", MessageType.class, Option.class); + MessageType repaired = (MessageType) repairMethod.invoke(null, fileSchema, tableSchemaOpt); + return repaired != null ? repaired : fileSchema; + } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + return fileSchema; + } + } } diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java b/hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java new file mode 100644 index 0000000000000..509a8afb3ea83 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java @@ -0,0 +1,46 @@ +/* + * 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.parquet.avro; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.MessageType; + +/** + * uses the native avro schema converter from parquet java + */ +public class NativeAvroSchemaConverter extends HoodieAvroParquetSchemaConverter { + + private final AvroSchemaConverter avroSchemaConverter; + + public NativeAvroSchemaConverter(Configuration configuration) { + this.avroSchemaConverter = new AvroSchemaConverter(configuration); + } + + @Override + public MessageType convert(Schema schema) { + return avroSchemaConverter.convert(schema); + } + + @Override + public Schema convert(MessageType schema) { + return avroSchemaConverter.convert(schema); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index cd3755d26c81f..32ca10777aaf2 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -19,9 +19,11 @@ package org.apache.hudi.common.testutils; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -35,10 +37,13 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.avro.Conversions; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -62,8 +67,10 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.Instant; +import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -72,15 +79,20 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField; +import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -131,6 +143,21 @@ public class HoodieTestDataGenerator implements AutoCloseable { + "{\"name\":\"current_date\",\"type\": {\"type\": \"int\", \"logicalType\": \"date\"}}," + "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\"}}," + "{\"name\":\"height\",\"type\":{\"type\":\"fixed\",\"name\":\"abc\",\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":6}},"; + public static final String EXTENDED_LOGICAL_TYPES_SCHEMA_V6 = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}}," + + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}}," + + "{\"name\":\"local_ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}}," + + "{\"name\":\"local_ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}}," + + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},"; + public static final String EXTENDED_LOGICAL_TYPES_SCHEMA = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}}," + + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}}," + + "{\"name\":\"local_ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}}," + + "{\"name\":\"local_ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}}," + + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},"; + + // LTS = Local Timestamp + public static final String EXTENDED_LOGICAL_TYPES_SCHEMA_NO_LTS = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}}," + + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}}," + + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},"; public static final String TRIP_EXAMPLE_SCHEMA = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX; @@ -139,6 +166,13 @@ public class HoodieTestDataGenerator implements AutoCloseable { public static final String TRIP_NESTED_EXAMPLE_SCHEMA = TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX; + public static final String TRIP_LOGICAL_TYPES_SCHEMA_V6 = + TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA_V6 + TRIP_SCHEMA_SUFFIX; + public static final String TRIP_LOGICAL_TYPES_SCHEMA = + TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA + TRIP_SCHEMA_SUFFIX; + // LTS = Local Timestamp + public static final String TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS = + TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA_NO_LTS + TRIP_SCHEMA_SUFFIX; public static final String TRIP_SCHEMA = "{\"type\":\"record\",\"name\":\"tripUberRec\",\"fields\":[" + "{\"name\":\"timestamp\",\"type\":\"long\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"}," @@ -151,13 +185,15 @@ public class HoodieTestDataGenerator implements AutoCloseable { public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6)," + "map,struct,array>,boolean"; - public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA); public static final Schema NESTED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_NESTED_EXAMPLE_SCHEMA); public static final TypeDescription ORC_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS = HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA); public static final Schema AVRO_SHORT_TRIP_SCHEMA = new Schema.Parser().parse(SHORT_TRIP_SCHEMA); + public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA); + public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6 = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA_V6); + public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS); public static final Schema AVRO_TRIP_SCHEMA = new Schema.Parser().parse(TRIP_SCHEMA); public static final TypeDescription ORC_TRIP_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_SCHEMA)); public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA); @@ -169,6 +205,7 @@ public class HoodieTestDataGenerator implements AutoCloseable { private final String[] partitionPaths; //maintains the count of existing keys schema wise private Map numKeysBySchema; + private Option extendedSchema = Option.empty(); public HoodieTestDataGenerator(long seed) { this(seed, DEFAULT_PARTITION_PATHS, new HashMap<>()); @@ -258,16 +295,40 @@ public int getEstimatedFileSizeInBytes(int numOfRecords) { } public RawTripTestPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime, boolean isFlattened) throws IOException { - if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) { - return generateRandomValue(key, commitTime, isFlattened); - } else if (TRIP_SCHEMA.equals(schemaStr)) { - return generatePayloadForTripSchema(key, commitTime); - } else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) { - return generatePayloadForShortTripSchema(key, commitTime); - } else if (TRIP_NESTED_EXAMPLE_SCHEMA.equals(schemaStr)) { - return generateNestedExampleRandomValue(key, commitTime); + return generateRandomValueAsPerSchema(schemaStr, key, commitTime, isFlattened, false, 0L); + } + + public RawTripTestPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime, + boolean isFlattened, boolean isDelete, long timestamp) throws IOException { + if (!isDelete) { + if (TRIP_FLATTENED_SCHEMA.equals(schemaStr)) { + return generateRandomValue(key, commitTime, true, timestamp); + } else if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) { + return generateRandomValue(key, commitTime, isFlattened, timestamp); + } else if (TRIP_SCHEMA.equals(schemaStr)) { + return generatePayloadForTripSchema(key, commitTime); + } else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) { + return generatePayloadForShortTripSchema(key, commitTime); + } else if (TRIP_NESTED_EXAMPLE_SCHEMA.equals(schemaStr)) { + return generateNestedExampleRandomValue(key, commitTime); + } else if (TRIP_LOGICAL_TYPES_SCHEMA.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchema(key, commitTime, false, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_V6.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaV6(key, commitTime, false, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaNoLTS(key, commitTime, false, timestamp); + } + } else { + if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) { + return generateRandomDeleteValue(key, commitTime); + } else if (TRIP_LOGICAL_TYPES_SCHEMA.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchema(key, commitTime, true, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_V6.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaV6(key, commitTime, true, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaNoLTS(key, commitTime, true, timestamp); + } } - return null; } @@ -336,9 +397,17 @@ public RawTripTestPayload generatePayloadForShortTripSchema(HoodieKey key, Strin * Generates a new avro record of the above schema format for a delete. */ private RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException { + return generateRandomDeleteValue(key, instantTime, TRIP_EXAMPLE_SCHEMA); + } + + private RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime, String schemaStr) throws IOException { GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0, true, false); - return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true, 0L); + return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), schemaStr, true, 0L); + } + + private RawTripTestPayload generateRandomDeleteValuePerSchema(HoodieKey key, String instantTime, String schemaStr) throws IOException { + return generateRandomValueAsPerSchema(schemaStr, key, instantTime, false, true, 0L); } /** @@ -354,6 +423,20 @@ public GenericRecord generateGenericRecord(String rowKey, String partitionPath, return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false); } + /** + * LTS = Local Timestamp + */ + public RawTripTestPayload generatePayloadForLogicalTypesSchemaNoLTS(HoodieKey key, String commitTime, boolean isDelete, long timestamp) throws IOException { + return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, false, false); + } + + public RawTripTestPayload generatePayloadForLogicalTypesSchema(HoodieKey key, String commitTime, boolean isDelete, long timestamp) throws IOException { + return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, false, true); + } + + public RawTripTestPayload generatePayloadForLogicalTypesSchemaV6(HoodieKey key, String commitTime, boolean isDelete, long timestamp) throws IOException { + return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, true, true); + } /** * Populate rec with values for TRIP_SCHEMA_PREFIX @@ -442,13 +525,11 @@ private void generateTripSuffixValues(GenericRecord rec, boolean isDeleteRecord) } } - /** * Generate record conforming to TRIP_EXAMPLE_SCHEMA or TRIP_FLATTENED_SCHEMA if isFlattened is true */ public GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName, - long timestamp, boolean isDeleteRecord, - boolean isFlattened) { + long timestamp, boolean isDeleteRecord, boolean isFlattened) { GenericRecord rec = new GenericData.Record(isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA); generateTripPrefixValues(rec, rowKey, partitionPath, riderName, driverName, timestamp); if (isFlattened) { @@ -500,6 +581,65 @@ public GenericRecord generateRecordForShortTripSchema(String rowKey, String ride return rec; } + public RawTripTestPayload generateRecordForTripLogicalTypesSchema(HoodieKey key, String riderName, String driverName, + long timestamp, boolean isDeleteRecord, boolean v6, + boolean hasLTS) throws IOException { + GenericRecord rec; + if (!hasLTS) { + // LTS = Local Timestamp + rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS); + } else if (v6) { + rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6); + } else { + rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA); + } + generateTripPrefixValues(rec, key.getRecordKey(), key.getPartitionPath(), riderName, driverName, timestamp); + + int hash = key.getRecordKey().hashCode(); + boolean above = (hash & 1) == 0; // half above, half below threshold + + // ------------------- + // Threshold definitions + // ------------------- + Instant tsMillisThreshold = Instant.parse("2020-01-01T00:00:00Z"); + Instant tsMicrosThreshold = Instant.parse("2020-06-01T12:00:00Z"); + + Instant localTsMillisThreshold = ZonedDateTime.of( + 2015, 5, 20, 12, 34, 56, 0, ZoneOffset.UTC).toInstant(); + Instant localTsMicrosThreshold = ZonedDateTime.of( + 2017, 7, 7, 7, 7, 7, 0, ZoneOffset.UTC).toInstant(); + + LocalDate dateThreshold = LocalDate.of(2000, 1, 1); + + // ------------------- + // Assign edge values + // ------------------- + + // ts_millis + long tsMillisBase = tsMillisThreshold.toEpochMilli(); + rec.put("ts_millis", above ? tsMillisBase + 1 : tsMillisBase - 1); + + // ts_micros + long tsMicrosBase = TimeUnit.SECONDS.toMicros(tsMicrosThreshold.getEpochSecond()) + tsMicrosThreshold.getNano() / 1_000L; + rec.put("ts_micros", above ? tsMicrosBase + 1 : tsMicrosBase - 1); + + if (hasLTS) { + // local_ts_millis + long localTsMillisBase = localTsMillisThreshold.toEpochMilli(); + rec.put("local_ts_millis", above ? localTsMillisBase + 1 : localTsMillisBase - 1); + + // local_ts_micros + long localTsMicrosBase = TimeUnit.SECONDS.toMicros(localTsMicrosThreshold.getEpochSecond()) + localTsMicrosThreshold.getNano() / 1_000L; + rec.put("local_ts_micros", above ? localTsMicrosBase + 1 : localTsMicrosBase - 1); + } + + // event_date + int eventDateBase = (int) dateThreshold.toEpochDay(); + rec.put("event_date", above ? eventDateBase + 1 : eventDateBase - 1); + generateTripSuffixValues(rec, isDeleteRecord); + return new RawTripTestPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), rec.getSchema().toString()); + } + public static void createRequestedCommitFile(String basePath, String instantTime, Configuration configuration) throws IOException { Path pendingRequestedFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeRequestedCommitFileName(instantTime)); @@ -939,9 +1079,13 @@ public Stream generateUniqueUpdatesStream(String instantTime, Inte * @return stream of hoodie record updates */ public Stream generateUniqueDeleteStream(Integer n) { + return generateUniqueDeleteStream(n, TRIP_EXAMPLE_SCHEMA); + } + + public Stream generateUniqueDeleteStream(Integer n, String streamStr) { final Set used = new HashSet<>(); - Map existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); - Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); + Map existingKeys = existingKeysBySchema.get(streamStr); + Integer numExistingKeys = numKeysBySchema.get(streamStr); if (n > numExistingKeys) { throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys"); } @@ -959,7 +1103,7 @@ public Stream generateUniqueDeleteStream(Integer n) { used.add(kp); result.add(kp.key); } - numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys); + numKeysBySchema.put(streamStr, numExistingKeys); return result.stream(); } @@ -971,9 +1115,13 @@ public Stream generateUniqueDeleteStream(Integer n) { * @return stream of hoodie records for delete */ public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n) { + return generateUniqueDeleteRecordStream(instantTime, n, TRIP_EXAMPLE_SCHEMA); + } + + public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n, String schemaStr) { final Set used = new HashSet<>(); - Map existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); - Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); + Map existingKeys = existingKeysBySchema.get(schemaStr); + Integer numExistingKeys = numKeysBySchema.get(schemaStr); if (n > numExistingKeys) { throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys"); } @@ -991,12 +1139,12 @@ public Stream generateUniqueDeleteRecordStream(String instantTime, numExistingKeys--; used.add(kp); try { - result.add(new HoodieAvroRecord(kp.key, generateRandomDeleteValue(kp.key, instantTime))); + result.add(new HoodieAvroRecord(kp.key, generateRandomDeleteValuePerSchema(kp.key, instantTime, schemaStr))); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } } - numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys); + numKeysBySchema.put(schemaStr, numExistingKeys); return result.stream(); } @@ -1062,7 +1210,7 @@ public void close() { private static long genRandomTimeMillis(Random r) { // Fri Feb 13 15:31:30 PST 2009 - long anchorTs = 1234567890L; + long anchorTs = 1234567890000L; // NOTE: To provide for certainty and not generate overly random dates, we will limit // dispersion to be w/in +/- 3 days from the anchor date return anchorTs + r.nextLong() % 259200000L; @@ -1086,4 +1234,311 @@ public static UUID genPseudoRandomUUID(Random r) { throw new HoodieException(e); } } + + /** + * Used for equality checks between the expected and actual records for generated by the HoodieTestDataGenerator. + * The fields identify the record with the combination of the recordKey and partitionPath and assert that the proper + * value is present with the orderingVal and the riderValue, which is updated as part of the update utility methods. + */ + public static class RecordIdentifier { + private final String recordKey; + private final String orderingVal; + private final String partitionPath; + private final String riderValue; + + @JsonCreator + public RecordIdentifier(@JsonProperty("recordKey") String recordKey, + @JsonProperty("partitionPath") String partitionPath, + @JsonProperty("orderingVal") String orderingVal, + @JsonProperty("riderValue") String riderValue) { + this.recordKey = recordKey; + this.orderingVal = orderingVal; + this.partitionPath = partitionPath; + this.riderValue = riderValue; + } + + public static RecordIdentifier clone(RecordIdentifier toClone, String orderingVal) { + return new RecordIdentifier(toClone.recordKey, toClone.partitionPath, orderingVal, toClone.riderValue); + } + + public static RecordIdentifier fromTripTestPayload(HoodieAvroIndexedRecord record, String[] orderingFields) { + String recordKey = record.getRecordKey(); + String partitionPath = record.getPartitionPath(); + Comparable orderingValue = record.getOrderingValue(record.getData().getSchema(), CollectionUtils.emptyProps()); + String orderingValStr = orderingValue.toString(); + GenericRecord data = (GenericRecord) record.getData(); + String riderValue = data.getSchema().getField("rider") != null ? data.get("rider").toString() : ""; + return new RecordIdentifier(recordKey, partitionPath, orderingValStr, riderValue); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + RecordIdentifier that = (RecordIdentifier) o; + return Objects.equals(recordKey, that.recordKey) + && Objects.equals(orderingVal, that.orderingVal) + && Objects.equals(partitionPath, that.partitionPath) + && Objects.equals(riderValue, that.riderValue); + } + + @Override + public int hashCode() { + return Objects.hash(recordKey, orderingVal, partitionPath, riderValue); + } + + public String getRecordKey() { + return recordKey; + } + + public String getOrderingVal() { + return orderingVal; + } + + public String getPartitionPath() { + return partitionPath; + } + + public String getRiderValue() { + return riderValue; + } + + @Override + public String toString() { + return "RowKey: " + recordKey + ", PartitionPath: " + partitionPath + + ", OrderingVal: " + orderingVal + ", RiderValue: " + riderValue; + } + } + + public static class SchemaEvolutionConfigs { + public Schema schema = AVRO_SCHEMA; + public boolean nestedSupport = true; + public boolean mapSupport = true; + public boolean arraySupport = true; + public boolean addNewFieldSupport = true; + // TODO: [HUDI-9603] Flink 1.18 array values incorrect in fg reader test + public boolean anyArraySupport = true; + + // Int + public boolean intToLongSupport = true; + public boolean intToFloatSupport = true; + public boolean intToDoubleSupport = true; + public boolean intToStringSupport = true; + + // Long + public boolean longToFloatSupport = true; + public boolean longToDoubleSupport = true; + public boolean longToStringSupport = true; + + // Float + public boolean floatToDoubleSupport = true; + public boolean floatToStringSupport = true; + + // Double + public boolean doubleToStringSupport = true; + + // String + public boolean stringToBytesSupport = true; + + // Bytes + public boolean bytesToStringSupport = true; + } + + private enum SchemaEvolutionTypePromotionCase { + INT_TO_INT(Schema.Type.INT, Schema.Type.INT, config -> true), + INT_TO_LONG(Schema.Type.INT, Schema.Type.LONG, config -> config.intToLongSupport), + INT_TO_FLOAT(Schema.Type.INT, Schema.Type.FLOAT, config -> config.intToFloatSupport), + INT_TO_DOUBLE(Schema.Type.INT, Schema.Type.DOUBLE, config -> config.intToDoubleSupport), + INT_TO_STRING(Schema.Type.INT, Schema.Type.STRING, config -> config.intToStringSupport), + LONG_TO_LONG(Schema.Type.LONG, Schema.Type.LONG, config -> true), + LONG_TO_FLOAT(Schema.Type.LONG, Schema.Type.FLOAT, config -> config.longToFloatSupport), + LONG_TO_DOUBLE(Schema.Type.LONG, Schema.Type.DOUBLE, config -> config.longToDoubleSupport), + LONG_TO_STRING(Schema.Type.LONG, Schema.Type.STRING, config -> config.longToStringSupport), + FLOAT_TO_FLOAT(Schema.Type.FLOAT, Schema.Type.FLOAT, config -> true), + FLOAT_TO_DOUBLE(Schema.Type.FLOAT, Schema.Type.DOUBLE, config -> config.floatToDoubleSupport), + FLOAT_TO_STRING(Schema.Type.FLOAT, Schema.Type.STRING, config -> config.floatToStringSupport), + DOUBLE_TO_DOUBLE(Schema.Type.DOUBLE, Schema.Type.DOUBLE, config -> true), + DOUBLE_TO_STRING(Schema.Type.DOUBLE, Schema.Type.STRING, config -> config.doubleToStringSupport), + STRING_TO_STRING(Schema.Type.STRING, Schema.Type.STRING, config -> true), + STRING_TO_BYTES(Schema.Type.STRING, Schema.Type.BYTES, config -> config.stringToBytesSupport), + BYTES_TO_BYTES(Schema.Type.BYTES, Schema.Type.BYTES, config -> true), + BYTES_TO_STRING(Schema.Type.BYTES, Schema.Type.STRING, config -> config.bytesToStringSupport); + + public final Schema.Type before; + public final Schema.Type after; + public final Predicate isEnabled; + + SchemaEvolutionTypePromotionCase(Schema.Type before, Schema.Type after, Predicate isEnabled) { + this.before = before; + this.after = after; + this.isEnabled = isEnabled; + } + } + + public void extendSchema(SchemaEvolutionConfigs configs, boolean isBefore) { + List baseFields = new ArrayList<>(); + for (SchemaEvolutionTypePromotionCase evolution : SchemaEvolutionTypePromotionCase.values()) { + if (evolution.isEnabled.test(configs)) { + baseFields.add(isBefore ? evolution.before : evolution.after); + } + } + + // Add new field if we are testing adding new fields + if (!isBefore && configs.addNewFieldSupport) { + baseFields.add(Schema.Type.BOOLEAN); + } + + this.extendedSchema = Option.of(generateExtendedSchema(configs, new ArrayList<>(baseFields))); + } + + public void extendSchemaBeforeEvolution(SchemaEvolutionConfigs configs) { + extendSchema(configs, true); + } + + public void extendSchemaAfterEvolution(SchemaEvolutionConfigs configs) { + extendSchema(configs, false); + } + + public Schema getExtendedSchema() { + return extendedSchema.orElseThrow(IllegalArgumentException::new); + } + + private static Schema generateExtendedSchema(SchemaEvolutionConfigs configs, List baseFields) { + return generateExtendedSchema(configs.schema, configs, baseFields, "customField", true); + } + + private static Schema generateExtendedSchema(Schema baseSchema, SchemaEvolutionConfigs configs, List baseFields, String fieldPrefix, boolean toplevel) { + List fields = baseSchema.getFields(); + List finalFields = new ArrayList<>(fields.size() + baseFields.size()); + boolean addedFields = false; + for (Schema.Field field : fields) { + if (configs.nestedSupport && field.name().equals("fare") && field.schema().getType() == Schema.Type.RECORD) { + finalFields.add(createNewSchemaField(field.name(), generateExtendedSchema(field.schema(), configs, baseFields, "customFare", false), field.doc(), field.defaultVal())); + } else if (configs.anyArraySupport || !field.name().equals("tip_history")) { + //TODO: [HUDI-9603] remove the if condition when the issue is fixed + if (field.name().equals("_hoodie_is_deleted")) { + addedFields = true; + addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel); + } + finalFields.add(createNewSchemaField(field)); + } + } + if (!addedFields) { + addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel); + } + Schema finalSchema = Schema.createRecord(baseSchema.getName(), baseSchema.getDoc(), + baseSchema.getNamespace(), baseSchema.isError()); + finalSchema.setFields(finalFields); + return finalSchema; + } + + private static void addFields(SchemaEvolutionConfigs configs, List finalFields, List baseFields, String fieldPrefix, String namespace, boolean toplevel) { + if (toplevel) { + if (configs.mapSupport) { + List mapFields = new ArrayList<>(baseFields.size()); + addFieldsHelper(mapFields, baseFields, fieldPrefix + "Map"); + finalFields.add(new Schema.Field(fieldPrefix + "Map", Schema.createMap(Schema.createRecord("customMapRecord", "", namespace, false, mapFields)), "", null)); + } + + if (configs.arraySupport) { + List arrayFields = new ArrayList<>(baseFields.size()); + addFieldsHelper(arrayFields, baseFields, fieldPrefix + "Array"); + finalFields.add(new Schema.Field(fieldPrefix + "Array", Schema.createArray(Schema.createRecord("customArrayRecord", "", namespace, false, arrayFields)), "", null)); + } + } + addFieldsHelper(finalFields, baseFields, fieldPrefix); + } + + private static void addFieldsHelper(List finalFields, List baseFields, String fieldPrefix) { + for (int i = 0; i < baseFields.size(); i++) { + if (baseFields.get(i) == Schema.Type.BOOLEAN) { + // boolean fields are added fields + finalFields.add(new Schema.Field(fieldPrefix + i, AvroSchemaUtils.createNullableSchema(Schema.Type.BOOLEAN), "", null)); + } else { + finalFields.add(new Schema.Field(fieldPrefix + i, Schema.create(baseFields.get(i)), "", null)); + } + } + } + + private void generateCustomValues(GenericRecord rec, String customPrefix) { + for (Schema.Field field : rec.getSchema().getFields()) { + if (field.name().startsWith(customPrefix)) { + switch (field.schema().getType()) { + case INT: + rec.put(field.name(), rand.nextInt()); + break; + case LONG: + rec.put(field.name(), rand.nextLong()); + break; + case FLOAT: + rec.put(field.name(), rand.nextFloat()); + break; + case DOUBLE: + rec.put(field.name(), rand.nextDouble()); + break; + case STRING: + rec.put(field.name(), genPseudoRandomUUID(rand).toString()); + break; + case BYTES: + rec.put(field.name(), ByteBuffer.wrap(getUTF8Bytes(genPseudoRandomUUID(rand).toString()))); + break; + case UNION: + if (!AvroSchemaUtils.getNonNullTypeFromUnion(field.schema()).getType().equals(Schema.Type.BOOLEAN)) { + throw new IllegalStateException("Union should only be boolean"); + } + rec.put(field.name(), rand.nextBoolean()); + break; + case BOOLEAN: + rec.put(field.name(), rand.nextBoolean()); + break; + case MAP: + rec.put(field.name(), genMap(field.schema(), field.name())); + break; + case ARRAY: + rec.put(field.name(), genArray(field.schema(), field.name())); + break; + default: + throw new UnsupportedOperationException("Unsupported type: " + field.schema().getType()); + } + } + } + } + + private GenericArray genArray(Schema arraySchema, String customPrefix) { + GenericArray customArray = new GenericData.Array<>(1, arraySchema); + Schema arrayElementSchema = arraySchema.getElementType(); + GenericRecord customRecord = new GenericData.Record(arrayElementSchema); + generateCustomValues(customRecord, customPrefix); + customArray.add(customRecord); + return customArray; + } + + private Map genMap(Schema mapSchema, String customPrefix) { + Schema mapElementSchema = mapSchema.getValueType(); + GenericRecord customRecord = new GenericData.Record(mapElementSchema); + generateCustomValues(customRecord, customPrefix); + return Collections.singletonMap("customMapKey", customRecord); + } + + public static List recordsToStrings(List records) { + return records.stream().map(HoodieTestDataGenerator::recordToString).filter(Option::isPresent).map(Option::get) + .collect(Collectors.toList()); + } + + public static Option recordToString(HoodieRecord record) { + try { + String str = ((GenericRecord) record.getData()).toString(); + // Remove the last } bracket + str = str.substring(0, str.length() - 1); + return Option.of(str + ", \"partition\": \"" + record.getPartitionPath() + "\"}"); + } catch (Exception e) { + return Option.empty(); + } + } + + public static List deleteRecordsToStrings(List records) { + return records.stream().map(record -> "{\"_row_key\": \"" + record.getRecordKey() + "\",\"partition\": \"" + record.getPartitionPath() + "\"}") + .collect(Collectors.toList()); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index 7100ab9af3419..23ad8febfb219 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -35,16 +35,23 @@ import com.esotericsoftware.kryo.serializers.JavaSerializer; import org.apache.hadoop.conf.Configuration; +import java.io.BufferedOutputStream; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.io.Serializable; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.Properties; import java.util.UUID; import org.junit.jupiter.api.Assumptions; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; /** * A utility class for testing. @@ -255,4 +262,47 @@ public static DistributedFileSystem useExternalHdfs() throws IOException { conf.set("dfs.replication", "3"); return (DistributedFileSystem) DistributedFileSystem.get(conf); } + + /** + * Extracts a ZIP file from resources to a target directory. + * + * @param resourcePath the path to the ZIP resource (relative to classpath) + * @param targetDirectory the target directory to extract files to + * @param resourceClass the class to use for resource loading + * @throws IOException if extraction fails + */ + public static void extractZipToDirectory(String resourcePath, Path targetDirectory, Class resourceClass) throws IOException { + InputStream resourceStream = resourceClass.getClassLoader().getResourceAsStream(resourcePath); + if (resourceStream == null) { + // Fallback to getResourceAsStream if getClassLoader().getResourceAsStream() fails + resourceStream = resourceClass.getResourceAsStream(resourcePath); + } + + if (resourceStream == null) { + throw new IOException("Resource not found at: " + resourcePath); + } + + try (ZipInputStream zip = new ZipInputStream(resourceStream)) { + ZipEntry entry; + while ((entry = zip.getNextEntry()) != null) { + File file = targetDirectory.resolve(entry.getName()).toFile(); + if (entry.isDirectory()) { + file.mkdirs(); + continue; + } + + // Create parent directories if they don't exist + file.getParentFile().mkdirs(); + + // Extract file content + byte[] buffer = new byte[10000]; + try (BufferedOutputStream out = new BufferedOutputStream(Files.newOutputStream(file.toPath()))) { + int count; + while ((count = zip.read(buffer)) != -1) { + out.write(buffer, 0, count); + } + } + } + } + } } diff --git a/hudi-hadoop-common/pom.xml b/hudi-hadoop-common/pom.xml new file mode 100644 index 0000000000000..b2959dcc61f97 --- /dev/null +++ b/hudi-hadoop-common/pom.xml @@ -0,0 +1,174 @@ + + + + + hudi + org.apache.hudi + 0.16.0-SNAPSHOT + + 4.0.0 + + hudi-hadoop-common + + + ${project.parent.basedir} + + + + + + src/main/resources + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + org.jacoco + jacoco-maven-plugin + + + org.codehaus.mojo + build-helper-maven-plugin + 3.5.0 + + + add-spark34plus-avro-sources + generate-sources + + add-source + + + ${spark33orEarlier} + + src/avro/java + src/parquet/java + + + + + add-spark34plus-parquet-test-sources + generate-test-sources + + add-test-source + + + ${spark33orEarlier} + + src/parquet/test/java + + + + + + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-io + ${project.version} + + + + + org.apache.hadoop + hadoop-client + + + javax.servlet + * + + + provided + + + org.apache.hadoop + hadoop-hdfs + provided + + + + org.apache.parquet + parquet-avro + + + + + com.esotericsoftware + kryo-shaded + + + + org.apache.hudi + hudi-tests-common + ${project.version} + test + + + + org.apache.hudi + hudi-io + tests + ${project.version} + test + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + com.github.stefanbirkner + system-rules + 1.17.2 + test + + + diff --git a/hudi-hadoop-common/src/avro/java/org/apache/parquet.avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-hadoop-common/src/avro/java/org/apache/parquet.avro/AvroSchemaConverterWithTimestampNTZ.java new file mode 100644 index 0000000000000..c1bda6ef0ffb7 --- /dev/null +++ b/hudi-hadoop-common/src/avro/java/org/apache/parquet.avro/AvroSchemaConverterWithTimestampNTZ.java @@ -0,0 +1,598 @@ +/* + * 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.parquet.avro; + +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.ConversionPatterns; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.avro.JsonProperties.NULL_VALUE; +import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED; +import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED_DEFAULT; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE_DEFAULT; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID_DEFAULT; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MICROS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MILLIS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.dateType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.decimalType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.enumType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.stringType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.timeType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.timestampType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.uuidType; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.Type.Repetition.REPEATED; + +/** + *

+ * Converts an Avro schema into a Parquet schema, or vice versa. See package + * documentation for details of the mapping. + *

+ * This was taken from parquet-java 1.13.1 AvroSchemaConverter and modified + * to support local timestamp types by copying a few methods from 1.14.0 AvroSchemaConverter. + */ +@SuppressWarnings("all") +public class AvroSchemaConverterWithTimestampNTZ extends HoodieAvroParquetSchemaConverter { + + public static final String ADD_LIST_ELEMENT_RECORDS = + "parquet.avro.add-list-element-records"; + private static final boolean ADD_LIST_ELEMENT_RECORDS_DEFAULT = true; + + private final boolean assumeRepeatedIsListElement; + private final boolean writeOldListStructure; + private final boolean writeParquetUUID; + private final boolean readInt96AsFixed; + private final Set pathsToInt96; + + public AvroSchemaConverterWithTimestampNTZ() { + this(ADD_LIST_ELEMENT_RECORDS_DEFAULT); + } + + /** + * Constructor used by {@link AvroRecordConverter#isElementType}, which always + * uses the 2-level list conversion. + * + * @param assumeRepeatedIsListElement whether to assume 2-level lists + */ + AvroSchemaConverterWithTimestampNTZ(boolean assumeRepeatedIsListElement) { + this.assumeRepeatedIsListElement = assumeRepeatedIsListElement; + this.writeOldListStructure = WRITE_OLD_LIST_STRUCTURE_DEFAULT; + this.writeParquetUUID = WRITE_PARQUET_UUID_DEFAULT; + this.readInt96AsFixed = READ_INT96_AS_FIXED_DEFAULT; + this.pathsToInt96 = Collections.emptySet(); + } + + public AvroSchemaConverterWithTimestampNTZ(Configuration conf) { + this.assumeRepeatedIsListElement = conf.getBoolean( + ADD_LIST_ELEMENT_RECORDS, ADD_LIST_ELEMENT_RECORDS_DEFAULT); + this.writeOldListStructure = conf.getBoolean( + WRITE_OLD_LIST_STRUCTURE, WRITE_OLD_LIST_STRUCTURE_DEFAULT); + this.writeParquetUUID = conf.getBoolean(WRITE_PARQUET_UUID, WRITE_PARQUET_UUID_DEFAULT); + this.readInt96AsFixed = conf.getBoolean(READ_INT96_AS_FIXED, READ_INT96_AS_FIXED_DEFAULT); + this.pathsToInt96 = new HashSet<>(Arrays.asList(conf.getStrings("parquet.avro.writeFixedAsInt96", new String[0]))); + } + + /** + * Given a schema, check to see if it is a union of a null type and a regular schema, + * and then return the non-null sub-schema. Otherwise, return the given schema. + * + * @param schema The schema to check + * @return The non-null portion of a union schema, or the given schema + */ + public static Schema getNonNull(Schema schema) { + if (schema.getType().equals(Schema.Type.UNION)) { + List schemas = schema.getTypes(); + if (schemas.size() == 2) { + if (schemas.get(0).getType().equals(Schema.Type.NULL)) { + return schemas.get(1); + } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) { + return schemas.get(0); + } else { + return schema; + } + } else { + return schema; + } + } else { + return schema; + } + } + + @Override + public MessageType convert(Schema avroSchema) { + if (!avroSchema.getType().equals(Schema.Type.RECORD)) { + throw new IllegalArgumentException("Avro schema must be a record."); + } + return new MessageType(avroSchema.getFullName(), convertFields(avroSchema.getFields(), "")); + } + + private List convertFields(List fields, String schemaPath) { + List types = new ArrayList(); + for (Schema.Field field : fields) { + if (field.schema().getType().equals(Schema.Type.NULL)) { + continue; // Avro nulls are not encoded, unless they are null unions + } + types.add(convertField(field, appendPath(schemaPath, field.name()))); + } + return types; + } + + private Type convertField(String fieldName, Schema schema, String schemaPath) { + return convertField(fieldName, schema, Type.Repetition.REQUIRED, schemaPath); + } + + @SuppressWarnings("deprecation") + private Type convertField(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) { + Types.PrimitiveBuilder builder; + Schema.Type type = schema.getType(); + LogicalType logicalType = schema.getLogicalType(); + if (type.equals(Schema.Type.BOOLEAN)) { + builder = Types.primitive(BOOLEAN, repetition); + } else if (type.equals(Schema.Type.INT)) { + builder = Types.primitive(INT32, repetition); + } else if (type.equals(Schema.Type.LONG)) { + builder = Types.primitive(INT64, repetition); + } else if (type.equals(Schema.Type.FLOAT)) { + builder = Types.primitive(FLOAT, repetition); + } else if (type.equals(Schema.Type.DOUBLE)) { + builder = Types.primitive(DOUBLE, repetition); + } else if (type.equals(Schema.Type.BYTES)) { + builder = Types.primitive(BINARY, repetition); + } else if (type.equals(Schema.Type.STRING)) { + if (logicalType != null && logicalType.getName().equals(LogicalTypes.uuid().getName()) && writeParquetUUID) { + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .length(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation.BYTES); + } else { + builder = Types.primitive(BINARY, repetition).as(stringType()); + } + } else if (type.equals(Schema.Type.RECORD)) { + return new GroupType(repetition, fieldName, convertFields(schema.getFields(), schemaPath)); + } else if (type.equals(Schema.Type.ENUM)) { + builder = Types.primitive(BINARY, repetition).as(enumType()); + } else if (type.equals(Schema.Type.ARRAY)) { + if (writeOldListStructure) { + return ConversionPatterns.listType(repetition, fieldName, + convertField("array", schema.getElementType(), REPEATED, schemaPath)); + } else { + return ConversionPatterns.listOfElements(repetition, fieldName, + convertField(AvroWriteSupport.LIST_ELEMENT_NAME, schema.getElementType(), schemaPath)); + } + } else if (type.equals(Schema.Type.MAP)) { + Type valType = convertField("value", schema.getValueType(), schemaPath); + // avro map key type is always string + return ConversionPatterns.stringKeyMapType(repetition, fieldName, valType); + } else if (type.equals(Schema.Type.FIXED)) { + if (pathsToInt96.contains(schemaPath)) { + if (schema.getFixedSize() != 12) { + throw new IllegalArgumentException( + "The size of the fixed type field " + schemaPath + " must be 12 bytes for INT96 conversion"); + } + builder = Types.primitive(PrimitiveTypeName.INT96, repetition); + } else { + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition).length(schema.getFixedSize()); + } + } else if (type.equals(Schema.Type.UNION)) { + return convertUnion(fieldName, schema, repetition, schemaPath); + } else { + throw new UnsupportedOperationException("Cannot convert Avro type " + type); + } + + // schema translation can only be done for known logical types because this + // creates an equivalence + if (logicalType != null) { + if (logicalType instanceof LogicalTypes.Decimal) { + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + builder = builder.as(decimalType(decimal.getScale(), decimal.getPrecision())); + } else { + LogicalTypeAnnotation annotation = convertLogicalType(logicalType); + if (annotation != null) { + builder.as(annotation); + } + } + } + + return builder.named(fieldName); + } + + private Type convertUnion(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) { + List nonNullSchemas = new ArrayList(schema.getTypes().size()); + // Found any schemas in the union? Required for the edge case, where the union contains only a single type. + boolean foundNullSchema = false; + for (Schema childSchema : schema.getTypes()) { + if (childSchema.getType().equals(Schema.Type.NULL)) { + foundNullSchema = true; + if (Type.Repetition.REQUIRED == repetition) { + repetition = Type.Repetition.OPTIONAL; + } + } else { + nonNullSchemas.add(childSchema); + } + } + // If we only get a null and one other type then its a simple optional field + // otherwise construct a union container + switch (nonNullSchemas.size()) { + case 0: + throw new UnsupportedOperationException("Cannot convert Avro union of only nulls"); + + case 1: + return foundNullSchema ? convertField(fieldName, nonNullSchemas.get(0), repetition, schemaPath) : + convertUnionToGroupType(fieldName, repetition, nonNullSchemas, schemaPath); + + default: // complex union type + return convertUnionToGroupType(fieldName, repetition, nonNullSchemas, schemaPath); + } + } + + private Type convertUnionToGroupType(String fieldName, Type.Repetition repetition, List nonNullSchemas, + String schemaPath) { + List unionTypes = new ArrayList(nonNullSchemas.size()); + int index = 0; + for (Schema childSchema : nonNullSchemas) { + unionTypes.add( convertField("member" + index++, childSchema, Type.Repetition.OPTIONAL, schemaPath)); + } + return new GroupType(repetition, fieldName, unionTypes); + } + + private Type convertField(Schema.Field field, String schemaPath) { + return convertField(field.name(), field.schema(), schemaPath); + } + + @Override + public Schema convert(MessageType parquetSchema) { + return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>()); + } + + Schema convert(GroupType parquetSchema) { + return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>()); + } + + private Schema convertFields(String name, List parquetFields, Map names) { + List fields = new ArrayList(); + Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue + 1); + for (Type parquetType : parquetFields) { + Schema fieldSchema = convertField(parquetType, names); + if (parquetType.isRepetition(REPEATED)) { + throw new UnsupportedOperationException("REPEATED not supported outside LIST or MAP. Type: " + parquetType); + } else if (parquetType.isRepetition(Type.Repetition.OPTIONAL)) { + fields.add(new Schema.Field( + parquetType.getName(), optional(fieldSchema), null, NULL_VALUE)); + } else { // REQUIRED + fields.add(new Schema.Field( + parquetType.getName(), fieldSchema, null, (Object) null)); + } + } + Schema schema = Schema.createRecord(name, null, nameCount > 1 ? name + nameCount : null, false); + schema.setFields(fields); + return schema; + } + + private Schema convertField(final Type parquetType, Map names) { + if (parquetType.isPrimitive()) { + final PrimitiveType asPrimitive = parquetType.asPrimitiveType(); + final PrimitiveTypeName parquetPrimitiveTypeName = + asPrimitive.getPrimitiveTypeName(); + final LogicalTypeAnnotation annotation = parquetType.getLogicalTypeAnnotation(); + Schema schema = parquetPrimitiveTypeName.convert( + new PrimitiveType.PrimitiveTypeNameConverter() { + @Override + public Schema convertBOOLEAN(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.BOOLEAN); + } + @Override + public Schema convertINT32(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.INT); + } + @Override + public Schema convertINT64(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.LONG); + } + @Override + public Schema convertINT96(PrimitiveTypeName primitiveTypeName) { + if (readInt96AsFixed) { + return Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + } + throw new IllegalArgumentException( + "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array."); + } + @Override + public Schema convertFLOAT(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.FLOAT); + } + @Override + public Schema convertDOUBLE(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.DOUBLE); + } + @Override + public Schema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) { + if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + return Schema.create(Schema.Type.STRING); + } else { + int size = parquetType.asPrimitiveType().getTypeLength(); + return Schema.createFixed(parquetType.getName(), null, null, size); + } + } + @Override + public Schema convertBINARY(PrimitiveTypeName primitiveTypeName) { + if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation || + annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + return Schema.create(Schema.Type.STRING); + } else { + return Schema.create(Schema.Type.BYTES); + } + } + }); + + LogicalType logicalType = convertLogicalType(annotation); + if (logicalType != null && (!(annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) || + parquetPrimitiveTypeName == BINARY || + parquetPrimitiveTypeName == FIXED_LEN_BYTE_ARRAY)) { + schema = logicalType.addToSchema(schema); + } + + return schema; + + } else { + GroupType parquetGroupType = parquetType.asGroupType(); + LogicalTypeAnnotation logicalTypeAnnotation = parquetGroupType.getLogicalTypeAnnotation(); + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation.accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { + @Override + public java.util.Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + if (parquetGroupType.getFieldCount()!= 1) { + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); + } + Type repeatedType = parquetGroupType.getType(0); + if (!repeatedType.isRepetition(REPEATED)) { + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); + } + if (isElementType(repeatedType, parquetGroupType.getName())) { + // repeated element types are always required + return java.util.Optional.of(Schema.createArray(convertField(repeatedType, names))); + } else { + Type elementType = repeatedType.asGroupType().getType(0); + if (elementType.isRepetition(Type.Repetition.OPTIONAL)) { + return java.util.Optional.of(Schema.createArray(optional(convertField(elementType, names)))); + } else { + return java.util.Optional.of(Schema.createArray(convertField(elementType, names))); + } + } + } + + @Override + // for backward-compatibility + public java.util.Optional visit(LogicalTypeAnnotation.MapKeyValueTypeAnnotation mapKeyValueLogicalType) { + return visitMapOrMapKeyValue(); + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + return visitMapOrMapKeyValue(); + } + + private java.util.Optional visitMapOrMapKeyValue() { + if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0).isPrimitive()) { + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); + } + GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType(); + if (!mapKeyValType.isRepetition(REPEATED) || + mapKeyValType.getFieldCount()!=2) { + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); + } + Type keyType = mapKeyValType.getType(0); + if (!keyType.isPrimitive() || + !keyType.asPrimitiveType().getPrimitiveTypeName().equals(BINARY) || + !keyType.getLogicalTypeAnnotation().equals(stringType())) { + throw new IllegalArgumentException("Map key type must be binary (UTF8): " + + keyType); + } + Type valueType = mapKeyValType.getType(1); + if (valueType.isRepetition(Type.Repetition.OPTIONAL)) { + return java.util.Optional.of(Schema.createMap(optional(convertField(valueType, names)))); + } else { + return java.util.Optional.of(Schema.createMap(convertField(valueType, names))); + } + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return java.util.Optional.of(Schema.create(Schema.Type.STRING)); + } + }).orElseThrow(() -> new UnsupportedOperationException("Cannot convert Parquet type " + parquetType)); + } else { + // if no original type then it's a record + return convertFields(parquetGroupType.getName(), parquetGroupType.getFields(), names); + } + } + } + + private LogicalTypeAnnotation convertLogicalType(LogicalType logicalType) { + if (logicalType == null) { + return null; + } else if (logicalType instanceof LogicalTypes.Decimal) { + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return decimalType(decimal.getScale(), decimal.getPrecision()); + } else if (logicalType instanceof LogicalTypes.Date) { + return dateType(); + } else if (logicalType instanceof LogicalTypes.TimeMillis) { + return timeType(true, MILLIS); + } else if (logicalType instanceof LogicalTypes.TimeMicros) { + return timeType(true, MICROS); + } else if (logicalType instanceof LogicalTypes.TimestampMillis) { + return timestampType(true, MILLIS); + } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + return timestampType(true, MICROS); + } else if (logicalType.getName().equals(LogicalTypes.uuid().getName()) && writeParquetUUID) { + return uuidType(); + } + + if (avroVersionSupportsLocalTimestampTypes()) { + if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { + return timestampType(false, MILLIS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { + return timestampType(false, MICROS); + } + } + + return null; + } + + private LogicalType convertLogicalType(LogicalTypeAnnotation annotation) { + if (annotation == null) { + return null; + } + return annotation + .accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { + return java.util.Optional.of( + LogicalTypes.decimal(decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } + + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return java.util.Optional.of(LogicalTypes.date()); + } + + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + LogicalTypeAnnotation.TimeUnit unit = timeLogicalType.getUnit(); + switch (unit) { + case MILLIS: + return java.util.Optional.of(LogicalTypes.timeMillis()); + case MICROS: + return java.util.Optional.of(LogicalTypes.timeMicros()); + } + return java.util.Optional.empty(); + } + + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + LogicalTypeAnnotation.TimeUnit unit = timestampLogicalType.getUnit(); + boolean isAdjustedToUTC = timestampLogicalType.isAdjustedToUTC(); + + if (isAdjustedToUTC || !avroVersionSupportsLocalTimestampTypes()) { + switch (unit) { + case MILLIS: + return java.util.Optional.of(LogicalTypes.timestampMillis()); + case MICROS: + return java.util.Optional.of(LogicalTypes.timestampMicros()); + } + return java.util.Optional.empty(); + } else { + switch (unit) { + case MILLIS: + return java.util.Optional.of(LogicalTypes.localTimestampMillis()); + case MICROS: + return java.util.Optional.of(LogicalTypes.localTimestampMicros()); + } + return java.util.Optional.empty(); + } + } + + @Override + public java.util.Optional visit(UUIDLogicalTypeAnnotation uuidLogicalType) { + return java.util.Optional.of(LogicalTypes.uuid()); + } + }) + .orElse(null); + } + + /** + * Implements the rules for interpreting existing data from the logical type + * spec for the LIST annotation. This is used to produce the expected schema. + *

+ * The AvroArrayConverter will decide whether the repeated type is the array + * element type by testing whether the element schema and repeated type are + * the same. This ensures that the LIST rules are followed when there is no + * schema and that a schema can be provided to override the default behavior. + */ + private boolean isElementType(Type repeatedType, String parentName) { + return ( + // can't be a synthetic layer because it would be invalid + repeatedType.isPrimitive() || + repeatedType.asGroupType().getFieldCount() > 1 || + repeatedType.asGroupType().getType(0).isRepetition(REPEATED) || + // known patterns without the synthetic layer + repeatedType.getName().equals("array") || + repeatedType.getName().equals(parentName + "_tuple") || + // default assumption + assumeRepeatedIsListElement + ); + } + + private static Schema optional(Schema original) { + // null is first in the union because Parquet's default is always null + return Schema.createUnion(Arrays.asList( + Schema.create(Schema.Type.NULL), + original)); + } + + private static String appendPath(String path, String fieldName) { + if (path == null || path.isEmpty()) { + return fieldName; + } + return path + '.' + fieldName; + } + + /* Avro <= 1.9 does not support conversions to LocalTimestamp{Micros, Millis} classes */ + private static boolean avroVersionSupportsLocalTimestampTypes() { + final String avroVersion = getRuntimeAvroVersion(); + + return avroVersion == null + || !(avroVersion.startsWith("1.7.") + || avroVersion.startsWith("1.8.") + || avroVersion.startsWith("1.9.")); + } + + private static String getRuntimeAvroVersion() { + return Schema.Parser.class.getPackage().getImplementationVersion(); + } +} diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java new file mode 100644 index 0000000000000..066d381f23f6e --- /dev/null +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java @@ -0,0 +1,64 @@ +/* + * 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.common.table; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.MessageType; + +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; + +public class ParquetTableSchemaResolver extends TableSchemaResolver { + + public ParquetTableSchemaResolver(HoodieTableMetaClient metaClient) { + super(metaClient); + } + + public static MessageType convertAvroSchemaToParquet(Schema schema, Configuration hadoopConf) { + return getAvroSchemaConverter(hadoopConf).convert(schema); + } + + private Schema convertParquetSchemaToAvro(MessageType parquetSchema) { + return getAvroSchemaConverter(metaClient.getStorageConf().unwrapAs(Configuration.class)).convert(parquetSchema); + } + + private MessageType convertAvroSchemaToParquet(Schema schema) { + return getAvroSchemaConverter(metaClient.getStorageConf().unwrapAs(Configuration.class)).convert(schema); + } + + /** + * Gets full schema (user + metadata) for a hoodie table in Parquet format. + * + * @return Parquet schema for the table + */ + public MessageType getTableParquetSchema() throws Exception { + return convertAvroSchemaToParquet(getTableAvroSchema(true)); + } + + /** + * Gets users data schema for a hoodie table in Parquet format. + * + * @return Parquet schema for the table + */ + public MessageType getTableParquetSchema(boolean includeMetadataField) throws Exception { + return convertAvroSchemaToParquet(getTableAvroSchema(includeMetadataField)); + } + +} diff --git a/hudi-hadoop-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java b/hudi-hadoop-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java new file mode 100644 index 0000000000000..b85b3b98fac38 --- /dev/null +++ b/hudi-hadoop-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java @@ -0,0 +1,162 @@ +/* + * 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.parquet.schema; + +import org.apache.hudi.common.util.Option; + +import java.util.ArrayList; +import java.util.List; + +public class SchemaRepair { + + public static MessageType repairLogicalTypes(MessageType fileSchema, Option tableSchema) { + if (tableSchema.isEmpty()) { + return fileSchema; + } + return repairLogicalTypes(fileSchema, tableSchema.get()); + } + + static MessageType repairLogicalTypes(MessageType fileSchema, MessageType tableSchema) { + List repairedFields = repairFields(fileSchema.getFields(), tableSchema); + + // If nothing changed, return the original schema + if (repairedFields == null) { + return fileSchema; + } + + return new MessageType(fileSchema.getName(), repairedFields); + } + + /** + * Repairs a list of fields against a table schema (MessageType or GroupType). + * Returns null if no changes were made, otherwise returns the repaired field list. + */ + private static List repairFields(List fileSchemaFields, GroupType tableSchema) { + // First pass: find the first field that changes + int firstChangedIndex = -1; + Type firstRepairedField = null; + + for (int i = 0; i < fileSchemaFields.size(); i++) { + Type requestedField = fileSchemaFields.get(i); + if (tableSchema.containsField(requestedField.getName())) { + Type tableField = tableSchema.getType(requestedField.getName()); + Type repaired = repairField(requestedField, tableField); + if (repaired != requestedField) { + firstChangedIndex = i; + firstRepairedField = repaired; + break; + } + } + } + + // If nothing changed, return null + if (firstChangedIndex == -1) { + return null; + } + + // Second pass: build the new field list with repaired fields + List repairedFields = new ArrayList<>(fileSchemaFields.size()); + + // Copy all fields before the first changed field + for (int i = 0; i < firstChangedIndex; i++) { + repairedFields.add(fileSchemaFields.get(i)); + } + + // Add the first changed field (using cached repaired field) + repairedFields.add(firstRepairedField); + + // Process remaining fields + for (int i = firstChangedIndex + 1; i < fileSchemaFields.size(); i++) { + Type fileSchemaField = fileSchemaFields.get(i); + Type repaired = fileSchemaField; + if (tableSchema.containsField(fileSchemaField.getName())) { + Type tableSchemaField = tableSchema.getType(fileSchemaField.getName()); + repaired = repairField(fileSchemaField, tableSchemaField); + } + repairedFields.add(repaired); + } + + return repairedFields; + } + + private static Type repairField(Type fileSchemaFieldType, Type tableSchemaFieldType) { + if (fileSchemaFieldType.isPrimitive() && tableSchemaFieldType.isPrimitive()) { + return repairPrimitiveType(fileSchemaFieldType.asPrimitiveType(), tableSchemaFieldType.asPrimitiveType()); + } else if (!fileSchemaFieldType.isPrimitive() && !tableSchemaFieldType.isPrimitive()) { + // recurse into nested structs + GroupType reqGroup = fileSchemaFieldType.asGroupType(); + GroupType tblGroup = tableSchemaFieldType.asGroupType(); + + // Repair fields directly without creating MessageType intermediaries + List repairedFields = repairFields(reqGroup.getFields(), tblGroup); + + // If nothing changed, return the original field + if (repairedFields == null) { + return fileSchemaFieldType; + } + + return new GroupType( + reqGroup.getRepetition(), + reqGroup.getName(), + reqGroup.getLogicalTypeAnnotation(), + repairedFields + ); + } else { + // fallback: keep requested + return fileSchemaFieldType; + } + } + + private static PrimitiveType repairPrimitiveType(PrimitiveType fileSchemaPrimitiveType, PrimitiveType tableSchemaPrimitiveType) { + // Quick check if repair is needed (no allocations) + if (needsLogicalTypeRepair(fileSchemaPrimitiveType, tableSchemaPrimitiveType)) { + return Types.primitive(tableSchemaPrimitiveType.getPrimitiveTypeName(), fileSchemaPrimitiveType.getRepetition()) + .as(tableSchemaPrimitiveType.getLogicalTypeAnnotation()) + .named(fileSchemaPrimitiveType.getName()); + } + return fileSchemaPrimitiveType; + } + + /** + * Quick check if a logical type repair is needed (no allocations). + */ + private static boolean needsLogicalTypeRepair(PrimitiveType fileSchemaPrimitiveType, PrimitiveType tableSchemaPrimitiveType) { + if (fileSchemaPrimitiveType.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64 + || tableSchemaPrimitiveType.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64) { + return false; + } + LogicalTypeAnnotation fileLogicalTypeAnnotation = fileSchemaPrimitiveType.getLogicalTypeAnnotation(); + LogicalTypeAnnotation tableLogicalTypeAnnotation = tableSchemaPrimitiveType.getLogicalTypeAnnotation(); + + // if requested has no logical type, and the table has a local timestamp, then we need to repair + if (fileLogicalTypeAnnotation == null) { + return tableLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && !((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).isAdjustedToUTC(); + } + + // if requested is timestamp-micros and table is timestamp-millis then we need to repair + return fileLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && tableLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) fileLogicalTypeAnnotation).getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) fileLogicalTypeAnnotation).isAdjustedToUTC() + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).isAdjustedToUTC(); + } +} diff --git a/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java new file mode 100644 index 0000000000000..1787a0b83628c --- /dev/null +++ b/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -0,0 +1,954 @@ +/* + * 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.parquet.avro; + +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.avro.Schema.Type.INT; +import static org.apache.avro.Schema.Type.LONG; +import static org.apache.avro.Schema.Type.STRING; +import static org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; +import static org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility; +import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; +import static org.apache.parquet.schema.OriginalType.DATE; +import static org.apache.parquet.schema.OriginalType.TIMESTAMP_MICROS; +import static org.apache.parquet.schema.OriginalType.TIMESTAMP_MILLIS; +import static org.apache.parquet.schema.OriginalType.TIME_MICROS; +import static org.apache.parquet.schema.OriginalType.TIME_MILLIS; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT96; +import static org.apache.parquet.schema.Type.Repetition.REQUIRED; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestAvroSchemaConverter { + + private static final Configuration NEW_BEHAVIOR = new Configuration(false); + + @BeforeAll + public static void setupConf() { + NEW_BEHAVIOR.setBoolean("parquet.avro.add-list-element-records", false); + NEW_BEHAVIOR.setBoolean("parquet.avro.write-old-list-structure", false); + } + + public static final String ALL_PARQUET_SCHEMA = "message org.apache.parquet.avro.myrecord {\n" + + " required boolean myboolean;\n" + + " required int32 myint;\n" + + " required int64 mylong;\n" + + " required float myfloat;\n" + + " required double mydouble;\n" + + " required binary mybytes;\n" + + " required binary mystring (UTF8);\n" + + " required group mynestedrecord {\n" + + " required int32 mynestedint;\n" + + " }\n" + + " required binary myenum (ENUM);\n" + + " required group myarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " optional group myoptionalarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated group list {\n" + + " optional int32 element;\n" + + " }\n" + + " }\n" + + " required group myrecordarray (LIST) {\n" + + " repeated group array {\n" + + " required int32 a;\n" + + " required int32 b;\n" + + " }\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group map (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) myfixed;\n" + + "}\n"; + + private void testAvroToParquetConversion(Schema avroSchema, String schemaString) throws Exception { + testAvroToParquetConversion(new Configuration(false), avroSchema, schemaString); + } + + private void testAvroToParquetConversion(Configuration conf, Schema avroSchema, String schemaString) + throws Exception { + HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); + MessageType schema = avroSchemaConverter.convert(avroSchema); + MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString); + assertEquals(expectedMT.toString(), schema.toString()); + } + + private void testParquetToAvroConversion(Schema avroSchema, String schemaString) throws Exception { + testParquetToAvroConversion(new Configuration(false), avroSchema, schemaString); + } + + private void testParquetToAvroConversion(Configuration conf, Schema avroSchema, String schemaString) + throws Exception { + HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); + Schema schema = avroSchemaConverter.convert(MessageTypeParser.parseMessageType(schemaString)); + assertEquals(avroSchema.toString(), schema.toString()); + } + + private void testRoundTripConversion(Schema avroSchema, String schemaString) throws Exception { + testRoundTripConversion(new Configuration(), avroSchema, schemaString); + } + + private void testRoundTripConversion(Configuration conf, Schema avroSchema, String schemaString) throws Exception { + HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); + MessageType schema = avroSchemaConverter.convert(avroSchema); + MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString); + assertEquals(expectedMT.toString(), schema.toString()); + Schema convertedAvroSchema = avroSchemaConverter.convert(expectedMT); + assertEquals(avroSchema.toString(), convertedAvroSchema.toString()); + } + + @Test() + public void testTopLevelMustBeARecord() { + assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(Schema.create(INT))); + } + + @Test + public void testAllTypes() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc"); + testAvroToParquetConversion( + NEW_BEHAVIOR, + schema, + "message org.apache.parquet.avro.myrecord {\n" + // Avro nulls are not encoded, unless they are null unions + + " required boolean myboolean;\n" + + " required int32 myint;\n" + + " required int64 mylong;\n" + + " required float myfloat;\n" + + " required double mydouble;\n" + + " required binary mybytes;\n" + + " required binary mystring (UTF8);\n" + + " required group mynestedrecord {\n" + + " required int32 mynestedint;\n" + + " }\n" + + " required binary myenum (ENUM);\n" + + " required group myarray (LIST) {\n" + + " repeated group list {\n" + + " required int32 element;\n" + + " }\n" + + " }\n" + + " required group myemptyarray (LIST) {\n" + + " repeated group list {\n" + + " required int32 element;\n" + + " }\n" + + " }\n" + + " optional group myoptionalarray (LIST) {\n" + + " repeated group list {\n" + + " required int32 element;\n" + + " }\n" + + " }\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated group list {\n" + + " optional int32 element;\n" + + " }\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required group myemptymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) myfixed;\n" + + "}\n"); + } + + @Test + public void testAllTypesOldListBehavior() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc"); + testAvroToParquetConversion( + schema, + "message org.apache.parquet.avro.myrecord {\n" + // Avro nulls are not encoded, unless they are null unions + + " required boolean myboolean;\n" + + " required int32 myint;\n" + + " required int64 mylong;\n" + + " required float myfloat;\n" + + " required double mydouble;\n" + + " required binary mybytes;\n" + + " required binary mystring (UTF8);\n" + + " required group mynestedrecord {\n" + + " required int32 mynestedint;\n" + + " }\n" + + " required binary myenum (ENUM);\n" + + " required group myarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group myemptyarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " optional group myoptionalarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required group myemptymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) myfixed;\n" + + "}\n"); + } + + @Test + public void testAllTypesParquetToAvro() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetNewBehavior.avsc"); + // Cannot use round-trip assertion because enum is lost + testParquetToAvroConversion(NEW_BEHAVIOR, schema, ALL_PARQUET_SCHEMA); + } + + @Test + public void testAllTypesParquetToAvroOldBehavior() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetOldBehavior.avsc"); + // Cannot use round-trip assertion because enum is lost + testParquetToAvroConversion(schema, ALL_PARQUET_SCHEMA); + } + + @Test + public void testParquetMapWithNonStringKeyFails() throws Exception { + MessageType parquetSchema = + MessageTypeParser.parseMessageType("message myrecord {\n" + " required group mymap (MAP) {\n" + + " repeated group map (MAP_KEY_VALUE) {\n" + + " required int32 key;\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + "}\n"); + assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(parquetSchema)); + } + + @Test + public void testOptionalFields() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalInt = optional(Schema.create(INT)); + schema.setFields( + Collections.singletonList(new Schema.Field("myint", optionalInt, null, JsonProperties.NULL_VALUE))); + testRoundTripConversion(schema, "message record1 {\n" + " optional int32 myint;\n" + "}\n"); + } + + @Test + public void testOptionalMapValue() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalIntMap = Schema.createMap(optional(Schema.create(INT))); + schema.setFields(Arrays.asList(new Schema.Field("myintmap", optionalIntMap, null, null))); + testRoundTripConversion( + schema, + "message record1 {\n" + " required group myintmap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " optional int32 value;\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testOptionalArrayElement() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalIntArray = Schema.createArray(optional(Schema.create(INT))); + schema.setFields(Arrays.asList(new Schema.Field("myintarray", optionalIntArray, null, null))); + testRoundTripConversion( + NEW_BEHAVIOR, + schema, + "message record1 {\n" + " required group myintarray (LIST) {\n" + + " repeated group list {\n" + + " optional int32 element;\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testUnionOfTwoTypes() throws Exception { + Schema schema = Schema.createRecord("record2", null, null, false); + Schema multipleTypes = Schema.createUnion( + Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(INT), Schema.create(Schema.Type.FLOAT))); + schema.setFields(Arrays.asList(new Schema.Field("myunion", multipleTypes, null, JsonProperties.NULL_VALUE))); + + // Avro union is modelled using optional data members of the different + // types. This does not translate back into an Avro union + testAvroToParquetConversion( + schema, + "message record2 {\n" + " optional group myunion {\n" + + " optional int32 member0;\n" + + " optional float member1;\n" + + " }\n" + + "}\n"); + } + + @Test + public void testArrayOfOptionalRecords() throws Exception { + Schema innerRecord = Schema.createRecord("element", null, null, false); + Schema optionalString = optional(Schema.create(Schema.Type.STRING)); + innerRecord.setFields(Arrays.asList( + new Schema.Field("s1", optionalString, null, JsonProperties.NULL_VALUE), + new Schema.Field("s2", optionalString, null, JsonProperties.NULL_VALUE))); + Schema schema = Schema.createRecord("HasArray", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("myarray", Schema.createArray(optional(innerRecord)), null, null))); + System.err.println("Avro schema: " + schema.toString(true)); + + testRoundTripConversion( + NEW_BEHAVIOR, + schema, + "message HasArray {\n" + " required group myarray (LIST) {\n" + + " repeated group list {\n" + + " optional group element {\n" + + " optional binary s1 (UTF8);\n" + + " optional binary s2 (UTF8);\n" + + " }\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testArrayOfOptionalRecordsOldBehavior() throws Exception { + Schema innerRecord = Schema.createRecord("InnerRecord", null, null, false); + Schema optionalString = optional(Schema.create(Schema.Type.STRING)); + innerRecord.setFields(Arrays.asList( + new Schema.Field("s1", optionalString, null, JsonProperties.NULL_VALUE), + new Schema.Field("s2", optionalString, null, JsonProperties.NULL_VALUE))); + Schema schema = Schema.createRecord("HasArray", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("myarray", Schema.createArray(optional(innerRecord)), null, null))); + System.err.println("Avro schema: " + schema.toString(true)); + + // Cannot use round-trip assertion because InnerRecord optional is removed + testAvroToParquetConversion( + schema, + "message HasArray {\n" + " required group myarray (LIST) {\n" + + " repeated group array {\n" + + " optional binary s1 (UTF8);\n" + + " optional binary s2 (UTF8);\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testOldAvroListOfLists() throws Exception { + Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT)))); + Schema schema = Schema.createRecord("AvroCompatListInList", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE))); + System.err.println("Avro schema: " + schema.toString(true)); + + testRoundTripConversion( + schema, + "message AvroCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group array (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " }\n" + + "}"); + // Cannot use round-trip assertion because 3-level representation is used + testParquetToAvroConversion( + NEW_BEHAVIOR, + schema, + "message AvroCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group array (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " }\n" + + "}"); + } + + @Test + public void testOldThriftListOfLists() throws Exception { + Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT)))); + Schema schema = Schema.createRecord("ThriftCompatListInList", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE))); + System.err.println("Avro schema: " + schema.toString(true)); + + // Cannot use round-trip assertion because repeated group names differ + testParquetToAvroConversion( + schema, + "message ThriftCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group listOfLists_tuple (LIST) {\n" + + " repeated int32 listOfLists_tuple_tuple;\n" + + " }\n" + + " }\n" + + "}"); + // Cannot use round-trip assertion because 3-level representation is used + testParquetToAvroConversion( + NEW_BEHAVIOR, + schema, + "message ThriftCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group listOfLists_tuple (LIST) {\n" + + " repeated int32 listOfLists_tuple_tuple;\n" + + " }\n" + + " }\n" + + "}"); + } + + @Test + public void testUnknownTwoLevelListOfLists() throws Exception { + // This tests the case where we don't detect a 2-level list by the repeated + // group's name, but it must be 2-level because the repeated group doesn't + // contain an optional or repeated element as required for 3-level lists + Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT)))); + Schema schema = Schema.createRecord("UnknownTwoLevelListInList", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE))); + System.err.println("Avro schema: " + schema.toString(true)); + + // Cannot use round-trip assertion because repeated group names differ + testParquetToAvroConversion( + schema, + "message UnknownTwoLevelListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group mylist (LIST) {\n" + + " repeated int32 innerlist;\n" + + " }\n" + + " }\n" + + "}"); + // Cannot use round-trip assertion because 3-level representation is used + testParquetToAvroConversion( + NEW_BEHAVIOR, + schema, + "message UnknownTwoLevelListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group mylist (LIST) {\n" + + " repeated int32 innerlist;\n" + + " }\n" + + " }\n" + + "}"); + } + + @Test + public void testParquetMapWithoutMapKeyValueAnnotation() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema map = Schema.createMap(Schema.create(INT)); + schema.setFields(Collections.singletonList(new Schema.Field("mymap", map, null, null))); + String parquetSchema = "message myrecord {\n" + " required group mymap (MAP) {\n" + + " repeated group map {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + "}\n"; + + testParquetToAvroConversion(schema, parquetSchema); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema); + } + + @Test + public void testDecimalBytesType() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema decimal = LogicalTypes.decimal(9, 2).addToSchema(Schema.create(Schema.Type.BYTES)); + schema.setFields(Collections.singletonList(new Schema.Field("dec", decimal, null, null))); + + testRoundTripConversion(schema, "message myrecord {\n" + " required binary dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testDecimalFixedType() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema decimal = LogicalTypes.decimal(9, 2).addToSchema(Schema.createFixed("dec", null, null, 8)); + schema.setFields(Collections.singletonList(new Schema.Field("dec", decimal, null, null))); + + testRoundTripConversion( + schema, "message myrecord {\n" + " required fixed_len_byte_array(8) dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testDecimalIntegerType() throws Exception { + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("dec", Schema.create(INT), null, null))); + + // the decimal portion is lost because it isn't valid in Avro + testParquetToAvroConversion( + expected, "message myrecord {\n" + " required int32 dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testDecimalLongType() throws Exception { + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("dec", Schema.create(LONG), null, null))); + + // the decimal portion is lost because it isn't valid in Avro + testParquetToAvroConversion( + expected, "message myrecord {\n" + " required int64 dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testParquetInt96AsFixed12AvroType() throws Exception { + Configuration enableInt96ReadingConfig = new Configuration(); + enableInt96ReadingConfig.setBoolean(AvroReadSupport.READ_INT96_AS_FIXED, true); + + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema int96schema = Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + schema.setFields(Collections.singletonList( + new Schema.Field("int96_field", int96schema, null, null))); + + testParquetToAvroConversion(enableInt96ReadingConfig, schema, "message myrecord {\n" + + " required int96 int96_field;\n" + + "}\n"); + } + + @Test + public void testParquetInt96DefaultFail() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + + MessageType parquetSchemaWithInt96 = + MessageTypeParser.parseMessageType("message myrecord {\n required int96 int96_field;\n}\n"); + + assertThrows( + "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array.", + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(parquetSchemaWithInt96)); + } + + @Test + public void testDateType() throws Exception { + Schema date = LogicalTypes.date().addToSchema(Schema.create(INT)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("date", date, null, null))); + + testRoundTripConversion(expected, "message myrecord {\n" + " required int32 date (DATE);\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT64, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", DATE); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", DATE); + } + + assertThrows( + "Should not allow TIME_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimeMillisType() throws Exception { + Schema date = LogicalTypes.timeMillis().addToSchema(Schema.create(INT)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("time", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int32 time (TIME(MILLIS,true));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT64, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIME_MILLIS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIME_MILLIS); + } + + assertThrows( + "Should not allow TIME_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimeMicrosType() throws Exception { + Schema date = LogicalTypes.timeMicros().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("time", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 time (TIME(MICROS,true));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIME_MICROS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIME_MICROS); + } + + assertThrows( + "Should not allow TIME_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimestampMillisType() throws Exception { + Schema date = LogicalTypes.timestampMillis().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,true));\n" + "}\n"); + + final Schema converted = getAvroSchemaConverter(new Configuration()) + .convert(Types.buildMessage() + .addField(Types.primitive(INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .length(1) + .named("timestamp_type")) + .named("TestAvro")); + assertEquals( + "local-timestamp-millis", + converted + .getField("timestamp_type") + .schema() + .getLogicalType() + .getName()); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MILLIS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MILLIS); + } + + assertThrows( + "Should not allow TIMESTAMP_MILLIS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testLocalTimestampMillisType() throws Exception { + Schema date = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,false));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MILLIS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MILLIS); + } + + assertThrows( + "Should not allow TIMESTAMP_MILLIS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimestampMicrosType() throws Exception { + Schema date = LogicalTypes.timestampMicros().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,true));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MICROS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MICROS); + } + + assertThrows( + "Should not allow TIMESTAMP_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + + final Schema converted = getAvroSchemaConverter(new Configuration()) + .convert(Types.buildMessage() + .addField(Types.primitive(INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .length(1) + .named("timestamp_type")) + .named("TestAvro")); + + assertEquals( + "local-timestamp-micros", + converted + .getField("timestamp_type") + .schema() + .getLogicalType() + .getName()); + } + + @Test + public void testLocalTimestampMicrosType() throws Exception { + Schema date = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,false));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MICROS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MICROS); + } + + assertThrows( + "Should not allow TIMESTAMP_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testReuseNameInNestedStructure() throws Exception { + Schema innerA1 = record("a1", "a12", field("a4", primitive(Schema.Type.FLOAT))); + + Schema outerA1 = record("a1", field("a2", primitive(Schema.Type.FLOAT)), optionalField("a1", innerA1)); + Schema schema = record("Message", optionalField("a1", outerA1)); + + String parquetSchema = "message Message {\n" + + " optional group a1 {\n" + + " required float a2;\n" + + " optional group a1 {\n" + + " required float a4;\n" + + " }\n" + + " }\n" + + "}\n"; + + testParquetToAvroConversion(schema, parquetSchema); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema); + } + + @Test + public void testReuseNameInNestedStructureAtSameLevel() throws Exception { + Schema a2 = record("a2", field("a4", primitive(Schema.Type.FLOAT))); + Schema a22 = record( + "a2", "a22", field("a4", primitive(Schema.Type.FLOAT)), field("a5", primitive(Schema.Type.FLOAT))); + + Schema a1 = record("a1", optionalField("a2", a2)); + Schema a3 = record("a3", optionalField("a2", a22)); + + Schema schema = record("Message", optionalField("a1", a1), optionalField("a3", a3)); + + String parquetSchema = "message Message {\n" + + " optional group a1 {\n" + + " optional group a2 {\n" + + " required float a4;\n" + + " }\n" + + " }\n" + + " optional group a3 {\n" + + " optional group a2 {\n" + + " required float a4;\n" + + " required float a5;\n" + + " }\n" + + " }\n" + + "}\n"; + + testParquetToAvroConversion(schema, parquetSchema); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema); + } + + @Test + public void testUUIDType() throws Exception { + Schema fromAvro = Schema.createRecord( + "myrecord", + null, + null, + false, + Arrays.asList( + new Schema.Field("uuid", LogicalTypes.uuid().addToSchema(Schema.create(STRING)), null, null))); + String parquet = "message myrecord {\n" + " required binary uuid (STRING);\n" + "}\n"; + Schema toAvro = Schema.createRecord( + "myrecord", + null, + null, + false, + Arrays.asList(new Schema.Field("uuid", Schema.create(STRING), null, null))); + + testAvroToParquetConversion(fromAvro, parquet); + testParquetToAvroConversion(toAvro, parquet); + + assertEquals( + COMPATIBLE, checkReaderWriterCompatibility(fromAvro, toAvro).getType()); + } + + @Test + public void testUUIDTypeWithParquetUUID() throws Exception { + Schema uuid = LogicalTypes.uuid().addToSchema(Schema.create(STRING)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("uuid", uuid, null, null))); + + testRoundTripConversion( + conf(AvroWriteSupport.WRITE_PARQUET_UUID, true), + expected, + "message myrecord {\n" + " required fixed_len_byte_array(16) uuid (UUID);\n" + "}\n"); + } + + @Test + public void testAvroFixed12AsParquetInt96Type() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/fixedToInt96.avsc"); + + Configuration conf = new Configuration(); + conf.setStrings( + "parquet.avro.writeFixedAsInt96", + "int96", + "mynestedrecord.int96inrecord", + "mynestedrecord.myarrayofoptional", + "mynestedrecord.mymap"); + testAvroToParquetConversion( + conf, + schema, + "message org.apache.parquet.avro.fixedToInt96 {\n" + + " required int96 int96;\n" + + " required fixed_len_byte_array(12) notanint96;\n" + + " required group mynestedrecord {\n" + + " required int96 int96inrecord;\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated int96 array;\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (STRING);\n" + + " required int96 value;\n" + + " }\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) onebytefixed;\n" + + "}"); + + conf.setStrings("parquet.avro.writeFixedAsInt96", "onebytefixed"); + assertThrows( + "Exception should be thrown for fixed types to be converted to INT96 where the size is not 12 bytes", + IllegalArgumentException.class, + () -> getAvroSchemaConverter(conf).convert(schema)); + } + + public static Schema optional(Schema original) { + return Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), original)); + } + + public static MessageType message(PrimitiveType primitive) { + return Types.buildMessage().addField(primitive).named("myrecord"); + } + + /** + * A convenience method to avoid a large number of @Test(expected=...) tests + * + * @param message A String message to describe this assertion + * @param expected An Exception class that the Runnable should throw + * @param runnable A Runnable that is expected to throw the exception + */ + public static void assertThrows(String message, Class expected, Runnable runnable) { + try { + runnable.run(); + fail("No exception was thrown (" + message + "), expected: " + expected.getName()); + } catch (Exception actual) { + try { + assertEquals(expected, actual.getClass(), message); + } catch (AssertionError e) { + e.addSuppressed(actual); + throw e; + } + } + } + + public static Schema record(String name, String namespace, Schema.Field... fields) { + Schema record = Schema.createRecord(name, null, namespace, false); + record.setFields(Arrays.asList(fields)); + return record; + } + + public static Schema record(String name, Schema.Field... fields) { + return record(name, null, fields); + } + + public static Schema.Field field(String name, Schema schema) { + return new Schema.Field(name, schema, null, null); + } + + public static Schema.Field optionalField(String name, Schema schema) { + return new Schema.Field(name, optional(schema), null, JsonProperties.NULL_VALUE); + } + + public static Schema array(Schema element) { + return Schema.createArray(element); + } + + public static Schema primitive(Schema.Type type) { + return Schema.create(type); + } + + public static Configuration conf(String name, boolean value) { + Configuration conf = new Configuration(false); + conf.setBoolean(name, value); + return conf; + } + +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepair.java b/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepair.java new file mode 100644 index 0000000000000..b31d37c835dbd --- /dev/null +++ b/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepair.java @@ -0,0 +1,600 @@ +/* + * 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.parquet.schema; + +import org.apache.hudi.common.util.Option; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; + +/** + * Tests {@link SchemaRepair}. + */ +public class TestSchemaRepair { + + @Test + public void testNoRepairNeededIdenticalSchemas() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When schemas are identical, should return same instance"); + } + + @Test + public void testNoRepairNeededDifferentPrimitiveTypes() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When field names differ, should return original schema"); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMillis() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName()); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMicros() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName()); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairTimestampMicrosToTimestampMillis() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with timestamp-millis"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName()); + assertEquals(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testNoRepairNeededTimestampMillisToTimestampMicros() { + // This direction should NOT trigger repair + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair timestamp-millis to timestamp-micros"); + } + + @Test + public void testNoRepairNeededNonLongTypes() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.dateType()) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair non-LONG types"); + } + + @Test + public void testRepairRecordSingleField() { + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(1, result.getFields().size()); + + PrimitiveType field = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + field.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairRecordMultipleFieldsOnlyOneNeedsRepair() { + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(3, result.getFields().size()); + + // Verify id field unchanged - should be same type instance + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify timestamp field repaired + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS), + timestampField.getLogicalTypeAnnotation()); + + // Verify name field unchanged - should be same type instance + assertSame(requestedSchema.getType("name"), result.getType("name")); + } + + @Test + public void testRepairRecordNestedRecord() { + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedRequestedSchema + ); + + MessageType tableSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedTableSchema + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema for nested record"); + + // Verify id field unchanged - should be same type instance + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify nested record was repaired + GroupType nestedResult = result.getType("nested").asGroupType(); + PrimitiveType nestedTimestamp = nestedResult.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + nestedTimestamp.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairRecordMissingFieldInTableSchema() { + // Requested schema has a field not present in table schema + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newField doesn't exist in table schema + assertSame(requestedSchema, result, "Should return original when field missing in table schema"); + } + + @Test + public void testRepairRecordMultipleFieldsMissingInTableSchema() { + // Requested schema has multiple fields not present in table schema + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField2") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since new fields don't exist in table schema + assertSame(requestedSchema, result, "Should return original when multiple fields missing in table schema"); + } + + @Test + public void testRepairRecordMixedMissingAndRepairableFields() { + // Requested schema has some fields missing in table, some needing repair, some unchanged + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should create new schema with timestamp repaired, but newField preserved from requested + assertNotSame(requestedSchema, result, "Should create new schema"); + assertEquals(4, result.getFields().size()); + + // Verify id field unchanged + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify timestamp field repaired + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + + // Verify newField preserved from requested schema (not in table) + assertSame(requestedSchema.getType("newField"), result.getType("newField")); + + // Verify name field unchanged + assertSame(requestedSchema.getType("name"), result.getType("name")); + } + + @Test + public void testRepairNestedRecordFieldMissingInTableSchema() { + // Requested nested record has a field not present in table's nested record + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("extraField") + ); + + GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedRequestedSchema + ); + + MessageType tableSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedTableSchema + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema"); + + // Verify id field unchanged + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify nested record was repaired but still has extraField + GroupType nestedResult = result.getType("nested").asGroupType(); + assertEquals(2, nestedResult.getFieldCount()); + + // Timestamp should be repaired + PrimitiveType timestampField = nestedResult.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + + // extraField should be preserved from requested schema + assertSame(nestedRequestedSchema.getType("extraField"), nestedResult.getType("extraField")); + } + + @Test + public void testRepairRecordWholeNestedRecordMissingInTableSchema() { + // Requested schema has a nested record field that doesn't exist in table schema + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "newNested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedRequestedSchema + ); + + MessageType tableSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newNested field doesn't exist in table + assertSame(requestedSchema, result, "Should return original when nested field missing in table schema"); + } + + @Test + public void testEdgeCaseEmptyRecord() { + MessageType requestedSchema = new MessageType("EmptyRecord"); + MessageType tableSchema = new MessageType("EmptyRecord"); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Empty records should return same instance"); + } + + @Test + public void testRepairRecordFirstFieldChanged() { + // Test the optimization path where the first field needs repair + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp2") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp2") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + PrimitiveType timestamp1 = result.getType("timestamp1").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestamp1.getLogicalTypeAnnotation()); + PrimitiveType timestamp2 = result.getType("timestamp2").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS), + timestamp2.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairRecordLastFieldChanged() { + // Test the optimization path where only the last field needs repair + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + // Verify id and name fields unchanged - should be same type instances + assertSame(requestedSchema.getType("id"), result.getType("id")); + assertSame(requestedSchema.getType("name"), result.getType("name")); + // Verify timestamp field repaired + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairLogicalTypesWithOptionEmpty() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, Option.empty()); + + assertSame(requestedSchema, result, "Should return original when Option is empty"); + } + + @Test + public void testRepairLogicalTypesWithOptionPresent() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, Option.of(tableSchema)); + + assertNotSame(requestedSchema, result, "Should repair when Option is present"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairOptionalFieldRepetition() { + // Test that repair preserves the requested field's repetition (OPTIONAL vs REQUIRED) + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(Type.Repetition.OPTIONAL, timestampField.getRepetition(), + "Should preserve requested field's repetition"); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairNestedGroupPreservesLogicalType() { + // Test that repair preserves the group's logical type annotation + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + LogicalTypeAnnotation.listType(), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + LogicalTypeAnnotation.listType(), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", nestedRequestedSchema); + MessageType tableSchema = new MessageType("OuterRecord", nestedTableSchema); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + GroupType nestedResult = result.getType("nested").asGroupType(); + assertEquals(LogicalTypeAnnotation.listType(), nestedResult.getLogicalTypeAnnotation(), + "Should preserve group's logical type annotation"); + PrimitiveType timestampField = nestedResult.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } +} diff --git a/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java b/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java new file mode 100644 index 0000000000000..75fe9ffde7d61 --- /dev/null +++ b/hudi-hadoop-common/src/parquet/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java @@ -0,0 +1,481 @@ +/* + * 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.parquet.schema; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Tests equivalence between {@link SchemaRepair} and {@link AvroSchemaRepair}. + * + * This test class verifies that both repair implementations produce logically + * equivalent results when converting between Avro and Parquet schemas. + */ +public class TestSchemaRepairEquivalence { + + private HoodieAvroParquetSchemaConverter converter; + + @BeforeEach + public void setUp() { + converter = HoodieAvroParquetSchemaConverter.getAvroSchemaConverter(new Configuration()); + } + + /** + * Helper method to verify that AvroSchemaRepair and SchemaRepair produce equivalent results. + */ + private void assertRepairEquivalence(Schema requestedAvro, Schema tableAvro) { + // Apply Avro repair + Schema repairedAvro = AvroSchemaRepair.repairLogicalTypes(requestedAvro, tableAvro); + + // Convert to Parquet schemas + MessageType requestedParquet = converter.convert(requestedAvro); + MessageType tableParquet = converter.convert(tableAvro); + + // Apply Parquet repair + MessageType repairedParquet = SchemaRepair.repairLogicalTypes(requestedParquet, tableParquet); + + // Convert repaired Parquet back to Avro + Schema repairedParquetAsAvro = converter.convert(repairedParquet); + + // Verify equivalence + assertEquals(repairedAvro, repairedParquetAsAvro, + "SchemaRepair and AvroSchemaRepair should produce equivalent results"); + } + + @Test + public void testEquivalenceNoRepairNeeded() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("value").type().longType().noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("value").type().longType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceLongToLocalTimestampMillis() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceLongToLocalTimestampMicros() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceTimestampMicrosToMillis() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceNoRepairTimestampMillisToMicros() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceSimpleRecord() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordMultipleFields() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceNestedRecord() { + Schema nestedRequestedSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedTableSchema).noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordWithExtraFieldInRequested() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("newfield").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordMixedFields() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("newfield").type().stringType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceNestedRecordWithExtraField() { + Schema nestedRequestedSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .name("extrafield").type().stringType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedTableSchema).noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordFirstFieldChanged() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp1").type().longType().noDefault() + .name("timestamp2").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp1") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("timestamp2") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordLastFieldChanged() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceComplexNestedStructure() { + Schema innerRecordRequested = SchemaBuilder.record("inner") + .fields() + .name("timestamp").type().longType().noDefault() + .name("value").type().intType().noDefault() + .endRecord(); + + Schema innerRecordTable = SchemaBuilder.record("inner") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("value").type().intType().noDefault() + .endRecord(); + + Schema middleRecordRequested = SchemaBuilder.record("middle") + .fields() + .name("inner").type(innerRecordRequested).noDefault() + .name("middletimestamp").type().longType().noDefault() + .endRecord(); + + Schema middleRecordTable = SchemaBuilder.record("middle") + .fields() + .name("inner").type(innerRecordTable).noDefault() + .name("middletimestamp") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("outer") + .fields() + .name("id").type().intType().noDefault() + .name("middle").type(middleRecordRequested).noDefault() + .name("outertimestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("outer") + .fields() + .name("id").type().intType().noDefault() + .name("middle").type(middleRecordTable).noDefault() + .name("outertimestamp") + .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceEmptyRecord() { + Schema requestedSchema = SchemaBuilder.record("emptyrecord").fields().endRecord(); + Schema tableSchema = SchemaBuilder.record("emptyrecord").fields().endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordNoFieldsMatch() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("field1").type().longType().noDefault() + .name("field2").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("field3").type().intType().noDefault() + .name("field4") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceMultipleTimestampRepairs() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("ts1").type().longType().noDefault() + .name("ts2").type().longType().noDefault() + .name("ts3").type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("ts4").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("ts1") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("ts2") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("ts3") + .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("ts4").type().longType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceDeepNesting() { + Schema level3Requested = SchemaBuilder.record("level3") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema level3Table = SchemaBuilder.record("level3") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema level2Requested = SchemaBuilder.record("level2") + .fields() + .name("level3").type(level3Requested).noDefault() + .endRecord(); + + Schema level2Table = SchemaBuilder.record("level2") + .fields() + .name("level3").type(level3Table).noDefault() + .endRecord(); + + Schema level1Requested = SchemaBuilder.record("level1") + .fields() + .name("level2").type(level2Requested).noDefault() + .endRecord(); + + Schema level1Table = SchemaBuilder.record("level1") + .fields() + .name("level2").type(level2Table).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("level0") + .fields() + .name("level1").type(level1Requested).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("level0") + .fields() + .name("level1").type(level1Table).noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } +} diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc new file mode 100644 index 0000000000000..116e98c519de4 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc @@ -0,0 +1,110 @@ +/* + * 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. + */ +{ + "name" : "myrecord", + "namespace": "org.apache.parquet.avro", + "type" : "record", + "fields" : [ { + "name" : "mynull", + "type" : "null" + }, { + "name" : "myboolean", + "type" : "boolean" + }, { + "name" : "myint", + "type" : "int" + }, { + "name" : "mylong", + "type" : "long" + }, { + "name" : "myfloat", + "type" : "float" + }, { + "name" : "mydouble", + "type" : "double" + }, { + "name" : "mybytes", + "type" : "bytes" + }, { + "name" : "mystring", + "type" : "string" + }, { + "name" : "mynestedrecord", + "type" : { + "type" : "record", + "name" : "ignored1", + "namespace" : "", + "fields" : [ { + "name" : "mynestedint", + "type" : "int" + } ] + } + }, { + "name" : "myenum", + "type" : { + "type" : "enum", + "name" : "ignored2", + "namespace" : "", + "symbols" : [ "a", "b" ] + } + }, { + "name" : "myarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myemptyarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myoptionalarray", + "type" : [ "null", { + "type" : "array", + "items" : "int" + }] + }, { + "name" : "myarrayofoptional", + "type" : { + "type" : "array", + "items" : [ "null", "int" ] + } + }, { + "name" : "mymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myemptymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myfixed", + "type" : { + "type" : "fixed", + "name" : "ignored3", + "namespace" : "", + "size" : 1 + } + } ] +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc new file mode 100644 index 0000000000000..606213cb16830 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc @@ -0,0 +1,108 @@ +/* + * 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. + */ +{ + "name" : "myrecord", + "namespace": "org.apache.parquet.avro", + "type" : "record", + "fields" : [ { + "name" : "myboolean", + "type" : "boolean" + }, { + "name" : "myint", + "type" : "int" + }, { + "name" : "mylong", + "type" : "long" + }, { + "name" : "myfloat", + "type" : "float" + }, { + "name" : "mydouble", + "type" : "double" + }, { + "name" : "mybytes", + "type" : "bytes" + }, { + "name" : "mystring", + "type" : "string" + }, { + "name" : "mynestedrecord", + "type" : { + "type" : "record", + "name" : "mynestedrecord", + "namespace" : "", + "fields" : [ { + "name" : "mynestedint", + "type" : "int" + } ] + } + }, { + "name" : "myenum", + "type" : "string" + }, { + "name" : "myarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myoptionalarray", + "type" : [ "null", { + "type" : "array", + "items" : "int" + }], + "default" : null + }, { + "name" : "myarrayofoptional", + "type" : { + "type" : "array", + "items" : ["null", "int"] + } + }, { + "name" : "myrecordarray", + "type" : { + "type" : "array", + "items" : { + "type" : "record", + "name" : "array", + "namespace" : "", + "fields" : [ { + "name" : "a", + "type" : "int" + }, { + "name" : "b", + "type" : "int" + } ] + } + } + }, { + "name" : "mymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myfixed", + "type" : { + "type" : "fixed", + "name" : "myfixed", + "namespace" : "", + "size" : 1 + } + } ] +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc new file mode 100644 index 0000000000000..7a98a74633559 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc @@ -0,0 +1,117 @@ +/* + * 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. + */ +{ + "name" : "myrecord", + "namespace": "org.apache.parquet.avro", + "type" : "record", + "fields" : [ { + "name" : "myboolean", + "type" : "boolean" + }, { + "name" : "myint", + "type" : "int" + }, { + "name" : "mylong", + "type" : "long" + }, { + "name" : "myfloat", + "type" : "float" + }, { + "name" : "mydouble", + "type" : "double" + }, { + "name" : "mybytes", + "type" : "bytes" + }, { + "name" : "mystring", + "type" : "string" + }, { + "name" : "mynestedrecord", + "type" : { + "type" : "record", + "name" : "mynestedrecord", + "namespace" : "", + "fields" : [ { + "name" : "mynestedint", + "type" : "int" + } ] + } + }, { + "name" : "myenum", + "type" : "string" + }, { + "name" : "myarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myoptionalarray", + "type" : [ "null", { + "type" : "array", + "items" : "int" + }], + "default" : null + }, { + "name" : "myarrayofoptional", + "type" : { + "type" : "array", + "items" : { + "type": "record", + "name": "list", + "namespace": "", + "fields": [ { + "name": "element", + "type": ["null", "int"], + "default": null + } ] + } + } + }, { + "name" : "myrecordarray", + "type" : { + "type" : "array", + "items" : { + "type" : "record", + "name" : "array", + "namespace" : "", + "fields" : [ { + "name" : "a", + "type" : "int" + }, { + "name" : "b", + "type" : "int" + } ] + } + } + }, { + "name" : "mymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myfixed", + "type" : { + "type" : "fixed", + "name" : "myfixed", + "namespace" : "", + "size" : 1 + } + } ] +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc new file mode 100644 index 0000000000000..ca1e505ec3380 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc @@ -0,0 +1,97 @@ +/* + * 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. + */ +{ + "name": "fixedToInt96", + "namespace": "org.apache.parquet.avro", + "type": "record", + "fields": [ + { + "name": "int96", + "type": { + "type": "fixed", + "name": "ignored1", + "namespace": "", + "size": 12 + } + }, + { + "name": "notanint96", + "type": { + "type": "fixed", + "name": "ignored2", + "namespace": "", + "size": 12 + } + }, + { + "name": "mynestedrecord", + "type": { + "type": "record", + "name": "ignored3", + "namespace": "", + "fields": [ + { + "name": "int96inrecord", + "type": { + "type": "fixed", + "name": "ignored4", + "namespace": "", + "size": 12 + } + }, + { + "name": "myarrayofoptional", + "type": { + "type": "array", + "items": [ + "null", + { + "type": "fixed", + "name": "ignored5", + "namespace": "", + "size": 12 + } + ] + } + }, + { + "name": "mymap", + "type": { + "type": "map", + "values": { + "type": "fixed", + "name": "ignored6", + "namespace": "", + "size": 12 + } + } + } + ] + } + }, + { + "name": "onebytefixed", + "type": { + "type": "fixed", + "name": "ignored7", + "namespace": "", + "size": 1 + } + } + ] +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 9e6565299040b..fc9290a5c3742 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -19,6 +19,7 @@ package org.apache.hudi.hadoop; 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.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.exception.HoodieException; @@ -126,7 +127,7 @@ private RecordReader getRecordReaderInternal(InputS Reporter reporter) throws IOException { try { if (supportAvroRead && HoodieColumnProjectionUtils.supportTimestamp(job)) { - return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(), split, job, reporter); + return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(Option.empty(), Option.empty()), split, job, reporter); } else { return super.getRecordReader(split, job, reporter); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java index 746066e1c1c74..b8d0241881916 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java @@ -252,6 +252,9 @@ private TypeInfo constructHiveSchemaFromType(Type type, TypeInfo typeInfo) { case DOUBLE: case DATE: case TIMESTAMP: + case TIMESTAMP_MILLIS: + case LOCAL_TIMESTAMP_MICROS: + case LOCAL_TIMESTAMP_MILLIS: case STRING: case UUID: case FIXED: @@ -259,6 +262,7 @@ private TypeInfo constructHiveSchemaFromType(Type type, TypeInfo typeInfo) { case DECIMAL: return typeInfo; case TIME: + case TIME_MILLIS: throw new UnsupportedOperationException(String.format("cannot convert %s type to hive", type)); default: LOG.error(String.format("cannot convert unknown type: %s to Hive", type)); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java index c31041ddc76b0..9aa5887b9cbcc 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java @@ -19,8 +19,12 @@ package org.apache.hudi.hadoop.avro; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.HoodieColumnProjectionUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.action.InternalSchemaMerger; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -31,7 +35,6 @@ import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.parquet.avro.AvroReadSupport; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetInputSplit; @@ -43,6 +46,7 @@ import java.util.Arrays; import java.util.List; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; public class HoodieAvroParquetReader extends RecordReader { @@ -55,7 +59,7 @@ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws ParquetMetadata fileFooter = ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER); MessageType messageType = fileFooter.getFileMetaData().getSchema(); - baseSchema = new AvroSchemaConverter(conf).convert(messageType); + baseSchema = getAvroSchemaConverter(conf).convert(messageType); // if exists read columns, we need to filter columns. List readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf)); @@ -64,7 +68,37 @@ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws AvroReadSupport.setAvroReadSchema(conf, filterSchema); AvroReadSupport.setRequestedProjection(conf, filterSchema); } + parquetRecordReader = new ParquetRecordReader<>(new AvroReadSupport<>(), getFilter(conf)); + } + public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option internalSchemaOption, Option dataSchema) throws IOException { + if (dataSchema.isPresent()) { + baseSchema = dataSchema.get(); + } else { + // get base schema + ParquetMetadata fileFooter = + ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER); + MessageType messageType = fileFooter.getFileMetaData().getSchema(); + baseSchema = getAvroSchemaConverter(conf).convert(messageType); + + if (internalSchemaOption.isPresent()) { + // do schema reconciliation in case there exists read column which is not in the file schema. + InternalSchema mergedInternalSchema = new InternalSchemaMerger( + AvroInternalSchemaConverter.convert(baseSchema), + internalSchemaOption.get(), + true, + true).mergeSchema(); + baseSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, baseSchema.getFullName()); + } + + // if exists read columns, we need to filter columns. + List readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf)); + if (!readColNames.isEmpty()) { + Schema filterSchema = HoodieAvroUtils.generateProjectionSchema(baseSchema, readColNames); + AvroReadSupport.setAvroReadSchema(conf, filterSchema); + AvroReadSupport.setRequestedProjection(conf, filterSchema); + } + } parquetRecordReader = new ParquetRecordReader<>(new AvroReadSupport<>(), getFilter(conf)); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java index 8f9aae530e412..4c55fe26551f7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java @@ -18,6 +18,10 @@ package org.apache.hudi.hadoop.avro; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; + +import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.mapreduce.InputSplit; @@ -33,6 +37,14 @@ * we need to handle timestamp types separately based on the parquet-avro approach. */ public class HoodieTimestampAwareParquetInputFormat extends ParquetInputFormat { + private final Option internalSchemaOption; + private final Option dataSchema; + + public HoodieTimestampAwareParquetInputFormat(Option internalSchemaOption, Option dataSchema) { + super(); + this.internalSchemaOption = internalSchemaOption; + this.dataSchema = dataSchema; + } @Override public RecordReader createRecordReader( diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java index a0d1b086e0357..7c11467fd30c3 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector; import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; import org.apache.hadoop.hive.serde2.avro.InstanceCache; import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; @@ -61,7 +62,7 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema; import static org.apache.hudi.avro.HoodieAvroUtils.isMetadataField; @@ -76,7 +77,7 @@ public class HiveAvroSerializer { private static final Logger LOG = LoggerFactory.getLogger(HiveAvroSerializer.class); - public HiveAvroSerializer(ObjectInspector objectInspector, List columnNames, List columnTypes) { + public HiveAvroSerializer(ArrayWritableObjectInspector objectInspector, List columnNames, List columnTypes) { this.columnNames = columnNames; this.columnTypes = columnTypes; this.objectInspector = objectInspector; @@ -197,9 +198,8 @@ private Object serialize(TypeInfo typeInfo, ObjectInspector fieldOI, Object stru return null; } - if (isNullableType(schema)) { - schema = getOtherTypeFromNullableType(schema); - } + schema = getNonNullTypeFromUnion(schema); + /* Because we use Hive's 'string' type when Avro calls for enum, we have to expressly check for enum-ness */ if (Schema.Type.ENUM.equals(schema.getType())) { assert fieldOI instanceof PrimitiveObjectInspector; @@ -336,7 +336,7 @@ private Object serializeList(ListTypeInfo typeInfo, ListObjectInspector fieldOI, ObjectInspector listElementObjectInspector = fieldOI.getListElementObjectInspector(); // NOTE: We have to resolve nullable schema, since Avro permits array elements // to be null - Schema arrayNestedType = resolveNullableSchema(schema.getElementType()); + Schema arrayNestedType = getNonNullTypeFromUnion(schema.getElementType()); Schema elementType; if (listElementObjectInspector.getCategory() == ObjectInspector.Category.PRIMITIVE) { elementType = arrayNestedType; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala index dd76aee2f187b..5fc0d0d6a6945 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala @@ -35,13 +35,14 @@ import org.apache.hudi.common.util.hash.ColumnIndexID import org.apache.hudi.data.HoodieJavaRDD import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadata, HoodieTableMetadataUtil, MetadataPartitionType} import org.apache.hudi.util.JFunction +import org.apache.hudi.common.function.SerializableFunction import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.HoodieUnsafeUtils.{createDataFrameFromInternalRows, createDataFrameFromRDD, createDataFrameFromRows} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions.col import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.{DataFrame, HoodieUnsafeUtils, Row, SparkSession} import org.apache.spark.storage.StorageLevel import java.nio.ByteBuffer @@ -145,6 +146,62 @@ class ColumnStatsIndexSupport(spark: SparkSession, } } + /** + * Loads view of the Column Stats Index in a transposed format where single row coalesces every columns' + * statistics for a single file, returning it as [[DataFrame]] + * + * Please check out scala-doc of the [[transpose]] method explaining this view in more details + */ + def loadTransposed[T](targetColumns: Seq[String], + shouldReadInMemory: Boolean, + prunedPartitions: Option[Set[String]] = None, + prunedFileNamesOpt: Option[Set[String]] = None)(block: DataFrame => T): T = { + cachedColumnStatsIndexViews.get(targetColumns) match { + case Some(cachedDF) => + block(cachedDF) + case None => + val colStatsRecords: HoodieData[HoodieMetadataColumnStats] = prunedFileNamesOpt match { + case Some(prunedFileNames) => + val filterFunction = new SerializableFunction[HoodieMetadataColumnStats, java.lang.Boolean] { + override def apply(r: HoodieMetadataColumnStats): java.lang.Boolean = { + prunedFileNames.contains(r.getFileName) + } + } + loadColumnStatsIndexRecords(targetColumns, shouldReadInMemory).filter(filterFunction) + case None => + loadColumnStatsIndexRecords(targetColumns, shouldReadInMemory) + } + + withPersistedData(colStatsRecords, StorageLevel.MEMORY_ONLY) { + val (transposedRows, indexSchema) = transpose(colStatsRecords, targetColumns) + val df = if (shouldReadInMemory) { + // NOTE: This will instantiate a [[Dataset]] backed by [[LocalRelation]] holding all of the rows + // of the transposed table in memory, facilitating execution of the subsequently chained operations + // on it locally (on the driver; all such operations are actually going to be performed by Spark's + // Optimizer) + HoodieUnsafeUtils.createDataFrameFromRows(spark, transposedRows.collectAsList().asScala.toSeq, indexSchema) + } else { + val rdd = HoodieJavaRDD.getJavaRDD(transposedRows) + spark.createDataFrame(rdd, indexSchema) + } + + if (allowCaching) { + cachedColumnStatsIndexViews.put(targetColumns, df) + // NOTE: Instead of collecting the rows from the index and hold them in memory, we instead rely + // on Spark as (potentially distributed) cache managing data lifecycle, while we simply keep + // the referenced to persisted [[DataFrame]] instance + df.persist(StorageLevel.MEMORY_ONLY) + + block(df) + } else { + withPersistedDataset(df) { + block(df) + } + } + } + } + } + /** * Loads a view of the Column Stats Index in a raw format, returning it as [[DataFrame]] * @@ -334,6 +391,7 @@ class ColumnStatsIndexSupport(spark: SparkSession, columnStatsRecords } + private def loadFullColumnStatsIndexInternal(): DataFrame = { val metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePathV2.toString) // Read Metadata Table's Column Stats Index into Spark's [[DataFrame]] diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 0098ee54c2bc9..0d0b595730a2b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -243,7 +243,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, case HoodieFileFormat.PARQUET => // We're delegating to Spark to append partition values to every row only in cases // when these corresponding partition-values are not persisted w/in the data file itself - val parquetFileFormat = sparkAdapter.createLegacyHoodieParquetFileFormat(shouldExtractPartitionValuesFromPartitionPath).get + val parquetFileFormat = sparkAdapter.createLegacyHoodieParquetFileFormat( + shouldExtractPartitionValuesFromPartitionPath, tableAvroSchema).get (parquetFileFormat, LegacyHoodieParquetFileFormat.FILE_FORMAT_ID) } @@ -548,7 +549,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, hadoopConf = hadoopConf, // We're delegating to Spark to append partition values to every row only in cases // when these corresponding partition-values are not persisted w/in the data file itself - appendPartitionValues = shouldAppendPartitionValuesOverride.getOrElse(shouldExtractPartitionValuesFromPartitionPath) + appendPartitionValues = shouldAppendPartitionValuesOverride.getOrElse(shouldExtractPartitionValuesFromPartitionPath), + tableAvroSchema ) // Since partition values by default are omitted, and not persisted w/in data-files by Spark, // data-file readers (such as [[ParquetFileFormat]]) have to inject partition values while reading diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala index 4add21b5b8da4..9c48f40b00ca1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala @@ -51,8 +51,9 @@ object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport { filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - appendPartitionValues: Boolean = false): PartitionedFile => Iterator[InternalRow] = { - val parquetFileFormat: ParquetFileFormat = sparkAdapter.createLegacyHoodieParquetFileFormat(appendPartitionValues).get + appendPartitionValues: Boolean = false, + avroTableSchema: Schema): PartitionedFile => Iterator[InternalRow] = { + val parquetFileFormat: ParquetFileFormat = sparkAdapter.createLegacyHoodieParquetFileFormat(appendPartitionValues, avroTableSchema).get val readParquetFile: PartitionedFile => Iterator[Any] = parquetFileFormat.buildReaderWithPartitionValues( sparkSession = sparkSession, dataSchema = dataSchema, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index f60263b3344e0..5ff0a77299c30 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -30,6 +30,7 @@ import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKey import org.apache.hudi.metadata.HoodieMetadataPayload import org.apache.hudi.util.JFunction import org.apache.spark.internal.Logging +import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{And, Expression, Literal} import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory} @@ -37,15 +38,14 @@ import org.apache.spark.sql.hudi.DataSkippingUtils.translateIntoColumnStatsIndex import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.unsafe.types.UTF8String import java.text.SimpleDateFormat import java.util.stream.Collectors import javax.annotation.concurrent.NotThreadSafe import scala.collection.JavaConverters._ -import scala.util.control.NonFatal import scala.util.{Failure, Success, Try} +import scala.util.control.NonFatal /** * A file index which support partition prune for hoodie snapshot and read-optimized query. 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 41e8ba902a7e8..b3e0e2785050f 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 @@ -29,7 +29,7 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieConversionUtils.{toProperties, toScalaOption} import org.apache.hudi.HoodieSparkSqlWriter.StreamingWriteParams import org.apache.hudi.HoodieWriterUtils._ -import org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema +import org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.client.embedded.EmbeddedTimelineService @@ -892,7 +892,7 @@ class HoodieSparkSqlWriterInternal { def validateSchemaForHoodieIsDeleted(schema: Schema): Unit = { if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) != null && - resolveNullableSchema(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema()).getType != Schema.Type.BOOLEAN) { + getNonNullTypeFromUnion(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema()).getType != Schema.Type.BOOLEAN) { throw new HoodieException(HoodieRecord.HOODIE_IS_DELETED_FIELD + " has to be BOOLEAN type. Passed in dataframe's schema has type " + schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema().getType) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 166579c867328..08559d0de6514 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -23,7 +23,7 @@ import org.apache.hudi.DataSourceReadOptions._ import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.SparkHoodieTableFileIndex._ import org.apache.hudi.client.common.HoodieSparkEngineContext -import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.config.{TimestampKeyGeneratorConfig, TypedProperties} import org.apache.hudi.common.model.HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION import org.apache.hudi.common.model.{FileSlice, HoodieTableQueryType} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} @@ -33,6 +33,8 @@ import org.apache.hudi.internal.schema.Types.RecordType import org.apache.hudi.internal.schema.utils.Conversions import org.apache.hudi.keygen.{StringPartitionPathFormatter, TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.util.JFunction + +import org.apache.avro.Schema import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession @@ -41,10 +43,12 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.{InternalRow, expressions} import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.{ByteType, DateType, IntegerType, LongType, ShortType, StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String import java.util.Collections import javax.annotation.concurrent.NotThreadSafe + import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.util.{Success, Try} @@ -90,15 +94,19 @@ class SparkHoodieTableFileIndex(spark: SparkSession, * Get the schema of the table. */ lazy val schema: StructType = if (shouldFastBootstrap) { - StructType(rawSchema.fields.filterNot(f => HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name))) + StructType(rawStructSchema.fields.filterNot(f => HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name))) } else { - rawSchema + rawStructSchema } - private lazy val rawSchema: StructType = schemaSpec.getOrElse({ - val schemaUtil = new TableSchemaResolver(metaClient) - AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema) - }) + lazy val rawAvroSchema: Schema = { + val schemaUtil = new TableSchemaResolver(metaClient) + schemaUtil.getTableAvroSchema + } + + private lazy val rawStructSchema: StructType = schemaSpec.getOrElse { + AvroConversionUtils.convertAvroSchemaToStructType(rawAvroSchema) + } protected lazy val shouldFastBootstrap = configProperties.getBoolean(DATA_QUERIES_ONLY.key, false) @@ -396,9 +404,21 @@ class SparkHoodieTableFileIndex(spark: SparkSession, } protected def doParsePartitionColumnValues(partitionColumns: Array[String], partitionPath: String): Array[Object] = { - HoodieSparkUtils.parsePartitionColumnValues(partitionColumns, partitionPath, getBasePath, schema, - configProperties.getString(DateTimeUtils.TIMEZONE_OPTION, SQLConf.get.sessionLocalTimeZone), - sparkParsePartitionUtil, shouldValidatePartitionColumns(spark)) + val tableConfig = metaClient.getTableConfig + if (null != tableConfig.getKeyGeneratorClassName + && tableConfig.getKeyGeneratorClassName.equals(classOf[org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator].getName) + && tableConfig.propsMap.get(TimestampKeyGeneratorConfig.TIMESTAMP_TYPE_FIELD.key()) + .matches("SCALAR|UNIX_TIMESTAMP|EPOCHMILLISECONDS|EPOCHMICROSECONDS")) { + // For TIMESTAMP key generator when TYPE is SCALAR, UNIX_TIMESTAMP, + // EPOCHMILLISECONDS, or EPOCHMICROSECONDS, + // we couldn't reconstruct initial partition column values from partition paths due to lost data after formatting in most cases. + // But the output for these cases is in a string format, so we can pass partitionPath as UTF8String + Array.fill(partitionColumns.length)(UTF8String.fromString(partitionPath)) + } else { + HoodieSparkUtils.parsePartitionColumnValues(partitionColumns, partitionPath, getBasePath, schema, + configProperties.getString(DateTimeUtils.TIMEZONE_OPTION, SQLConf.get.sessionLocalTimeZone), + sparkParsePartitionUtil, shouldValidatePartitionColumns(spark)) + } } private def arePartitionPathsUrlEncoded: Boolean = diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala index afccd43ca3ea2..b27c03deb1e10 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala @@ -99,7 +99,8 @@ class CDCRelation( requiredSchema = tableStructSchema, filters = Nil, options = options, - hadoopConf = spark.sessionState.newHadoopConf() + hadoopConf = spark.sessionState.newHadoopConf(), + avroTableSchema = tableAvroSchema ) val changes = cdcExtractor.extractCDCFileSplits().values().asScala.map { splits => diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index b4e09f6d1f656..0edb6fda98526 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql.avro import org.apache.avro.LogicalTypes.{Date, Decimal, TimestampMicros, TimestampMillis} import org.apache.avro.Schema.Type._ -import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder} +import org.apache.avro.{LogicalType, LogicalTypes, Schema, SchemaBuilder} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.types.Decimal.minBytesForPrecision import org.apache.spark.sql.types._ import scala.collection.JavaConverters._ +import scala.util.Try /** * This object contains method that are used to convert sparkSQL schemas to avro schemas and vice @@ -40,6 +41,73 @@ import scala.collection.JavaConverters._ private[sql] object SchemaConverters { private lazy val nullSchema = Schema.create(Schema.Type.NULL) + // Reflection-based checks for types that may not be available in all Avro/Spark versions + private lazy val localTimestampMillisClass: Option[Class[_]] = Try { + Class.forName("org.apache.avro.LogicalTypes$LocalTimestampMillis") + }.toOption + + private lazy val localTimestampMicrosClass: Option[Class[_]] = Try { + Class.forName("org.apache.avro.LogicalTypes$LocalTimestampMicros") + }.toOption + + private lazy val timestampNTZTypeClass: Option[Class[_]] = Try { + Class.forName("org.apache.spark.sql.types.TimestampNTZType$") + }.toOption + + private lazy val timestampNTZTypeInstance: Option[DataType] = timestampNTZTypeClass.flatMap { clazz => + Try { + val module = clazz.getField("MODULE$").get(null) + module.asInstanceOf[DataType] + }.toOption + } + + private lazy val localTimestampMicrosMethod: Option[java.lang.reflect.Method] = Try { + classOf[LogicalTypes].getMethod("localTimestampMicros") + }.toOption + + private lazy val localTimestampMillisMethod: Option[java.lang.reflect.Method] = Try { + classOf[LogicalTypes].getMethod("localTimestampMillis") + }.toOption + + /** + * Checks if a logical type is an instance of LocalTimestampMillis using reflection. + * Returns false if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private def isLocalTimestampMillis(logicalType: LogicalType): Boolean = { + logicalType != null && localTimestampMillisClass.exists(_.isInstance(logicalType)) + } + + /** + * Checks if a logical type is an instance of LocalTimestampMicros using reflection. + * Returns false if the class doesn't exist (e.g., in Avro 1.8.2). + */ + private def isLocalTimestampMicros(logicalType: LogicalType): Boolean = { + logicalType != null && localTimestampMicrosClass.exists(_.isInstance(logicalType)) + } + + /** + * Checks if a DataType is TimestampNTZType using reflection. + * Returns false if the class doesn't exist (e.g., in Spark 2.x or Spark 3.2). + */ + private def isTimestampNTZType(dataType: DataType): Boolean = { + timestampNTZTypeInstance.contains(dataType) || + (timestampNTZTypeClass.isDefined && timestampNTZTypeClass.get.isInstance(dataType)) + } + + /** + * Creates a LocalTimestampMicros schema using reflection. + * Throws UnsupportedOperationException if not available. + */ + private def createLocalTimestampMicrosSchema(): Schema = { + localTimestampMicrosMethod match { + case Some(method) => + val logicalType = method.invoke(null).asInstanceOf[LogicalType] + logicalType.addToSchema(Schema.create(Schema.Type.LONG)) + case None => + throw new UnsupportedOperationException("LocalTimestampMicros is not supported in this Avro version") + } + } + /** * Internal wrapper for SQL data type and nullability. * @@ -77,6 +145,11 @@ private[sql] object SchemaConverters { case FLOAT => SchemaType(FloatType, nullable = false) case LONG => avroSchema.getLogicalType match { case _: TimestampMillis | _: TimestampMicros => SchemaType(TimestampType, nullable = false) + case logicalType if isLocalTimestampMillis(logicalType) || isLocalTimestampMicros(logicalType) => + timestampNTZTypeInstance match { + case Some(timestampNTZ) => SchemaType(timestampNTZ, nullable = false) + case None => SchemaType(LongType, nullable = false) // Fallback for older Spark versions + } case _ => SchemaType(LongType, nullable = false) } @@ -165,6 +238,8 @@ private[sql] object SchemaConverters { LogicalTypes.date().addToSchema(builder.intType()) case TimestampType => LogicalTypes.timestampMicros().addToSchema(builder.longType()) + case dataType if isTimestampNTZType(dataType) => + createLocalTimestampMicrosSchema() case FloatType => builder.floatType() case DoubleType => builder.doubleType() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/LegacyHoodieParquetFileFormat.scala index 046640c11c1ba..e911ba411a983 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/LegacyHoodieParquetFileFormat.scala @@ -38,12 +38,8 @@ class LegacyHoodieParquetFileFormat extends ParquetFileFormat with SparkAdapterS override def toString: String = "Hoodie-Parquet" override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - if (HoodieSparkUtils.gteqSpark3_4) { - val conf = sparkSession.sessionState.conf - conf.parquetVectorizedReaderEnabled && schema.forall(_.dataType.isInstanceOf[AtomicType]) - } else { - super.supportBatch(sparkSession, schema) - } + sparkAdapter + .createLegacyHoodieParquetFileFormat(true, null).get.supportBatch(sparkSession, schema) } override def buildReaderWithPartitionValues(sparkSession: SparkSession, @@ -58,7 +54,7 @@ class LegacyHoodieParquetFileFormat extends ParquetFileFormat with SparkAdapterS DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.defaultValue.toString).toBoolean sparkAdapter - .createLegacyHoodieParquetFileFormat(shouldExtractPartitionValuesFromPartitionPath).get + .createLegacyHoodieParquetFileFormat(shouldExtractPartitionValuesFromPartitionPath, null).get .buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml index 9a2a9232619a6..f2bc4b816331c 100644 --- a/hudi-spark-datasource/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -335,6 +335,18 @@ org.pentaho * + + org.apache.parquet + * + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + @@ -350,6 +362,18 @@ javax.servlet.jsp * + + org.apache.parquet + * + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + @@ -365,6 +389,18 @@ javax.servlet.jsp * + + org.apache.parquet + * + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + @@ -376,6 +412,18 @@ org.eclipse.jetty.orbit javax.servlet + + org.apache.parquet + * + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + @@ -403,6 +451,16 @@ hudi-tests-common ${project.version} test + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + + org.apache.hudi @@ -411,6 +469,16 @@ tests test-jar test + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + + org.apache.hudi @@ -419,6 +487,16 @@ tests test-jar test + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + + org.apache.hudi 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 index 6917a4360bf95..6d3378396b3b9 100644 --- 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 @@ -29,6 +29,7 @@ import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.util.BaseFileUtils 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} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala index 24820c1c03204..6718c441f1d05 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala @@ -227,10 +227,14 @@ object HoodieAnalysis extends SparkAdapterSupport { } if (updatedTargetTable.isDefined || updatedQuery.isDefined) { - mit.asInstanceOf[MergeIntoTable].copy( + val mergeIntoTable = mit.asInstanceOf[MergeIntoTable] + // Use all parameters to avoid NoSuchMethodError when method signature changes between Spark versions + mergeIntoTable.copy( targetTable = updatedTargetTable.getOrElse(targetTable), - sourceTable = updatedQuery.getOrElse(query) - ) + sourceTable = updatedQuery.getOrElse(query), + mergeCondition = mergeIntoTable.mergeCondition, + matchedActions = mergeIntoTable.matchedActions, + notMatchedActions = mergeIntoTable.notMatchedActions) } else { mit } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala index 3f3d4e10ea9e4..15769c4c8f411 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala @@ -95,7 +95,7 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi } val config = buildHoodieInsertConfig(catalogTable, sparkSession, isOverWritePartition, isOverWriteTable, partitionSpec, extraOptions, staticOverwritePartitionPathOpt) - val alignedQuery = alignQueryOutput(query, catalogTable, partitionSpec, sparkSession.sessionState.conf) + val alignedQuery = alignQueryOutput(query, catalogTable, partitionSpec, sparkSession.sessionState.conf, sparkSession) val (success, _, _, _, _, _) = HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, config, Dataset.ofRows(sparkSession, alignedQuery)) @@ -122,11 +122,13 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi * @param catalogTable catalog table * @param partitionsSpec partition spec specifying static/dynamic partition values * @param conf Spark's [[SQLConf]] + * @param sparkSession Spark session (required for Spark 3.5) */ private def alignQueryOutput(query: LogicalPlan, catalogTable: HoodieCatalogTable, partitionsSpec: Map[String, Option[String]], - conf: SQLConf): LogicalPlan = { + conf: SQLConf, + sparkSession: SparkSession): LogicalPlan = { val targetPartitionSchema = catalogTable.partitionSchema val staticPartitionValues = filterStaticPartitionValues(partitionsSpec) @@ -141,7 +143,7 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi // since such columns wouldn't be otherwise specified w/in the query itself and therefore couldn't be matched // positionally for example val expectedQueryColumns = catalogTable.tableSchemaWithoutMetaFields.filterNot(f => staticPartitionValues.contains(f.name)) - val coercedQueryOutput = coerceQueryOutputColumns(StructType(expectedQueryColumns), cleanedQuery, catalogTable, conf) + val coercedQueryOutput = coerceQueryOutputColumns(StructType(expectedQueryColumns), cleanedQuery, catalogTable, conf, sparkSession) // After potential reshaping validate that the output of the query conforms to the table's schema validate(removeMetaFields(coercedQueryOutput.schema), partitionsSpec, catalogTable) @@ -153,14 +155,28 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi private def coerceQueryOutputColumns(expectedSchema: StructType, query: LogicalPlan, catalogTable: HoodieCatalogTable, - conf: SQLConf): LogicalPlan = { - val planUtils = sparkAdapter.getCatalystPlanUtils + conf: SQLConf, + sparkSession: SparkSession): LogicalPlan = { try { - planUtils.resolveOutputColumns(catalogTable.catalogTableName, expectedSchema.toAttributes, query, byName = true, conf) + sparkAdapter.resolveOutputColumns( + sparkSession, + catalogTable.catalogTableName, + sparkAdapter.getSchemaUtils.toAttributes(expectedSchema), + query, + byName = true, + conf) } catch { // NOTE: In case matching by name didn't match the query output, we will attempt positional matching - case ae: AnalysisException if ae.getMessage().startsWith("Cannot write incompatible data to table") => - planUtils.resolveOutputColumns(catalogTable.catalogTableName, expectedSchema.toAttributes, query, byName = false, conf) + // SPARK-42309 Error message changed in Spark 3.5.0 so we need to match two strings here + case ae: AnalysisException if (ae.getMessage().startsWith("[INCOMPATIBLE_DATA_FOR_TABLE.CANNOT_FIND_DATA] Cannot write incompatible data for the table") + || ae.getMessage().startsWith("Cannot write incompatible data to table")) => + sparkAdapter.resolveOutputColumns( + sparkSession, + catalogTable.catalogTableName, + sparkAdapter.getSchemaUtils.toAttributes(expectedSchema), + query, + byName = false, + conf) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala index 0989b8b09aee4..954b731dd4c33 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala @@ -23,12 +23,12 @@ import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord} import org.apache.hudi.AvroConversionUtils.{convertAvroSchemaToStructType, convertStructTypeToAvroSchema} import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.hudi.avro.AvroSchemaUtils.{isNullable, resolveNullableSchema} +import org.apache.hudi.avro.AvroSchemaUtils.{getNonNullTypeFromUnion, isNullable} import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodiePayloadProps, HoodieRecord} +import org.apache.hudi.common.util.{BinaryUtil, Option => HOption, ValidationUtils} import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.common.util.{BinaryUtil, ValidationUtils, Option => HOption} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException import org.apache.spark.internal.Logging @@ -487,8 +487,8 @@ object ExpressionPayload { .zipWithIndex .foreach { case ((expectedField, targetField), idx) => - val expectedFieldSchema = resolveNullableSchema(expectedField.schema()) - val targetFieldSchema = resolveNullableSchema(targetField.schema()) + val expectedFieldSchema = getNonNullTypeFromUnion(expectedField.schema()) + val targetFieldSchema = getNonNullTypeFromUnion(targetField.schema()) val equal = Objects.equals(expectedFieldSchema, targetFieldSchema) ValidationUtils.checkState(equal, diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala index 4005ef97e4561..c5211d8852e11 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala @@ -36,7 +36,7 @@ import scala.collection.JavaConverters._ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface) extends HoodieSqlCommonBaseVisitor[AnyRef] with Logging with SparkAdapterSupport { - import ParserUtils._ + import ParserUtils.{checkDuplicateKeys, operationNotAllowed, string} /** * Override the default behavior for all visit methods. This will only return a non-null result @@ -51,25 +51,25 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface } } - override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) { + override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = ParserUtils.withOrigin(ctx) { ctx.statement().accept(this).asInstanceOf[LogicalPlan] } - override def visitCompactionOnTable(ctx: CompactionOnTableContext): LogicalPlan = withOrigin(ctx) { + override def visitCompactionOnTable(ctx: CompactionOnTableContext): LogicalPlan = ParserUtils.withOrigin(ctx) { val table = ctx.tableIdentifier().accept(this).asInstanceOf[LogicalPlan] val operation = CompactionOperation.withName(ctx.operation.getText.toUpperCase) val timestamp = if (ctx.instantTimestamp != null) Some(ctx.instantTimestamp.getText.toLong) else None CompactionTable(table, operation, timestamp) } - override def visitCompactionOnPath(ctx: CompactionOnPathContext): LogicalPlan = withOrigin(ctx) { + override def visitCompactionOnPath(ctx: CompactionOnPathContext): LogicalPlan = ParserUtils.withOrigin(ctx) { val path = string(ctx.path) val operation = CompactionOperation.withName(ctx.operation.getText.toUpperCase) val timestamp = if (ctx.instantTimestamp != null) Some(ctx.instantTimestamp.getText.toLong) else None CompactionPath(path, operation, timestamp) } - override def visitShowCompactionOnTable(ctx: ShowCompactionOnTableContext): LogicalPlan = withOrigin(ctx) { + override def visitShowCompactionOnTable(ctx: ShowCompactionOnTableContext): LogicalPlan = ParserUtils.withOrigin(ctx) { val table = ctx.tableIdentifier().accept(this).asInstanceOf[LogicalPlan] if (ctx.limit != null) { CompactionShowOnTable(table, ctx.limit.getText.toInt) @@ -78,7 +78,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface } } - override def visitShowCompactionOnPath(ctx: ShowCompactionOnPathContext): LogicalPlan = withOrigin(ctx) { + override def visitShowCompactionOnPath(ctx: ShowCompactionOnPathContext): LogicalPlan = ParserUtils.withOrigin(ctx) { val path = string(ctx.path) if (ctx.limit != null) { CompactionShowOnPath(path, ctx.limit.getText.toInt) @@ -87,11 +87,11 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface } } - override def visitTableIdentifier(ctx: TableIdentifierContext): LogicalPlan = withOrigin(ctx) { + override def visitTableIdentifier(ctx: TableIdentifierContext): LogicalPlan = ParserUtils.withOrigin(ctx) { UnresolvedRelation(TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))) } - override def visitCall(ctx: CallContext): LogicalPlan = withOrigin(ctx) { + override def visitCall(ctx: CallContext): LogicalPlan = ParserUtils.withOrigin(ctx) { if (ctx.callArgumentList() == null || ctx.callArgumentList().callArgument() == null || ctx.callArgumentList().callArgument().size() == 0) { val name: Seq[String] = ctx.multipartIdentifier().parts.asScala.map(_.getText) CallCommand(name, Seq()) @@ -105,14 +105,14 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface /** * Return a multi-part identifier as Seq[String]. */ - override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - ctx.parts.asScala.map(_.getText) + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = ParserUtils.withOrigin(ctx) { + ctx.parts.asScala.map(_.getText).toSeq } /** * Create a positional argument in a stored procedure call. */ - override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = withOrigin(ctx) { + override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = ParserUtils.withOrigin(ctx) { val expr = typedVisit[Expression](ctx.expression) PositionalArgument(expr) } @@ -120,7 +120,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface /** * Create a named argument in a stored procedure call. */ - override def visitNamedArgument(ctx: NamedArgumentContext): CallArgument = withOrigin(ctx) { + override def visitNamedArgument(ctx: NamedArgumentContext): CallArgument = ParserUtils.withOrigin(ctx) { val name = ctx.identifier.getText val expr = typedVisit[Expression](ctx.expression) NamedArgument(name, expr) @@ -160,7 +160,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface * indexPropertyList: index_property_name [= index_property_value] [ , . . . ] * }}} */ - override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = withOrigin(ctx) { + override def visitCreateIndex(ctx: CreateIndexContext): LogicalPlan = ParserUtils.withOrigin(ctx) { val (indexName, indexType) = if (ctx.identifier.size() == 1) { (ctx.identifier(0).getText, "") } else { @@ -189,7 +189,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface * DROP INDEX [IF EXISTS] index_name ON [TABLE] table_name * }}} */ - override def visitDropIndex(ctx: DropIndexContext): LogicalPlan = withOrigin(ctx) { + override def visitDropIndex(ctx: DropIndexContext): LogicalPlan = ParserUtils.withOrigin(ctx) { val indexName = ctx.identifier.getText DropIndex( visitTableIdentifier(ctx.tableIdentifier()), @@ -204,7 +204,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface * SHOW INDEXES (FROM | IN) [TABLE] table_name * }}} */ - override def visitShowIndexes(ctx: ShowIndexesContext): LogicalPlan = withOrigin(ctx) { + override def visitShowIndexes(ctx: ShowIndexesContext): LogicalPlan = ParserUtils.withOrigin(ctx) { ShowIndexes(visitTableIdentifier(ctx.tableIdentifier())) } @@ -215,7 +215,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface * REFRESH INDEX index_name ON [TABLE] table_name * }}} */ - override def visitRefreshIndex(ctx: RefreshIndexContext): LogicalPlan = withOrigin(ctx) { + override def visitRefreshIndex(ctx: RefreshIndexContext): LogicalPlan = ParserUtils.withOrigin(ctx) { RefreshIndex(visitTableIdentifier(ctx.tableIdentifier()), ctx.identifier.getText) } @@ -224,7 +224,7 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. */ override def visitPropertyList( - ctx: PropertyListContext): Map[String, String] = withOrigin(ctx) { + ctx: PropertyListContext): Map[String, String] = ParserUtils.withOrigin(ctx) { val properties = ctx.property.asScala.map { property => val key = visitPropertyKey(property.key) val value = visitPropertyValue(property.value) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java index 1898a276a9f6e..b26805a5cc4ec 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java @@ -18,10 +18,13 @@ package org.apache.hudi.functional; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; @@ -33,6 +36,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieClusteringConfig; @@ -42,6 +46,10 @@ import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.testutils.HoodieSparkClientTestHarness; import org.apache.hudi.testutils.MetadataMergeWriteStatus; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -50,15 +58,20 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.stream.Collectors; +import java.util.stream.IntStream; public class TestSparkSortAndSizeClustering extends HoodieSparkClientTestHarness { - private HoodieWriteConfig config; private HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0); @@ -165,4 +178,57 @@ public HoodieWriteConfig.Builder getConfigBuilder() { .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } + + private List generateInserts(String instant, long ts, int count) { + Schema schema = getSchema(); + Schema decimalSchema = schema.getField("decimal_field").schema(); + Schema nestedSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("nested_record").schema()); + Schema enumSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("enum_field").schema()); + Random random = new Random(0); + return IntStream.range(0, count) + .mapToObj(i -> { + GenericRecord record = new GenericData.Record(schema); + String key = "key_" + i; + String partition = "partition_" + (i % 3); + record.put("_row_key", key); + record.put("ts", ts); + record.put("partition_path", partition); + record.put("_hoodie_is_deleted", false); + record.put("double_field", random.nextDouble()); + record.put("float_field", random.nextFloat()); + record.put("int_field", random.nextInt()); + record.put("long_field", random.nextLong()); + record.put("string_field", instant); + record.put("bytes_field", ByteBuffer.wrap(instant.getBytes(StandardCharsets.UTF_8))); + GenericRecord nestedRecord = new GenericData.Record(nestedSchema); + nestedRecord.put("nested_int", random.nextInt()); + nestedRecord.put("nested_string", "nested_" + instant); + nestedRecord.put("nested_timestamp_millis_field", ts); + record.put("nested_record", nestedRecord); + record.put("array_field", Collections.singletonList(nestedRecord)); + record.put("nullable_map_field", Collections.singletonMap("key_" + instant, nestedRecord)); + // logical types + record.put("date_nullable_field", random.nextBoolean() ? null : LocalDate.now().minusDays(random.nextInt(3))); + record.put("timestamp_millis_field", ts); + record.put("timestamp_micros_nullable_field", random.nextBoolean() ? null : ts * 1000); + record.put("timestamp_local_millis_nullable_field", random.nextBoolean() ? null : ts); + record.put("timestamp_local_micros_field", ts * 1000); + record.put("enum_field", new GenericData.EnumSymbol( + enumSchema, + enumSchema + .getEnumSymbols() + .get(random.nextInt(enumSchema.getEnumSymbols().size())))); + return new HoodieAvroIndexedRecord(new HoodieKey(key, partition), record); + }) + .collect(Collectors.toList()); + } + + private Schema getSchema() { + try { + String schema = FileIOUtils.readAsUTFString(this.getClass().getClassLoader().getResourceAsStream("schema_with_logical_types.avsc")); + return new Schema.Parser().parse(schema); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index ec93ea229accb..fa4482e537fbd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -153,7 +153,7 @@ public void testTimestampBasedKeyGenerator() throws IOException { // timezone is GMT+8:00, createTime is BigDecimal BigDecimal decimal = new BigDecimal("1578283932000.0001"); Conversions.DecimalConversion conversion = new Conversions.DecimalConversion(); - Schema resolvedNullableSchema = AvroSchemaUtils.resolveNullableSchema(schema.getField("createTimeDecimal").schema()); + Schema resolvedNullableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("createTimeDecimal").schema()); GenericFixed avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema, LogicalTypes.decimal(20, 4)); baseRecord.put("createTimeDecimal", avroDecimal); properties = getBaseKeyConfig("createTimeDecimal", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); @@ -209,6 +209,31 @@ public void testTimestampBasedKeyGenerator() throws IOException { assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(baseRow)); internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); assertEquals(UTF8String.fromString("1970-01-01 12:00:00"), keyGen.getPartitionPath(internalRow, baseRow.schema())); + + // Timestamp field is in long type, with `EPOCHMICROSECONDS` timestamp type in the key generator + baseRecord.put("createTime", 1578283932123456L); + properties = getBaseKeyConfig("createTime", "EPOCHMICROSECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); + keyGen = new TimestampBasedKeyGenerator(properties); + HoodieKey key = keyGen.getKey(baseRecord); + assertEquals("2020-01-06 12", key.getPartitionPath()); + baseRow = genericRecordToRow(baseRecord); + assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); + internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow); + assertEquals(UTF8String.fromString("2020-01-06 12"), keyGen.getPartitionPath(internalRow, baseRow.schema())); + + // Timestamp field is in decimal type, with `EPOCHMICROSECONDS` timestamp type in the key generator + decimal = new BigDecimal("1578283932123456.0001"); + resolvedNullableSchema = AvroSchemaUtils.getNonNullTypeFromUnion( + schema.getField("createTimeDecimal").schema()); + avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema, LogicalTypes.decimal(20, 4)); + baseRecord.put("createTimeDecimal", avroDecimal); + properties = getBaseKeyConfig( + "createTimeDecimal", "EPOCHMICROSECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); + keyGen = new TimestampBasedKeyGenerator(properties); + bigDecimalKey = keyGen.getKey(baseRecord); + assertEquals("2020-01-06 12", bigDecimalKey.getPartitionPath()); + baseRow = genericRecordToRow(baseRecord); + assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow)); } @Test diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v6.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v6.zip new file mode 100644 index 0000000000000000000000000000000000000000..1de8eadd923e640442d938db4c68879691b27ddc GIT binary patch literal 103597 zcmd43by!@@vM)R&NCE*uumlf5g1c)%2rj|h-F{s^L=iKMJ=a1VDYu4)S>gwvM>a~Vn)htR~ueh+74`TxlQ)Yx7YpM6k{l`*a-7tO+ z00R7aY4^*3M`2%v{sSZP2S(O%=F$>oMkO1^$FMph^NRSuV1GaDsZeFJ6|4t7osI!0|KI%Z}%W)5wb3~Wq{%&bi8telLD>>TX$Fa_H> z8an(-tN+1r67=FmIyyRfjL<>HdJP8i#odaj7-94Q+|;?rB~=nz_CEPAOYxfixkZ`1 z!7Qr=8#V#biM{@s4K%Ip2p1sz9V^ou>qYt;S8M1Ue&Qug_Tz4^mq^6?m*GAd;LQ2! ze#y@Paob@n=c&8wn(f)UniI_elWj4luFHCuaca51R0WXaT(dpGJCXNXh5i9tQw+%; zrFKkn+)GMc}DIN88 zxg4Lm-44meR3DdkeYgwhTVpZZY+0*1yGFC>x|T>vHTGH0uPNuo%mSHvPwMgM4KZFo z=^>$+ZQ9A3(il&{B1;J4tXLoxk~gAj`rN^dZmX@?O<&g4*Ejh!)p7j#l;dsBD058% zp7)itth{8}BtppRxBih(dOYX1C*NSiB6<64%xnImi`>#$#tS$j;TZJYx*g{9^=%~nxbEDXi0Fr6N9!=s z;C?I@Vy#oBWl7fu>Fv;@aB_o=Z7|w(*R|JP8-b+q&L|D4>;)308`wYy?g%r0i}Ch- zhwV5@-D$5Phmxew2vKJIeJi0E7Wv&tAQbW~!4;Fwp zg?u*U+Bej~u}6ctI&YNkhHqQ!pC>lwpAmjZ8K$;LncTOUbJ}Ubb{ra8UzFG-+V05q zbRF@#IPX)R-y;Z=Y2eT&jQbvOruNjWXfAt;@c?@c9HzUSt|fx;oX2egH>-j$5~y=c zB8Fr$r@m@+_<}%0kwpnYRaMbpG?5|^pIX?*puQc>U5DRheF;v`QFAX}Ux0s}c-7T5 zSUqa#aoLERSiR7X01bEytdb!#>~iNZ^~Eu`^}Bh~v+l~Ql$`RJUdiQbAkkW$FS%)h zp_O&Ea(oGqo~M_6QWJ5=v7?huu%er)eW|UwJm4;gv2=rKlsy+uQJl@ih<6fL=-)HCVR~SD8~XHFOduG)j8dq zLyq=0S^sc8J&QyNT}=NmcD3ND>UTSm7XZgHwl%cOKWLaSULa?IU^ZJDW2J zB$P0L{jTUJwnMw&Kg4aH;fM=e;#QmiVHtnws0 z>6CMbR*w~ZDi%-h4$!33-U-X_vDrbYb*D`!t?(SjhW%dBq?ldrr{p*A?XO%2+-~H! z?HnmtLMmLeu2S>awHDl0{5*C*(VuxJ!W_cd?AtlA|MzFGP;!MW0jH)EbAng`-nq^E&&BJ|o|$Rq6a{HVM86!``ea zpI$L7?-$P%p)@jDC8qgSJ^to7nwdP6C9*12yAf%G{hySQu(ThN53$J}9w!m%S9P6~ zZpGBYV`XQ*CyC=T@7M()FRISvW20lYahylKfzB7xGRI(n|xmP8lwSdNUfQ({Ivu3ODHn&OnfxIpdxD#xc!P0mP+MO5;5Zfl=8 zG&veid*%6uw(VYM-X<4&o`=&!O!Q7e(fIUFMT(|`=ou`vv zk_iN3k#}|SokkP7r0plDGTcoRewf*Xlv+)1W2iNko8-Kav|wS8=T~FCZU#Z@V+%(& zFpJ;k7#udwb)kFRt%L9CJ%f8o77E@jbPQXEZcSbzw*`o)r@jxr>xRBOgv^fi-n{xcoVoBbEo7ZQNHj~8Bvkf-(_9gug9E;ezPv`YRkkv}@ zUbU8hcuF#(p7_tNJkJn@H`{16^Y&88>{`)H2~Otpeb`~Y06roY=&G_ zEtFC=%`Eb)OrUcF;Ad*IVDyebP{+3SXuV^VUD^n)T|ToI8c=etVy?x7`Vs^%ia~EVQU6 z&qR3Ff9drvbRL0N+Lax}{wnnCQ%)CLw}Gn3AeHn%$?Wb6h~l^s`;CUYoO zu3TcuiIJGPFRZ@)po2y`b4c2pE;*BNK@%w{EE@vA!1P)THIhN#>5}&wUGgfKO(nsK zSK$hRS7gQlXpf>d^`wm9AZ{-M^lpnHlcN) z6_)BnUW$oH3D!+bNpNVr?DUOAE~VA7uLYLrP0ht)j94*F%MNHIvT!p5f#YOFIHGm#dthX zaTzPxPjs{AUz8ino)8m7vjk4=5sMkpY`AV%ViZhy`bhkxm-)(ayZN&tEN*|hTIcsc zH0rVbZx3gbxcp z%d`ZkQhHLww%f1ae=YSidH) z3at{dE1S%w;b`>~)8;8 zLR4AS(BN6?=z8;{QQuSE!$Z^FV>G_>?%@#p={ilW=Nk`+c?)ws7`pVkbT&PyeE7Ta zm$P@q$Tv>Mai&nxtrBm*Hx4N?r2Af-eOgf$k+Zw!x&^ObRY6n@TtwJ892Ms{`6rC` zlL$bCuJBP|WDE_^MX5SQJH!oWT385u4(lT^w&8~NR(nnh5WHiYBo0S<=1=K>d8n=g z(X**=ovX90K)dH4$iLY3mAaY+Ptd3*zkCJY%^QvDFL3Qg3jj>fFmTgM`Hwxgr-KP3 zYb!(rFhJSuY5tbR^UeG71AuG?(S2u=x9+?P!220JG7(zraH()HLKj=`;k~qg^`;+6 z0Du8+Q4lIS_q+QRfB_oOG7?~Zase6M+oFYy@5yUwtXfHB9$44m^RT+#Dk{UvV0+jU zu^3AT>sWp^@N`~u-+>bFJ)m5^>#2e;tnmqb$|3@QWTFK4Q(>@{KZ>A|>Nls~l|xLK zHDaNp$B60A4RAcR%E5iJ#0=j)<_ET?`XgqsbZ5K$+UM87flRFMjKf!%Sc}>V0__NVyHW3?<4q{?%~*`TWU~P6U8&Xg1ED zZDlW%JupP1?i&sX^qb)Xi*%PA?rVaf>VL|4-|){8A2_sz!6LB(vr6l0mw)wM6u#NH zub|-ra+YN{PrcM4FHJ$VyguJsD5e2V*1MI{>va;Z+Lef0RY28LV|Nk)32EypM3-f z`lb84Cy*fobeV6X!PFPN#>oIUwy2%{rPa&qX{n{zLNtJjxgV@ame}JBpzHZ@KTNZ{ zvl=jFu)_R&&;t&@44 z(k|e=2|6M;U~YW5)b%vpJ|rMf3vRi-&(sd?d(b!lfcPXw*`~VM>|P@V2QtyQp|6lL z7oy<-yLP`;CCENciUbhU)R6sIKe4&ugpt~uKgYb6;=YWwV#)#luNS&u+qFpF-5AzH z@{ld}-uDH^d-wdYUu?nysuBHyo53PK>6^vg0z}(cMg>C`nC}~4#jXh>xo&m#e)In< z;Q4plzpn+gmh3P9PpCaLFn%>PdCybw|B}vT8E+I#kS$61VR)U%0?cWs*FPDMDfNEi zPS@^#=*_8S^4)(L3DAj%^yFcfu38z z88@XY_v!xugMhyYi*qIIjg_$UTkWdGmdj*&ZuP=AkLiWEr+T`cH*ZJo9{w41 z;rmC>=5H|*j$bhor~j*%2?xXfdd%cr=qz&LDhrsC|6A+Jife!xdA1vTUMrCn zS+*xbbGRU8;EA~GE1T{O)@6ZoKORBOGBPLT z%05zrMz8DAXQ}m_#ZVjI)Oddq^HVF{c)bN*8P7}2A?kvItGnK59~%bd*hi~0tk_ou zU!+2pKB7)S1nQ++zA({DbT0DH*U+J((*cII%0hXYDH6+k%?%cxE-)DCSvQmd$#nh zgy7IxTFXz$eJn`W{!@w+L`JmnU|F8|Az|pcinyuNLw=Wrvp|)Vti?l$#gWdXbU!wz z#}=2laKTI!D@`9ml7k0aaAVNT_zN|E5yRbGBC|x79nhr9TQeM^X&wyoxjwB` zL?zd|!$^=BQ|v9{391OiEj5D7DAu6FCDrb{1lr9S7mn4V#U+*U;0~}WtC(X<87`kL zI?i*C8m7LbXTN|BlXK2|@`D-?fGJczjaU@1Hj_NR8rbw#{0Hv*A0vh^@Be=_VtBu6 zW5vhP$zgaoC+Ht|IS0f4)~5Z5m;Z+iyA45iz*KB#IuJ7%QR!`L`GocHEpb0}6}gzY zcqz5zXMOZ)krGRxw332(;#o!z@Agvr|$E_I@mx>+z%gAo4pFYk@gV1J63!b z+ldK3l#Wv8eTap90Ey3xH^p?8MOhd;v;PK;cUwEFW3gwk&+*Ehh}PJOj#~CPtUMIB zt+l92Q(^BBi*#538&MVb1GiQDdl#s?fYnJCF1?2+$Vo1U8<(oS^ijfu|9-4!t&VJL8t#Eaq~e3*Wnv{6TCZl6!1zts^+ra&4-*du?P_E`08_#hPuS%Pt z3{8Dz^2UodYo=)Shnuq-z${){^&I66zQ)^_xfy=di&@_ex*IQTyMq+CFi}jc>y<-e zrQ6kr;HjuSV2n%(`z0_%h6Ad$-?PR7BzC!*iRL55nq|p@faR6qFLpn^FlF*kuQA^& zfvVjW%2S%ortTGf3HB%BQn$K2_MMP#dC_Od{AIGWH=*izYseY?vJ|2VWLIj0#~;0_ zW~bRR=W5f_$aE6C*EhHxnUoyq^h2|DVIWU{8=u(K>T1{Y6@x|z%Zc|&3Rpi?tHF6> zSJ_{)!SpuIr(^5#^p3M1BN}C-k>6cnbIf!?;B*l^O2o2TyavMI02SdNVC7wMKm3s? z?-q{KrQChCcQ;7bp?8q3m?93HEUnZIu|Dre-KO?%=;MnY1|l)9W%;wjFKb+M4S2%YR9g;s1vLrheLU})79Plf+KYX&aUDT^f3vFJWh$U zhsdpOjx9lR!*NKsVP?i3^=e$yk2+mY7T;c6NL@gTbO=jYxLjEjL&;lR^9mR zJKi-qv`1Yz4dA;8Erm@S$3Cb`8=S^&NK9g2aI?HgADujxDbmz5Ptbkt-U(Uiqa;#` z91&iRGjhMRQ=47*P?NrBnLA2`jYk3Y)arG{qY$Xr`+VV7O|w0%1nfSI84Qab+j<%F zCQmYch=KV@h@SX91hiMMjwKLA)<0)FYx1-`c(>z;IneOqyhHV2vhPt4GV&qPTBkY% zJ_~IG_VQP8(8vyFk6oNqJKX1nmIq!R3=IdudA$s!8;rZ=3Bo4Udb(*2s}J29-Zqx& zm6#|+tn~q%34HC7us^3yJ(cm4j$kNe?$nn3z|^LwFwbzz<)|yKiDukPA~Hu}tecNp zOVO%P zdNT%+B|=o^rkuY?=ueM-0*Ss5KiSJ0B>ur`DU-jM>U8Brh`eXpio2kfzvGBN9#7RUw)b|suDUISCoRni1PAp&&2@1(Q zEGBIy%(c^(Sjrw4!U?MF(N}v53}V-cB%+hvdpT$##+jd^g~lFf!^BlNz4 zvs|X4dH{!xXak?l3M^Rk1xs1;^+d&SMyh*;9G2T+$w^+cU#*~Ja7My0=- z#_MI&^VyiR%}qhyx|gy$y^-&yqOqedQ!F>PJeJJgULtPLp}zYen>^u>Z(Y_fJuIiW zm23Wp75*(-`Y59`G}BE2(~D%?e!{r|jm%JN{3x1ejKew4JZ3y{Y`X|W$|5nVcPO1^ zD+twzL$c^(BaR|}PvYj8{JF`>LKM!Jl{X}GMzrT9Bn93I_K zaC%ncQFW0*Cva-y%p_nqqd)#o&UZahj7L<&#VZezU{u2T;+^?bi$UTo>34=8A^YeA zP9*&hA%ftIhszj|9wDk*Lfel?lIBcQ4Y5#n+@#pI6a|1vku>Z!*YMgxox;x&J@HPd z2dtp!+ovZFmj#};m&#Oji@i`4TYUnFSA;;8XldG-r7Hy-Q=}42P033uZDTvSPZ`K( zaGfsfx*J|8a&@U1t>{7Rrj_jZtSVUXgipi>_D7QSniy_BbI&JooBI~_en4`SH+HjE z>wI$jS-KRRT{H8^v9RWt$J4n5x;Rp{EyCCn#1EEUH9=C_e*3J<%=y8Xb~Voq7lI7q zI3pZ`{GMocj@neljxE-Y_45}dN^$IrVz=Y8X%bORag$H^1+B<+H<0lq(Aotkl(VVK zv47#d+T3No!BWyzI#sz&;Cun1d9jE8eYJ7v!>Ch8)r*zLtiF3j-tKG1D%{Ka zMq>Y)MnQjH z6@hss@e$vttUv`!U(Bx3wb*xX-qcykmbKA`*#BsE(0ancHsu%0-W@#^5^Evk2c9w; zvEr<#d&;KTH_X|6n?{xxpJAn%YEo9kQY6|l(SCYTDlj&a4QVq0vWxdjX4%R~_hM2_ zt&owga3_Lm!lde$hIeD5qe~^0GLv#LE&3GYh&IeI?LFDDtdzbRU4^U| z`hEv7Y?`A$%|Bb$zr%s#a}K2{rZ*Y~4L@>_^=$t`{+}b|(3V-2hE7L9(A;jn`3c<|SMum%0BZbB`Ch;o(CY zOM`(p$=Jdf27&VCirM_S2J{(~U)m-rxqNnb)I`*bP&&mkwA#bQNZiNxl!DgTl+2e= zrM^G9Tx3NaBgp@-%zbvp&gB%nUc)(UBA)*+f{tGip^fVJa=&FAt~a27z0#6#g?1)X zXVdIb$Fc~!^M#xy&DvM6g}tx|aQ-Wi`P$9oSDysGui<7A9^ULZr<1SEvYhfp8Via~ z6gH*3YD!BtAA*7qU6{c8iiqz}h6Gg^WEtj!b9@iq z0Rf!lPh=c^;b?+18JNHDHqM)9irU|3`yYdnr1GBqfyVuf4XpA_D2DtG^7*yr$?Gs! zS^ONo<)37~>I3Awdq44E@6$Z-Uo(I6{g({?#h3vl-pQA6&T1sEJ{u?MduhKrGyG%$ z`_DD!_tf5PD0j6_8yg=ay*zil2e!Ep2Sfj0XG6bap83c1-%m|bpZs?oHmQE&9l8fp z|0vS2VzT1Tt9J0?{I&XU|52GYSY$f+c|Fg0@WTXll~i-C zU;DEqsksLe;-gO`@l&_n_L$ILsok(n0`!zJKl`VxM1au2b(rn{y~ysbef)*E{_bnq z@aY8pa)tTm*T0zl?0dan8~%%^NvtJq4glEc@MW;6rTXnn_kA`U&r|Nb;ddb{Wc{sX zQ4*prFgg4a3!~d<+f)9N z9Ck%1_0u$*kPvCu>gQ*h`UrmaC!ILIhDAL9-l<{4m48(&=f?ShHLPPEi%QA?NxCb3 zOdY^^v`_xx_sE}T&XC}HmVXpZRX@r<9A5CiYiG7DbVT+kqKeuQ}IV$?qy6oo@*A0{EASE`5RjD#Lf*c){u;lY^NlxYq8#s9pc zjHm9SM7RDT_P-|8Q6>6GFz>29&KsV1_DhhY6w06d-ZYZ>ur=Mq3cdP6!d_5;c@98; zH|5?_X+H`p!A|F>w~&jv3ImuDYIsv&9gTJZst|WaUNrw|0RfHeoJryw-8*hitz^Yg z>dx%F5pLqE3X?6(X*Ao!RjImu{eq4kJ4FhhRL{KKLskQcJzL9Q`TB zlPaoCYCnUMV@n=zT`=+Rk?9SS^d5R=kmKYhcg%5MU2kimE}w(&_oHw&p(#ZdjH|Nl z^0cBdPZRyg&ngBdj1D{C8EtEbSfqeg_iAs*@{}*`0{-7~^MUbX?e4IpI{Y)o9fq{A z>gyWnF)^~xv2o}z)3L&m-03*=IN9l#Sy?z(V0f0Uz7ZQN#~qg74*TQ$E6)7ixz|kh{ zLqeEEP!nCH*bCzG&(t>5#XH6?9V7flJ}oz1C&?DE3h=!Hv}I#$XCsv z1DlJde9Zi?XeK7H#Mze@*5n{puv znr65D>I=WqTHCcY60v&t7TOH>0CK(znknAS@$<@oEVB5N-dr*aQgjCI^`|IRFWZeZ zmNie}q2s-M9AA0eEF5iTJV-YPIv-`I*l-JwC>j+nB#fK=z#+mXcXP3@uMk|8s&*dV zX(ZxM3W;K{%#6JNSN6PZ**mk7)$53SOu5xHc6Fr8Bv5LGlCxL!9+xHYVu>o4Pe$-V z@G~!iZhG4XLtbGNmm}2t3kR-CX%Cn-W2H8@q@~=5XA_Pi6!;MG9^jV*_zt7d^AW0X zd@KnNDTG!q-n}`M{|D~uW{=hq4tq!)R{R@&`@gjr{)c>L2Mxnyzgr-yyj1bk^Jcusd^8p1GA&z#S$zdO`|HlRCXyDszTbw9Gf%p?VN-q3^hN5 zA)%P{F4E7FqR04G7X%hV${%f;34bmF8XbOR9f=m)sMy#8+9mLuUG1LT4T7OJyWsw0 z9$b)?941v*5_XM&R3@z)5i-#vGQR0vewyB3Noy|cyK=4L6yefx#~YC z@^Ml0wVhq2scj~yN2>_~Xl-BGLHk^XAx18y?0bzaZ}8%LXdw*fB^O7@n1z_hbl!K;Yi9J$;C zXAdbo1+xe@E3OTXH5v69C!Nz9AjG*NgSh@|$pb{`b)JG5+IY3_A`HlXI+k;4$!BSN zmQ&o|ajBiTrhS&Z8RmYL!XQu!oU(S>hE(Qw^}lN5_4-x=VmSSr%6{W|M1=x!FWZhh zywaKgP8?3mRUZxa-dcAUhTSf(DI2v$i>$s^3f*`hQ?XVn2sy)FLT4K%BzAQkAI=>K zvb@TzJV>~-Uz8ZkHKk2Lt2%`fN2~JaeMO8S;^fz9;4n;xp5!oAZ`zrxU0J?0M(k>{ ziVpHWHP{4t91()8)hbM_ZxR^19H#0AXPt~!w$EEf8l0g5S|hSGdp`IB*W0N5G!1tb zl4~4y7l-!Rd^_14nB!P(n7HHQm>Z`F=ff%<`fip5MMY^Z$4BQzpXL{;Pghq|RncQu zEU45MvMb?VvsQ*|s7SC=T(VnAqZ8D|e#CZ$H*(w$!7Z3%O5Al6fH>XO7Y z)1*Y(ZC{CKzo3(ihdhV@YE-zLcDAPd=u#*_4D2IJe)H6AEbxk$s+Z%!f%9;Fen9m? zd@sq(NDI4|7V5Ey*|_r+xtPH+UE83d-9C}d!JN6$(ULdHx1@@w-V?D4I2ert$S$N zXR;SAd%VoR9(mHTl=f7?Kre*B^Um<~nH8a#~?aA~pvN5HkckwET>M z3n@QOZ>IHhT*Nn)W{7PcRnVS@ZMH!enJSd6H>htQ46Ap{G!~(m5MYI6MZ-XT+`^=@ zabvl&#sl*;LttE~y$IVyfTa`)kJV*+fI75sNhruj0JMUX9Cx#m8+27X$@uE{dP1Or zf*Wgt?sLQdN-F2p@SQug$x$9l;}1;pYa9CE$(jsO%j!G22`|sqE#mWzl%UhXngt+4 z9i;W+YN_#}RbIyLj<$BIu+?Yo!}E3HuoLKEGX5Qp$)U&w>fS}0+ui^I9uH9An+@x6 z5U!v2=I-?kA#m?<%0;L*>B?Zqawt9Ddge;mO5n3v#OAd7T_tsI zs(!u2IMrpj2fOj2w>qy^R>rP;;C>6tX?6RgYXtl1dzU#vLqjm-`8t7HM zd3m)veRGV`znA|=YUZpPMS37d5kpmu^O2v|)Iq4ERkhm!?nSPJn*Lho?(5u8scCkv_!Xqdzy6{s5QXLl;&KE^HO{CXSXN%EseY5gM24T zTjSu1%{2yoy3tCeqmjjfvsurbu*O5mDUQp282NRneP@W5U8a@A^`M9OSx2CG3#zh} zwzev*!v$~vc~0k|F*U}m7ooW0F*Y*dir)wSI+*lh?|r>a@x`nx8|{}sZzn!kHM))b zAR!E+C9X7B5!1eSGb5ZxUmS=gmCT(NLV243OW?G=S~T=xbGtwS7Ng&6C(-1f@2VO` zlp~)O*t|Y}JojezY@POPTBB#)cJubE=v2~LIxdmGkBQ1shU8${$HLikuY(56)Kj}N zYlkhnC1qqL1ZM=5WnAC9)^^(TG`!i60>`Jd$onDtb})P7R-tat_yy+q1%5n=^C|Kv zlC)-IEAjm9yeexeq71ApH7+R!~v^w=AMRVsQWd7#E+G4$OxiA1j}!8L^JB7@vT_Bg1gwN zX*5oJ%DxpCe#B)GelW`gEHjE ziqpJD_*}4hGwqvshgt_$7Q^u3Da<(8Yi z;GbDDgj6jwyu8mf(Q9UT-lkU_v#HayRY3oQ)7gwsPQT>lBR8hA7m~Qa`Zr#!ud~f| z^~e_wA@U{qufE^p!O`5!oYzB{A*6VnJ1g>{g^^oQJ{w#U|M~0EZhBzJG zf4YZq@;LojfhpcViHmx~g8s4%V;OjwC4Sc-9ppMQ1NxlK!mzi6CwfM=(G7nRYY-P2 zYa9khTPH?kADT<f0p=nKY@i;Qs3F6sFF)BJ z-{Q!rja|m+VZgju9~c?vuC1@Ebaa6nj$`Mp2Jxt?#)C4 zh<804{f!rKQWU&){1>vq7%0*R04%91Zo+fca)RNVskkMQ{~Q{;_}&hN@Zi+IvQJ?7 z!*Z(X_kww6M2oz$vlu@5&2v87mgg_{QPc7o4aWSV%Gyy_^6jr;6Rq;$e&{^o(Q}XA zjq{AQz5IaSLEq@#j9@)q>)l(Pcq<2o8NoGuXvd`>lpbdXAdJUy|D^GLmE2hT7wV?o zwKuS(!%7oVGchcuSSf}Z&b!ooj1AUg)AwsqfX>@lFBlR5khJ3e(e$CuYpz-Mr-~`K zmhO{D0ZqO00;<2chT`lV>y-U~<>iW7xa492v|dK-zwyl%|Cd@sJ`36VACg_S@&KBa zmrviqRyDaOp9bR-HMx0jh`$#}(yWV9Z0Ng@a z28}gfRNnf%f8`VP$^ZO;&C{qQmtp=ul%LF-B045?U<_?OslqG=2z`-Kbf1UXl(}Gv z2Cxaeq+KNs}^Q-sT{U{dc2x zMq=IIf~-C89*ny$1V^)ffS!PDV_}$D?u&P850V8;++-dEKq$S)vg3B%pX;~z>IgF$ zMOfu8@BKO8evDY>L;ZVQ>uB`;)zzAq`jpRq*&s^R_3zvEM^Lo!rm5ovCB; z0wyNNkYlCm?MH{D*x0&5FFR^g(p^kF3h0NTzf?|eQ0Pkq80c%}@ef68e@79%bJkC{ z9$7sWEXc8*i*?ZUQqBfy^+zVae$e1zTxGqxDcvu@%9g{Oz@3m7H(te&$%BU%OE1-q zzlhquap_!E11_y$noF^g8KX@&n%%BR`_X&%*w9Jg6TKBUbM+4H&pcAWf8>$gzrEv+ zOtfD~r1$BQ|3wPrpB(;~iHh|X4)ms028Pbs28KrVzqsB17W%(^o5(%4XRf4vPhncW zKVJO@ZofFwo7r1i{p(E=KYwWb?KA&A9JJ`n-CFck4$T{Xx2kB9kn$5#EuzrZX(;}%| zmd$eF?A~)jFIgC%(MLoR?WCNx(^+|bs6*6W^Ss^e0|hZj{un~a1h2YG%xS4(oC{yb zoe{gd&B?NW9hY?Xke^ry+o2WWWak&j`Si1bZyE`z&;ygGDi`srtE{N5_D}o5IpSNj zc3+9r$=dg(&cQwuL~phPg~{HFS2nfzo1cjBpphuyrb{ZqBP6-d6f3Ye<1Dyi(zjEP zant;`Kw-^U7kmR(SmhT^U~wV(Xi7bTC_t)i*Ol{OWcV|*)eNuDxPx2#uf6>=LaP@M zb3}fA4EdLuo<9MnUoVt_gQ-a=)|Qs04)k`0u-AszI~W@L z%bsvBTizf4KvjR+^G_G#{f86Zzx(C46aQttKazz1Ip0s4+Qa6#KmIY-pT2UxQ4AJe zA*H~ce@(C_4axt(96ve#hYhr&tZALW{nP`hjd4IA*W^y-VF991tff)3R6a|~4cGS@ z&1=?Ja+nD@gibg;@&QOW&2-WTKj=^5Vo793$nDWX1H#lmoL65%XY5vyzKXy@~9m--&g(~{vsPM-(-LwrzFMT1L zkxy0K=F*zyPTBLjrG0e$BlY{vxix}a>Swadxp4RPNq(?5?pS$nPu`aVvrSH2EVX`N z#zZ5|^xe7bH0~{fd8;D zYon`=#wO}6S@-@{i9E;dgSc8mOTM$(>HzFSoz z&Sm7@EYVwkZwWXvGe`Fo$k#QaW6mN+GDgdgxiO66#!&aNQ0J7?YSX}-(d1_{lxF5e zTCC*|1Z1I4q>+nn2y-c%obW`p%?RHN7;k<#MHLkp%-c>Lz}T<>^S54TO;GBe_D zk1{h>uG=J_wxHtrH6T6M)yg4UIAJSQPw695Yd+RFk7z-*AATdd_#u2elgd^NITZP; z`kKAO*E&{EhdoW_)Hnr#;u>YgS2-QA+&=Uzinn9R;z`(u@=;7F3~S>?v!6OX-UX*F z6o$|$hSFE1z+&;@EQ`8Ur(<88O2an!e6WhYDm>b;D&Np=n>k;~MUWCqdLRk!!G&7m zP2R8XXNaG)KlZU1@O3T6`=yNZ3fqqn6~h*n+RA(61 zsL)QVx_JvDRwwiZD{ByEq;4i-z=!coeTjdm|5qZA-t;F{?BsLEgn{Nzr6Xepkzes0 zRBWKT#>Q^_xnFl_cxkX|txgzeEMuICQMRo)0kRtG-;_7#HGDJ5i}F;sB>hpg2|ni~ z?;)5T-LGiVilA_x6X@gHj(xy=0FfY0A2AwA$#-=7{Mq5Csx3?rPuXVrmN=>n4lx`R zE-zNg;0z5$u=>IuN7!GO4(l-KM;9?pv1*pm&oPbDOO-$4iY*oww-=|0c$Y~%_G11) zmGHh8?Fn08{QpPVR{-^$ByB_R;2PZBJ-E9DcXto&?(Xg$g1fsWxC9CA4#9%M2YZ)I zHoM8)z2CoTszRk|X1b@R=Xs`k`d278reImDf;h!%MmwwJMSz!dC5E>;S+G-FW|l_| z`CfIsrc)20FqbM%_CoAX-^cOrje(&T!<%7QUwml;>mAC%6s+c%jK|`Pc0d|C-?6%< zoDDVXQ~cs(l59bBa84iferSHk`w4xPffI`jCx?{__?KNPBzkL_{dt&(>y2s_P+C@D zU)K8GmQUI)qCVOqv6))EJ4R-;2w^HfGe&ceU4al$&x6}vRx}x90WWFSb1l0VKdh2q6VMsBP(H#ok)4|$<0BQ@B&Y3`yj;<$Nk*w&^( z`N2k!OM;rqRmI$yZzy!mK9h_&)1f~^RIfBj1pfdz=E)f{?%=6PH~7H4yI=wYYvG-! z;mg3?D4dd7eSOfDcA`hT9j(@y2r5b>1A(632gln?cjUz-VzzU^LK)uCE})GZwFMAw-IMYk}D^@~j3lvqlNV%CcYcGIw^{SN?8 zMJoCFivnx`#8g*$#!T+*%H>;wpG<4HfCAPCw1^O!phdn$am8PtuZg!dt^iAVgjl0< zj8c%Q6`B%_P09&`K|mSjjf=lh?h=Cu=^fG0R>A5!L_7oc!2*p>w=WeNVU3v6A|>?s zI;WlN=drN#U^st#Fta$bxTR1OIDFPBvX6{z3DQ<$SuQlaMfl+`uo>qvb+LG%OYJhV zX5qbJ>X$x>3G57W0J@ezi|#kE97R=v$aZz%zGy~z6XpgqF(;#ZbeTw!kE4{_P)LR*!dG&{Q~fOGt}?<;ki5@?X<|mbArTkBl*Dy|6xCT^MHR2 z^juNzH&yj~NX1ttKf{6Z=2Pbofai-ShjdbXAN*%}0`ueGb_N!2zmt^T(h7hlA)e;` z=MVbFest`e{?!e@(^UVbZcu;jhFaIk%Ffo#M(5uX`=5Ix{7sL~&)fc0yYSy^>bdvl z_79--Ur79CvGIP_yP=7>fr9|N z4oY^n9;V8+(15kRJH}_53{ad@5r1-waZeQE7l-n{UG%rH|L~B%FvmkhNwXJpNG-P% z327j;Ied|-kbID9W#?~PM5;%blW3}}1SuXaiDi^ZD>;=oKn%AeH3L}nHN+8g-w8&j z7fZA&M?~TG`Vd}z_*@y$ZdTV6ZrD7&=k&!Mj9s@6B+LdDL3Y_UV68Avo=gCwDovq` z#ILdf)tHPG6k`;tM4qghPp8BgirdQzm5Yg0uz{9Q7G(AiruC(8xV)eD!=CvL4@~K& zuOyHYVS44IFT$%j&qX~Yyi8L2E$udPL+jplqJD%sZ6My%`jV=TA=K8q4O`tdW{$09 zm{m#zeMAJ`D*$;SHOFDNO0jl{StQaDQ@r{D$xyvZMIwfpCbmv~=S!>$@O}6X)epu~JLV|#R0Ys`dP$C*^jBaEOE*2dVU$&K# z-x4K9Os*0V)DHiFo&IHh&%4*kyS9P!Y2Q7!Umc}?^;O@F(q9m#Zx#{*xBqq{VMqBY z3W01>p>c4m$O{3p+|;8TuI)AXVeHs9UDDk z69)tR|FWx}IqQ#IeXC6RudeL=lUI6H^PisW=^sDBgg?9b24Vi#1iw)2ca=R;_#eCE z`9aXXbxHfz9~~{VzJa-&&Oh?(7f-@Gw_m!fXTNetOi25bdipiKVIY9vhgpE3y`gF1lc2;bQQjsN?74 z_13z>_T?1Mi{z|s^(upp4n%RDOGH(sm!Gk68}Nu^;!GtIOwCTdyRBzv zsXHhfEb^49my8bp!2CPe{th*N(CVL9^BHgcq{~H>WoZmnIFDm3oz2aoJVHM{RxB%? zcj&>Ixp1EYeaMYwMBNTPM;}VAym`1xt!(A4CR7u9FLx`e8)?+H?1mj_l(_^qq58(4 zVO2J4^Hnc&GxO9A3M-K`E##RTHr?%QG0useFC5sST_M;bTdN{Fvho{-(L&fJ!6K{H z)Anx7*D=*c^H!}RxK z+)ES@%anYkvkal2d<+KUp<#@SRgfb%_c`6U>#?!u-s1Ur@2QPWbazh3kLkmQD*OzK zOD@GmgearPOU_;rFR!BmnDGL2cex~+JX5G?g8N)u$!>EKw5W`42RfK1X6cd91Cb2w2*lyph@0sm{vyyGo7U?JQ zJ@?GaUB%dHXYi)OYfIt`90Qv~--7`;u+tbJ6A4+ngGV!)%RYinFGKQDCqx+7hp@wP zaFVlm-lz_!vL#i|!qj9~s%=?!TB;S+F9pvl-qsNrAM|mFpxGqsp~e$2qT1`q4mTlr z468V1__c^9iaF?^RSmohq_5C5>Y2k$p;i%5To`J^H5?H!^$y!5)O8q-t0tYQINF!% z%y#{1Y}7g<_*g5WK6Fr0`v|KsVaW!5HuErg(WEi8GhGiE(m+d8-%rcCF~x>n7*Svx zAFuL)!&F3VHzMSHyANRxd_agtznzXh?gYeSyyPLw%Ts&Y0vkwUbXk7q(CBlKeAk|X z=p*>oGK;7)M^fN}F2u4zVIlPD-sW6pT~+3Ln! zI)&r~N!JkR13gg(#AALTW9Unb$Uzd|GSfAzsOIBE;d#ligea5z>$aG$WpOU2m>>7qTP^ zO~o__tWyp{3k+a?X+fR?C=?{;%N?+pW|*sH8AmOnWDPJEiv<^P522(wUgHxIr999$ zO&Owo=N>TU9ghpA+FiTa$*j8W(|#DKk-=oB3U4eC0vxU)qGXFbgf+_SMrcAfF!r~@Hn%^Rvr-ZsJ9RfWUZ`%#GcL&~T zKnaZ3xDmidtCR(Bk$E+dsuCAXmQW|(0~=3V35FR~!+iH93-J(QYOT)Rg(0JsuFL%z z3WOA=d#;!oyeT=P#q#B80fzotdcd<`4%pHI4-H4=lxxdO7wAahkfUqursn$Kb?RHD zvjw7I_ojy`5&=ZZIYrS}$sEG8Z8TWpbM6oP?kHbSF&M({9J{#nAf;=3+w`M+sDRJBD`!4F^YZys27M;lGwLe|2ZhQ>CPV3>$kp71gp{BlNgb zIxom={pfG@Uf#sEbDfY_k&7w^8JS_4Xd4N-8)yRMQ{8l3Jd<^JG_wjX6{NoeZf^|P zx)|+SG>4)MK5LHBxoUH>6G5ChB(2&_g?#;@1R#u8hg%?Z)>P&N^QMOHSr0BsJR^{a zHg>y<@nAQcAdaTb{UzLR3PGLwlXK?v%!Pe)l>LsYYS5~?5hCuA@Z!u(hndtVhSP|r zYobF&01W3h;R!cx+LCKAyoUvQ8G(<85@Ne<5*tsjB5fXhs`PK8g|D}4_Z6_Ws5Mk6 z%1>x}s&_J~nL!!`b-SX-NgEJpE{P6-GKWVSt=JfK zHIF`GPRGRR(R*K%5$!gm0yQ`?Cg;uVY#EM`l$FmC)z_~Lg*RNQWA8!&$;T2@tx&!t zMAd*EI5tq4_ZW{?UmY!>G10pSWYTG_E0U3a@FSi{FO;xcv;dE)KSglEZ@l@m$~(86 zRr%>HJX#_kQnw$-b$6RmWo*x3gV}%=L)}}zE2RAnNe~hOh#^qodurBhVo7ZpmgU{M zi!acZ3DRzGb}zjj+2B8P)II{rD#R)_ZHR|3`3{oQ++#_?aFXA@h{HS%QMlOAdKO`3=(^K3cOP5)>vw;CwJHom&%2HBLs+;Ly= zE0Gc(tHsRf?QPcyz0=Tx?Ka5~0|x=Aax6T#xs=CW zdTov|9dc`5gDfjr5`N``@?FqpSq!{zxLso7ea-tV-j@_Au0x6nm> zEw1%YJ3XL<8~L!m3-G*B#^R!O-$<9h|Hn%G&0hY)l9`0u;KN{XDydn<)Rhc;5qAlm z>~ER3LAGQ&=UaObc=tR@q$j0<13asl+nJc_`l)i3A>Mz0!oSt>9~RrNmLxdqd0t`u=hRs!{sGL|>&3e%mZ@ zAVYWLGE`u7WI5{oP=ob^XlcMrb+nW^>yddNR;6%~|8=kg@ckoEm|rDtn6HORIV`v9 zz0~VhN!G8a3NMTKv~FDkU9H-_?sJ3PFCpcNAqBG9L&?3aAYnjs>o4MofnZQV=&qhZ2QJ^Kb_8|}KWL4kxY4t1pLqIP!m zBgt|W63n)Z1L(ox>4bXNi=}ejuEN}4rkn#s^4qrA3vUW^K|aPUb90)|Z)&%6o=p@N z2Nk>NtV%;bBdmScl_LOXbi29eoTqKiTNZZ!YENu9`D`c<5dCcUm3mtiw1eCDhLF1jvUIPJDeF4|+|s~R8Kd!osY-57m92NAIL6Gs zH0`z93%yrMA-aA1!g{>vOY4dJpibSa5boh#pCdx9Wa|o!C?P#C&gjS}TwE5GbI{4B zzQz9j{R4dt6cE2cyBrrmHmfwf4o2!{4^q@TAf|B)p6ZVK8x%yWsE?OHtI@~sug4dX zYPhz0?K$;I5jz#IsAD0BW#Vh{LM0mw^ze{EZLtPZH>dTQigq|YeY7s3jIEH1wUMw0 zi?rWTj^3kOCnxr;6Qk-Gcsa8Gw*is{pg>N&qN|jLN^zNZTV6N7fLf`EU^OnIs7- z8rV12eu53nB8Q3kO61Y}tb}RuJ&&HOzHu0?hJ*p*1&7*6^vN7$G^+)3$DY#Bm=Ww3 zNLT3m;bo_Cjz~a~jO$G3Bs0RAHEOLD!h-;R+7r=UlYuL!YxD=;7yS}!b+ceXI6$QB z$_}Nt7FM_)T6Ud@P)Rq#G36=#X^{=wK{qiMxM~C<9-bd!e^&@>hTt$ zDMX~j(q;Xe^mQXPptD1cNKuFSGFOa4x59df!7+hxEUW8MX}isqa`ZO#y)z|+!54$X{cL(3UR92*# zPZcjw9T)AUDS%OCDHxglmJte8g#3Ncbiv1z>pqc0s;mTmKH>>e>aa5+)CQ<#Mi)au zA+r<8d1begTj<`IrwY0N3hCVVYR zuQl@@79^m=T(G4yioY1OUB|%QTm=!-zs(T7-m~2o#NOi4h>1^hO43T$t2=)TL79Wm zDo@n)!GZ!;63DOE$*hihac4k>B7%&x5R&*vf3CSCC9QAFR->!A`w?lns(X13b^10T z@7NM%WvS9U%_w(&zZ31uy%!rJe5w-XQq+$)QEUb^#l3vuw^w3!dPiYkKXQRUD|9R9yMfZaKt$Pq zbt%^+5uIqTdIGQV4FGOJ2c422(b0mIAhFfh?0v+N0>k{%`q&3aR^iNHc(2%^ zcLwU8z-u*e>G|w7rWs(Ow8TBuAj}JLBVXGgf(_;D+ z4V0bHIxR8S>*fP^jHgz6sbc`kEp1mRu_jj9TG}o%jOH{Vn_O!w^ zQ^5%(LS z6DLqEcMmP|r6w z?dvURh#xQd05}yhhIjOX`baBIom zDrgf6U8JTci>05t`yq?(=gVZi-(kML5tN@{>o?T*2eSGFL->w2e;%9v|7+|&v%vnl z!GFH`#`)dg)c;k$@L%}!nett;3cw3LZJTuv008dab^JGt{^bJvFY5dS<2#g>6bGV% zYrRyWbhXq0k;a}cd-F!wL5(9wWJrpMQA@%|+-%qLb@2SLJWZ>KrL)!6@W#nNQ!HVT zfryXaxZatcPZqw=8DpdPbcFq#ya@Q^Hi}85%wZ?vp(XF&PB(v3v<=v4$}w;xcxVMG zYCbRs+UQ}vU2Ye3HWj;HATC~jwD~YNHDmZWW|Y)h`W}7bo$`^Culf6Rm~X?Yb?zIkgHt2;PI$ee=d5m(e<(KPN9aQ>Cv(Pb_^Cj~1r z^&q9O^?vjaXVBtw;$-(ii;mmvB<=$%FlvI|0eldvqw-bqyo(;7%2ksRbC?8)V0_~G zOwLd!K2AaU@T>1`?>USB-$K^h@e~~tq5=S5{h!wFrycQ!E&d{caHy(fJI?~+d0e=1 zwfWT$v)^*Uve4?fzB*&jk|B7Ki!mLFYMhP+p$LzReJ-s17BGuYyvm%yWMOevT?;)F z7vKfA7eHR&>o9u+%QQRL;^+t6JVd1k_IVwgImD7Km#kby#d>DXJ0>+RqWC_31OF39 zFkefPRW^u5#F{Z0Zf@jgW#^-s_3_5GW+V>G56X9y4xD4e>eW}NVSb$B)?Gj(U4|(X z*NL+NYqc;8tZGSE!fXvzvReWevBcCQCRq(rGK^cD?uelfX?h~~<6m9%uUfh8=Jy?6 z=J++6NXa%BwM%)&4hseb4Q$kBEVoR(wt>&?p)iU0;gw|H zt~^v259)iSQ@x@QLq1l&wUD?Z?h-8lJuF)axKN5vVg(rHPcO`!*NVN0)6A?Fi0J9Y zrl~Vg0jEb2xiR$zA#Xg#H885J9&xIh!{#~}^iX!Jz~HKJJd}r+0kZ6e+Sr{g>-)@A z9;9DYAIxKfExfh^Nu=dmjYd9K=3wc5gp2Ty+;~&pNZ4w2lmJ*mY~!MSU5S zDs@?q>vWKLUuI7#II>e3W;#14h-C^e8jFs`SA`1^ zFID_?d`M%Fb|WzJ+PcVCK`@exvCi^LVGQ5pTC^fV>EMl;QC?H7Ld>HU*Lv$6UWt~0 zCpF3NTGnl52{@jFf*kOGoC<1~sCwc?LZI>tCu9jx3k$rm#iFPWU=r`fk4;D?>! zUaGNAITYej2Abx)MjhtUmXAc;9g@99qjNZASi{=`i=8I9uJ~wFVo%ucK4mb^M^vAP z>O@!0a!wh%w=1oGZC_`gy5V^kG-UFoN#aZ4mUT3U*zzNXfQ8l5qvNX9=x~I9SEI!+ zAQhr4KGByq5>DuV@3`^BiVIn?A!97_W&|9957G5+QZ>z!tI#A{PyaW)VqH!imxdvS&d7o`_ zD4&=V?@+`Xk7Oaw6&Tu>KH}tmtZ1S|STlZ8FLIVpkb%2vc|i<+h{XMgEXbuxQq(r_eS3+&u`iWr{5&QC8D7S*en+mS2sRtC zcUYy4Ren3k1H7=4Ddi=7XD)tN3ttWBp(`#`@U_!eAQ%d1wE35^cnlK|RD3))>i(6gL z7qK_s#WVGvgASGB7k-|_B3qWx6*dD##3Xk#2G-kwLYq2PMeasUB@Sku#dFcSCFVF2 zD~%~mt32((XYa4rOPvQc#9g7!q;*MV%IYmj&y}w_@T4VGk6^P5Qi)C2>Acz+z z7m>g=%<7orsyo1A&B)QE9@`iB@k^l687$!wi3t?Q++oDfX1?kt8LawDfXU>mJ?n>> zPNnA=OXzm$?h{p2%E&ENE=e~pQZ0qy0cKcmQlOh%b9tDjL6P2dgBH9;`a$GeCGQ2a{y z67#+AnvRH^2{!Ux%frBq+?rrTeoX*`v&Oi^tKn^*x~>v=HSih{o#AA+^C zCiKYt!4|535=7vg;M6l<=V%lZn&%2^Ir}qv2k}lnwTl9`vJLv(XMknSf?tuycteVW(@5gomk$BfI~f*dmH!PX}Z3S)dr!OVJC zNstx?CU0&~p6bDM8}LNik`c`UUCo8zHR0g$r=p z&6$X7IG=8tprUPCTWPw#V8iK=(Er)dPhN025rBK&dd#$VNXmY(QrLdH$@UHIGj=!F5&dFja}C*1WC_ zIh2kV9(j4b7MzakFlM{(y`@;?;P4Wu<75U_0-_9Ag^9UpK{RpCB)0f`Zc4OXpuMbp zzYUq}ak(~*0d7t`Oy9PLLuU@SfZUm!+OI_M_C03(6bsG0SWi7=cxJF$povA9OK(#} zH^3VNWj^J^!}i75Wt8-_Vv2Nd{*YlqLy)mMKo z#eZe%{G9Xg<7=3|F-7V>ljfe;;ct`Xe(sm-cTnt)et*uQ`=5tk`d^0lTUy=oaL?_h zpvZU1^Rqf#`-be^Y_>Jzr87cdNcK}?DRh)Fu%yg`4*V?wZuOr68_yb ze$1)*j{-h3t#N3hnrGhi+zjawl;yix{Lv97ch#5Qg?;7@PZRm` z_uq#5{vp)$S33KPoX791<(baDij=(6eR^m>&iEgqB0s3&?<4;~S$+|%IaFPgTW3M= z%xve2Pn<6n2TvC8Z*=pm5sze80z%q8ZdjVD$G zRv|D@3$NT?8Gl&;7_C!1Uyh&S@M*TE1d}tr_}v9K{zF3~AJb~;yu2muR)EZ71IB=tB4*N>uF+=T8IuO+qa+nsMd-*)br9UsTFIdI=socpuv z^WWT@e?6~7s1l!4BFJh!7ql!T@7sNx(sMSPmp1O(8Z@J;ZkZgGTf7MIEK&kn`PA7z zvT`|$sO-NQbHJn=-?Aq=iSKxDY(?Ox(orU%3ew5#Ie~uAN@W7gET$@`Y!AbB%e0~z zTZ+zO>j*e(h;iy{L9jz3`#Eq*C@RSoo-5m1cZ<$-D(0b=5d>*c-Ryk}8sxC=^qHhM zcke;L#dQIZWK)Alt(7V~^kFr({})zU4(br3mwu}Bc!e4-ABUA46Rujsaq@Nox{SQ> zzWN&tzy!zhBDrgGCEIeY!;dovr2*6Dta~L(B96S!>TI(bW%WOri&c@V9oa9h*%+TH zLBD>?hppMi?z%C!Ior%n-!{Iiw+(a3gEWoQ_6`b`VYCZjGeb^BljeFt-HtgN`&K*< zydaUME`f~90;H;^Ax)OUBw=|j_&j`hv72+ys8>%V@LxU=tH$GCNOcO*iIzu>d^xXfmoCy_LHTLz4Fy10$6hKIZd=@Fio7 z6ANk}W(!|%&PLgW(Fv%mp0k?#(PBr87E)Ex6zm8YBY=*ytGh}Zwo&ZsKs}5~voTxx zTsQ5(S*ive$$8buvJD)sF<-4R5<(-U%GY>>NyC-zN&{3#HxQp;@totKZfb?73r5u` zd2M8uI8#5xg;WY3XeWl{x0B-Qit5P7Qr@|NzOrRVKY6)aHc=;HoO6NWodqq)$vbhl zXCc5ScPBuI+m^i5mc3ZErzpg*Cw}IFe71`%2;1?v#%ONJWF5g zOB^U57PL)vY5XO2BO7t+1daqX{Jg$rc^!PTc2kqwDb7DyVW6Zpi%LmoXqy)nn>0(H z>cNBHb&4qVWu7srnaDE4aKN!eR7o+#JU@+{Dfm)!@%U@brnz&h(e@8a_QX8NTRJ8K z7H=Ogd7XnK2gMdS=bxBW9Pl%<(t!a^`3%VK=x;-|Y0nm{tVCb%#H_4``4qB>sj$UC zR2pZC;^05+H`?qf(8N~xn#roTOO$uGA+z@O9nn@{5joVvzzb#D1PammjYRR>Q|FjB z~y7UHHMC-dr4`Vw`&b|?4w$>JT5=qA$=T~4H63=FN= zonEYg<{`Y)3NNiWUlML=t_O@b*uyO zs=Nk0NQ7(n24Vf9y{S2ExR@{)sCjXL4ABDtSNbl&TM;05EhDHel!tI>!t7Dtvd$~w zd&Edd3Q=Te1WNdX!G6@~Q!QEVIuGA5?k?E9u|G^INe&<=rkU85QN^WBzs(ea8{XP_ zm4Iofa!!gmdrqhgfw7;!eN{?1L(z0a0W_qHR*V1oyuG?t|eNc6Yq=t#7Spg9dy}HL!tdDRg zF|ICLCxOWzp(fHWt@)MnAro#&Se>Q_V!vvIFCOiH2+UO<>h{nT?>59_N3RLBIrU`E z)QOK$w`pUGn5{PQdfms?`zX&9o%+Mgvv^dv-Suw5G_by*A>Jy}8Nl1-+YC<8h7@3oMzp_73DG`03^WJ3eEA&4}9XRfz3 zXHz%ZNs*ON;@uqJq9I4(S@JzDuDcv-JbthwXXm@Mx7d5UgvHcoKE;42b8o>!&6gnj z8l=~VEnF&N#u+Ng>+jC|<=b7q>O%uFZ`;6YZAPL_0~}3sB!4V)q<9;GIWxsl{=}^6 zC~Gr=TRLk#7HL*cA4CDfp(emhJKvVS{(2}*al0qhBY|!`YnxPFR5n5*O(W#E{i<+# zyO5+5mgW)cJC=M-`Hr{>$$0uO8^Ck>eLB$(knZ1aSNN)58AM(Dxfm&o&Y_F&9 zpu^Qrh~&h(cv~i}#%IU#{z`Vi>_u4nTwqyp9YJaTYqUg%(7AtEdMa#~@5 zUpjmsB&IoF0Kg1i2M&nm(8qu8HA?4Q(~l6r48cah7Ja+-)TEfHi= zk56CyNJEU{h0p*P%(j8}(3B(8P(nm325W-WX8{I28$DEi=gJ6xR~2tMNh%o!jkhpS zI;R@n8ybuOTktsAzi{EBJxS&3y06+fYoo|#gKSa-QmxWMkh21m-_s-;lK4`EzBgoN z{jy2r1vmUNyLKTBzp8f)w88XM~BXlol<8fqH@#IFPE8WPrOuR#A$ZRXi?7g{O2 zf_j?ubL0FcU-j3S|JxF@{(MiX^-aV-E#eo^-ftF>;!6hkyeQwDln5b@`}?O->_EKF z2mSx1Ji6a0kNyYc{o+DrK^1~U{`NLgVBCfvp4lEfkq}zUWV|;?9iJ|{CNr6J=)+K& z$q1TSqiETD21@pY=_{FF`PUYz3xtP6R6+QBCXo-zwj)q_rAMWY$IfNjXrsxtOUJkF ziKDk3%a@#bD?1;o>9yZ@)|Gd*JFMiwaG9P5CRclj0*k_LDHlMk>d^yE#*umVZa4}< zA!ZE_`gwuS0@^BMuE{1z$7s`Qp+J@SgNu{C%A%ar3aIQ!0)PkP1MCAMbbuRv_4FN9 z0RWIJKe%`vKREBWG(6ZO626WXI7sjiZ?|UAAmi|d;AQEDR`qo0#D?(R(1{8Uvip)n zhYq;EHY?uKCtQ1s-cKn85XnGMUHs$!MTEaNOl2zFpW>k@7t3WZ1Wm zU2ddyLGe3pI%NF;i(X>;AqFW!WE--58YB+LU&-dCj3H1TIYS zrHnyOktfM8b329|3r9=diHEdj&K^P%Gm=Xq-v$@@)MTJ_4GryP@FSyUVoXQIYC zMPFa--}e`cvnqfl73YQDOyGOD0q6%H(H3+kckmqfiKvrN1C9a1iSYm$0ysPHoOH5m zkO2l)2)?>~9aG80HvqNpK1&K=ZICZVpZeAwBRZ014BjfCcF> z0qk%mi-PVIEzlEA^5p^<19%Y{ZNc6HG^G;zrVZ*g1QpjLFqmxZ5!N4hkx$P(7!<7y zfC<(B!+OG|AvbuW}Zq5x2-sIZko}4Ug&|0{>Wn#roDOm55GEHDp z4XIG9oEzFFm-J2dJCYiR5DDM`huRYx(+=Y82lU=Uh@GmRY)~if)z{zV#fY|?BV~VG z#-N~vutHF&F3~J7XEC51Ow;8vOACQJhA6iP_9^ZS6}aD-e-J9O{|g=p0CLe4Gfx4b zu}{=q*t2Q@q6Y5}k}WpXh9;#kcNeqvS4R$IhLT-YZjc_5Yl z&V`PX_QeJ&m#A3IkQQ2esWox?17aWHaAG6kMh@Bg>U`$Lbzov&ckj7((Xqe?Sq;mH z4X$EWNvLT>EN?q!jtP}iE=uP;dh>Y4)n&gJ>rG^RoCLZ+Mljc2VxE;k#s{#ccZmxU z8{(lQ{|bQZ9q&$oHc7}Z#`3}vZpVTMHT!)eEM-@8!OrQMbWZopFP-9z6`vaM8{MiT zyu{v36#DP(07C?k?J@Qk)4#=z_5c*hHPz1|!?K=IMb6go_DZ?z#P8i&)+41=5v%q? zZ>Xm=Gsl<5`0%+n<2^h^d^HN*h0=Q|DeB5PLE|%HW9Cv*t`ApmVUlMBZzUsmzCHU?k_HD%{-uy!Ee;)izmf&!x;d>lY0Jp(w0S3gk%28OX zxf^l@fx1YK%asKy&6@!JmDc%XA!6#lA9=_z#+Y2Ih8lglQ4R;H7)-+0HjfeB_c*S&rDzp9xsa39|jVIv5{A^g*8sn zgkB-V5LCe(ZuJiYPG(Fn=#5Oj;z8yl;3M~`PK$P<1nwr08Y+Ett6Nw#3AVll z)qvG-Lu61U;oSWmAtF5xYrc2Ruk%fNr?xToZOIjZCl3DuV}a?)OG%Jeit%cJOHLPc ziMAY*yKOG1(Wf{`HgR>LQF@7*JE7xcN9t%fiHfFXec|n)0-o~fp}}i6WEm`5uXXE)FEuy(uHddhc64F#thOPd6RT31+- z*otmVrdvJRK^O zxQM`1t1jo8PUI~jdEwfo5_E8%y3lo|>HjqtfBGmLD)urMhU4|wX>chd5C zc%Cyuzuv~G?S;Q};k?&_Xc;ORoZc;`x8f2OxGXQlu0K2r6tBETO&|k$4a1Qk&=i}tPV@pKTPk{3TP{HGu^X#hkl@&K{zUp~Dw%rvib`|ct zr82z;eJoRVuwB^*<5<*cK!fl&^?~iad%Z(fWyb7km7*PGFQCNGvQO?YeqNg<7S8&B z2Z>EJfApy9;#w?7O>KCLnM-V42i^GE`qn8isO>mdB#|Lz9OvcBzH}eyc$0M1DJojv z(h-a;dLdhC3i9iCes$RoRBAyiw8_y?HLR0C26Sy#=jn%TPG=-sxp}!I(^=-(!_jAT ziBg<48P2TsgQs)bMEqq&bB}Hh(iXSvPgy-~IH3FajZj#xO&3nvy4o)|KdWIjxZ7Wt zUzU2hzoQy#v1hs}Nw$Jt-ZJ0w5}f3>fnifRM;)5g-C!Ito*02Xo9ccEsd+rwOuMoq zBc(ai-^f7zUSqUfO5!pdvE6fwpX$P-r!iM0zZ~HR%CP)xn@xqo$>32of!O+iEOxlP z8l5Qr;YrbQjQvP&zsHWZ4P{~o2qb2DRp`>d;# zO@mpT3%1fP34r=jyP6hBN*K~KFLu=Xa|6xE{UPnh=NDV!BJwh-(vDVkhx}u)tdP(B@`5l!rw8C zVEe+%CN^6^so-lW&cH*ss+>!tO6g0bnJj#WB<2itEg-Ew6XUaGZZdx& zj5F7Xh=7}HrOX?}lyR?^iHX@-{v5`-eET5&{b>~S1^^NZ%nmjOL+MVhEWCE4!hAZ- zJJz|efN}HgrH#u1j0p4D3Qqou(asMBKC#6>%I{c{I%!$v^s|Wzbyk6nDPlYnKR;x% zhkY`$ahz+KX{gMdoD$BnN+{DcCy~+7ogOe#$hnb+@)tLw!A4lA9d+08?F9oneJFg&Ds3<4tQoDR!~5o z5{2HNOFdm<6mz@pl>hQr&0%rMI%;!?Oo+k!uu$dro@&1PBA9aA(E64mqY^t(?OsIj z>ZIw7th#fGneHT+qLrM_c(HQkD}nsOp2H)Q4CcXGq&zFElq$|KJqvaZWtZvhLAhMV z+YL&I*SGGY66@6tW?_$0jJl4P9=F?H$0Z`q5x}#~MM$km<_awg!MpcUcI{5Y=k8-n z25gGWl(1fYCPJ3VnAWLJ?@QBRI&erUGS~c4J!ju{9ec|@0>!3%(&01_^L{0w2z`q! z6?il8=yVh`veI*TzR9PW-rRljnq^yo%+=9ucVknXb1fybP;)M*Uk9d1hqDXIu5M&X zrEGN_->pO^`dZsN!9npzb2yhD8wZ8+pjWk`hx4>_RnIE$3oM$0Tz5o~Uf?X2Ck%@_ z3qx;-LrvX7-qboJ6G!E%IJ4-iCC|&s{p%$Sa}O9%hKTf1CepgkEj((tBXw@Enj5XO z^B**4Z}}fiq6|(p=bNO?4^s?JG{ZMd>1Z2Ir>>A$$x`T~%h{bIwOHHJJ4DR#ooX_t zD=*$mG#x?>8t>*oTA}f3EOJ4o!<9_8dX?5|NJkCJZHjhdj)kcgq-6|Qcd=5{4 z!C1JIqNLi(m93D@**U)veU!FOP+iIr_*!@?rTQEOdNmRyzf2;8J)6mRR&Fk;UZJ`W zO(K-iQ9EVL zie1+4xf~5|Zc{eNbsa86Bl%IcS(WJ3#i#eS?5>B`#}uIXpIdeP4>irNLf8c!AHo`_ zFqRry(^L2QcTp+CzGUbfbo6bfH`Fyd4OcXKWN`12+}}4L2`{Xjh8rnG1W{9Qb0cI% zwH6xp-yj_IVx=%7Xraa4*~tm?bb_fqU}E{kAE>BmV4>b%y-KuBB(c0mS-!OzwqCqr zxrDt$E4<`z&944Tlm1c~=dn>2X|e}z{35(E#gO(`NMtBuh~SOk$hET%H5e-PJ~ns1 zYc>Pn==KkTudDEwm?Q5&ia>-2#>!P>rgdaV!HCUK7uTXVIKr#iyJvwf% zx~Zhz9x9DS9;_a<4`(|$jWmC}-GWKA+>DHi<2!QgIrc++*4=LbkaeSq#))ubxTxQAY2F-=2b2HJxOT^`gr5LN!^b=>eViXmM= zE7PrzkhI6Lax?z0+i(h_!<<(8Y$7yPExzdKd@2Ev0!RYv#Q&r19-}kq)_vi}Nq1~J z9ox3kv2EM7ZL4E;Y}@GAHt*PY^Q^tL&NzGRz4jUJhq}kOzg5+oHAc<(uitfr7ZyD( zVFTda0h1HP-!HZY>a-i*m{0O8QYf}L3b{} zx>v+IKXz5u|v(#x}yRsklyd;~Wn`Gv@S5|z$w1E$opNuDSWTcau zuk4k@*8_*mpmZBIEU4@0RV@)Q97Gz?Srr$FrPFCW;5Ws_i*_IQss%uOy7qH^^+2e-vt31YwH>>oTP`WkMFcJb?I0>i|(eYqq3xfR;Cv`O@mGA zPA|ERo&N22`m;N-v!vd2bL9>sHcitrl~9Z7K0?*zM}hseN0m+*>heZyAHw%P+fxk) z$BMKYE4=p|)=3}O(Z#T;>)3bb)^;$;b6y4RLocitf@O5Wy%iVvHfu_>#G>}l>gyLy zUJ3ii_E=pr>O_CKcH_|3Gcq>Wnr{h2>olHK6t)`_OYcEfXE$7&PKzAUHD^ve7%W6@ zQ#ra{$?9eFWR0zz!Q@o(dY50CGy3ZP@|GE;eYLiCJ!Q3O^nSkS1sHDulyq9OHE(uo zhT@-a6WPu;yj3-9x<&3|;WfMPT$p|I4NYmTF7-v8A7ra!Ulr)o=Qr!7lb_MOiS6C3aqUk=@)_p`QR3!L(mUg?J zZvo!N|9rH1^pzdVZGPNpdB-=NWa`E3&NM!_bT|8Y>3lYqsoT>zRS1bAlqQxG?oKkq zYHmZ93_y#(ih@Ew%R`F7>miemL3!^9nsnhgSxCXp20s8vD|v)+937#Np#1J` zVf(+w4rYISW&7Xl=KpJd&K6otJ70JA-0b$fHs|s=k!U_HpO{r|>7+r8s|vD$IPuXb z*PrO+HkGe7#}34lV%x%=&VPocGgXpRw)^<-skz|)!-f67CMo|g4EX;kN%{8}qe71v z#pi1{qP`C0|9(jRi4f@TImW+@$$yb${Og#IYx)weW&v#W@Hrj_yWP-yrJxoasG=I&>PPbLna*GH6WbI#SnI33 zFBt_8knDeZGyYQ^_SYyfd`-{)OZV8n((%7d&wt1Ymk(sM08Nlry`6IF8xb%X7-dJ#avT#JZ(pG=BYhH*xkoqE&lWb%GQ}jd zgtP=D;$eaKTY-Q^igP}{sz`^uQz&U=Ksohvd&@S#7yKc?(Mw08r(#(6p>JBGnmGI0 z>d=~YWRLu6yQ44n;oBzr6{sKznMw7Z zjFPNDa7~kWu(zB-TB5H`=Np(U-F$9sY9=vU_+U;JprPu!bn5DK4F+WjE6`U=rS@{= zUZ7g>oH%ZP{Je1dMZ zVajK~!K9;cTAHnUf~@fRl;21*|Gv{fK*7wOg#i|~)%++l%SCIDyE^%+j3mk!R7$iU5d8sp?|HOx&S;B-bfq>H-A|Wroc+AKA z79RG}%;2cmzw9;>9hzCUr8GCo2 zG}wa@wsdg`Rq#kGe?4mK2)e{u^mIZB;Rf23hNij7j&#Z}dS>troFD8c?_u^CR2s zc{A?7)?XzlbTlgNOU#E79R$bJ@gExSC>xqMDm+of@B4l^u-kZpMST(MT@@LM>kEn)r0=|o)^ka25jvmuIHH$l*Y*$OnOOFLBO z^mTkrAxGSJ(4nCsiCKa}{iQjsOrvGQgiRozWgO#HLK6 z9f1U4$@#PF9qqa-l!PbB}c0p?x*z%A5$_SDEHN5{q1ZM9=| ziqoLfOD1AXozn2#$9<=!BjcprbD3Mjy9~OdN*bE|5rM(j>eTL#y{{0ygW!d*H+P=L6f2*}2KjWu{ey$uXF7c?~ETL)SO168^kZbbA5N5M;qX$u-H z%qY_#?(mw9PLHvKj0i2{PG9%J(4ssn)e|oz98}Yi*5gnXkCH*Qw^v8#{_EyKPt@-b zekL!AvmFgNOu>tHNnR4B4E3Nk zR{rYR`vIk0eosOFyTn(rV9=UcK6tF`-oT+NE!d8tY^9D_{P<`PvMbjDdl?&3_vB% zXnka{=iYN4@xBEh7D_PX3Bmn+08esW4=K)94G%;T%mG8GJ#2A4gjstY1}k5t6q+A} z{jj#DK(IK_bw|0I{7}yC04msTb&iQvm8EZ@mg8motax)ufY+@_04Fas67tEGs^9pR z2b~e#&}!lR--yR$&9Q;l)ySmA(aMwX=dJ@v_?aD(5GmBhot748%LaO=v@tIOU=u( z8+3@x?7fj7X8DnSqwdn*+YtUcB_@mx_kNc@&s->g2S*NOfPb-TfZ8z>WZkENf}r@$ zK9kh96rgzy73Co-you;B!cXzTh9b9Fi2Y2EqE!VI(f_Bmj%9vrgoYprZwSqgCBhR=%c^-I5jU^}ypM@apicky;1Wn*T6ITXmXh6i!$olAX0Ir5 z8J;y`uZ&?KMtP9i`5Nn5u!B&w;Fh+kOg-PL!ct#aYAFgzbadC7VJe?Lh2tS7#W61C zy3NIoNG1#IC5cUeH=_deZ96q80YnXjTfFdd$SUqVlE7_HhenC?Vr^=3L8f@X4k=N_+8QN<$=GIl(T&}!&IA5o*YYi`22X&!7DDwD~B0HlD%pFvc>+dl9t;X+TEU><$MJsK7b z?v9bxjZE8H#v(;i9nt<6##8Qhs^`Q(PMErLh9g3)xZKX zSD>jIxLa~tsbg2Q1LWpNeMvT!dark;+jRw{Z%zWUKGePnM7&l{+;4}l$)X|kqJik7 zoUfiplb;#E{&p&VkY;Q>svH*U{v2L0u^}&PA0n``oc{*{b+Id9PYK*$7vcSqShZ?t zu>1XXm3{62axtqp3vjAQr&O>?Sj#RVXP-_*Xt^^$$J-dsk&3Tyk*o{rKp8E!?1(E; zAy~IFeVhocO+zRz#bWu&@pj=Hn1;#7;@ngrb@;KSPfScg|I@{8_%R+jOr6qB&l{0` zHhSeFj81*D*#=b=NaD({HO8YLY7Mg(c)BN%IRt&q+U*Ps<94 zGs%nt()pEH($q%cX4;AvS*X6tR}8tj`w>zSJy5nhBp+~N>Jv@V$CXIQV$G#W?*u}I2O^ELyWmY$Y(@6jsFDVd?`-tr6a&MceyF@mOV(%g zl{4W|>G>oU4!GOrTfS=l@KX_1YV0QjI+94r_Z)r<@eYdU<}J zAY1yuB4pv|R{lt1sUW7m;eH0X2!7TP=!TQUK(!NUHAhR%wTnT5xoQ~NlkXbPS-vBY&4zmyI) z7!P{eHH(sQVCRq=A)HZG(#KY7vkTQZO%Im|dG07om%SeMYrUxP7KtPV|2Q^V(8vuU z6NU=D%G>*mJMY1)W3bwKA8dssrBW9xxxG>Gf-!xwy0;iNRWUbdaCx8yb)hR>elyq$ zOtUjHC&4P?mPjZU^9B?%Z>X}=L;2kQ#`Zjpg~RAw@(#jxXXWRFASh(9x?j#X-sP@Q zguK)hCN>f9NX5GThQ`)J=lH^<#d`X^F@{PIM(4vucd`Es1g1NP0!j71PEgS)_ix4OP6N}2km zt@*177+o%#eA_r3IL_4DU4MZ@V&O9|e?j$sZHGWfZo`|Z7zCm0HZh!2tdL81#VWvM={g_ zraHBS53xZ9dRcbM-LQH#kk4Sm3)`Xv-Up;19%2IN=86wNFfDGvcG5Uv1oauAf1|)@ zfe;1($R)m$4gVmn1^%jtxjZ5Lh6wf(Q!XCI!qm-^36?}Qt3<3@DP74#@!>(b_2djRWOG^}{G(+26LaLbMEsLOZs!vY>}!M^x6+tUR-zHj%cVYiwk8ZcMC2k_-87K{sE>uiU= z!9|jE#{0^9rAe?p%xTD|2DV5Vi;li)XwIo=dNxTPm%iI;z35%MD&4X1&NUOouBO5o zdqSP7e@kc&M9y06_kZlp5km4?T$MbPaN!%`WJX-iQ^;*z7F>jCU@Y`YR&1&)!nL6= z^GCHYaHljDt$7bJiTpUzqsEXTF)a6*g+^4OPIs#Q-c%WzV{T9=xtt4%!E1~Qe%^(< zXs!85+Sz%nLO4(0uuq+H+Sv(2-DDROyddJq0I#qSKJycaacL`>v<7Wv_mg7*FYJaL zbxTM1nHS5nfpy*_h#NC^aO6pOb^6GbvZ>m8V_pW`jC-`Sl3CxqfRJ)hk+4odetCOX zj}lLYcj8;$(rTG22B8K3r5y0|o;VL*luT_dq2I zzoBYMG7?WxA04UaHEJS<&6QQ=c3?{yb^+@?OD;Q#3dXn#4staR=YSn{|JM=`3zYxKC3^ydU4TE1%6}FPCSc7msH43>DkV zu+Q-Q!`gM?mDluh#QS&*N|Toug_wEGrb3Sa<0n0m^ZW^SAu!E_4F~Ne-xl#(VzQlo4VTO7W+c*|_{c3P98gjTgVK8KJUCIG zp;p8kf8U7QugcrZ4}_Fi&x2=@f*KUYFDl0plS^7<28Yg&1KJbUeiybPa{W=7OAq

CVGk+dN0N7Wsi&Y9*?db|H{LY-1JIva-uDHV6~ZX#!X9+ zNOwSf^)^*g+@S3~ZM;BDFP0`ctH7EMpS*d_j_WNdKv})##fk~lc^R^BT!oR+Um7A8 z?fa@aIlKpxq*=V?G!#vv3U_L0n*C<`>Scw)8XO*;ZkCiW_>C>BQX+gBfp`PiQajcNGxf{fvkq{i?79s4;kK&YFCVb?hkutKM9<9K?jxypt58 z#e^|GnD_RP!Fhh!5|&?e9xl`d7M>Lol4DQB@}k7EI}0B#dc_ zVuP;aimc}Tf)=B8rVI`NkE`Oyo4))IN!LHbYGU4vUZwF5lURS?$QW=)JtY+}m4-ZT zAj!mtVSk|UAb2)CyA^t?Q!E)^almHsa1HuLbuKhbVq288#> z$nise6m$ujKvuo>v5E5Nwz8>tA9AZr7$i(+e@X$sK4UT&y$C>`TMHrX;yY8;z`8mo zLkJFj>v}0>Jq+5Esy|{-j1)kZopLbr|75j9VP4AB)uhl?ekq;WPbv{zu`;A<#23~0 zc_>iE;S{Y_mI?s*G%zm{Nq@<7nfb9e<|EZPpZin;`Qa`m7pf3O?B(AWGZgHm*-Ej2 z3A4h|(?m8WD$n(4+>GT~afm!@L>wslBscYH)c4qzK<`eXj2c65+7+YSXF@e4;OZd< z8RxTf%LG7#2j8^WBxDuX&QkSEH9>ceN-D0t0Gup<87ML@B015pLQ>pqg5&zwNWFap z@e)ctCs%tB+rR@n(!IWquvqP2ZTPK3mJ#MX$5GJlY8aw-1 zd|)n`HwD&|X+kvenlIBcug_R0b?MAjtv8sWhAq$bh|~)AmKH;28V|><9$#QHm z&>iR>kqS(egyD<(1Lxvou4{?x zi2ck45%xcXX?b+W&@F$J)t9wK|M)Ys(T8Ss2Il1K=)lm}1UAOOp}wp!CXv>rL1`hu z$-$o4miFOp6ae&eR^9#a@}GbB1YZ6_aooQ&)e+p7dD|DlZ zL|M{Se)d&*>CF6BraXD-X0BpWas6o17FREcZn=XKTID_Wm zZ5Qx+9}ng^*@q$IY5oy`!bY|~_HY88AnW4N!+;bqAuvb-jnr3tE@?q9K_*Ef_X7^J zr0v_5WG4rOL(+%g9TIT!iwmp;4rdX*4z^>5+TrglO^f_az<};=ltsw492@Ef{z2Zi zx9t@9oX3dh)lvwPAHW~v05gxdgLi&_vI*vWP!W?(Ulk9sHk1Px3g zO~>@j&L|$}l@^8q*Bc~04$vn96**9fF$lITIq^k{U8{xqh+`$eC4MUU} z*WiQMBdqx);vEm>FK56Jk$6T(8ztleMdZgm+o->(ubt7l5;{lkk0A=q9t6T8?3D(D z4ECyz#4ry-ixU8j<3Oy%OZyFaJ4{W-jTFBBsfW@)_|7=d)moE7Winz7!G{#}fj9># zb(=vBtF)(|>rRjg!5bOLIFs?9nP^eSDz0U}k#rW8Hsra_w;53iC=F^rj){vrEfR{H zj;(+j!!3hGP;cAFJ3J0Famog9usV6c=;<3GF?*+w_nRCr^>=@$?@(yr+i|u*22>_t z{4jsf9gY_T0>KaNG(Sz~?TmU3sU86|WO)L>Uvvkomm4pfmg2C`Epa;tw9u5_G~5(x zDDeusey|4Pvl%MMZ~+)(WnU4BT{3FdAZdf)4Qe(W_s5*Apm_atSP&>;r34eWqdaSB zpBR@pfg1UKUeX#D18U>cuuyDB6 z)jK!|+$?RdBZ$~SY7sG913n?1yPOL}E-rjvcc0e>*h*|%oY8t?7;74s)q^o62gyb( zLc;EPW75xEt)~b*!Vw?w(zqjqR3ZhA!*NChi4#buXX`=B)HEQ|`&3lIQhqD&Ush?4 zeBnOWLD4t*o2ufHh#&m=LThOBv%l915lX;-9)tN=QOMFXFtGjpW_N_f&8&v`xpFNqWS*zU4ZESFR!S1^@*CGw6fx{~{x8hp(d`)I@+X{i^m; z2?-vk0;H7Cvm_8E{!_Uf$V1N-krXA5(a1&ls4tSM%JGvbX5JE)jK(Dt-IuLB5aE-x zA~92@pJZRHEP7)UWr4aFn-qd0L^=+M96CY7*fcA=`ZI_@;MVxkz+lGec|{?0-JXFo z0Mu_~Bt8*n_m|stYD+uxG6w#?;+=TSzvG=%t%R#%gIR* z*9K{{xPIXsVkPuZ4;dz9Dpm(m8o^^a#q2aK1MSE^2Qz zR{Qc5mziZ9yZ3sAQLo8FT?np`R#^f3(taaGGN^(+Q1SFoeYEHNYB7WMK>9<%gBKM! ztQ;lAmb7ZX(#-yEJ(L!dGO!EkI7C+OMn_@oT;mY=UQ>P@V2ksxr}oes@y$K37@<*w zgb_6qx^0}b1Q&}pClcr!`n>~7hu=6(g02VP_Q!!&QosOm2E#Sq@+OV+FDeq64&&-R zJuI(kwm-`mpvoUY2pDziaZAaqlP^c~X(TsG4&(J4>nMPRkO-0aH-Vl~Y6dTU>h=3o zhGYK7q~0WC;n<>q6pxM`cS=^-<(7j;Sn(@gSt|m8U{*Cz1WSU$ao7(R3Q@SX z-nR-tI`XNp8X<0UQau2>CmsCVztr=acvvM2k;#Hz;=E7H{g*6g0!U&r7js!^FA5Dk zS1_r%(D(K4GU#=FC#&a83ff$}?Q5EfCfE@B0XyuHdhz9m0%Ae@5G0&1m~oDg9d4fF zXYxij%oi|}pMnX4gnEm4zA$${=xLyEtodJfM`}52!tjIWR73{a%~&l(znvvb^Z5_| zl#(EfSkd3`PCakkk030@5;*{65uFu?aGJ0<-rw;K>=)kQr{0#@empb>77+XagBJoW z8KnIf^BWqB0UaY@2yi@3kaW93wN>==A~!8?1Jl4!#DTZIM{uCmbkUmP?s4lhQjO9bZm z+IlzO=VMmkVaIe-=ke)FPdU$~{dwh8oUyH!P|EW+=ArcT)+N5P))zpqPVe{WS8_E9 z{EHp*B1VH=TF7K`2ApUi`}Rkwi%uw!B%!$|RW$}>Ls{m^QMMJ!%@QR#K#M*M*Z#)1 z6Q4KF%PBciyTIjX86K!8ou-a&(I2bp=Vy6{jr?NQTSaCL3;iO87%sM@sg|Arw>2q| z3j3U&(y!f09uo5^@Dk&1=9A-W6Z03EMCnI#JX3B5j3+l6 zhC0`F8nS##8{x6D+UkZjZn8P{>jno5#kqSdp&Xua#O+-nwN>S#z{Hhw>>jc%)#SWO zPWOOl@-~+@qhu=^gH`IHwP;t%Gmx<xrPUBJ|K#o7sLs_~KL zhym}8XMEx(2P)&6@moS5X8h>XbjO{nMg*W#}MOpx622IvDQY~<^m#GI|8 zpQ14bUp;-sCG}Z6GdEU!1h{;o>$v5xiyB$t&DS+4X72ZFJ)=V&lv|xFT4hwOGI;0c zC;=-;>}%eN6GZS8$1J%X8h}5QS#G&_Iki3ye+Y;1qbj%Pn>~7>Cj#G7FCVoJsyc75 zT$9Zd`m`1+JxOc3_l2X?5Eu+tE16y9^wywYl?Tmh-PcTXf`kh$VR4fST)sWJX}WVT ztGRnB&Tm;Tba_rKQYd0Z8}IF$o}Zh5(Q5aYT4e5v?KP9|HQBl~Kh?Xud#JqN^Q+)K zHEK|i@F`LqdfBWl&MV!fy-gS=WBC@u%KLsABSx@Q9{paw6Gxu>JvC9NtQ6zPo4jb# zk2vtygq_i3wVX}*X*%psXFS;e3hik*XJ$CvpuW7;WURl?LFB(ma`5YFW0hyK<-9)Q zekeFSt~HnU`8>!6|5s+rouwkYnariGMI)8jTM*=C{vJG!>3y!nL1JB=k=w-^TXP_O z2?n26lij6~_N^O>nWWDS&rY4z!mbb3nXB4|_Hg;;iSA^TPGxh2%`?e+pj1dPU%KHb zdi}>@A2H|pRdJc7WJPXBfvlFPm7BM?DZp=8N-Ncy&Zh3iivXd-=)*|?E)uWC0_TGR z#`7@MWaH@2xA(d&V}a!J-12N{nRyIdy0TWBbK0_srUiW!4D4{RvWKhhthg<;Ex!o0 zhixa4)zKc`wfdS!#8&Wgk#G;S2cu{z8?>4pCJ5q9x``Q-hmd)aWvm<4&_?8}s5>u? zuW4T;rwTF?DlL-7ckE?j+r-$Ol6G>Y6rV2Hw%)@BD!Xb2kNwC8pK6{i2^+L~RRU=*B z+0SNAkg7fUvyThX9sv(s^2++%8;R`XhbB$({aBvK!yy{-Yq>eTJ8&!QrX>) zAh*x-f-GVrU!^3#Fgads*_h)Eq-~I?EGwcRU&DjzA8~ zo6B+faCxO`CnGLX{5oRLH`BU(2F>4XRHKfUHXo-kz0@7AIZ>g>+h2(%jR3>GqIRJ0 zDN+zW>ZIGn1}5e@DzqC8L%_|vbi?jXX}^EeWw`p4Lr(pm2^ka4#wN*ZC{EC@gkd;AC{AG zG0mJJAO6gr{$QQ=`ZKf%%z-Z>bgqvTj>zIo2}#G%B#+DYDSRF4OtxOvR|f}|+2!GO z5LF5!>VGyo16vEj?kZ`eY_wl4vLc@+T0sj#Rf(HkuidINAdYbF(WT%RY|yn$);w;y z?&N}wStTS>gZCK4eepc&_*-EyZ>NoFJ?Y5R_i)x}sMwsjYx%VPP!=;ju_5=+zHP89 zwVuG=;K?ys3SBv>++m6`_?{v!yIMn=XS-H8VP7i&)KJbf^p8D+FZ1t@Z0Fg7)pgh~ zNE*Z9l6#P~+}JHolI#3jU)~KXiKahx0O3pP*R7vEdn9MYbwbW;!?1X_Mt3{Ejy1?* zY&A@?j(W03bB*lluG0*T;{%eQFIN=%;2!2yDpOR^UAD+ioKhC6<)01c?90{?g;6p1 zOmyBCD=LLK9B-a6th-0lU!4`F9yjlS9Ijo~ljMt;^fD;5K*b!7WQ~@depnBx9i9JP z;6Zv&Sy|Z+PCbIJDN^UUb%02*mcc=D=UF3e(~Xd9+1oDCA9VU41YFH;$teg$sJ~?0+YguHZLK>PqSV{(WJy%yqH5`Oi=379EJ%-U`Rx4Y2d=#f*sXi# z`cSg|j_otE(`3mJ z;xqQQ>4YNOyfrlDE$d~UR)3y}H|b9{59Z+a)btQ+DsRb|4_9jV9{5oBbTCZWc~2?$7Zf(UAwS9P@h?F6`H^XvJl&!RDO%HG)d|=bu%( z$@`cP$gDU!R9>R^QkD)=mM(hf*`s}$lSXxE(x4_S*O4pJIoCJsBt_;+h5gW;ZjJs&{MvZ(` z8K17YsNu+WkJ#wrsTHaR*)3}Kjy4|%-J-b1yRyD5rBp26lyyRVch+tB*rcIh3fOKs zYTK{nwRzE_F;~+udEiD>Z!;;g2p=_*(fx|Mqsvj-Bw)+fgl?_L!NDWWM0Ws9u?Q(O z2d6B7%aV+4vxQ9RG^y;d;eaWc@q@$0quhz~LiNR@DGhafcXsZQj@+w$NAhM1*+td^ zZc3((TZ7asM41Urlh_>6b}A?#Mo}ijo^$-F9#*;-6O-WLIWQHVRaKE(;dKW>)TZI%a|v zc`OBIM4odOP_(>B~5$~g~_=h7)8_iyWpB1_?54jWEEAP&^ z__bS&sUzd?Tg|6atoUUIAMGiRAK6(dUEL1D-I2{U8y>GXQ9t zUp0L;9}j0&6_-jjO#snbgFa2I=g*P2Y+E*8>nkaqS0gXWu#JbTn~#-FFA&WqQakg* z_jflZ-|BbYLs!Kgq<~->52$Y-&W}k-9ULi+&~FDxZof-=&3g0*PkWAmtw2RE^KA7; zCYAza98Lr{=kR_}NCD`N-qr&K@`ZmXfe`oq7mVjS1TueyvYiq5{{bbSDFQS8Dw%Hh zb+G@pP~yKXzx9tq1kVK5cRzXtGo0+DWd)-VDBG2{}*2(3TLy4 z?Fu*oh3XOo*92p|Q{2AUKYwroH$o~B9dlhXOCYkny&csx6-4CwdnQYF=A}hE zRe@K#w(=^9xq&dS$7~NcQCo!yD!MzjH(wE>su2O0#(DXx!lK_ooj;(n&IxM=o|Ty5 zGCBZHZ-Lalo}ud(p7K-l@f&_qIJLVXd1MpTfD_cR6BeW}xb{31XOL!FAHniN;Zb81 zR;C%%k>umq2yopHmF2J4Y~9u4>Dhhewg52iPNItROz=-7-;TJNKg826pwf|_=cjSq zFICVI(HPox=8Hh1VIGKWELW}-=f$R`mTJJ@B0GG%eMewP2~rXlnuh#!EEy4n+CPhh zs?FRP8IdQEs{`_&0R`>VW6Nogu?e73tIdVP@71Q$N9 zCPotB7mL9TYK}kQPK*K3NTj*`jzU3K-HR4fPcl_etZ@I*)pqJn_s=K6>$miP=v*Qz zn$=S@sw5_cOznb7OO0{6fMZH>p@xlN9RNLLlOh)_*JsvB!Ayw3<40OJ!)>P1qPLga zYW>2P5v92A6e;+}qySe$`u_HQ#TqQ*>>sy>LzF;IXCtzZ?&#$Z2a){Gg(AaXrcHU* z{IX|8e4_~|M&Z;SGMyD9Rt8zYyC>!%C(n37gr${YZaj|{h=!Iru>`%hrXZ}Q%95yM zMH-TC)>^1kjrW!N@YB&#T#=eI%fuX?-1u6SWe-TPpGHojrcKN`8Ro^&{RxS%=Ts|{vTX78_$NyMJ0=V|HB$v3@7jlA|(<~2e|?|>g$AA8W(k4@vXK`4TapO;T7{Z6Y4 zh8b(de~h+Z{4S<{PT~O8r(M2E$N6%@cPNjIdzMQ_dK^s%X=uyb^bXX~9ZlR{!SqhFqQJt}_oq#emRa`dqsz((qng$34+)pC4 z-11=j{d=!uyv!i*cN~o|8*kG@?>~(BG%P^U-u_FI(qqruyut`5w2nbe0kw;oxWVhWfE)(m zW?jKlyENXg<(?;clj=8q#Yhu2c-R+(G#03~%Gr<3LQ83w_uR~xB>#GH7buSX(5 zHxMqPI&h>S`5i{NE=n~Cfox}>(5-;8v<$g=-f<2U?e>m=m}4h>`C#Z8E%HvcLQ~?@ zx4DA?=xPxWJW4Z}SlPby7f^b|Kq=dVi1nftmZ z-1n*If{v9c*#HodR>ka@?Sgoe@~sQ1l4g8>hR>lB*qeQqj@vbX&yDBX{zg$kKDwR8 zQ#~Ppp7De^rv6@F?5MJIi8t*PTGe}3GfDxxAHMHz`(cJ{8_Fn}0SlVBq7}~DLFYC6 zaY#ot4~|7znVjY}9pJuD-ch_lzdHjSd$~X8^+)I6mRlN5E0dn=z>v2a6fqa~4;wfT zHJ~bDqCE~4r4)}EZcP#bYiyHAsvo-a$E?3MCrD}EX_NT5L~67*SnHAAXZ>Uf`K=}j znmo)yA~l)E^5NCuv!!i<0i{gfPH0~d0d3X3v9M4YEK6a@J3?dv0l;l_gibY!BHUvX zw#{#dGw;q7Umk8}@*YBpq6!Iymfx#J2y0&k&stu2y2KMYJT_{3EMK8)xL$a2@|(4L z$t&0jJSUCyYXRP^tZ<}%!neUrfb@zSkBET3b{FnJ}gbazB%@n=FtxKG9bL#=- zyT`jyGuFCPJ+&5P!~x=5PWPk)t#RoC^Jf6A$o>~@2A1?`Fs;?ZVZch@PX&iTehur9 z3g5TcPdNk2^~IzM@Hq8U z3J~9tosJ}UO9FeUN%ZV>GJZZlu0~3VE{uQ%z|7gQl^SRI)L!QmxT9De#_eJ1Gy`clFFBK>OjcH(?MUzncbN$_8{Z|h?8XYm!uQFBn8P3lp&{h{5YEP z@7y|u*{Jwokf5?2AhwkVkv)Y z(^&+`l5irPA3m0H-Bc%&lFPuPoXlp~dJ=j~^CqZ5>CKc1)SQ@WC;Bz6Dkjf{AgvIDM#ykbHQvflR-h#YFSue5;q;q6KebTNr3tm@WaGM&M9a? z%0_PWlyQTssXGwbYli8i>v;hx)Pa7b$5)TcNYcn2j&u6j)PfDq=0W`V6j-U$BWaNz ztg0Fp{MtLh@QlcqIb%=gzW2Z@|GwW!{z}D=BPOW}v=6iJN+gn7bP-o}n@ZP+EZ0Lb zRPQX6`v*-d_3yP(DPPBHp4Z9DIYPxIC|LK3cM~Q@Jl=2eCCN0zbpYQoJ&M!)401cN z_N-6GMEh?+H7xvFxs7dtEJgFIq63f{3)yJhYi974Hw?pG%i~uWEq*sqjC5YtYkzGo zhHgZ?e?J%ybDfQ%LB#|HF-C04jC?=+UW{9DJ6uI=&G(*>jcXRZdi8^Sv<}47IRAGw zAgrP09Bs(~Nf&|O!ipcux>&?wKCFgbQ2b2;CZ7?rWS-Kl<&m(H0SJ~t9Cc4~_qtd+ z2J-Q(E0TPd6tS0rz%GaLRVk0_;?*;&+%c{B`WA~JOBet-@*_!5w}8gxtyAHlzp zNm*4D{ME<;(G(jQuUJUoO((*T*#kZOSYqb1wjyz}w3x`99%FT+9UHaX++f@4SBlzY zu$g@qZTB+g`k9e^^o`_9(Qn6B0*yvuDcg&tg%52l7c(WGx-jQui`fZmyTk^cQt9Bj z5-*Vv=55^BJgR#4g6Dy$lrp9RRB|?Pe^17FusY}EVuz}e=2$+xw(2-E3Qz;n;&)h1 zYolG}PRY*6y^VG}8JDU3*}z|5+{011_=u))3u$YXN8>~edj9g;iuis z+*<51xyy(T(8bn}Jo`~z<+~<0f*uE#tpJsx9MP{c8BJCvkrSzyL7(3@96OWqq`mlo zk#I5hhelpOrwH2RY%uKRv)F@h0J>W762DIqs?d4wR9lXP(rzY1@tgx<*!!%Y|LOs| zv9p5j*aVH=P*1HhH6ACrq0DALh?^`yiO`)Y>N(^lMor?e6s&!ea8O5Oj~;jizwlOu z@!5*hVw6^WB0wp^`B-WCdSSp@_H&nvM9|BchNMUSw5P}M7-Q=T&R`Z8XKbAqt7cvH z!;U-^Kc;?+-!B0N4_2ink2JYiuTs;$eVDf#xwI{AX*k~Wydm-C47kTy>uc^~@*J2r zy1I_DF{IQG`|QVwb!2+Qdr!w^V(=)S&Rs0AAk0Y$jo6yluOeG$&=Q85~j zffvliiPxJK%H}fX)qE*BN`Q1EfP^rTU4W+{I;D^J#S4gOSw7+**zvrB;J?rYZ#g5= z?-L;&{KV(R^z`o(d)FI*LSebk}3Kl>#?~HW<&LVB#l#uyz z2S&*S#heXHR=y}#>J`uA97N=)73z)F$<$jK+|nEU2?>< z(CT6+@s*q213f6G(bx9Z-fK46{o+bGnLSpmpRo3xE-0`8rbum|E@gKr>KKpjyHE7WftWm7VVj3PEn8BY}Yt5vV3R z-}vwE0lo`5^UV0TJo5spe^A7I;*x**fCGJCgoLJI)Kik#`fTpSkOEN00)^aN+QhxR zhBjp6C1ljy@vcu(0|2jJ6CARAZ*5xu->be^xgDB#`&abZ*_6qjDu)otMj{C75F@OO={Q}d>LBWP4ncXx6B zT8(cM<^{{t(N273Ysw^rq3bi8LJb#E3g6%+j-&n8W;Z}oTSE84`(e9r0JMVzth#@A zBFC!Wjj6`Eq4`;UzbJ=&|GhyV>ra97!;@aDb6fEY=SaZO8+^da^6h)1q!d@PE&A%P zAWwY@y{PE4l01)g!!lwUCV_stmv5pSsTsZ`+hDxkkM9ni3Qv0$M!q9R71~ zfEEsDt_B)MNNlrrZ()!49U4ArRdjkSr&T%bPNyB^5q!+9%!btsQ_b|!VzT7=zZD!D z@5)1&c~3v+vG2Z_FQ!YJ%Q?K-?&`!{`|ni>;1bytD5BzgXD9f<$dhJbS_2SS^F!R( z1?Y%qFgCF5D~Gr~obn1;0DMdgQfkl79d9`kI*PBd53U0jzg51mr4uvTp}aS?O}lo#bE+^eLDjw zeD+|)QtQWsBNsZ0%bssKPyobWESXG(xWhw|PTU4_@I$fikyvE2)Gnl?a}AxtGzi)G z8IB6{2KzKy7(!qhq&Renl{-6vqn`HnVxhD$c{EIv(@MU|v`OZq#@EyL`46$a+;0bR z2gysi_Et^ML}27(RZ@sGHxck!BQ}s!=9b)jL!jr>K16j<4ssk_=j6Mn7vXU0Z zlsfoW3>{G%PFX_^fT37G!u}O%Y;&l!z@4?r*_1>Kx-%|l(+x~mk(BxBmc*fu;!1gN zb_m%Wg{B4tt8H+PkYh>l4HpiKZH~hvGl=;N^Y|P3J_I9j=a2fhp+c+afrQ>@{R;1G! z!ggWPpdkkmX+BExhMrIIsKVXF6e)MCOp~AhaYBYOg#dpj!Ccx{bzVQYCc2U(TvP^g zCttrel;(f-)AQG6oz=*28#iBgi^9y=(FLD15(rzEY9Ukv3vQd&pk5oYK6}@D$Ur<# z#%bEV6CFEtC|9mCNk_}fu-A9pq!)aF<4JZL8&Y)9kQV9>Cz`yn43G?Ivdu0y1J;7b zC?$>MH32e7{mjN$CvIU&9{WwpsoUQE*-vO~kHtQ~7O( zNtP=9g?YN08?$S_gjneY+Q$iD&g}#aJu-3744$#K#+AJAlUw^ePP-yq)+2qH!WdFU z#-5UHJ$_(9b1hv)@nIa2>X8wy6W0rL6SOcj6=_Yhhk80uHwb`E6-5lKRvtZ5$WS+w6JTrt7W*1*`^mx@we7CE?<^1AU zkXrJSXW3oZ1DlhQP8NdO{SHBQh*ZEw+e`QDiS{WMXLZ?Q^~Xd{{)6@Cr~|M=d?5Le zTsgvq^k8vT!moA?1>1GN-z7Gf3y?&TT?lBNF4Yv0MIO~LhV}V0lURc>$F#WLQA?Sob^OQL zt*RXo%`g)y`!SQPeOd*ZcvD{bfQrjBXAJKqDSV^jg&|I49@jw^^T^%5%g`KPj@cpM zv2wG=pr)&+5RQsYP8DE7uSz0?E?^kkTez976X#P5bE!}v`@~=_j$9NWM@7)zI&_CD zScFAOSgo7fiD`kJ#T}_o-NY1g@|V5*NeqJKfPN_C=^n$ zt2$m29x5Q1k04+9=pbdMNZ1hocH(MgMr=p3q0)pDY)l5c-KEdI4Q3%Aj8HA+cT{&c zO?xtq3a_6CsU+G=o}b0FNSi|ya-Sbgz-D=dIN2Icxc50aAFG;Jx?MUvBN%Q&L%Fg& zAxl$%Oy=%R`gj_ZDEb+lwuI*BHC8U_-*q$>A2M@#QW}lr}Tet=9Gp z8->X^BE{OBpIIwBgDoYE6=BP-H;}7`YdmTn9hDlF&u)*i zDK{XA>nsE8W?FVnBw44JrEtz0XcKPs&y635xqgU7ixD5XD*}q|pYK0rW~p*e6?V_M zNkX?lh-!$g6hjGyKak5EIdHV8)gv<_q$|hOE7v9})GKk+wb!w=BWNbnjc{-#Dkh9b zBD5z^8#Xw9b7Hike>xZi{Kv#3AHO@^fB3}yUFHexKRH$Zpg8>h*LV7dxI0xQlHZ^H zKi0plkoK?rvw!!Z{e@1|pYlli!eS-=aH{?xmNoYCd_L$))jAmy0D%9m+y0wLf9grm z)3DO8=>DJTS^pr<`=c8U|EE&>d6J1-{0iofE+0~wfZ79+d=e%aQUL@x33cp7Is`d2 z8#AOK3ua7S&Po7br>ZtZo)YRnVMss{1Tnry3?U&oIRvC180=OYmdkqiSmw&RGw(jz zC6mddL)PIZ@8Ng{)6h=ND?bys2Ri^jSGM0P20sAb^+At%APV9ZJkazPWv(f>B#8<8 z<0FaLifPX)^UDi3;xR_AvX(d??0hp4oCJYz(87E>z7mBBlxkRKP~uk@A#qdvW3G8l zU|>nUT+ZdFV7RQ#3CF5EHjPj7m+oD!EkFWU{eTYrX*jn<-9)LTCh2zTyI(pIN}9L! zk2fIDAjXS=+VLT?1emllUw-ZI%wZ3;)X=S>NTcZfp<;6>GHK_ftSjx3kazw%g_hDf zvx}p?6Z`saHK;%VI3H$jDFH70KOOe+m8@PK+|XZQbRda^r<|G zSq(*gdad+osQWy0a8~1ZyYO{x-gNi+4D9Hk+Kyo(JP#PU7~b;*P~bA1U+f#3SXuGH zvv~)gL(u$$e60S21Uil&D2eFL)oS}ipfFnhTK~$s;n5o^p>`$UR=cm#WFv-*gLy8 z5=I7Q4h-OyPm>P9{Y=Q}W{AXAt7{&!1s`ef-1#7l5R*;CE~8eFf`=Gk4G}V*Y!POn zjDL}q*cYMG6HmJ;rwRhx(x2|~n~;cV7*8MB%nAsY1M|jaQrJ5sb}9syjs5dMaQ39S zoDNH*25bM2Fl&AB`}gB^ey=4f7%+dyj-n_L%<7NyeI?F@WuDDpE;fiCj_?=5tlG>v zS|>5eQ72)ANj6{#89T?ERoxmHnohsbNwL%_dB)aCZcPs%JY+yFJL}xsSkCVNt?}y> z_$aNv&nGVDz@h6rUBj3KZRYJNVR2fpg%JJGgR zfM)81m#mI#KmZEsSy#r{&P%4Ys!5PW=>#rQOa{02$QD>{|{u@)3xRUG4|MBBRm7woQ2Vf6#Xew=(RI8JwLKwDh2`Bz`n07XOJXf zC1bZtbAFRs4DS9^A-fVsV%AaiEmNpLU_S(!b!ya|b8@;G)?La}kyx#VC5hf1FNlPY|-D&s3-Hh)ZO9X zxm?5j)<_wCuk>nW)}7kf>^*W(vZAfvWwWnBx>S$&sVmc((Wi%!wk|*zEkx zemn6lQYT6*m#n3@Q|}vw$zp}*{d^$nFmiXKA9q9vCc%VzsHm94Ih4H3;sCae;Zv&Y z&)9hL2P#skzW6YP14d-M`qPQ_cX;_}-c&WWS#<$anC!b%-l3ORu#gXvp>3DOoAJB5 z=u|bFQ7`x{&Gy=p&1CnDh&5-_$XCSdmdQf8?FSOS;NqX=!-|Qu;T8gLA1oxavVeWy ziBYfgE#i&|LMdu=x&BH5p9k_kQbhCY7ST285}vUnGN{^jUvsP4oLyzT1deRZtZ2+k zrW~yYAHoMR6tk4$Jo$SXYEF$X##W-O#P(0*G#?xek!F-<#sw^qJWZM_1GK^=&~|iB z^bPn98j8zM3rrZIMk;g7@AW1+?ogMj((;w)}^p>p0+@WGb}SWnH}R^Sv`I ztk>;#f<7gYheCWc*PE2@K!Dws8vWU|uMOOzof!;^K-++;bF5q}cB6gdY)r`FmR*uryLBSBJZ*J6jnl0h@H{EiGN2yu5s4So;M8R60FbqVQu&?%7fyRn>dw=6_XAnbl_LUbyM2zMG7W1!44}=QPWKvns7Ts z3N!W*aR=1R+-=XSyyn*((+ut4Y5ZQrk!m+VlMcWHhu>q8dntQE;Tm$ z5vWVDX>Q`6j2S_7Uznd4SjSLfc7rd)z~fM(^Q$o>9hRGL_Qd4~hU6k@^=Q{ug825k zk>i`#v{`3v?Y52wNlK&fby0lUK$&Vk%v6a>pfTj5)U7>tk&;Ab%SKAxp|%zy@KA!; z$+;-z`V7q(Z}tp-ZXEr zcOZKHw3Intf;0m5i5gq%4LWwQIj~@87|1$?TWILINBSWg-s9lMU7Fm_ZROrzcsw4@ zr>pJ$|NimZP4MgW1Na{sz&}K=YeB(>9lka)Y0LlsLVvvhr2W?&;Ji$yY$PCpIF+4gn?R&Opc;yN85}CC0jOzF7e#10Pq_kL4HXX}Z$QQ$W--)^4q z?Cp3pRtzHHWjIUhZGHA^tI4FEA1CeLLD%wzv;uveDFI#!n@9>NqM>@OEub(w(1_5v z5z!hwvFyeKgHIt45H}3C2zGwxod|%Ugr4M3B!ML$gnrb0r4aZ#1kq6KqUx2GVu{jZ}X?zAwNj3Kfa^$CDMl; zAOdf~yXCJc^3N(%>JuR;2&4MgLJWX3bq+~9lbcqM2Hb~26pRT`6VGk-RM1E6SGZzC z`Zj_f0BirK;j|e3B zdu_-FB0ynI-Y*LtQXgoCFa%_W>R_~gZ#_soQ4mc^psyt0r`#4eZ4d<`KRsG}Y)qII z2#DlU1A%~-wgMtPf&xxQ{g*H%J|(f=BlgXOL3xqHehXEi$!~qy`=l2L9pGYLXu`ZX z>IR}24+eR&dG&P^7Q7wbzQPL@WorneZ<3!`1W_R5u`w|cI4C4S#8OGh)X_kx2ip)Z zaa`i0`~8k}t2wr-!qOwz=tu<1nJ@jqB=oqz@<<2~ySC#V3bmPxLIObAda$`~;!uPF zK!{Q^m65h(=995kv8LvzC5K2?82tol#Ca=)TN!>fjD8Y7f50D=UA5>DrmF3T_7PvZ zV3ZV~>v2c#E-!ekmt{l#}4K(0f< z1C)17kPwhZ3V6T)IK)eo0{}@LkXgiDha<88l}l-x%*TOLXA(oM49&yEXHInfErV>i z%w0s($`lV6w^QCZd`df@_Y#2zkmP{@VK%Hi_$X}TXKfb4jah3nVBE|aw}gqgu50js zK&V$jp}?i&x`3g%)D(JZ-W?)e*XDMkrp(XIHWN{DikhZAd0grjUiJ4X2zb=~JQu!E z2yda9S{!(KD8&z`FeZQ=2a28`qyz#{zT|SCj7CC9R@_kP!oGgFHYr z-iZToQcxTQTrXi4KuMtIHTRC_l>BpcDpdpry_n4zff_p%!kl-jIYP7x6-X6&IMjT@ z?Jg;|CI96zl=Rbo0YJD0vamV~OBZp_ph}(p=t@Sy7Zjx+f&yriC%X$8AD=4lwaT19 zurv*)@XQ+}_QIMF- zL`Af>DtrqzCmfz(#+e60v)HD-&w2*=Q8GOS81VAOQa_p4GOYH_lg7>!OjjnFB%uJg zbe2?~!ia_N86rS`VhpT8EN>Hb8wB<3jbHj!>Nmb9(yR z&{2`kwjP#NR^pJA(7~8LXsM;GRj-)N`+@k@H0avCmF>$qspVD>`8TVfUh3lwf{!zG zVR{Fm~bkoQKqX+fGU0XIZRo*5tkwrx}Czq@90w9X%$)HK5Zp(qZ9g1Er}$^r4>d z;%>RTic}mugqIk{O0MC`yYDl8++u5_yB+d;9%7@3Yk1jrYD<=&Z5`*a7SrhpKEz7* zF}76HQdO?2R9>nmQt%%@dgZ8Nue$X*oVd+~(P{?qA;l5sPh0(lxv`Uoc~O$s6c}0* zo~;#6=2H7{vVhe|1lP9%BEC=y9kxl?}%mbI+&JU`m`hYuXb% z=a`Fy^t^wN1Bt#+sCVV*F#9f>ptmTqFlvotGO2tj#q)Dr!$uHhp0L(ZpTnHDQQ#r)*XF8{S@zdc zu=U8OT=<}0*sa-b>DRl}p=s}kQM`1I6q6xiymsFH1~H=fo2NsswBlLpG@VRsa+`zf zTjB5heer0RWYrEf`NAO$n?8CbZ;)I9CRM?_ZEX+8LiK< z^`h#at#s5d*!m7CFv~M*N>_Bd%lO~vv-0ox?9eOx0 z&-AQbL7y#a@5lBGCFugAvBgmSY}eM5sJj_9INiek8&0d9dF`=wx5rr8@x>Rdc4Bvo zl`m9%K=PqV$c9bi?sh>&V`~>Z0CdfDf{AP& zJ7%r#luE@f8P6jq+lRaM>r88JVT|D&;fpkTsILV+jqkI?k4_yctceSy9_2f-+xs*s zZ!d=aAd0(;@vz5loEyIvc^q^!vhQKUJdC`o5?{&6g9h5+*C}@ryuw_ertTuOkvYph zrzOUyqk%OGdKJBwR%}a{$6xoB5A<9rVB2HFgs9aua zDZIwvK{1ng-bqO8CaZQaTyCbr=~lcjk#7`-q^Nyz^`D(Q-S!$nJ_2gxwRb=zMS-+W^Q?_g%Gg%|sNf}7|kOTHIn zv!&m<22~v!nYNByNxo+^qeUJ}Qs`2rZR;ps%Q3wQ9KF0X+PmJDDz7bvBioaCop#uS z=>@&|*<{sT#px0a`Xr$ypcN+8qeE+^6}=xci>4_5=CwLeFCB9&Jsw$aooLUU z8a*Tx9Tk;IK_tQvf+(>q z&sw9Dnj5X1R+|BW7{P-Z(p3_N_x+`J5q|H_-qSb_d^;`j*{wjmj$f3?&<3^Hk zSwEz-l;iq-`y@NtD{Y)Q|tx(U%Fhf;aWtiSS z*2*dktSh^a?bPPDH3>B_!Y1?W8=KK6@Ssxiw_IuW z_C+lnZQi}9d;Q=$6;SKOSP{{Q_lez^Xw@djWa(FR|9Y}5ZxHXf>3nB>-`jP(!2?4# zn(-!qj_e8iyLw8E=ti&)LHh4Isqc%ZSlqKvufLKOXx#5f?{^k&MKb)sS=e8L8u^yM z3fNTxrlQ$A(1%uxaD6y@-e9U6hGkt?Hy@bXzq#%5&sBK_J1RMTs?>gdNA|K?GwJtH zBQ4W(U#6;OcmaA{4e8M~26Z>F9#6z`5zlrlGe6$)+7J}Mz_jAFuZQCw>z3eld^g

YNl3q)@W?*TGOFU5YPD$O^8x2rxmV7U)c@rG{QnQ4T|jl zfCX0l&7#I#^zsO&GvDPd>Wic1pDwOH4Nc2*b?YrzKfwQS;QzzpQz9n3FY|Q}*ZV3A z|HFab)W*ip>?`PA-`>&8(agqL*UHFI-%#IC|8J-Jzi>1u(uz@2j!=(`Pbm&d(}+un zQ;Y8@I8%)(UCGkQ(=*D`GuJcL)-cj5*g8ReUDGq}OH58pOi)9Pj7vAN@PP(_Y1z(c zDL_jwTEe#eQ;yRgNCgTfhNj_HryakF`2W!H|0}5C@1C>2fGhq?`g27#U|9JdWv@W# zGBsL0JT`QSq-V3slc8}k89t2<(Tc@iAD}Bccg_|`>?r3i?OF|I^ON3vJlX!1%XQuY zy!TIL{xLtl@TJLUzf9!&DkA@bng0sN{7){qzuWj1Zn-~g+`2}3t!*qk{Pc+$u_#*J z$SfdfF>F3Nyr288#Itvw&g^xSP?+tv6apjwfM%}$XZVQeU*My^kRdzQfc%1w zmcUVs6zy^$Kymzq`C!^nugdHEEk}zWP=X8vz~ZEarPKmS@F|7)LD_6=0PS?C2v0V> zO0fnA<%BCWQTeU35tc3|q>%cel)FD&++7a?;>XuqHnqJkyF4!Yr-_CK=Hx;#ky-tD zN4Ms{J>6&U1`7?rVu^*&A7Bvs`h+ZlFa!Z^M>cWM2SpI^0)DR01HMvq4FuAM6ZZA`b2uSO$PYte`{3unREjLIm$3vL^gQ(ozCI)B=gK3j*3E#|0v_ zl?xae#CXUZVMHWsNq|A5{^Di~g#g-*cWmg$h4D=ht@1}ER6>}5NUjv4jkMKch}K(< zCq}A|dfFDt0C=T_F~cGH5nC|z2Q?!^0v`k5lZY)Zv?Sc9AYq9~cAvC7KZk9>?F4$l ztHG9Q3Wb;96huz97>cW0y`dC($5W@O6ujwP;D+KWKs-!nstx(lQP+bV#r6{n4yGXkI zmiWwafZ%PGplSH-_7s+|U|8D%9YFJ15y%4S5#r->aFY9mLzvQy z$!7s@O&}4_+t#(J4F^$}u!SQn^qDih5hFrOnyO}#D*=3^V#ERY2@|p(6Xh=ZVG!a2 z(A5OXHMOFIAOwTQ+ggBSaHV-bEsGJhLZf&=UPXlBrwrv!lMG~_*!;+F{CNj+M1I9? zz{Q}8z}ExvQ3|1?w*UvEKCcPE9>!+}WYh@H8#mwFsrMoe5sMWK!;UbL#;6E{W@KmP z*$@ln7bt)$(cTHHIOD*R@7U1GlkVxMgdjDzd+|CIDssq-s8@x+x>qU;^b9l%Bd#xH zV*&(Ri%e)o@(%TLFh~wag8mE71qq8D%*6+U2sm$J?+T2#!4VMIf&;xrLC}T{jjP?d zg%-mt$b>qCjVh}uB!X|CBfxc%bEeG3g7t5p}PqpMRcaVS+pJVu{ujBu2D0TS@my7^5(+M~YSNJxyjtT{steeka8 z+zs6)@JwHg6*n88E4RG22pQp6%180b46w(6AO?0oyy#)^TaFAM^Ut2yCWl}UKdv!w zqtF`bGr%}!>TJS~ZaFm#nuhsg)1+MJ63|h`;b{5jw5xrmGm)QxG+shkOM!b>Uz8T7w^-QZKt74m>nj}+ops?ZhE^x81b>ECmaoteO(D7Cb5Wo+WX z8pYT&a4#$2<`%(K%Zc)w*sYHXSab6`aTWI%87a-LULlGc+`E?+nV0cQ{ z@nf`q0J(8Qh6f5r38~;VARfm>);WxEDb^AL^6Z2^H%J;T1VMZgB9a=0Bn1R9|55gm z6NXd<;%oa2ZM)xC4u~9-j}+1qCprY;`zq)(uthk1E<_?heLWYNj3w{Xdo%lN#XgT& z6KBTL${AeK(~$a?+B^ob;+OQ9bFn~z);M1T07H;)TpW3L^7&{Kdpz$aq(NMf#RUil z1z&CRJcUa}kB}`hJ=TIf5}zeX^m&*^%+J8<-qbK)G)Etj+P;ct?Ix^DFf%*x-u!qO zf5Is~<}^umz+2}&XG}D}66BdmAtgOH)_h0UEkrey<~WZuLog`O9$28pN&q61`|Tbs zY_rHJzJU!zwsiR3x=@@4)RALvLRQKgg>2`x{3@WD;*vRp{E;3vp%+l}$kABeqPwX4 z14;e@uPOpsGO4rORos4q^&lP^xjGp!$);nT-6 z(?yyKP4U;EI8>P6D4#yRYj13fX9z6!*4L(%udYSe4aeX>bQCn7p&Y?Opbi`fQ37KC z{fWRtt>EgVbL0xGRun6bo9U!M8}sd2=8)5eUxdv`JIS)PTICxaipj+kLkqoT zKn6l$AA{eW8rw5k`-0QMsS66A=+AWBJw60PXpWkYX6{9j#PIIp*pEVz?m^A(|Fjg3 zb75@FGqkMBkeX$sGP8oCc)if{4<>ojjO15nWC6i=-gC}zF$=d9WwDSy;aHCb30D*b zq)&ptfZhS?X-*MtVDN=TwkP&Tpr$;JY1;YZUNrP+LdR!erJYkw3%%vT6cYyiZeoKB zl`_qoDvcQPy{0sY3{G8Q!9~6XLn7-HvW_%*em1iZ`A%9wuYIY9#wDs8fyiV2XPj1G z^bKDo7%?cRjf3fIaxe22C1WJ1I{)7&87X~_*UqXGv{!lCS2eyU8A+Kz`|T2b5oQwk zg+cgIlRK(05_B88T{p?G=8vG5{YF>17MP8f?yJ!LR7(M*rN_dyG<<<&{#K8WI>b)+ zLRgH^!ALz;vyL=Q<7t|!M@bNrKbzkg-a@t{${MI7c!2U!rQoiQOw;#E;G{-G4;%ER zpSlLzj&u*w6Y`Dr@>>kCW6zepT7}xO+xaphKYmM3OhweCQeoV{7pz6`(@9?bR>jPz z{x?Gs5P3vjUV-c=mWKUTX6w^3mDzVrn@gFO>@?V9$lrbTdO;8^x)3_CTS^U9XhAT^ z)1SwA%)_I7ql>eBql5Mtk(rg16YN+ot=3)P208ih&B+u`@`VKjXD|HUdoIO^2W}Ce zl?$bQ?d!yQaGFm%3pSXyCSy9-c=F9WHZ~z;M!E^jh*hExQ>zcyh|}5^f95LXmj*1h zM_yARDkcV)OKd`=lIaim$3lNtc5oSZ?ze;=-ZD0QX(f6}`HenA4EK7Qbk-HAxU-*$^gLe5Q^Cdtn zH=&%WGB{_{)}5`JgZgAHfSK30VLjnqPk18}(9K zX4_jj4=$&DGqzKgyieTbCev0zb=&bxc67i;>yCT>RootXwh^#Lq?o4X+V(+*cg7Q0 zGEx**Isbj~8_&zs)6>(V4aarH`vExH^F-SFYk6JSFNpSSOn4m;>y4(_GU0L< zSV{9A^Y_;49a?lPXs%f<3%>qmMk|MswTI zKFur<&|HX&z9KpwJI&SOexFWqL|17{jCz{5FIg?a0K4F9eDGfr)mxVJ&FiA#PX2Bb zY2i5)7-89dW++b(U0REnv3T1ujQY-eUOromD{sV>!E~PN=Wt7VUMKPBaJ|qVi)Ef; z9AqIg^X1)EWf0G{j6RdE73bYob{Txl8jE784vyH=vu-f4wLS<&6|}B$>>u)!)7ai* zzCX*op$87XKYEvpTnU_VYP?3{t;9_5s+>QWfWP?NwkU#8ehNPo7cjKZU9%O*`WQ$f^TNVP(~h&FypEJ@=KVV}+i64Yv9kx7fy_~v@Sa1Md;vY|4_tK7slKC__VmaLw|~az8T3J+ry+?2u!PXy?v6mi1|#! z?`nl}zeeRy=%Ue!Be?Vc8Xs!%mJXIf{UnvBiyi`C?CPusIJL8yi-h*#thvw|-fC7? zH=@W6iig_vy2)dR-2*lf37(Ll-=ZxznKkxnSz#qou*1KZ;#>%Sou2*Xu^ZcbSdmM^ zwNNhiROLFRbFkt(Cn+=NQxr&2qn&TFotG@+-@Sr~KARmj>9lU2M0fBqRGVJxL&k(1 zk6-*g{D9JpGDR7l?lsBnZ2#SE>ova9Nw!B5G0(@Uh0YmO^PQQcs@SXmWV(=T*cZ%BX!}E1jhE8n3X+y@Z`T?e5hN_9YJ= z5*v+!*&7xX{5kU4n?l$}8s}4* zcBa}DS?||RGYkG&XzC^5-L)7+Qb&&~J4SV6-h&j&%QfD8P5f`S880xC(xL){8u^bq2LrBWrPN zTXb53sU-4@FvD=O@Lc}>O~g`eE}Hd={l{BxtT0+}{tzx!E1^6v)Zyu-uatL1^XLnx z`t8y_l56b5;_UJ%s|j6o2n7z)qU`}dRO7bkPz}!bR!ivbm9%D0f9t}|tFd2WAWWzC z_J?Q*zqx28>s!;Ab4nc*)%RdK-w!_LnqBofuV(|n%8D<|d6e(nkUJoZ9w!)4Fq3o@ zH{Z@0U9Kn12Ya8TSyXRish1_!O395NNR5LTI`P2NFqun?QnqwuA5K=;j~Yh_Wxh08 z*_v5c%7gdt)?$~~o$|Ct5f0}$_dP_mqZ=QyR94XAZwY_VkTW(@t2A4`C!J|kBRCWX zjZ!@Kj)|ToRixtP@t$!X)U5K@Tsqz_biCCWlqN5zY<}m|t8p7YX^hP$!`Rn$*wLx^ zNx(jqqeW~sFAcB$ET+`z0_nZU?Y6X*F!<3%mIC*iWRfn5IsnHro4V6}zw_qevS#hR z%Eb8mdvrK$0FN}wyNB`%W_vl6C|%4O4)eX(RaUT~=V#lv15$;Xc{{DWYd2JXVdU>q zq)l!X7HR{TSL1E3=#xyfPl(NQeroa(5ewz`M8U@EWgzfVtH+DvN+;eA(b!O=u}1H` z^$$_x6TH0=*r!gts{1{rF_=k=?&@`Ir6x_Ek(%i~e~~eG7$Me-)AnowofKADipxi% zD0rKFy7gabTu%dhGcBB!>j1H^aZ>!N$(Vk8(4+h(~hS60FUh%KL z0ux;oVlLxw>lsfEY>Q)w1jBCZ3dBs<#zFxXX9zQmB+Ff zOqSSPQGH&@_QjFSO*A~*r^{A6O&RBBTxq*CQO>Eh@unlip(AB!r?`)^hbz|YpN`j{ zQh|9=jjZUVDz&~mH+p9GHB^tkaL@5RX9gCp2XB7JD^!+qy!N(x zT@o6^Tg{$`0#0(XG%eJ3gsCiO-GgO;1m!Z+Z{OrXtLkxJnQB z0Ur~c)IV;pH|u{XL>sJL)|?`rZbUU#Gad1Mjvw`RLmd()OI2jOZ7lA}P%5yt@5CCP z1g#wV;@Zi6d+qdjY8vVI&OT_+dAAu4{66w4f?-thUYf2_b+ljRo>8Sq4s+8Zt}b{M zr4U1<@^b}!AZ6B>IbOztdw2}ATF)ecEIj27-{dYLYv!LWK4k^B ztMvLNKSBQ$?DRvd?7;s5(&WC1!2cRd`;XC0|4mr+cl7f=#5O5DKsa418UF+z!#5V{ z!^b!M2?Yb2gOtHH9#Mb*!}V{_*@GirpVL>r0=~-sE1vmx?CkGY-(TQoe`0+p_JrQu z9GmNdy!IF2KDV?ml+1|luPb{A5u1CN!E-4ODF+KjGAYV8^-cOJ#l1uKOiO6?ULRQ9 z>CL37<6Fu)CyunO%~n1%KGL2zS8BTLOsCB7mtP^-^X%s>hl9*on+;_RKxkP*NaWyu zv-=KKxhql@OhtM`PZJTs@6tB(d5LY;WwKm*l~P7G;=%!;$A@O9v9cwojjzYC+nt$ zsA`l&DZ@6uR7X&>s465Z9KX1HDkh1BV9BD`?EEQRHG`dgOtNcDQ<_TmMt9KLpF3WF zG+jQ;M;Rck{PG_riwf=_ufwtEprNU0V+-`S8KIB*Rf||BpMnn#TIKJ_c4sgInSUi@ zv$u*g?~N)jV$Jj%cl&;WggYga#9$<()as?&H77))ZC*9F$L;&gycY4p!Llh`j}4(a#B$Ww~f5aK;yV$ z_|-kL_U#SP-l?CF{lqjV`z78!l?sQqW?W~9rT4kXw%Bwt`NW4BaB;!ZK zXDn|g@3HJ9st=gX8`?WY=W~7W(q}MlbXWFEYgHKa?Aa;yEs_-aSJGQHzdkqc+7rK4 zoX4y=l3iE)#aNvwu?|v<>V!4ouIP>Y_*+q*RLtMx68(#QWpuu5n&^c`oaVHX|ic|Ir6yAys_$ff* z2WOC`3dSuwvtLBYvkyj`O6HHvZ|<`LS|^-x{vEVVd?|w%YRXd*DmV#qIh1ofP=W3P z4%BF!u+KEt9tRqpZ_qpIExF$AIXtasevD#jsdBf<#plPx0=7MA#$L@;-RbTGRW+>a zrbupibn-WH5ZqVDUd5@a^N|8ew?MopZR8EkOZDV@KW@PTjoEtWl9vuu*%sUHW&e1t z|7(%q{(7#-amZ}G9%+C-3hqBW*Z-YEMQdwsV{7!~*)nqY`-=G&w2yzT9W6cYK`-5H zNB?N&4{yy^^?(0R{!=@DwSLgr8`(J-IXD^_{%s`z{8~l+DE}5D`;Rt&{;dtruQnLz z8CdE5c02Ibt-Jp>w?q7SJ3Z6?vW7nf^Y0dZ{eb?b8W>pqs}kq`O*c;2jspK&_Wu~E zzgh9sjju}nH<@jJnEFTgw~_jDc6fPy60ru#ULZ`lRt~(PluSCU2pyINO$y}T^LBH zU4l$Vbf>Jcv{YSFH6IbC_=IOE2>O#b`QDev?Vr*2eeeJOcCUNi zd!Fa~o^hUYo-@Q81;4xJvJ$-_Vow&Mo!cx4UNFFPQAr$iA8{XPzXbP^uVp5evqNk$ zcd5Y_u{Y)Oz0Bu~gP)=c-`q_^^BplftMryD9W%b=yl27C;uD0FvS9|HXUUCy#75bK zSz%oQ(=p9%?twU-?P~g%Qp!Z(1(V55m|tpM`+kyoEvf8TLTqxk6h%JRLbCaM8h-%m z)QYRDrBR)UBFC2-^#Z!9o*UD$%Uc6xdD~sfBX3aI=RZcVwOdug`IA=o3{jG0<|<7I z=V#s8`81=^AWy35qNuU}%7mQ)G#%-Bf(VWObBSGQLgMPg-oP)GE#)-Nt?%u08bdou0 zVV;L3SPV#k92(Z9lXcX%ys>#plldd$QUf7r(>~0#U1=81{k28X%j)Va!|Z8c%93xC zt7G5L7nPxQCXH1ji-rV>nAca=S~Tldj>^Ubn;BQkR}S!&L_BMkl1~=8j+}-Q( zU-s3MPc0#$^b;AAonO_f1aWI)q%5ua`$}E#+0Dnkc(5&~C**9VZnDQPf^oq)(f$i?fNe1O~1uEC4OPs zuyW+XpmuBYh(G&-Kz+aYXwJJ7Sy54Qva|uHX7d%XJ1Xmic>R@}k4O>i@EaT=rwks8 z=bCVQ3Dy~=U(fwn8Cxpdk#slXwy3dsp@^hb=I2ie29pt_$m2g+DFQUwWGINI_|{t9 z-G1&Q%J>}Fxcl|@(9{%uT<>vL*4&Bp_NC1bx}Rwt+>+s|Uv{p+Eet-Yuf?9N(ITh# zsi(7DhO765K9WCIwmC0sEG@)$>vbzBN0)l5`t9!2^>t!ShE7{`*~vN-lh`kXQQG*5 zJs6hFgj_8as`=A3dOqXAW>bh3>W{_kj3)Ej()4Chr6s2b6|Lxo-#lmR#5c)XeZPKL zeN&F%>2&k7a0p2WPJ7m*XTx{g2L;6_F33bQBQ|9?Jyfq|+0C{@XQYSB^+UE)r<;T& zpY6Ql(AQZFw;h?z;F@0_Oq-dqGWz*)V@3w6uHg~Y1?~LO&Gi|SJ0%oZMix`E!?kuc z@3AqlcnGR@L~ys0hkmecbP)San#_-LwBzR+Z-+a{25>ClE{ChASC8_Tw^HzMtdC)J zZp-)PZf9pnpGg@jg$dbK8L*5L>mL}d!6>P<-z!k5s+uy7!>I0pc872Na=5MRWVf2z zreS9y+Kk_3wc*CR(_)Q5j!c>vCOucLsjj_6Z*a~`$~*NLDqW4Lj&-Zv4x~ubt~|Md zPq~jUzHU5O$2T!6alcKpE|Y9U48~lqnm7?L zp+)2Fmh-0v^<|Hovd{MQl@k7x9ffydxww7x{LPnRy0BkM&dg$Mh__CfZMl(mrlQ7; z3{R80$Zp!i3#!QA_%}#OaKJmcVVedHP`VOt&roiyf3NYT7p0V8QvR&d*eEDkLTOdB zKx&wfL&EflGR^Y54=(HC$KjA${)!d(v!6sIF&BV+W`o68Ww8=fm$1UU&k1Q`VEYe! z)-JpA?8UO#7wnEXss37J)8Twz0-B1t=trDVuZpcM`x7-6;& z)N0IfJw|bK7NS`|>~h7eWPJ2^9537}K|Cd+eM0*MJAJK#GtPC4skwC}E?Ziu&}^}~ z5KOXfG40~L*Lh;d%(=dE+-4cwG-gJyistbOUyFLoOhZ7~^z%`kS|RWAS@nFmd5Nr= z&c0dZ0N|8e#<%$)c#N5yW$*Ig%j`HcUN@Opuxks>^^M(1u+G=~G0vV}Ij)x)>5PrZ z6Hj(gS(U>iy{m3ghc#e$0LCWy34NGH6M2qT*20zcDb-k`b@P%fFP2V9re$YWMT?qy zL0_D)30z;ivh$Gc6qsXR?n-{H<r&rQ>TFS0k z+BItulBhHO=w4@9`KWJq8J$gE9JTVfg2GjYR-CB%$#^>`E(f3MsXDi@%D2!LNbz?H zl1E01z&F{j}Zc zbaIhidq9}3s?SCL>;Z7>d3R60Gi~$=D#_Jm_>uyQ?mY4f)qV6N-Iu&+dVIcbD+R(X zBcV`K49Gpw>STI1&k-1{&QBU{ma?8ep_<6GCN0w049_N{Da%Ys7vJxP(f-~3yhLon zw|IflP6fl%0VTbecFp#;l6Ei9CWNyinJbtiZ8l!$o)@#wl7xvz=KcIuR45yOQ8!`g z%(B~n*Zic3okISshqiY91NbS?@#$**($33m;*K(3I_=AbCbbbTs7`@UwSpq@WYJ7URI$X8O&YcBna;q?Ks zVqwCd^Jfjugq;@?R^XPizbPY4rb!yAM@b-1#YjTvP5#E!DtFv{0PPAJ{epMQ2aR)! zGlSOzMLsV|aF#q=@(g2uiL1~k8!IgG^5i?Rg(fBA>2}Fji3`N#8bK_p{C;C1-sAIjup!%3 zEg~^Z%&dKsny;(fCxX_ow?@(68T}tAIu?El3iAk)^Gr0??nZurV4Ry=+bu-*vZL!q zCQ}e6p)^yt+9Tu?bM|3%PY+H`>G~zYM_T#ZYaeKSkedajm{&hiw0Sz>t?w=S*{BsC zBi*Rqlpx2ISHexHy`_>RrmG|SvS!v*;(*-jurXLkR=6o*i3PW3p5GS@u04ssU>BkO zR5~wqt`NC9Ng$P8XX<^|ws;ylGasH+QPSl!2G%m(*OO8m3}Kq@M-0U-5>|w;Bqn2T z-Zz%(gr$AU==f-XMFkDB1J2K&^$lOuZe-e})mz4GP zak{o+SUv+ii!^^gbVFONe=djc<&O$s?$R=Zb{;oZS+zX{DxFHbNXxwmDpr$MVcs!x zv0arTNYt@z(Slb^O@AcDOj$D${*75xRwm6d|G6Q<)#@C};80wq+kKioaFI)vgwkrW zt)>QF;V&+#WW3~`SlF1b?Yu=OuG%cjP^2Z^tG$^fDkRNr5;*wkXXxGbe?_3PppIcQ5hxLouwQhc zhuwV`0xTkaw{RG9Hoz7?j^Q0B131~zC1F87{>;X6o>z#Fc^uupDv(H<_#dfGp55@L;1I2`c=T}q`5%foyF z>B6no4lz-}M}qJ)DI0F1mr;+e3|Lb|=CwVxxpGr8iy-lLR94N5I+mZ14~gMeZ_Mcmm;W9Qee-1B8Ri~DvH7Ye_)+g z9L!V~U|htK3_scnJlJsE9wGT!w=wu7Krg2_INKMtc5G0s2jZTc-N>W!dKY->)KV+} zCWyNllLb8RN`Sld8vMbpi{>2Vmr6RRP8i5N|ABshjAvOTHc+~I8dEAPNW)*8dJYfyUXUUU))O{cwc=lWU!`p=$TGRnaE`b z4I(dwsxi($uQTrFqlj36a;+I|tieMfnEh!$%J*}DS)~vHXOuILn`@csm%z_{T{k&( z{S;99$3N`;9g05d{!2uVdn|nt4{tin!V>L0+Q(#^2CA9$m6E3>;73n5@8X@k7p1TP zLUOaDnGmp;qR5F0?Mx8>ht@r1Dd;c|HaiCRF$9z~;FAa%iXPy}F|%<~U0mReb#dLw zuC_0dEz`k3Vr+Blz)v5}I!fLUES*x&CAb3oU0!4h0|PntsULU`sS9Tk-G^}OEmkFD zF>}vEA>)LD!K(G-?aa;KbMDJ+LEN-tUsDUh_N^8C7{h@ZP~*L8%qRPo-QH7Z^X8?N zNvbh6$58qiUfbLA*E<8)0i@1ZKDz(OqCG%4LLWtQxsU)LGyi49mL9dkQa*J(Kw z$h~#NQ8?*O-T+2p7Ufo|bgh*jT);JU6PptMN!iCYSKM7uy?M6%z`-=g(+|bm^^Ll{ z!~r_KumUixyoWfoCMz4h0$IN(iUMol?2Q63y+@DbY@P$1oG`&3gCc3wRIIQLqc;@Y27K9%mbSFHq&_+K!bXs{!NdX zTKmJjt2nm1T;6Nx3q}A8zTXS6LE6Wlj)@#JKgbhw@$||bZsUkuJ%XI;XZc7V7X=4# zmu4~oD^6G6C@qRJ5v-`OGJEPJ6LrlNr_xnD$}ROl%Gx z?S>P_9zM&mS~c{y;d_i2!XB*u$?la{0nma36q|Pd_9^)f3 zw-!Sa;=5`eI9?g6A}YlgeiC{VEOFM#2|ZLj3DRzyxai~tArbWtY-mp9Lq234kpdLH zQr>CTysrtwOHDAik}$DEytv4>9nvQiekn>wT5Z$n_Xrp`Y^PH$+~x*2Mss8cL=1Wa zjN&zp$^udV^xTW(*V*whuj&q9%P?o*ke#+2nYinMvr_D$tCC6rgLUvD+E zUB%)?5*bW^!9+lk(EalHFvsDT+9cT1qq4 zbu+>C53;M1)b!@0XB8JVH;oVcLk>i8(3>6J zi5x}LQ%>dN&fL%w99>`HwQM?6*j7EFVq~K_$C1`^bA92)JMN*(j64e6NJZ|?=nfW^ zEBTUCbFt(q*O(WAeZu2Vl~lDJjVvVinVbz->-~r;qHu*HK_nl|H+VHeG|qi3B|SIi z+6^*EteldLv%bOK71$M>QpqnASavY>v+q9lT1>$F8zpHAatoD~eukDBk3%)Vt62d)5{qF9W53v(vx zv8X8LnAcBWxwX}wFGl-xZbJ9dYs-t@hNcU&((JipW~%&0VD1WT-vni_@1cN-~V~+2W!TT^VNV7fQPw(UvG( z)MNjlM*+_VZ-EV4al~efI3OOcu9*)>P_kub5j%5o)GYDrb4I?Iham-Rl<1bM4qvz| z*)E_`Yf0utpGj~tDKCs;*9?-NVqckxu5@o^;7C1dm-H>g-%5^lX4yr7R6j}bd2GLK zX86JryvRAVp{be;g^=$==C9^ODd!1VavdFK`c|EtJSPbrZ#{G?tNr>u#M>_u#W%R< zv98;_Mdi`Yt(i>(4u~}`CP$QQnhAfbCT-XXUEka9&4r>i#^}4kMK8Ig z6Rup>Dzg`ra)oVuuU;FN#_3DiHEM+I%l-l*k8`-LEC@l6^Jx+)c1ZTs&S#G6;?lZh zvlXpLd9cwJGVqXJcU>6;4%eHTW@&SELuG$gP#kBdJBd*5%fwMz7`gJm`yx}#$Rx94 z15xKC@2WG3X3v8&9u#hLaw(ahv8mK%*+gwP=*n)TdOP0qFIPX;lI_1y+3y>zhaUG$ zpik>&>0NWN_`C8r4n6}idN2=nJUCOuv!rH1@LK}e*FsMFaTJa& zmqsG?O=nk_!AV6uXUE5W&%LGK8%!EHTc$Enk*A0B!yF+=kn3yApg7S7Rt~m@5b0bc zℜt0*c0(1Cv*$ueH<<|2&j|2U7%}6xcb$f!0C}xC#GokjF27IwBi!+T~9oe_Q?p zKZlI$%!5?TLxzM@RaKqY7nK!(=B3nzsdsqM4d~bffQ02&VD|Ty<0v?7$-ldvwv{%^N%$ zmUp=_FD~($V3XkC;W1u-%joASJ#(J(q1OQX2kLIU*KC@w0(ZV`=i!n+uQ93FvGM>C z8^NBH<;d-fL>0H@d6R{ERuq=b@%m(5!h701s!Hfv)6yj+=s`?OU*hHU(PLBM*{fFy zucL3xPY`($GqZ_xuqYU8?_i6{>pVYc zFKcDw7@6l7dV8~y;Msp`H=8cLenO=qV0?Ze8~@N+Y!G%A0JmFYfQ2z z`ZNVEs?6C@b}7JD%guP?CwYgnS9)_!=fpEae;FAZ)6vb6Z#bQ8#O5-iH_N89tK3Mg z$Kqx;>Mu9>2FL_ZEq_qw#=#bozWZ|PA+x4T^FVTS>s2+PuH;+v>q51DqSXP*@a|H39iHMM4=Fud~uri;+=bJH8X>Xf(KG{f=V2YnxWE-yc+>axE(qNv% zCDZ6uvJ4KMXd>&JrkKwSp@lZ=3O$@0ul%Th<%sCi59dpFwq`CR*hhp1-oj;~wGb^{ za~0P5td#p;#U)v&=~9_mdWuWq68z&}+&Tg=6r24iHd0Y0>dlLxpxx1M!=QkH5 z5Ohd9^?K*O7Lv?pBb_FT{Gs+t`NAl+MMd|Ew6xATGdHBE802#5twrvv=WZB{wwA_Y z&1t=f>*nLzGttvuwYkL%d%ZDGm3_HHaV!#JcS78FYXWnGaD|hzR^F&&m*OV;_@*qh zF<6qQU3xPGZacb5W{L3Kmp3}X(DS}&YZMnG6Sr$yr0vA> zp*o~5pli0Og>>b|*Sd3sVWm$k`q)=q!13D&vfA;`uB>k6Bhlnw&EReDFk7-RtuM4(&8)zfm8c1;Sa@*Ni)ZXg!n|DCYe`&V4lt5?eTKq)7SZdo~-APT0a9+S= zvj}mc_rm$M4mykmpAej!DbJFE^udwUl-s@2i`5hhwigNpm{1|a)E!4J&#^1;C=Cdi}P1KK}V4w$&RC(p(#j<=W+noJuW>v-0rAfh( zGQ(2o7(S9x5FV1ivs!GPoZag9gSf2P!g5U3LbV%Zp`~|*HP2{~K~E;E{|!roN&3vJ zrnc}5jZxP+4`XLU_lPj#W^I47x?z9{TNXoNYgb0q1UuH(Zwia~Rc1T0GrTKb*tL90 zgQb)7TJx&{N}scc+0#YNHxJ@YWwdUN4B5mEz{vWKYVAlbt#G0EtMI6<3>y`>1{jh1 znHU$l;rf|XVbg>z7CVjDc+{m>N$>W6LRRjt_h~fKSSx4al1_5=*W*AuF9hbf---PxX*BInJlWWYzClMTAu+a3nl79B;4Y z$ZB-dv&&tmu%gwrrPCLC>Og1sa)drPx*7RH2y5nSNjeN8^I~m-toizKoT7?;Rtu9J z-{;g@+eszUBbWrqTeDl~jJ9y29uXXYXV;%BHVW`WhkhL6s~{k&_ww@i=;I+z6B!xZ zEal_p@9*J*hgdS!h4sS{Yv|)^k5?2%ngl-4u?&-%$yW)Y#V(VVZbl3}RD5PB5lmg= z;jhg@FHWmbu7)Of|9;qK5xL4IRX?6QXCRV1&r|AUV;)D{s03^_AV4nSNzO)3o_6#3 zH3qVBua7OYUR{*bX{E@*-f#TUy%vX37auChydiG>sPHTala^+_(PgbXebLKY@U_5e z)9iXLMH-^_J2ZNm)8&!&FJ@c~2bZw2?-ILO-ShoTxZkuQlC(p4l`ZK znHa$~-(swZ@6|&Wj^iZRAd!4GIVq_aCpWH5G}=pG@Oe3LraH@VVCar4f3yTSpKkHa zG`wjytjk@Q+ssZXbn}KTIl07QGU$^Oxay?nnNF2Oxg|e!OupoZwe!i0$4E8TODgh4 zWITT4m60j9j3zoZ6cqe6I%l;a09gKFCv7nt-<=y!Gd)eNHPugE9 z>ByATB)!SPhZBCa1YeiI)VOoLq0mD>GB5CE`P6_-80)CB5z`mD^gweietvlqJ`?#5 zMHM-;bnW#jW+;+keA9K= zDVw<;NY`@KhEXXu-tEjR9JrAanLDD%l{8@}DVfxq(PF?YKH4#r92oW>=i65@iq}J3 ze5T*@?(zd09vXO+JR*)8u!v)6!(Kx(<8#r(Z|;mGHO6~rig)JydHHl!&5~he5$-JV z1mIAcWTpp5`3l@|o(- zZ5BkH4~1sM7b;(WVaSQ^dg-lyiF*BGiZs#%I@2s^o>vYGbVPEE9-R7@vePiTTQT{Y zLz|{-kXFw98c!q@De<&V1H`yr3s@Q{&b4T^ zI%JFEsrAjR+$c&jMk}w$viXtCDD2u2pDRJ(jTB&GAr5!n&Yu7VzbZT!7R+)x0)PH$Tr!f_wuaj^t%q;Kv%^?gzC1IOD0t}L{ z+*{K4UbJ0OE|(MWD!zuMT&`t|7UHFcE4E;gE|zY4jzfT9SkaR?ty_a({N{0nAifCZ z@+0Cm!h-xuH|Xz;4L=)W+w$U)x8tL<&yujw&eeY`JK~)8z;`&HaD620S@?o%EPK^N zLy5eMf`CNlIE5wijgr_K(K$+MBL#WoNghR3F)U@hUN_8wm>J59^^8jmgx^(Di|56p zYFoxD&C15s=tz~%8i#qv%eP8kyy~*IlTI;D(e1q|UEY^36eR!n-H)$je)8&!!svZ| zoS8Y+E|^IjFCtqvh?esF@;pkc)U4t12{_dRpFT@&?^Y)i9Kp|1O3A?r{`PE!_0en) zBa`PEIkLM(n#Ux*N^(3~ZuzU)4_(BqP8+R7IuSWNMjidE===ifl*sO{<2>g3l?aEw zv{@4iuLvy0rYIoSE;nuPDVvWnK6|zk2_s+`8__k+la=QyQ`HbeEWE&^l723m9y6&& zhe>3Hh4pn1$$PR`NtiIJxQ8JDcu^bgZktUEpB3GRW5dvox*kq?(ZI`9lDYnMuwG1O z`y|gLHBHAZQGG$(H+amqook+O(`O6tKRq|hM!OJ{5MxUrXoJJYKfT$C_j%|%rrf)6 zXY~e`OU=o}YC)^dy9`G2f9j}v={%3FNb;Co5|e&=Kdw{N;WGVXeNeZps?BhX0M`TY zZDYv~gr*ib#7WPhu{$|h=(U|(+=PjU2hum_o|4_UliD7yoV2AKG$4*HCqw!(3$`0m z@@}~nI%DO-w*$IK86{;B*lP;-Nk3kZe&tE_En60i<#FKN420t z>8GGys>JM#P&O&70uO`WeSPPvnA75kYY)?w-C8>2dArKUy{ zQ-v%;szDn=D(a#fdp|)q{pWdeWv#&Y7)nmHzzx==^cVsC=<-Or;q->Lc=n%tLp89m z!{$<+5>aby1?ZdFyBAP|Zs2cY;+`2N8NRf^TX z{^7bGvLXGUuuE_2Z$NbFKYrLHZS7<4TMfQr>_ItWeW9t#iVgc-_Z$fxM2blN>p^+< zJQ+SG8HL6SeB*=o3ggSJUxP@H<+0eU7u15uwNzO155SSqcZ{FzC0?YM*abjnQjK{w zcFDX7ybTM{dAf>o+nA0{bj37gJTkVjp9dBaQ%fvAeF2LEE({|>wAC)Mw@ zU*RAUSnP1N2*m*&5PzHcT{VWt(>ivm-35IRmNAlJtIXa~%- zg}%oEstrlisa`^CH+~8V))%)`<_x3>z4U4y?5qbMmV1Jw_Kd(6m@;}>I=k7nZ29Tk zgy?74K(&GVsctsc#(P&GeL`K3q$gL2?}W})lmw3B%;0Q~5C0vjqeq>eKY z{Na}(C5S^vIpU6j7aXMR+bnPR@Lnp;>OHl~&C-wk2nre`7?!sIy*`U4M`G%%7k!IA z^7}1!M&JOC-3uvj@tmBS5=vq4S|f}`(6ej1pJo!SJ%g|rps_$8%zBVIZdW`wh&4{K z2O{LF6^aj{VW}1hI4v^Hl+-Ygh+`>jjC=7vY zeQJVOf9l+L=2@@;xr>e6ZIL0x;IzDGuB)59zVT=?4X04J^1i07B8l*I_*kreZWmXfF2x2?q` z7=;6PQw%+Pcxe-HdUp(F3RLwsYR5+1Si>1>dtT1IEA(3B~RJ{Ko&MjN0*& z$`3MX;KcIXH3`rgXzyp#j3a8cM%3(IwQ_Tzanyv54mF{vS()81?ByQZc2yo`Fmtj9 zIk)4Nz5TFbex=uNwQYo)&$TK?Z!P-$TvR#wQVoXTW^Y?l#;j6IijKegCb|^U#1~#n zimX~LT3Ci!I+q7%osAkj$YB9m*ghTP@q_R$7eb2E5os7c%=eyG%%NmdBM`x5$pr_u|IehI5%%wU&;ixzRjpqDz zZf@4XXbhC)US}L^5WL|P<633JlKe;|sw%1^1y8oJzhPl}4oQ>vTWiwIjKs-nO^o6`- z9XtnV%kVqz#1_(}_^fyxx00_k^n^KCa9B8?`{heNzkDOX9A2WguGutpygDuOE-gy& zw*HU6D?|xbupy#Y!+EHY+aK9xrjQ>@H=uX24!&QiM!RkmG)&)Etsd6cuQu(H$e3*E zX>=Zsiec1$#WTszWaVR@o<4Qah9r;tWRb5-89!!7S(hc*&4`zy(b)XO{zmDfrU(wf zsi9eZ*O$aGOv=8#SC~X0R!oF>SWXX5j{V$)wc;F;mJRfCwN3L z^4QjAFG^3BbiP2n&e~);gP>vf(Vl{jh{3YZ<3ZoJy;;m6n*N$OO7LLjqD{r@P8VZ4 z9EQIn@l(By9HA#9D@tg5{*K&QGkTru-dutSZ`x>+`8<>;kgUi?I)m7ReUQH1$N*oq=!;w($8Hs_hgD!x~wpWk5;_M1XyDiu-m+ z;JCa9F;)7c8j%hQ_ezG)Baz1SxiKpZQ$a&bW?Kq`*oAwG4?c+BG12st$)~1~O;KMd zv`WgTXjMT|OzloLq*>rj39_7>e|V{9xzhKVOsD+h#f6>cF7YmRSFhkGR&U(=qVsYP z^Tp~Q?SjN)E}2qZu=H&a0|fP-p3Rmu+ixhgM?U}LyS}ls5s#Cfz^XFeDMkvWreI)k z;PxUH&`17iigsV%AwU4RdtHP~-3Q+QtA4;mL+}u|zbxiCuBfw6MRDvGwY!e#-%Gmy zRoa_FrR^@Q0t(#qzR;fcPxhc!H0{?Em@xFF6uQ6@*Ebzk0U*SLoQ3>1UAadOXm|m7 z48Y4Svfz3u2Fjxk3OgS8-=tm{kF(u~(Q-6`-J@p;MxM`BI+zB{h>GIh`V~RF0X3k(r!YffmpKT1Lf}^1tK` z9122pSnhwDKm^GBNHl)S9XemX$K9R*r~*;?A$b391^@!#^Q<|H5Ku9u3^2uh7>JuV zP^A99aM)F`Q($1^M^Iu8NJxJ-1U<}H127}=Fa~sr^2!R-RJ7#uj0!a5bbx7+Gbu1J zkkio7GSUKU%PYR52P_kaMuGpC4om2NZ<%I#D247ocWplPPuTisp}711V(v7S`RAdH z&`rq}CortCH``;+EdMf#?XX$?Y3ElDv;(wr4xlTzUmAs)mXflutdjCe@H9w3#s3n` z|5@^7Q@nTJG)wTtKPVYAK$jy*-lIbCN>5vBuj_Cep*>pytMJu}Tx$m)!UgCO&M!iH zwzPf(q21c-(m73Kz{SyzqDD8*0yMyz5~?!%j}O80-}SY7doBE4D}7M0zpj=39e>;f zhyt_@E)poBg@=e9qWQ}o{Yi7Tl7BhGa!+D=K!Td3G=Ps742%nW2d3QLgaVR?>vtuO zqPI)sNR9g~uC|X0-e4~~5$NjUK~cfsK8}i-)=LdF6;)uV@^LP9FM0#_!!Q`pDk1~$ z18+ho^iX+!)1f$29jNUw4g>{mr~KEp?RS+BW*HX1afNsQJPfER`5z*+XPJlA>z_vX zKs`8G`4XQ0y(UK}!a<#mREFQdFICHv25`U@yl+Fnhx_91M1R4zFQ)@yCxpj;`ZSaS zVxSiaJWNbqSySg!o(=Mc^&mr)Idr&8c?09UA|Xe~c@L96i@Xk1Bq{$POu!;36tuPV z_4HrM|6=$bODBLT-Qt+i!RgC;+nWD!&HqU078Iq<<0ygqbgLmGO*3V@t*SU>UI%IO#O#R=_#rzYs&wxl=u=O{mW((hogu;J;T0h1!Ib-`vre1cAx#zK>X*XC-)A(0E3jP zBEP{n;CjD!=xw#AfKvmNlr{9_|Jv|^8#(>8{n!&UXed)e1XN0ZK7lto6c?_?aq-`_ z>rYPhmF>uIZLbXjn@fdCuJ=!Xp?g0(2!!Dm4}dv={Uy<3kod1b9YSJP3P-k|zuOvs z%)tdM1w7gRE$!bS*De<)bb0@5X1SoG5O^FHCrjZ!?(7tLA3u%CtPg+!-mFl}5Xv0` z$zf)GInzV(IB1fO4Ab^x0$N63PQW#AeH?g0Gk_w090SLj#=b(HU=*O`t8zcp-*>bR zp%AD&mVrYW`4f~Gff zZvZ>Y20AzUeE{H3z$|;zj-mbcSo0tI91oySFH%2--XYNU+SNZj(LV0G^*%xW|7=b; zq4?m?_`mRRXwxd78*u_7e>On42aJFXQ0{SzP@E!G-IFpn9=gW-@Ddjw)$?=QG zo|La?{!4A4njt92kMw_gOn|M>rCj2i??CkQm>kHiLNU>-^)F2PA^Kg3AK3u?b~u1i zA6)z0_`R0?tno4ADGmbo6SVu!=AIZzm~to5_%ovaM{@K~4hmcYQ!Uf=(q0Xx!j{#~D-J2?cvQu6&s?mwX& z#o=)JgbIVETaT=g3fd5OLs!`9STg_dn18k1g9;yMzk3x1?f1ZXfH%-l)FrcHDtw5M zzqFUo{_=!=80at2-FR0NfT0}{x<4CBr~fqU z`FkAoUv7?!VfI>f&?ZZe48|gXc%m4{XF!J+LC0}(yomOF@@b3{v<=bQ`CAtO;{+q9OW-E~Y zWkLcdG*xiW_Oy5%uSm}$|BbU_@j8_v{gz^!9Nkq*04qRiAi_|1or^k=tAEh%Ut&A} z>;%gLMTw;XZzlyzVYp2UEMtda4K4a4)}Raz#ETk`g zPG;@5j=L0hqW#?`V0Hc@YsaGYzwim_F68G1uLB8TU;q=^C*0{ei8UxqJB@cgUjO$* z`DYk{e&DFUku0yJzc96XD(_yN=14ENS2!4Gkp^A1*;nEH^Z{VyK%g~zbm6C&>UCs% zwWkOm?MmyZrguP>(-)Z7b^oAZAZ_{OBP+IBz5RHH>bOIPckw-e4Y2o_1P`273Pjlc zbbpQnC_rHTJK(T~?`e((0Tqyzk(`!!my;vAp}zrvJ{TL&L*Ux;vw|0o=Hz$>dkO%c zkG0+-3zCj_gLLU=vj2p@fqys!3ed+-A7*Fa70 zEWiC1^ie*C*=Zs=8kW!PEbGvu24?yKe z``D{87!N@oEONNY%)n&${Q=TnuZN~B9l&}*Wbb5;(BB7# zfWWf%FX$=z@MJLHoKo<(5dfYOCVKo?6!bjHhx_mk_C8bKEA*88cZW*bUr}(1 z!LFmH>{mh;c$!A`NI3TV1_%fAAtZ;jg_ZmlvU^9MoPIyX<5!RE%IV0Vwmrr`WTE$l zJj@u|Jt)ThpLdD`wCG4N{f6!6iQY1gpt(D~`8`2GAP~@dsT~Fa?J*Ro(@W^2oz+f< q0rXnK!;HB}LNWFa5;|UY4qVh<^#|A;8n`rp|3-kR+(~kvYyBTRXMfTF literal 0 HcmV?d00001 diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v8.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v8.zip new file mode 100644 index 0000000000000000000000000000000000000000..165dd4376cf296cc59ec2d40d76823ce6de67343 GIT binary patch literal 139172 zcmd43bzD_jyEeR35Cs7xrAt7%I~D2fX3^c}Bj z*M_hg*M@eEaDYcpXO92?sQYqK|5*Lm@9MM+|Bnc8e?g#Q{vUvR`#wDN0z%yVP5!SS z)YOI+7P>}yKuaxaTO(T|3v-~Eo~@RymaP`GjS_UdcdVMA;EjE2!Ds62R1}0`Y zeP(7_Mk-n$9Th!26+H`(o|c}8j+UO0j+v2_jsX&+h7f9Hr)T>Ib*~_l1^zpAqSPY# znwpy0O8CuR+DDI8+q>MX=qyN0@|}9)D>y6`$lSLoj$TFUCv>^zHy*7=CyQcN_L{wRsYWqpiPMN)bW2+r$e)2Y*js~E3`YgYB z!~d*h{XmOG%QW&EN#vNrc2eUdsVt?d@v^_M|)+TIEctVU@ze~bI^~$?yg1!1>n)yHlX|X)R zEL0isex)gV@|#8T;>*;Tts8|IvvG7M*Xv8vOmmr;uuvq$VO*}OPL*dN97Q^Bmn=23 zh3`TcT6CSh3AFD-(O{QvbeHq(OV0Zge*Zp#* zL3U4H{%nc45wO#m>m(-WZ@gEuQvOI7SK85S& z^yg_>KjY%fNoH;4iuT$h#p3Zv>FH8^%@B??j?mEKC?w6Z1X8TL>eEg|*W5Kec(44j z&SPi)?o;ofQH)t=FK~>R*t^NCKd78SmAbjrxptm7yPK}}(-A3kC;~5w!2%D0B^UW^ z+YU5!jZ8T?bn_&r!mQ=pX@IQgMRV(^eWAXfLSZ8rwdcSd8npqE<}S|`K~LD4d4>3G zNWpEG^^xprm-0z&jF)aVcFS(sEi-dd4Lat+)26mZdde=-McE4@rXdx(-2Ki+nA}eB zJT=QA1?txTZM-eGVPAiqHJ;j+?cD7Vw$Izw)jaNiht|pnuNS66J%~Tw7VWvie|FDk zocp3$d&t0CJ?y(D>ayV$UaqPPj9a7~ls^0&OMaOjc&P{~} z33)$TQ1>JZq<_0L5C*bq0}iC@uo0bOrqz>)p9*lLc?m0do@f+dL zs||L)Q18irxmvN!W6@X|GsN)f8-)0O^t^FCblkb?sDzmqw zY8D!u7MV+&5gh%U@!inPjr{vVduO7FF=0yKRT+DVZn;BAYH3<} zX~h9<@+ zZ&!O*ul}64*VD@n7?A(~{PL#&*?*<_e{0@;;PD^k?7ca7SJ}0?#-j*kstn$>y-#vE z+8K47LVc=08SfO7*NK~+xoQ_$n8(&KsYZ{D9=)34#b9SXTb}#M2CUr7bdX4358IKF zBDD$dI?xyu_cMI~w5;BlStGi5V>F%4W=2+;Pz$cd<7Q;r=sE%0^xGtzBvg==V6De} z$Y+)2^oWWfHHKa)#Y25gmsSgh_PS5NrfkfRS(ys)b-s^&@cDTLfpiiX?6PH%dAJgx zN2L@c`AF#JbViIm1=--Te%&x2x!CY?NK0Y`ugBtp<6WPviOUd5!d}q zn8KEQ43C(aq3DRwv$~Yi6ch5~8e_b$3{d(n*QT2>B2C;PtIIgDF3YRRjdGr{xI3%{ z<*#_uRvNW%uwW(^Sfav4k9(YVwG^?I8@1Vigsp-cuJcHMcKLmpCBhosW7<|2YGov$ zO@vNd^~Q>c9oxBMp)57IZsjT!J@{27!B0&RtEQz4%9US?8%GbU9y{k#M-Y>B2h0tW zp~Uzy*E{y9^T5AwU&-VhxqXu4H_@P9#r~treKvJ(W47q}gs%pVo=2_0p)NyHRj*|# z%M_%w2CsVH;?Wg&j>_Y_{AAA8V4f+|__{q{L7uyq%5l7Ie5O2!{d~L_YagfBqCayD z=E4-O=3umIXaD-N=VC+SRHbFU>a2&P)(JGTGhg89u(JhfsLwlEe-&MLd499cajV<_ zs=uMxVsUf%B4<9%GJ8oAjaKWtW=11ke;DO(d!3$7qh{AOX+Y9&HJ`N6(8;qkm}Zci z6BZqzK#81{A@{>=-w?W_Ud6;Ue>{$cF+odRIaRJApie;dOj)2uIWw+x!rE+GU&7){ zkivZMb37B-EBF~u>u32DU*oKV+I^mGiDY`iE#&6l(9wgr+abMV#N_Hbj_7^L!n)gn z&@8TVdGC!n_>%dnBgqPc@jNG!j$TrO_!mT>Q1KND4W`FN2L}uV@foK~`qM#4?5g?7 z^q0m#$8pTpIq=l+116R=CRAfgBtY@tA>%JQSxu(QF(!#*Oclmq6t#~2qvIZIU4gXl zY1?jRU2x2LYl%1W-DHZ}{hb5oXob$ULM|LBsHAP)JF@eg%q@?sI^MR6H(W7I!{)*`?ERe!OGrBE!erYt8axw|=pb=@8hHnH-|O#E3`)AIU_mZFqnVeP95*X6yOC-xcfVA~FjMDQ z-2+{n&W?8kG8lAugfAZn!J(m0Jx5fn_}1vU6RB6hl?hs5dfL*d^ub44-XJ4v!lKDX zHfx|x2uN$Ez_(J>wdPc@`W$_dIk-C%)vlP8=`6SBdg%E;zcGESoY_|g&anX)JnT$4 zHWm?EWW}(Yz1RJ4mr2|qYvCe7CXsDO$bU_pY zTSl_VZ{`cOW@S$KgG&k>zm*v^@BvS+!>{pNuL`oT!}@6KjAfGI*q0Cm1?JDr`!2*> z=IAok4l`kBoTjW62J0N6xsQ`pBET8rG!38vqNEPW1I1(u7uRT_q!DyfWQzh>1pV6q z-(U@{JEE#d;LU;0N|gGMHDk4(m?vraY=?(~Tlr+^hq+gkPMAA*Y&NSWx z_q;f5AR4L0#W=wxq*qFQBB}^<-)S2MY)S z8No2t;0X9RO!ORZ?S9@k^EyZ5>yIs3pwl^IYO^bv=8#ATFD|=XcvX|K9deedc$cOr zo{tzu|Z2lzsX#%@IWQwwlk(_OOB8M$Il#apYDspt>@P{pLaDMQ`s<+Mc=iS?=0Nx zH3-yio}Vj*9m{5&E8SWS#mU@GIdIqMNOzUR;UJo7$fwJdp8NP8qmS;#y}s#we&GC8 zVc#yP#xYg`&x47b zEW*vgui+?b;by(Yq@DGJ$-n@*ImNB5)b)Basf5_%6{8ne@)lvJq08!7vz_0uMt~UV zi3sA30fI5yHM$RRs1osKK8oYhv;;UlfvX0WNrSo>Kz_N14WJ(05M=5^hEK^b; zhn`8$P>h$Oll!7*l=7DJOa#|yOO4toF`ue{_gkt`K}lO!RZGOiQ?(_1;Uo?jY?=vJ zwwjQ#WeYp!49`&#CuntHBjv3nx=4P#<-uWNGij~tWfk=JuNP1pjX`&w4=!~JT zBI@?J;o8@8_x%LY_M_d!e)+*Z0im#dH2F*^)z zlTkO5M!QR3im%Mc_wodZ=M-kdGj8O}lCiOF3W|I%EEdzC^G7x{MmkUS$Y4}pr%YKh zh~g(T2;$n(Fz2c{os#N8JEUNtP1*C)JkN2&OFu1`*z$x$w;P@7uyV`1)$7^P)3!40 zbJ%Z4g?9TkG#5@fP;s%ONoG~H&qhSNAw^5D!DTRcZMhl|bJ7$LlCW|;`pFeh0h01E zBIY3Ox%#nqfI}aY3tosI(n+10cJn&cf=fqkkG#{ds|y~&YZQg$c42kc&$duVu#>D^ z^L|erXH+YVN%Kx<2-U_EGFg+m;IZF|EVO)A5?iiNCicKet3$kT$4{$Sh4enK{QqgDVI&1LwU#)eLJe#Xjk~=sulD2qx{y%kh#+7iKMYDi_xmw zxq0=4zmvK=QG@?l#-LfCK8rZ|mx!qv+e2jfAv5|Nkx#>(?sC@}+1+J~*g3{$lwVcU zu6W>;rE!fIdvJpEI*TU56!Sy$^yh)2EqK_<0oNmCVM^3Pbd~E8JX`rD5!F33(=|KMFF0x}Z^G4&sq=xwyPJAA zism@#{vm4JYb1qLoJEc2{k#SN=V0XF)1_-WbRk!kSy{TFL_Nvp+8otMGPb*m?8r`h zqO*^L38ddn%BG&rNfZh@_h+QN+7l`zG~yQvzhk^!vp@8jYz;xx|I8GyMp2`WmN<^)1oAc`TjG}`@g2=j7w;oxFDW4tah<8|ToX1-%8@xSJ_8bv z{z(5o-~_gYWqY@3bE{Al6+NNY)3x^I*tNvkbI7+Q`Ek$&D%_9#?~fVB>^>N(W)HgB z?jx{IM023P3KeLAx&swquU_>u-!hY_jUk1P$n$gOPY9d?yl%gqQ_g;r`c6EI(AAY1 zUoiW^h1l(=^dMouQbRV)CP{iQ6y7I3SM8WZ^{qUW+VYtqKM%@v&@E{LJ1Fn79y~m} zY`!cOW?lJIA3Qu075poPw7YA<=i7vZ>Ar06?-roPpF%#WeqR!v0%cErNB;8mTp3Ac zwYz#ALb&q76Y!0+&lu)DS97sf*je!64oWMJ2omLmS3|!?`|^DKH0i;BcVFlm9vPyX zm#*v>0-%!^V~lc$9nd(m6!IF9g>PcX4&|xxni9afKs$-|<;k)isV%a#s=%DKWo`9B zja4baJpwQFa%&8EB?Tt0b9Z*h8o(3#E3zLC*q0Ijh@~K9r-=0%dwgFGLJ+SN9~nRw zcE7vHGlk=ur_W=6R6Fi{CU1k*q7%Rqi5eCcK5E#XzYw~UIq3L-cf*cB1ppwOs~`}b zk^RF1PC6p)Dhyz;bqN;AQ>>YZ>y&~Vgk@nr!ctql;jVNs8I<-XA$&2fgt~lk_F`#EVLVxUsac+U8zR%kXAR-0;Jne%Ydei6!MSu|c>|F%3lK+kO#u8vPQ@`wq zR9}&+4T25;z)El_&xrr7+U#Wl&oAyE_SYqQWx!T&kuC(SdfA@g9`)zN>)SO~gz>we zRI98uz|*#sl?LGXqE$2CwGejwPbKOS;5tCMYW5}p{)Uwq%GM{BcL03xAzmWiECj$R zqrD#luG1<39Z=08|G*FcP$#>WX|=BawoDCu6$} z5a7w!v?&h&eD}iByhnMNz=2_@M)wFHZ)yBk*=J+@`qLiXPig_h0^=`qVwx%t-XL1F zF~LDnPoXQw?nx9vRJN&m;NweyV>QNodC!EQQ5Piqd`n3d&)c>-C4i-vA@?KvK&w@F zfKliz>+pvwHJ#;5MSwDmTbvqTJCzCG>E5!1icM7zcV8a^jI5Wl^7%0tEAb=G8nyeR z`$1dT`3N9|jxKi}#=z8`zoTU;k-*II;MF~bon`n-K={!MrZ0Mx_r`L_hRS6Sy5RxXZk^UyqU|HNcek2u@D8}2e0nGwBnUmF1nXA z;3k;#^br6fSWwaPwt3N;mzq6_DtTuFlEd4-EQbvMxHUUD{Dk|4=ve>!Ua^)nyoi-A zUI4lb(OnvwfAzCsyb4YNQE{}>P^fVwiJmaAi=}*?Y_FblYE&^`d@MgP{en4ul}U*TExJkG}^oxCF9F@exNb(37CYT}9sTYbp?cY0oy;5(=ZE}!OI->S?zm_HpZ zqJQ(({ENGU<)^zO`2WUh^N;Ql7MlNccga1`b^`iNb`Tx^o4W+k9uuRE785-k3l*&{ z8ygiPEwc_4E1ixO6+IIztsW~gix#~u1H@fIOU1xM#l-v-4 z|IL~6+g>-}x9{ndEBsX~SN{dVj-a^eRylcm-89 zH~hw6=bG3i8aFw|F;P`fNnqQwrAlq6Z#K&4{W0)unO>+#6PN;-<^Qg&8+6 z&2^oZ7u@278XoFQdHV%qw}8~b`eG1okn$Zg6uniK6;3Ya0Y3uhnRiJ#ZnAV63?p(n^o}AeK8-TQC(rp)>yw%d zS{E}NA%Ct*Qk>=JvjDPJEonql9nsP^oUIJqt;R*#N$bE>PqGn|dyfj?l~M5LY3-Wr z$@o}SHQWgoMC=X-`D-Hcv|9!&Gxx5s2GEB;a-!yTO43ly(_?udIe2sUS;gFMX@lVf*HM>?zuo0alkp3i3T3OVSS4UMX@{Ey zPrpSo@x_89J7#WWX)+$M*0EzO9&6G(PtwOSv8pS210w3W^%4)4Oe&;Ue^KsZ+XD8W zd{CAAv%eXiKOvIayF!M6?ccqIkYS)tug^wH4{;MRGSahBF|z8?Q)w~LLOg_Qw2*tBY!|Et%KbO=NVo(`!>LhV7ELo1scK1wJOrRxJT^b~IvRsbbO zOX%6%yRyTsm+T^X7lHbrp#^s-N^X}m{yMB48Jk%YrUdsxfLqwQVt?)AEmRR%TV_9# z{N*ZikA-_9>4L3zK?LSL{?j{_nYqush1^XpUK;kYUPKloBr+`tpgxDZw@fXAjgA=f zcY5zSv|m!8Mzg&7FprFtNt`sLdXst0Ifh83H)f~7&objt1SHv|R+ZPZ{c+vsyX+dd~#HUko zS1>D9^emYNMS?YL-ZV4Cp3(bQ^3Qz^+cNB8vkY3K(reP8I#IA0S|n9B+qf_D?_H(E zmFF~|F8u1?h#})6mG!FCxP~dMDJI(@<*vM@$9n`6 z`l6^=zu33YZyei$%OTN?a$M&F>blpB24!82_C)#zE{2!rsb9Q|p8Zf%(ebPb8ixiY z_AD4vD+Hbbdoa$Ey4~TJRa%dlZj$a;VydLYl<8}X)$%M=@lIN=I+S&RQuP%gDYK>>QJ?);6vJxJFAWhzy32WzOF0#rT!= zKFz>UbJo8;nDZ2`T|W^E+t>(Mee##l@E;vWkkRn}PY2TdQWjt&A%h2TAhA^Z&4I*1 z^S`x{Jvfm5VHMk*gL9RYPij1hAIwaNQe65Z=cO5~j}jk#`c8EZ-^|Ac`hsY?ntl6g zsjz5!DVf>JFYjK3?JdGtL2(D|(*_aNzlH%)){~Rt@$0<`)56jF@Om&31lT5L|L(Fc z9_KOx_0*Vv(|sW{D#E4z=5C;lt!iU-Sc96ZVU$RwRmv*6a>rYh=Rkv`myloj>72ei z;nfw1gzZMnNg(VTD5=?t@IqFyl1pE2qIuxSdj&ZDQ#4}jwX{%=(Gdbl`mg*-!PgUV zk_oIu<>smUPJK6PE#sZkJz}ZSyTG$y_3UkKk!825Eb7;07fkbGFiVf0TtZeWSZ7Z-gkRePo%syL^h*X)#mID7N{{Fdju zQ`E-wcsPN~sgH%y>2zy{#&xSW5Qh6EaxLkC#eI|WFg{5BVol9n^(+``pW6Lkx9-Lq zcW7lTu&tep@5+Gr&NA3e40iAtYT$=?pYm_6-;GJ z7X7E9QNA@?97es#7UU`?!=u$UKscHMjoNdMg9Dc3S)+bnEVbipQ^##v{V96=E3z>R zXUi3+3;5rWgGUr*dWmv9=n}Pv8x-iv7x0e zgw|~Zp$krQRN+n21l~bhIo2x*Ht9#)5tF7!g$MJTS%Rx|<@T;N4_4K(#9*+YTMOP!|(UZZCMjaPixPBX9(vSP96Fv9M%&IsEL%Sd7AxfU5Uq3TX(HJvUldo2+*_zz&(UxZW67Vs`$W*5 z24cd?UaO9hCOR?iD4uq@X|E$Zi;g$rTCBSVwrC%O)aS_@R|otIOAKdw^hX3!;S zwf7yO6SAQ&`8YUG>nq-(c(`iaJlQ%r!gJ0q8yeQebYIyO9#2Ee|15@awH4LC9T8^f zIGR`BE}#A5ZCj=hjlBhyBvD(POR}1-NM%b1Vm!p#GmAViJAbDUFPtrGf`N_Wt6xQU zxxT}q;2@eHD7`;vGOrunmdCwL>4InHwrI71?(B#5q3`F!ZJFm zT8WwB2(4P=2AZj0I(2(H96X#lt=-V^l9_8^j|_92`t>@Q`9<@fuCBCJaK+mN*gBxH?m>XD3i9>+^kh1MnkC?6)yTcc4xw(xv~`+W)^Df*TT=oVrgbg=Zxp`6nG{CL4Cth9Mke1 zwu@OS4Y5T+`SoSDK?F7WZsP@4esg6!i2VJPTMSyzIRAh)5#$(rn{5ogi7LqZp zR=NlAX^@EMz(Uncc28*?Fa!;LjLZUF{z7buwscNo?rWdMh>f;*UOhf0g)DREOAB8QD_Kig+bMa|%^$ah3d{X?~vN4`*{ zK)t$FrlIBj`J#3TdY!f@H-1soW~ZCeoZVGvm^!N&D8=p}T5rR-F zl8AHC+L*jQ@Tu}V4Wt-Lkdjq6mGNM+3J+#=T{HSpdfsDlJZm;R+651 zYh*l~`Q@o1lj9je`s&;BedEx$mia4zvNG)o1tyn9j#o}Pa z((uGTB`1??G+{(Czk4j#8Jdr2UQ@otjUo6N6RVWJ(=<>s3V!%z;u@=2p4(zFIf9P+ zyZL^0zv;OKw?{B%%LuqWXWiL?HYS3abm?@~WVA3$p>C!E);WF~Tr8jYnozZ?aQYLO zZ0r0HuTA6t-l^XWcbk5fwJy%?K>~5sLQo!|NdPg$(jXEUbKsstgju{Wvz-dgbNvCR zc++k+!BL5>Zh3UVW^85U9{G~J8Lejhc)RlN3JEo-ZnC-yNC+nRHt}NR86;EXrj^zb z%gR?TBdWo9&&~J=O8On(YW7x2_#>7RX2b{L7E4ms)Qrlh7<;0O`-$SRLrkX`&JBI0 z)B2{gGlcECQX7Q*E?tf1;`EBm7`w*%uVrxBg$I?QEcytHk!msW%|ZITxP$@}{R+AB zewT2(Z|bJX+bOZwFO;y)>3Hnr!$}b>k5N1-cL=-qcpkY#E|ywIOJ48rwR*&%ZY0?H z33YWBW$I!S8hHAkM*z3;arJ`(sZSX|6qDo&ShM#Ek!L%(>s5I0OraUw?7d>onbKAag}Y-HiUgJ`Ya&f z$=;_5aekF?EVCW);%Shaq#q}In4Ye!d@TMdvTwg4BlU8HK_s zg%UNzL3$mk>7OqKoY{eMXL<)#(D7n2_L{L7gQBu;$w)QhPtB-oDj$E7o7i*3ajFV+ zQ~$0Hse!2}zMc$-qpwnYri>Gwor}dabw#-$QN!1;G=TAz@?RPa$L2Tmr@bvbVoe+=oYQ&?>m)u*c!d3p1S z>1vwO>11)%Ig0Nz=tV<_s3=vNkZPG}n#SVT4D%jBKWo3rn9%ict5jQ+zErBv^f3eb zH!NK&CYfDk$%YGt_Sw{SHF`0wb5!rOb)jCU9(k&~8T$-V1(JhTYaYj)Wjw)4=iisg z5s;^>xeO|l6BK4XHr?n3eyg*vE$BPZSU~AEDTrMyyqE^|#)Rc-%#;iagisz7+mwuz z1-atXZG>YPKe_!@$aW3RH7Fe;R(jK8AI7U_z#{hi=~|CIMoJ3x_4KSM@VmhN8x|0_ zY@V#i@*HFw5nJUKkVR!$^X6N<{$9A{Ue*%za|~7P+wX=&^3s$d<+Y{bdRWK;6ZFo3 zhOT_^X{_~pcIjqimX(atlD8w;L$)2VOx1?abNo~8s? z->;$L*0SDSg9)e9n8VIM{v6-?UaOBbl`DlIL}5$Aa1j^bM)iwXIt_2&xt%Kdy?@tG zLEy4p#(?UQoGt$GM*~?Q_nVouY?m3?Q4E11#30}A2)^&nw(TUCN2uYC=?_k3Mrf+H zfju<1s>*ggszr#;n8wze*22V(H^k$X;Z}_}_|J>(uEH4nvNNNiqZLsUzA9dc&}384 z;AsF0s)epqa+%*si{2hTUxP1;MvwY!V1F<%lX?|(?E@^WF(P+=TE;!?N#6YQI9V=d z3)%n8kHi`|O8kdE>ZgtR_mej|N9xsp)e`)P^SpaL|VeNx49KfDS zeP0w}tf!_Y{~7wD)L#Ou@<*C}Zx)R{Led(ujZBL<0K6>*4~Bn8f>A#Guc~i=s}nw# zH>itmnl5Z0We1rY?SI+#Jx4GWh@k(1`Dajqn9~#wAn`~iV(yo19g?@G#5R@`+z;IU zne;99h>P^0ilweZN^)!VZ*KpD`PUF>V1Vv}9pAIy7mI)7TV5@~L_NfKt>tkqg3>}< z1n#4Pe)1e4M7S+KrS3`8QV9H`&c6q+ML9EZACj>Na4{j4H_r(4yO;-G7l5<~qSwYH zIg$r$HE#Jg0Dg%9fc-qF*mvY*3oZa+g850|->{{Q2B7~{Z=T3cLJxkrQ5vnJV}JD( zNHwYc75ty_@znWb0r4EM2g3jtKQJ)=MU{=hI&u(w{c+ir{SpksHRA7x4>36aBy_>= zRSk-UVLacgbVz^GqWZJ>#mVoY5B*9)^Iz0+@7ASphO|8>40it)wOaN*u@XE681hGXo8c5BuE{{j)2Z?=$-+ zzWj$GoJ7zlD5Bn9DwU4hlO;HYfdjGk4z5Mrx`e+73P3tgOa(~>BZjX39G#8DoDT&W z&&uWJrX8Z?TPnYY-UO)`WuZJF@yOkg^S&c0K?UYPUWZuVniB`3 zE=bVEi5rsPzUE$B4cd0ZMzHd5ubh-D;*n^%wFB3(=W5G`h%aQ>ai1Fvjn|aoEHn4T z+sAh(1*goYXPfif$jz;#bWWQ_`xabjs8{e$x7JE_6z^=C$%e?k=-S?zv3GXB;7Q*Y zxzb6P#|uH<`SBb?l6PPq?Z?vMycr<1L15xroO?`ot`}jzr>=K1*!}~qsB>FubK0c> z@(CF6#^W};FB+?>1cs)W<=EY#BpWiViGw${c{ysEZ>v5?RFuiAY-xT&`s?;yiN zemBuQVAdnx)hzVZS}om4Ih|aol3b>amYGzsgj#w*sv$U41a9rHB9}_eZrD;d%`7%<22(|g17B{*iL$G z%Cj^*KLE5p$wSnYs9zp1;l53qX*g~iF#y|7jIhknSMJ|-^y30atr1xn=JH!hmCkZH(_cqJzXu`u}}LR=?n`u@I2-svooWXx|{V? zJqG>cV~9j7Qjh%ZceJs=&M)1Rl+O7in{As z6OB&_QmDVH#^X7kU0wjXuT>O)TKmsx@OIP~?pQX?C}YsYd0YovWg!aOc}5-rTH zhj7qm9cI!FDo?5q@Af+pTO|++DbAStw{i}Qg-0DSR^Z{!`i^|pZQVGhV!nq?RJ_4N z_LoyPaLc5|HRV3KTFk1_Y4(+`VPpHDXQFR-SB8?;>`j}CzF)7|Cb3yNaqHSKbr6!o zs3Tz5lfi<+Rg11D5QXaejeX+D*aEl11z6IDFQq736aH%q2e&u=`Sy9ue&Jh2kS?qB z#o@m4jmSY#MQ;_Ow0f1xJW<1rO6f_|M^)g>Y$R~<+Ah>lS6fzMHG9r3{V(aHl~%xTOrp z$z;7y)1o-5>NL)Q*gtoC*@=OCEUd?DJ`3KV;*(VL8Ad$oLOsr-(Fw$&YEz^X|-`=X@sk7frnelAZFCMH@VbxwX zPo-?s9}U?Rg$Y@ZhuA7yZdPI4Ui*LR6P2tgpqW1{a!vP#*!rpu3s}@a8F0Xh7K?f7 z{Ukf!rQ5;zp2?NRRfUJ^^3frOHcOozdyTP4b^wy=L@%I*@EsTO<9tzE8aUA7YzpHaX8LnnLZ7nsLZkaVI>pL z4ebYS4{nuCyE!c8ZdOmI4su~}N;XPNLMYPZQM0LVp;|)i>bT4vl}k~v$m=Gj+N;yDo{fli5%rcH$^+nzW%5E>`>^tU8Zo?vJ;N@ z2COU1&Gnh`pcljbo!wTV3V1FplI;C+*3yj3^h_3NSfSvFMk7SG8z9YPJcxp3p^ofy zb1S+RbH0R}Fbi!@c}eOJxz-zeaHRa{J)V2&^8W0sP@9*5&IUFc<6wysenh~iFJT8| z!RNlc$*+BxF7!Crra<6;RELx#P~Ymi6thKPaZ_*9?XJ+$5s}Y3ovDwu9X_#vf&r;S z{7s-9sR1d;a-JB>X+{}NgY?SWES8a>WJ;(!HS7SK$Utt^ue&n|xfr%?9lYnM%TWfA z#&i{GhneiY)bj}#u{IxWc^Qs#a%iVFH2Qnx4|;7cP9083{MsA(&(SsTVz(qiBhfV0 zx?#x>>`%{5&(B{_Cn<80GJU~GWT;_@7df{N8c$$lQRLYCqPgGOTP}7dCCJY9RoPYS zEj`6VcG1+97`YP1L2o+8)9diMxykcaTrr`3% z*%GtRe;a|opNmGRcDM^B9)lz5pft?OD^QUto*;@G%GpFk9ZHs`%r>7JP>hC)T08F( z;$N~%G5+XN>L|Ne!Ud3Rl-%_-su%;>rg4&V4uCr%ez1*A`5)^g>=7f_U)|1wGIZx>bRT@3TIn zu_Vz>8!o*LvpCGOJmo`ur@Vtgqb&k8S!c=Ec&J5+x5TluNlNb3JRWWNQO0;Kp^}!q z3b3DIXgR0#Cmf@T zTT7{FcAwm^W%~neqeAF?&F?=7xq_{Qn_|`8&@m+@Y`PKKII|_t!weBsC)1^W&3v3z)_*o;f$f2 zL9hAA*(Du0GwT`Na(mh`vXST@F5A=dxY?H~HUi>HvswDe;`>nej$E$NFMBt)yG}D< zO!^%mlTk5JR=~Tu_k0%lTMN@OShd(MR5lL^oMwvSY?)ChQF=uTs>4-X{EYiM<)?~F zfqQEMXXm2cI;6JO8C6Fxl{dqyn`{$3#%8Cj+NF_knw>iZWC*OT#*s0 zbfS{IDAILlQc4bNZD~U<&&W9@VQTVYL;ao48gg-Gy0aJNJD*u=F7GA<4I2R+DX#BG zUnim^Xn8CTvY-T~$=^(ZQ<3saaQlAf3HmST>{`-}0~4oAEl8$-B`ane?~B;b;ZI-a zJvF{Aepfd=J6zN~`wAcBI%0{0-KAuzewGzQuIHe-i8i6XRlmUb#L27>I5U6L{^9L; z&MvQm8NH)HJbhVwd8Ui;2^?yazuWROvcy2u_xyY_y_W9sbK4%!oS7NL)wH$3+TC8a zykWn%-t>6InS+_3;S&Vuy)_BXF1u)sKfdBTpBS1|AwxcW02=fYXOLIpqtz<5`}psx zv8S|n7P858<*(I#XWqX(>5l<+zY`GPAWi&^0NRf>(E*y)xjl&hKeVi901dyx&&D1~ zcv{U0K|bUBh!pR?KmlSDJCJe!faT}o&q4^2<*pBfAdg9l*^ocKe*v^NAQ325p?oTb zhidnwunDIhs$y&6ra``feqPx4=2I}9T~1%6gvvPr9a` ze(Sul{$8R z;IW&*8#vgOa}XhD#xe9iP+0roHUjk*>Y~(=1Y#!K3_XB1HlNbraU+E;Cbyxb-5+9E z{PM$NUn-EArzWPV>i%z|#cv`#jnX+6>X$k0?INDhd3E=i z0syYpLjbIfpg7UY*!^FuL14~Z+wbiHthIT65go0#<7XNGjixiF^6*T$y+*4Zjq zz~Z2_ltxDZ^`9z-QG#?O%_}9~zltQro|E=rV>L3`8i7+*YPE18*AnAM#mW=-g z&KUC`(jDZ)qXJ#>Fo~=kq+q+Q)QpG&>7BexR9AGJxQZdj@M(hFQ6jh_Y=d0R^Y>X$(pQh^Ch*sxT zzxLE^{P~cJ!{v23*wKZehQtZ2tpmC)7F}6ua7lx`>I~ISxb}P;pdJ)CrxmwN=BX+ z*+n~X1>fm2%UYhT@>sJ;bPf3m|6n>ce>&OmS!^-&BJZ18obtkvVMMu;NXB(WL}%Nl zL;ft0ooeeCJk>XC2UC}IA4zcE?eOxy5fCYBZ1poa6Xrm`S3pk{lZS#%aH1%bV{k-U zaz&yz*=NC@j`9wE_3w^%Vf zeH47_ps;ydUtgNsD|Pq3Z0EWc>$STj-M!He@|ywx#$Uwxb1N6MrL~2n9^?QPJ(~xK z-gmP5`)`|cBjvSS!or7FHn0?9#cXXv958-*AvaeE1OWD8r`fh)0HM zscHe*CJ}U_WdU3em=0Dq}tz1==s8=my^pX!W-JjiDs)V3^&T z<7x=)KDH`#zywA;OTyQuYhWG7ntG9DWE*y36t#B581%8vU}#r7$a_CQD4~+QoT*gi zkq5s6g3?vg+mLWIErj4wyz19G1tIbWT#V9V!M3P&T)V`E*fmcP-?&SBKoq+T6j*3B zk2#3SwXk&(Z74vGKb7Bt=Q}?p364^t6Wklry>4>B*v)KkuDNwhHrEL5E6YjF{!BLY zH4-E!Ps%l$53OXgJQ5etG`)a8H!C9Au!N*CHmSAL>>Pi>!2eHSUjH42THC_H*2dOa>lb1FW5hTA0r7s<{Lj;ne$CVO z==b;E2IQaO`#akN{|Q~+$W+fr_x~08-Eb|^sS(b)f=k6J$Y2epC zwR2A`xw*>a8wjQMH|_r&wU9+s-_*#!@c%P)Tkf97e=f84)cB+{8T&)lgKdb*jpDze z=HEp32Qd#T#6O6sOmWV9gcgPRUEIs}Y6Jz9uodzZ^kzbJyKhsGMlx?3+eQONR!h&e zC&O0cC}ay>4m80v!dZ3^HNt6;nkhsvn3dB)Ht@m) zv)7lc%?j3N!P!nm4|7zA^e%79j9RduEQ3;EUIfqU}%vdw7QLRB@R0EKZ-F%FndD`Xo0E4IQyi}=Tkr!k=pZzNGIJqconmC!MLhU=HUa&O;qxY+M|^QR z(R%B=c_&184$HGFiQfqw3lY4?IHxEaQr=~xTSJ|hmmZj*eN)nb<&aDlu3}(nOSU3- z52XG-qAdhW=}#h9Ap#Lb~N^2QRp86b#2>tu$_Csn`N3<#?rDb z3ZGC`cLDA3LWO@JnNRfFKH>j(fBAf(h&rq3{`k~<;TTj~iW^?)jbamfnC~y#?|+Rj zQK4Ky;#vDa-90+(ncGFj=eC)-OKi$cGjYQjwu>J|;L(u#K-XR+d;KZ=^_OZ$e}InE zfI))qZ@q=~w@SwKKd<)x%3JyiQ2fsGU}T_Y{4b#R$LjwBn*JZ~)TrKRyDti#8qrTM zNB|LtFA^WwNbyT8b#c&*cc2Gns`jFY9ep4Yd(lG)-8r2LLt zAhUr2@!%U{`vRQJ+-tsU9YE5?OV^95tE*`_TW1B%r4r*_Wd>*(eA0Ffm(E5>o|`i) z;C_Br1sQSblO7|gqWjcddgL;`xHw2gv?&E-5#)XlY45p|xh|~Gn(_bxG=X40ZwUFs z{nUCpg^^@SUSQopw9jO@e>Fa#(71Mw77vCfL^fokL}U;V;x|AG(>TcE161&Md|KX` zB|azR@-jIhc38Z$cu3i#-|0FTItd6CvbZkyw!REqD~m!WNe0CCvB7a}&`79h>qN)& zV`^qrw220lduqDGJci*qFg}0?3xm0(>9+FAt@?q&P~PtC1ic8y(aZYjhp(foPW-3d zgJ7kUPv^48kh*)KWzf^hl-cGz$pex8PmN0-?4$l_$RR0Yj_5yk+GsyH_Px_1DP&|) zl~6@^5Ve<62%{9!e8Gn$CNE-EcBpnRNcAwJkbZ)^mTXVwX)C3AUM9d8W7^QH77ouj zTUeYs-BMtHIKYuPMNIoZMc>hWn#zRBbYJMa%p5jGPJG&Oom19?<7K~ zxxLN5tvL3pVKrdI7v(l}NZR*!HUsv0P)2QVf}m9}12%H7l6U_y6%?endH^#~U#gKQ z;?K5ibeM`or<%Ss^>P|v+{#?@Hv+VY%=6M-lty;=Gz0k-l>9lNq z)Uv7d8sd#l-3>fPw`oQ}k6q-l_evF7+d(gRJfFgDY{nd7`!>wfk!qLKWR}rn)yO?> z*{stZG2Cw#iqGuBd(cD!PS*(682YIEc1j2Lg_gpKx%LhhhILo?QCnY!y3Tr0@OxdC zqY^7EZ1au!jMUs;Q+&OlQ%rvgH`96x#lXczua1yiQ{{(QXOiK^7=SfP$jp0G?XQF{ zhdW3#)GuoC`^*M2qm%_bJ|$*n-$?g9NcP>z2$k_o*{y%Se2R<1yCh3jTCoT^BMls6 zKq8hkd&yjm7Sw#p_Bq$cnE!*{z>IaMSO9;jQ+ z4*$MBy==gzrvU7jtHPb@gTSJ0J=3Hb1`g44n4;X@zW@|d99zYwbU&XXbEidzhMv${ zo?%iGs8qYP=TQ=<>QQ=EKq23+T(*XH@?HMJoRN=WLKb3WQg<&8zl8_Tpd*5Wxi_J9Xcq#LJ)MZuLdt>E!;xp#B~2Ukv^a z$OHZA{_OhA;NgGUOa23c|1(wdPif;H5Et_s;{Is=CI>aDjoYsABNy8ui&&r7anW!A?SzgBDc!J2a-zK{QB5}tz zV8#O5IU~`&zsno)nRp_KJ1{c*?j^q1M=Nj^V*2rFFoEtLFN`>)`CgBYn1yIurb-G_ zErY56nhkMBAfE)Bhd`M*Y6A~+wHQozQ+hqU%m>M=rjRDj{R>qJq8Puj^k_e4vj^a= zP|Kx0112=5^3(v0kaDsu6s}$gEA?YD0UDdQw7wpm`fmSy5;swzwU0KeODxjj=Oup% zR>?QoRuIgfxa($Oc9g&5fmVG@rrH2%wJANr}b16fQlTGE^d2ib>wROsRFA6xTk22tW3+ zezSNre!BKecrzXo6J>D01(Sw)!_ufRXA>)RvYYu9so<8}c7=0E#b)YPu-?W?ylc)y zUV~C#<2sS|ReUO{-mrF>8A3xvKs94`-)6h_xE!13Ti`Y4fa3q+@mlZ=z&4Iq@XkvX zK)3xrLq0yONOvq8(6m-+A8Txi^6)Mi9AtGFmbQ${*rl;1bh07BoFaQt4}q?W{L9g9 zCbv?T>$}VxN_wTIXD3OsAaaH@l<;x${Ga&(hPbaC+(I*ghKJR&KE8!fkebw8dh z-#slZIw?>q#|(OT8^*juNsX{i7DY|pWqg&E_ot25*lSdNq+p`vXrtsOQW=a+ElQ%W znQ2WajQ+GP%g>=QhN)L4PpuOCX{;j2GOpU6ZtT~|j#1TsO;v>%m2yRAARan&{;2Y+ zVb(JE;wByP6gyw;Xp|n4!ZHL$*w7L&&qcMNH1M~Sm1Nr)s`1R}QPZtq5-`zVW zPexxlUEMz9IOS|TpDA3T6p6CznTG=Tc6O4uRwpn=^onCho(TuuT;nAdz{sV*Pvqu< zLZPPC{|p_WQ!rfvc;fCKqRy{OK!6rWXb(XO%^oW9@bDTg>s2?+NIX94qmKfHW_6`< z0!$Mo-2hYz1TkmJ0Fs-Ttqko&Pj815!(SR^JJ$nUD>Lt7cRg{1{K+CMyXoCjPbHrJ z6WcOT;IkHe5pocccq6eSjPA?|tEv8MwW*7Uw!EDHw|ov=N*6o!ZGDe=1@NT#bQlsRLV~-R#?r zhiPmFu`6kwaCEIMIOU(Wuvy_3&+~BSaB{YD^!UHlhrj9izjwLOIF<4JvpVGa zohbZwl!%AeiH!ZN>EQimP3OOe`irLX`=7sIFT;F$YGK#~Hn@goc)B}R`2H*t(mwf|`kKgGD0CuAl*JdZk z9e&wgut?;Tp;CFAct378BRB_0utp1K)$!D99+| z%_Sr-XnF^KAq33}MV6gkQc%SIN%PG&**Kk!0tyS7BagbJEXr?;9k0ajtfVB(NVGgI zFse=Yu3&*oZ@;j}^ih%Z8R!>GVofATggUSff(Sd|?I6jCVQ&K()s1;pV41 zf|kwmo&O_n^1{BVGGuK2md)-C^JLV94|cZA9LT}$O2PY779^CD<8(85zsJW~C{*lx z4(rq^ijh%_*}lW3e(xsqE|VyOq-qst7b12F?aos>jh*T%ZV_V)B@dY$>l^4Sqbyy3 zdL&2UHVDU&LGzpS#`8hvINu51rXAep>*e)#KMNh!Zm+qGx0xB|(y9!bOW6Z^3aZILBfE7Qp3|PNJ zSC7G-RHh-SQ(1@+TzujxxjaeOH(yeo16kaH**{fOG!v4rGfgZxKqchJoJi%fr=}zoY9z(R_AlnzdzPJ|tUYHc zzLRau*)32QsziV#=91&Bkr=Q;GU%DWJ2Gn8?Ct|fa7P}oDUuiq;nDJ`6e`|o7zQWV z=iph?twoJ=q{~Yl#q66J5Z!$;-~$$r4enU?FQkKHn~Xg-F>$;COzk)UE7XkDB;u3w zhpeCoWpD3wbjYZ_pfvg<K?Pokg1ER1)oPiZLxBu~14lRK4S=9g1j$$mkUXnjv$t8SrEV^M1BaU5pHKVf-%ncK zy!HDRF#jRe*>lK)2gKiYpXwEoFu{#1Sb3CMq={`|>Z{tl4CxT$Eskrh4vMbRI8 z@1Lya_sxG0-Tox#-`U>(1jhe**uQ%L{%T-1a5OTva5XXhU)t*b-HrU$uKqls;s18^ zAAQJwOv+zh!C!q7|Lp3Y9uGYyX9MT|@=g3P3;)qQ-9LP_^GC6)KiXg20Ds%{|E7r2 z-*`a(oPXOdTg`tl-ap<|m;Kgp!S84V8#(|0wZF(Ve@n$b2Tb#i0n^a`zWz~3iGkyH zF(th}e*X1hpf&%)ZS#+H@HfO=t{%$aOf|9H#GttSzS`BLt|FXa^I|PXEJq_5LP4FS*xeIUu=yv`AU2_)mL;%cT zDhc4PiY-F0nVq3W(O;vfJN4O$*%kZ4#jIVc#m*V1AQoumPUT*g&Hq&(3dw4EXvx^X z=~p|pk~9xNjusUTC9R}MEaf!R_t26q$W)nRB>wAadE!QTf-g6~0Gzggz~};6n2;dZ{t3LVshmq>wW+7>!E_-6e77fnmXly+)OI zSV?)bh2G%2_tO`dV8__p}Gv`m8Pe}OP78{8i8PdRFayG)cz!o65MHQ|er!5)0nnHA9 zYF>wB1PvrtayJwSia}TkihjbTBr?2B4=o)a5en38GpvoHS$6TL+Z`Ym52JzP7sKj9 zCU~B8UHEdUT^RSwca_9l$*JL4X+g&VLPmxwNn5{fcnb^Ahl6!fv_locK7Go{15 z)9M;9@MbQy+_Z$i{CaKjsC5tcM#SvJQMa|ESR4j|92cjhDnPAo9txTWsq0yp)_TMA z5BL_?9|BI^pp(#|UHNe%})WEEiAQLO4Imi@FJ zoteHlfJRV)Op_D%Kg^lEm>;q2ezyjaTrg?cPy~zdYhFt6^ZIEGQ9>4CBe)L(@wp@! z*2mKW(`$5~uw}BhHdF!m)?l7pmv#a{bhGhRdKk4km8;gusVm(lm$Xq=>pa9wXf{t#3tg2OV6VB}hIbq>kAWk_zc3b%MT~q?AK=uHyY z!@!nH^336^I-)%-Wj@Hhgn(>nFX-P8aNUjkvViJJ@P0J=OCCsX{HN}!S<9Kk!}R8` z86BS*Qe0eKT@e9D`xn2*Y%9GthPUaYt}kH@^z}W5=g8yL*!ng#bS3R;w)|R6c7*3) zPw!my&(@uSY;|QhBJd*Tns`AsGcgI+%=OdQnMjB!=b3N%s#|L>o;P&_3E1pzxa!&T z71en&c+^CgRX#}-ObrFT&VZ8gQYyb514eHz&!DC4`G}2cX|Pmb|QZ@p3)%7?>|V zLcbtXT)6ESoHXFhrm|?3FHVxvwcXtCkd=9^x{-@Xe^tO`K5^;6O!=}% z7=cw0`YcDNs!Q3T%o~lPH-^(NkGH~T3{%`T zZfS1!YP>oboI&nU4YJ$KHnWo_L2QxtI*xVPKrBS4x~VM*oHUJv&G4^98)V|IW`+zv zfU;2l2=9|GOX)I~FAHgKFruWl$rIl}nm9;D@`w=khm)CQ<6YsdMpR>(hFcG^M0AUa zl)1x(BI`gqzBifa2NLBNPM_EJe(9U{+dj9{cfFda>15t;3EUbzy>0BbJMpG>2sF3Q zcRp`G>%BgZMw~vW?zCDhom=V_zIa{`>r@H=wqpFqUc=DcDp)Hyh=xQZipkdN8I%pzfBLX4TzsfJ!Ju6ljS}RCJ^5nEH67EBY|LB^qMJ|4P}fXNtl;Yl?}{}!kj#qg!nRAE5LV7>L#G% zf!PsIDfx8qzJ+R0k%-t$faSVt_{)kScb1zWux;Z_haF|-0nSjYem=)cTWO|thKd;> zThDR6f%&nTM}+=vcmaDs2Am&OTQWP6^R@yo&}IQ@?0){yyyMv_^P14*MmE002TGGi zIgpekiN7`h4&DJRj^&CN7i6(=+!sW{9D~iZyc_3+pcbOST0pfyhK7+Hj9^`YbLuLf-+_zq3JI2_e?xf$IzTF_iMI7=Dhm18C5Fz*^_%T?hFFHW+p6nh1Zw@A zo`RA`V#JZCX-uX11)6pq4a>NwGVpy&_GL+E3oq{ZchKKQwd6p2%qU%Ds)7bK#5^W* z&Pq0_3aWaO+AJ7mi7{ehJYy=P)r+%?LA8ex*{qME)bjbMOp}|jj^{1)RB!J> zbL>g^u_9Y}SCGLtoI zoyK0a`@ZIB_pA`E0PaZA&TLFIb3WM5#~*5eXk)?(ub9(`)vo5dAFyoczRk_7Sj9~1H`gz zu3Uoq0KLQOt=ALF%+c4}C=#XI+mD^-n6vZ7v9#n3qR`7OK1NCdq2qoLB~DRCH*Uxm z(|6#SE-8)1GZ6DnDT7Xyp@{C7G!wr72eDfU(p-%fiKWhOR+@)iX3Eoh?)}$$ng*z! zkPs^?$NJG6nc0D*N}fmkvvO&#UCSZmH%Fp-WBjbKyy2@K zN|C(x@1P9&(bH01u0o%7VpQC%)rpi!zd#qK9p4rdjhdzv9ObBr{o^$$UXMO1EXgS zQ^_flJLV;D!N%N%ZeIcF;mla#URngdhIR=Gdr0k^STeToSlHRiBV+NSM-9P7pUpLO zX1{ zvBoV;F#C81`181Ck=u@eHxq7>6OaZwd*+~e9;82 z$5hWId;HrO0oqNOzIt~!V3xko_ktR9hUAP@SvAkY$lN-{{-*FbYj>7uL>J zT||mcWf`hP8vRmCKlf6PVP9U4AwOrr9FEy4zI^9RUfj$LCiBUQ1;2>5wx!)b<*d~# z!x}O7KEo5!2f>0iw~aGVYdBY4Pxd=Rtmw8fs$sXs(s@{cZZxi~s>npn=XHYvvEVhI z8862z8sF|F>3F&dL)v7uVuiP%ZTk=Ji0&1eYJdhun|Ths65;Zls=KeHy}xTLn(7YRX)c zh-CFiL5<8uK%d^p#@IKBsj6*7Y;2T_A7}Q_Nh7#v13&QPb@SKA9ExCPU$A0fe+H8Mm_reS$J6<9+bcf-2f7SkH1eOJBN$f^0y|Xt|gAKq|xI(NwR7bS_Hf zI6E(#m-&U#(8?gU5pffPEN>SDS%C-(He9$MB-}$Z;2);j`<4w4MvR_P-mXK~tvk=) z?fA8eCVF0)3|JmVE@|4a+KG47;_8tG!-I56zGn%OZ>so!&96^Nvrb#3+Zu-S4gqn28LAAHXD!n1o^Hp zLWTut&>ot0|BhgmC5o1y)p0~TBcaTQVV^i6Aw+VS^%eRJXNlK0ri;%NM!{)>7-mtA zTTWmkwWvKy_08D5UjIAfToyxq8jIB{_S>nFlyj_myF2q*>5Ms4HV~)BA*YB#fPn{$ zRk2q|tqg}QkROMTn4=RL+t*OG0z2nJm$e zkeUPo*=)!$mr6U;ut91ixQty-D zW00AWNQ^e`^>sWQJ8BWR0gx(L)=*nZG<2+ajG;!{guy8t11$B7Cy@6+Hjp{tp{?Rfpt_!podkLpK7$p0fiZK}%m@ zfRCQLKRSId;anmc>0UxM#BLL0471KQ5^|o{rTZSwsp^}8Nhzv9G?AgGKW7cZ3yZFq zqkdCbFm@T>H5kga=4)D=r>rEjS=Is1B|#b?V96ClZY0}+4V;R1a%!8K9&C

fvz> zMAOb%u3_3NK^_T&mdF=DvGT(+M>MTOtxt(nEns5j!q}W=Jj{@cBBYusHe z&y+4DJYif4v$Xw9ZDDnK$cQgp$HL6DGtiJgkv4{Z9RnF+lzgMre#IzBexQBKkCD1s zUlbApt}Gokq07UB_*@c+skPx_Ntf-{uzm3sS#bPX#`%8OXEy5IL=TS7p1Y;oA$cR` z`08VnOOe(5)vNp9`Szgl1YjBd>%{pbEhjg7I=HIOZ0se0i;0&&f%A3|3gF?#!vTyn zVIRJdYeg(av&rPvBPBe*UL77-m|+)1YE4`;hh7*`z}+dsp(Yy|m}NMRZ6ILHNfWG* zTFJPHF+6JyUmq28I^i27aA(G#{8_4jeg+JPrr+#(#Ke!3RmP*e+Y+95|i1cG_ka_A?L#Yq9-MZFdH?3Bh@|KhDlE zbRTLOU#+>4DkO`N8zO=7Mmw{6{1w>1t!_VDASi=NfW6(()i&V=m4^@w$y=?2G(8&3 z&`Yi~MQAEgWf1S9^MRdKrFFgsRb22r{qzuM?bX6m$zAog3Z&7fmiT`L_h0Zis0K zGpZ_zipS4pqfc_PpUX3_GtbjAwx-wsv)TEP!9OiEY!QBE+w$*o%)L`A_aS-BCxss6 z)X@yPp3dOMm&^_jawn`h^AaOok45s!~HTdipx#5!A1t%7SS>otF^#bWMbh z;UR$`GQ($KuL%HvGV52|+xkvsBQy-r!4`gnD=&`bpgse+Dj3w}s7a{A{3OgT71w}k z?Y8IA=7=o)aX=iihErMY!{g3Rx0CN0KphngmB z?<((nf!|Ff0e#~QxsAm54V^>yB#;5D0BHbnq?CofeyB{k5I8g5m_3Ipid9x3&2$5| z9%C8v>D)T4W-5<%Hw+|jHU9p#|6ZY0UWL?c8_FCJ5RS4(NXh5A@)txWm@}w2u^4!{1VQBqEXMh_1wY$@tZ&sTzM4R#@s3J}&9 zfRBiFy2b)UhPS#%m}{l@8bT1GiEHZ1Lbc?6WL1_n>7k5VHkrM9`MiJ!HQuCnp3+Q* z8XcC=TJ+~lM6H{n63_GF=N$1$=~5te_0LU^NEr4#g@_{S5AwsDX!Qra*N|Fur$jZO zfYAfo#BaFYj8eKh$0&lcI=zE?cjMSUJDaRQ?(BaRI*-oLMl+{s5V!DoUB3#wv0y?T z8Je{k(q^g@0ntMlTkakBSX?CLD5Z>;b;r&N@NpQrPYHPys+;Sb!ycKat1?d?CGI;O z7LX-54CcJcs+bk9Awiq&BI!L%4Lzd*V;m-ij54~2cPI^$12E-%%P=Fb3kf||O{95B zmJB1_1JxoS;~yuWVXe>8T^&-$vI7P-u}`8*oGwmTRrNsB9r+=`Q#verC@Uz0tc;MQ zLV0k-g(^3R0;G7_q^JAJqFFBp%Uv7lJt(auK#S3DNJ=m8>TM*A6)}HicsnJ4?H7_!&?{hlze>Y2eW2JNk? zNxdulK3CI4c&m5oQOGgLF8>w2IL$JSB!`*60; zfT;8?_`s=W?$=g1?NvN*It?+^Wbu3?N$ zz7OosN?+4}&fn)TM7aNC8Hv8t#8v!A`VPu0mzdG4Hm+i3i_WvY5YVY5jWPqJWdR^E z(NwT19|mY>-(;I*!5MC*%eya1P;Jn{lTa^Dt5?d@CM?ahiev5=Z%n!ABq^?f^P|UF zb}~`P;80MN-lG2<#{jAi+GY?xTmZO$pEgCQhG}oD#7w!#CHe0Yq7ZHm|v0g6wKZUHMKE5J`=?K~R`Ny3{5EMAZHG zDpu1T_t8v<26hkbtoC%g6rtd)p1NDKfyiyD>ZAWcz%IJvFfh=+ldq4i2kbb_UE(?y zk;8R=o&H$=9W~^~NRv3?XvNtg_RAt25BEi%8i*O)rpp`ny2!Q#<2A4ks6*JAfqb_T zI*Euq?Kj6xqf9*oNZ?5hZ7rSGn=h(QuCvCqX}m`)WzW57|$_XaUGXkUXn5E;_}#|6z$2< z-hxkCPo?)}zzRU}(*>=}gTVR;e0SxmbK? zr}ssD<;!wMMXx9ldMCeH?`>fKQ;al$E}SuffAr5=M>O7$@!*89LN^YAFkIrWxuU<6 zG&bIyjg3P?*P4{N1g28@Wcw1ag+vL%AmnEGyYx+(^^?8?sag*yluXCO6+CzyAY}%L zy^yzU8#q8JHAmyPoo7f#rGA7mCAvRlgH3ZV9;|?#_%aZSrdVtSHJh7aopUU_xta0gMHoNr!I zf5T^m`|R*FyPIW5rz~I1kNVit6=46-o*sGkD`u?gPSmKv-D)%!Q1U6Wj56>YrBi1D za=5wb7z;hSb1*KFpHi)v>ORJJQU}d?gbB3h8*UJ%AvYrt&G-F!w_m0@pxe-P?!R(! zbYvL(iaL@zryp=^24Opg?nHr8982m9So}PAiyzcCm~9h`*|2oH(*t|c_(&bd{E!x;I^}`x4FE$#GrN~(JbS>cJMiYT7etLD1LF^WWY9T{Zcw= zQdm+`Ra9!#>Z2v$FSR0fcpn(oin z^Z)+!1-bC`Z#mgon3S=9)Hjg--BkZKAp`$P3;nm4fxiX{{Jv+PwYD>(W%zw6)9=qe zA_o2@MPBINEGD=eG%`R!?C@_W_t zb5~7;-eBh2esl6bS;2euN^pJov|}WchtX6+`3NeFa+73~aNOh;0&b$*uUDLhvtdSE!!t_|E zy{vO`4uBJc;`I4gz+lVR!M(S}g)l2p;SBERSi3-itV2M=r0|1ieFk*F-#iKlVaLLR zu|X2;L#++aXn{~$3Lw&A=3~-cB!umc;u+cN>2cvuIHKfn1;j?%qo^b?R4>1xiJ1FfGyafJt{zEs>BqIHx2(Yez3+C;B0{ z6DNF!SJX5|W)o@&hcIZaa_Oa~1^TtVfaXaEFG(%t;K(x&;4lIz@C>ij!P-NSC&twL zYL7aE)=Q2TCZ87;500cCsE;Y9E7b?YPMC zUo%1yT>6cOJQdeSR?rOvU30+(mU||Nu>OuQjvOEsY0f5Tq!d?$#LYgHpuK0>YfGND z`Ux}wGCVbMU)MIopz8#D7sL$^LvWqT0DrJ{D?MN{Yfq!86Y2MuXLtqtODTUdX^38J z+6Y>PqXG9{1CZe1)Bab8)6~)Dqx*D-y@c=Egz}LpXb{E;as)f%2$50ps>?S-;>T_& z(%p~*RYOFGu)?*ZlLwOm8rV>HIYj~b0sc{nw%0?-3z^sCI3~5%qBdJ$Pkn>qk&knWnMtNXY9LSpFT#SMJZK-t2~&)^sEJaNef{IN zRXZNTA?;QDG!O!z22!!c0$ab5PS}Lh$>ENCLyC}h;gFYVi~xzY##CV(BmwO80E~qk z0?TiSb!JKdF!}S$R_cYU1#(#6Odx;OgaZbuW{Dww`H@p!p{|`%Stjj5%h#tU$8Q(l zw(bm{KchhU(tEL3lp`*panOLO6B43fXA%_^$1c0P)_Vw)(w{zR%r+D)VCub zg)cxfepEj0Z1{ZSUvZp1i^z=$r2mjVRS?IX`XOzT`qRj0)fQ$An1kyCm0b~PzX8n#Kp@FxW9!-q1U|iMn@jj(LY%nooa|$Km5IWd8SuT}VM_O$1 z1;1ou9=>p-$3yrT6g{RgA)rKgAi=6QICRz8aSPmOFv~A4zXu3#5YcjfYq0WXkUS6Z z`Cygu<%)PtBSe0;u#2H%!~?ZpoS0Clh03wSP9m;Z+N4z(MK-3&EtMywG}f8t=5$bmWgRV+=e3 zT3JxCw;-Q0O4_$1;uyQK$}ovPb__F;U?;elhufTh|m?_iEd%#VQjyTog&w7Iv} zOT$3ak>S?B@)9J+X)xpf?7=wTXex*upW!foO}A>D|CEY^m{a(wUnfgT`jg)xI*5vy z5PU|>BCK+9la$**Eh@>i(l7B^j#Xp;BS-|Bcs7789*qK7QhFnP(t+qikyM;Lg?GQ< z%!}((;DMnblD0@H2U>FAQEUFiiyTEj;JUOGWWZwJENqXx_*``#CW@^5f!1ppjJt4} zO(hP2PKlqFcP#dABMfNz1JRiBi0wD>v?ufaMLh>Ourg;R^dis`viTBo@j%Ui^C8sw zo~-Sy>dV#BS5r&LE1`fi1NPV?wc?B6c|?NvI*2%)Fy;bT9qwl2Rtm;*%sDWWAA*Ua zr20$6e$I^mXzAZzSqt95Fum&FBZjp@7NW9QZzOA}dtJTgn}6k3Czpeu$415-(H(TE z`VdP&un^1UZ>dD;t+YTg$N_q&6Vb;4y%?Z|3uFT_1keR+Yn6h6K&H^Qpdam0oV2=G z>*0s9_rzCTZI?%q{NnefPy#qF=xdTcy)kX{qNX8afP(h<2Hav`X_rq8v-(ym~@a|&oAh;N=uDPl<`nylx#KQ zooQy*yS7J5rF;Unv^_}L?(TGYGN%hIn8@Z3#kL~H+DY3im0l zrx&VbH|FUaCZ{{ipH2lN8t>Au5OZKN#Ey!JnN_k>_hPj8z}efd%wlP;$(S|^?4|VA z!WLfSoQal6Ry1Q3$yeDA+(?N&r-6T^me{msDUDFUdBdxFt?0K}W&2q?%rK30kK0N_ zub1qsFW3GwZR$M@yty+~dGM<6EY&`BI$A5rH(h_&Z7fB3(_4wl9(E_wh@1r1L?&PzWRb}@o+==0Bm)6N5XZjSNibi#kK3&+J?oKNwdt@kAU_?cy87Vil8fW z-c*`}{rxASm-|JCJ(uWKewd3vjS&a0+4p3u$^~&h5BGs8Jw1D5Fqv$0oB*FQtM147 z@h3IAk}er4w_Nm;_GlGfUf6b5qmXus=Q%IN(ZgsIJGe9JbX@#4?zZPu@YaH&({sVt zjO&eD#;s0D-rb(^1XJ1js$w_fZ_O!`5j=QO6={k>d)K$Ir^>!l%X=8tjgvE*R8IaIH`BE?{JDG1JXcr&rq|xEEjZEuC46OLOn8qh@l4&#oS! z?3f;ft?Iz_Yq(heRWW-p!-F#~TjZ758ec4f)?~2y@qpVbAEQeYMf!TEmLpYNJ85{R z@?m2yYv177-yyer)%9cB?z>*G&gRq33^Yu8-mBZ`lN0!cV$$q9vt9bG^}W`95|xYy z`b-JhU7D+Oy&HaOKOH`Qn9*6BO$_ZDrB;}u ze)nS07i{Xk4OBU6y#Tjdon&#e+OvBhUec%yy0WX=WT!iwNV(8ss>}A%>a<=wK6eY1 z#YXzhktfhB9eho(oye)Q{P5E ze8Q#GjIItFvHzN!_{@rg*cNlg4u`||43`S!`_GP*qZKMBR-tz{@VbfY`{vx~PIeQQ zsoD<>6_z!b0<1J7)zHprVO1LI7Wx~3^7r1hxe9Ph7e*tkDbEGUyHrr&61DwevMm$d z`GcDG+I2F!-d>p+BVSAGRlloAvY|*S(JWNrTqb5Os_-Gb?(o*!EC+{X*Zcyb)i#UW zFwGXEiHa~|W$WPT%A2-!D~bJ6iaS}dSkwH^;Z8}pMNhHmm$xag#pDf&>qcm^vN5mF zvFE$$Dj&53y((W{p2@VQy#MkiCfm0 z+_+rpLA_Xp1n73@b)%bBGN&usZBP(!j7lYnx3LtA6+`>9E}&#e2ayj%xyy*sXdK9E=(rQo7JlIr}XUd;*X3q z>L`DNCbX|SZeTu?DbQG{vQ%5FaGg#cUp?+}sYrrRT)g~B=po03&UB_-tZ|*s#5l#H zWcpRF(*G8@#kQ2O9x|<|?KM-t%;>l((B2i4$&0prFc`Ctb)3A_qixfC+ZHibDlZ|ZA6;V>a%2lN_qg7!p9_2JPQ z!EqGlLlcO&umsL4Q!`Z2 zqfG!`>g&(&gy3iDPs}hydt6M{wy-B(m@~AxS*l% zwSHJ%ozAN70Nx=ryPIfVE;45;P4bHG=UuD91)Gbhxe#h^Zs^mRQld%lw2ZwiZklS$ zpkSb^%V=Y1o(34UH*Pm0W;$X)Ks>yOzROplyf3Qx1hb_yYE$Ud6GS!05@E-UC6->drQd_ z{#2;Qv41!)cdf#kaJyRG69*3S;RgKP%*+79Z(vHTf<~A^B^I`U>mE0M?JIJ8N#iFy zXBIvCfG|Htxx^0QhMN#Dx6MGCU7Jk$Ma>vqX`y1~`bODdqw_wq$Jey4bNX?Qy^bNT zi;+8QG95Y>%C6&7Elnxt%XH!wAUG}WqU)XX#&2;_!-b(6U0Qa`uG*RQzt}Z}@Z{gy zQJL$5Kj+|nDM5*@VgNKss^hyEmhaoHB%!M_X)1JMz{0N1prOvY#$GPOPaI2EwhWF= zez(1^@gh!`+36LYQBkGV#v+Z-3*X1zBf859 zCD#Q==ilD87!i{N6h24x?()qIr>ScJ5Y`B6^Nz$;d(Vb?r9K zY%8ylADRU)VWK-ka$wF?;C37l9V_?|_2PObKp2iUsFDi`*ECkOE5&*2{4NVgZMaJ? zhUZ`@!jg{dL)`t{TW!38Wn~9!tB~+FY^iV8x|VlL7HzEKW=gbEm^yeY1(zI34C|uC z2H=+@RahJsZ!5xqu|e%rN;r6ON_M~SQOF}1RB&~_C6ChM$X z=)-3n)@=$ngiOqV>-sVoqrOT%gbz;SuA%w!?y!lQUEhLTY6`+z`fsb!1{b)P7JMYu z)77r!oIO0B-IiS1zV&V2jAEl3#rJ3BrPryID7`;$TiFR>;hfF>EGIaHuGSa}e|aJL;xcD$aK4^VplhD0Z5`=xZzZ&>9<2dy z3gc{Pf2r`vM&ej9m6Uwp!}%bTjL&WXTki6lhbiGkYOE#$wxr=N0za%NxCX&elgCM4~oxSaW&M&r5wx?z+*}@U2>BK!w{AnYoDbOaDd4 z$R4vp-w6j&BqQ?kSrM!H&>~_R1Ebx{D$4js5;q(F62H~9RqpE@IONyJ zoJMr0yPj;8bren>_TO(K?3s*~aX0TTYOdtnEEAQr@2Rv-I~KNA6q=K->k0IqE~EF+ zN#<9a`^|18uY>%)OFQKPo`H-E5A~IOtZ?Je5Hj2~-mP))wkL1a1KeGrjx^V4a-Y{o z+WaRJgOPwdZO`rZ1|GOJ?ki2Bh5-c@sBk>FaaNEaXXG-U**&Tk>sz);ZRk77im~g+ zvapv|YE27`%k&u+IN;`18DqP6`@=_EoeFyiC`|SW)Cj^4a+veQRuYs7NL{x!x2Xo4 z8)r+dXFndLG=24KH}2b*x>UHK3~0=idHs}AKY7!z2@f?;@haRkio0h#P@cPcgH5J$Gm&C9@6bJ0va=)pntJkc-=}XDJk_VRvb-hZ$57oNw^}kn z)(u!6XB5BXmOzgtq1#X2D=acTYZa<+Z<8*~Bn|R#%59wnBv^2(yQllkMFSVR+RfNb zr|NX?MG`OOJTuE1&tErZN`_5A;qKy@m{2Um#Du|HlNT|srPSNqO=TjfjdRB{n#Ucf zI}@^;ZUs6;*A#MJyO-=d#j*)jGQy)-$r|4+H)m|MB`z4H2A{syyH}oWl$~^&TGqIF zxUCqm8L=7IU$VN~*znm?VX3izdGgxiEmVCjK0c}1Rv5`0POHv5nF&7AK(iUlUl_?> z@k~!nzGvIqiIYKB0Hu*{szJA%Uy0E2bhMk>$98p6nFKv5Y&^~gFk251 zt`E#2^7#}C1{?-;B@PBR1~A6gkkJ}>!|i>3vV{x$R9A%2$vme7drxbAeK|r|N}otQ z)DT}edT8M`atyFc-QRujgTfzr!%qAA@#in?oa%rMPbom5Ll2FFHztwAg zQpWhTUX$P?sva`>}+vU05QYJ2-uqSG%rfd*&*2~Qy{kr^ft9VF}QZ=lY z^jb^$`n$G9u&x|Gf@t^f@bG%qLr`{9q+0UOi7u{BJG%~v-8G+YSk)qvKinEPI@lO0 z+UmT2Lo#4M+CQ`PF=0v_=2K@|vcOrGU;qu4?L1;}t9rha&=(MGqC7VOF+X^B8T@1< zRb1Nzg##*hgjYNy%b8{JH6K{u29)1I?WJf-Pc&0_)p!U_+>nNTQiVAHJ(%DZFdrgY*HS&HVeIYm@Sg4GN=NgJM{#yNMAg2xFuC6RrXUidB;o3bC z>i4%xm;Er91AWq8k>if|)EY-mu3rth_&!H@UDz&mvwC^%@65R-d4YV^4rw{8 zG;19jdfZUJHHSA)UBV9A><}%{!dP-)3yIE@ELK{n^a@!s^b@ZZMTM<*+EIzsUX;!R z8>cIwGb&4^61B_K$x1 zPvmZjIsx|wb5frz$%OV*H z4_FB5J;C(f%m`Kq!4&1k4i4y>r>C$4 zA^$8P=0I_UabjeDx92QWBjY1l5{tYcoK`7L}727OZ$d!?&7bvN_Y0)OUKXb9Zp z@ZNe7*}W3am#&eWs(w;eV*IVI$MxIm;t)9-`T#$+@omb+)2?*NqUc_(QRi}4)EyK^ z8~1E>1*~$L+1RwHfvE#v2g}bTAx7f|K3BOD7z1hLYT6hVkZ;~E zz7Y_>!kwxQo;(cpd}nKNe^{?TlW>qlKE#2th3`hi-)aR`DV>n@7Y%NquLbzhPQh;Dd zcUZ+Bn`$jL5=u6^?_w-WBMa6v?P||b9707i+54QzNqfx-Gc}qbC1$sm&?wU8BzbdrN(aTmwi{w(6XO+QlN(Z*`jjYcZw zCrg?b^7=`MqW?q{s%;Px>x-@~19kvnEs3}cmTr3PY(jSo3!{Dpxm| zOu^AW%$&`-`pH4v74J_6{hF<>BN=L1v+-YLq)K}Cn3~0Pd|??p^d~_q*x0|M11VBs zHpW@_P~qQeN=oL)voOa(JgN%IBF5__8ss4G9+rVEYj@Sm!>Gep{Y_mrF>h+WK(DId zEfp^ETEBjcATN=K&H;YqR$y~mC}S&=t%Y{f>N|^UZj899r>1_a-VdG%(b~%Q&Dd3- zItM|kFe4+uw9Kc;>Mx{z4=<#CdfB1&E?u*3;T_DEaA=m&ouU+B_%&O= zCc+*4V1O+8ndkI1^>7TTYnTP=rRI0aqYh}%fNuo%GOP3>Zi^ik-wz9dbkdW&#_?K? zWsaAJw5OM6!pe0HFTdE&?`k`y75|yk4Kv z{)gim;4lC3?)Zr>`l~zrGg78+%!)AeWV}|44oVJ9z2|E)HtM`zQAir* zu`42;9uMsoxP%HlW|aq{`)!=&eFJYEVJJL?!;CQO&PkX6t)HX?Z@=P3i8`!gpSTRx zZ_Ms!4Y0@Xzr z&H%zXH=;;!2MW`QKlrR2%8R1-$+Qz>`WjtuRQUaSg#9Z+oe8KzQrHU-qwL6pI#+UvA&<4OM*lSQuZ<;sse#>+pkG0^)70-ehchmun)~PKH6%1 z#BJ5Psz{MHy&7?Q*Tw45y3Q4S4YGf~kBf@o4vwz3bU3W+>HA6(nGo1rXW+;qBQYNm zS^_T%UG!LQ>9I#XkDZYsnWYZa5D_--x|9|bRy_Uzdm7!s)dv6ACSiQleXB_`A+TsC zGtzh~VVvASltqwjKB)rY^I|0f3pJ6mwly5$$hES#mb8PzxC&yJUQ3OYQjlhfbOOB! z7d4F8g?|K(b*V;Iduum1oQlak|9cybwFMa0UAZz`R&)|nKj)fPe*2JqiQkVgY>?eyIxQeR5x=grp##Y3`3obS)di|K932m|H8LxSIGm5OAV76a)N zscX|-G>ID~#n{1L%ONA;5|O-SUK|CPv&Y{emGltI%>;pNN;XfNPpfB zV{obQ^j_C!F*|H+S$V#Nd(erPoo@rHv4nDrYC&^E^@8e@-oU&h%hisRHzI-I88Yu<%XC(wgMo9Y#Rn~=c*UYJ*@)2_mJ zyw@kDQu;!GGqOsy^|2#7@@-c*{`r?WNA5*xTccN0oXx?QYqhBs4e23yq=Ma`r_oKe z2-wF(TMpC#8Jec;YdDmZdOo8WA#i>36jI+#ittmNKqi5di#SeXJqHjg^%KmMU7D=z z3(Nh|zoZ`e1l0!P>0!ddi9n*E;<=EiSFu+KxVgyH;sN{X&|}8&+QxR6GVzZFVN$~O z%|`AfK$&a}4&>^Zji3oWX&)rDa>eNJTjV_x(@1r>1Zv7&zKig!758LOp{DmcLve0< zv;L&L{?QOs_Crl#hf!N5o%_|t8hjgx6O%khIN*J)Pa`BxXco+T$aZKXod(!yRt)ud z5ZkIs&R<~GKn2-n0^O#Lg0ghRg1_2tQCoBwv=a2r=bOFVUY@-bfr87BL$x)z3Lrr~ zauNCa)suHuZLjwVZ4?6H6%mi$d0kyM-cY>%zTPlCPxo{Aa750hc*TSxA zHT+%VJ#2zGRz&sgOGFt%*!CobyEtoX{5LzUF^-c}okslSMVhaZFLI#Lpy*JHwYu=T3jQL5(6rGty^W00YW-s?ntFa%*t);23IR? zB%85Fo^xGi_%^+ETfWnsofaz(^=-cOtiTGj8utt{4%AGlu1FS!9_pUBDZRjVc1Xo| zdBelXuuM_P1FjTgtpVdpO_pT`8mL+F5J8iu$^}`)#R}TCvK9nj&1=16I`!bAjg+?-_rimVv2~N$^sY>er+yT`| zZb$nKw3HG*e0(9gKt>eRFz-Pg zIIkqdz{s@h{F%f@;@jWVOv0}Dmaic)}(08wx3o8z> z62`^CAsXftHG$dF$SW1Z^(kk+AL^Pod(8?Vo|@ zLz_Hmq_7fgD~!;xe5FuwHL+PZRo75Bq{yaaM^Lr>{d2Lff7r)JkNr0gT6R1g1p~#4 z?+sm&q6E$ni0^so)v`UVI9mhmB4$e;0#SN!C1_aXkNAfNT@-YrJsQ3bf?*%+=+s6f zCHFh+Zf^iQ%86X1bt;TumgUBZ_N_A9wSM&yce)a#ZmcwX$msOI6nN|3 znw_Z8AdO*zFVJ`uq$0RW$#S7bX;mr;7_SW-%ILJz@v`^KGV3#Xjr4X)_lV3?QdPL4 zwN6=aQWt7e65U zaO_?ZH_k${29to~3J+lFs^~v*?0$ryzxFWvJ`DYpJN2(x8LCwHtta?U;lPz%L-1+g zgKB?e7FVAaSF462L{`cq4iI{+#Cs*fc<9`X6`mPKsiNEFi(>l<%~I7Cf*1`aDDnM9 zAgNT!`wd4ux9z6qr@`y{=#8UWq8&_FFgl@SAds6nkHh`~#HDS$MLVk`Vb)&Me<3nAmw@O3UX%*x)J?}qimDu$`dGcDapNM;oyC`znPBD;Dc)By%Iu{Y)&uYg=1Eb3zsdtzn1uz zg|0NkiQ^qEu8vw-vB|Maw$JBJQxLPplz=g3_f~EO$`za!uTbsI@MADr_uB^kdIBrP?@xVlYYIXIy0p>c(|$|e~w9jbr_RKvMk ztBcy1&w$zqn((23e|?Cu^1($jqt+}|IcPJsQy9Q;i!;~i8z1`I z{0=Z4>b^jG&%1OUG0`$w_ib;xIOHlLgnFkW<*M)Nv?T!Jhe0lO^DUnSEX+(p8K1A{>^g_bRv)82(gNDZ5BS_F<`Gy{rjN&jKGf zav?33)mO7o>E8qC9952vi z{d|^r^Egw`Q&rNE%DX>2AJ}7tQ(7Uq(3zw_&c5&3tbp-M@{BUFLG{K_n=(vFuvATX z#<*^^cB>&i@o7G|EGoV1gpk{xG>@lDqAFpWTD(@D?=a*nZO<>P=o=hJb8NZN@_b%Q z&9b7SQ-AEzQWH0m15IYk=Cq6Fxam~VVTEwTwCN=LG1>_45370!4UGdlu)_gXRRb8$ z@ZVb1e@gEB!K!`|`2H(n=@0YSOjez3VfQG7ixR9Gc{LuK9l0Audk5Js-Hn1Bu5$lip*GAAZ?X_zD#ZJ zMJt2QHaK8%Lyh4DKV|;6RgY6O`2cP(%26VeSMmNqk{}BFxHt7oz-V7{#PIxL=qKJJ zwGQy;0PLw!ek_5J%V9-|Qe7C-3Kx?J={RPt@9v?h*VFRB^VO=DVOX_B%C zErYmYZaE^?#roxd4YHnq0h^#qgr3?#>&$MTksnotch~B)rHpTNj6j^OsC|=y5q#%c z-cMp|!;j@CQ=oVkpJma~4q_WdzbMFa`dY`&Un}5#(bR(uARE_jucQ6UizXk~tUexN zxC`Aays8-=t1p=+Vsjc3Kd4PgicSua>h40lWe#)U&IM0@Oren(hR+u*981Ef5uc+W zAsE>jzG02HUOOzEO{G>ZHRc3qZP}u9Deo3jL$sMM)8rmel(^b9glmdvm+hmW?Cel= z%+t9IcmNd{!8|fT*Y3pERRf}qBav&^oE?^ ziIhGecj;)Y(YU3I?&;~_G4{#z_UP$p0dGIZNR3W104`0VD#<|?T)k?G?$%^}fa4vFyACiQ z8jt*c%pWlPK7Rl(kw2$O{wgK3q&{R#B!mb*jf=SMKN<`pKY-519+vsYu~~hqRwQ&K zB3;fwJvN?Z*3*=jSY7XS;7?B1=EF(X3W7hpSc|{x1A;ulr{px^d{Aa`PzE>dnjN~N z@JsgKagMIxnCIJ4t{_EH|2`mlaJE`qYy)1{tP($nGG4;AhXi+e&P3oo4F3+p?lHw| zPeE!)f#nNs6r{^s)=Be7(&OxoR0lgMAbU{SqqbUbx17A-zu_%D#DYGS#~!a)fg)7} zEDp0O#T^VXn*a>G3xo@AhRCEFO{+k$3&#ZnuIK2%`lL82`Hma6w3-*scRWfY{$w%b z;LV%y9F6&c^jS71^w9bdS7!JR9tbrnEQ}D-k3$UcmZn7&3`lY~SdUwwiU^mK2niT*e($8Bl)$Eg zq(uNMoJpuvYC&U`1Ef=|u1$SWM8XJX&jArF)d4(ov@m3e3__Z>3rHWh4tczG@?tE zlq^L+@kbUh4P+tHO>BP3eeiFu z3Lh9(GyGo@m|B%POx%z64c7^0YDUi@CFV z_OTz^7qwHlW1wywl!<)J8-0=B=$KuExXJr#L+Be?INZZtlNo22@@h~ow5U;h`sFS6 z;aiQuQ6SpQK?zC-!jB!js2>Vkbpxkh5iSN5fCI-h&5ni~d5JyvO;D{ROel$PLE!nP zg{3liuJ5LK?Mxxxzyezj@ze~qa$CGJd0oxwKC0N*?c7-Vcva+-CZU@Kg>6QPy#~@M znS#BrUkKJI?d!RfH*S5lE*8*(>+Mf4x{FaWuB-RreS)xIqA4ArYzG z4(HpHN~@dlO<`vp4X6+S2%D{GbJw zlywA>qJBg#HxgXpWc4gige@%vc+`fK9SvXG#e2uZNIEjo@hiYg2yyWVC|$})>7c5< z*@y<;3NDAR&>r>d8H)+Z1Xqq!tPBXku;6C&(k5znAb!LMeY#bwldt@mh>C!4eQWWs zmTjRH-A+Ya)D_B{>5=Qhil$cVd*08sjbw?yDAcqR!w2fh2f$V-1&g6>idGwEBu~Lb zDRAI z$|SAIXs1@HIi+*PHaV;dk~b%l$iTlu);BSUgfQ}&M!a#MS*q_SDSOZM0DVRnU--Cy zF@mz><7To3n?}0^C1xOE?Me=ByBaXtezhFL8flqdIq^c^EW*es?R=+VN0Yg9Nh(H#fzneYDp;q714Lq;nu7Z zJVoHP89cVRfQ)H-;MrMv z(@#BAiyXb44{nlc4$;X(QYUt8dY*o%nEoGL>&1tlRCPo9wchg2Y7oDxEdgRxR#rB) zKNca_nK_vM)5zx+DAk`J34d2s0tUL+A}FY?UgWc0wsy{+X_&Xyz#*87LqXz}7#xkS z4;q;fN#e0~?n;LYdB61CQYL~t)_NI0da`6?^`orxQfm4ATr8=76p_>ZsjLLRpZ%wh z{GS5R{us*0zN&Xx1$b^s0MAYGABppa1i@b_{WquWKPddA=mT)thQkom2|AJ9TiybT zh8fs6R>(_l=2gXz6mU5rNK{su%sbi`=M){`auW0hF!B26cq8(9qmUhaT)2t?>{W}erohk8LQz|o+}dFhTa8C7mNVE zxlKgGHGv?6to0mbzWkPRsxg&G8wEL+CwJ2@V7}D5!9v82hv*MGF%@J>Jc z#ba9wa3-lU-bpeCi$QDM7st451$bXQYvyTlp@kmag2OyPU0v%Dwe4=^w>|W%7e^o zouxa@3a0a}Nw|!-wUk>P!1c%0P^?2sx(jr9b`*OhA`e~X?aw@|Q^*7kJB%{M!Bp1r zou9vzLDOGouBfCD1H_-$5cH-oVkJ34-%db2;THeC*|r3Iix*K|m^<~OnrwgK$s?LW z33`C9*lvnfN79<_*#75X3DVb~I%)Ai_5PQYOUl&g3gT7O-}EtrEnpnI)Dz<30@YB% z_i2FEU5_-g2a<|K?{WGW4v$L~CSM_h-%(GZbM;bDg!+3gQ@7D%%7Pet3LsOtuaH}I zZ(~EhSj=A2pS6sOfIVA62BvjO*xW|4WsyLlk2m;q!AVUzDQG(qFWQmN66e$IyQ~gl zGOs`A-d%8iFBDrh0XK%+Zo*7m4%RRhWsG%lTfa3Yzaq}4z^h_kI;%)CO07fGLF_6i z&xC?MV!q_z&~h4ozC)+e_;gx*dX#(Ybz`-Iu(Qzmo~ISM0A~l~+^qthH`!sw;@pht z{y@sZ-tGJ}S>#eg2J2EJ9Gg>SFElkZSH?ms%Z~DLEM0iX<+~+&*m){Ki#`nk+v?i= zu-l0th_>ZwxIIITk;!>qGGKZE*tb*z-N$=^UUtXj$0s1rwhoA}^NiFr&n*%)2B@@( zgM~z6c9~kfa*;yKucqlPhOG|qLGPR}#fYO}<{7uh)8CiOS?x{NEhb7q%*}qx>=)?x zM)1|D6%;CA7TN&dAuB~Fqcw|m<#a7Bz%J;2g+e9nAb6<7EWKfnA-zE|L}ONQ1dJ32 zW!B2-iAN&BnX6-b$dY(%#(;)Qw;iA%>uM2F2!dqZFDS&GA_%@Mk35eYNxDm)2?@uY zDQeUKMB~kE0M5nSLI_k~N;)8=ZMhKg?P4|PyNUfOT5}ig8Ve3q*s2W0;~aic4GXjQ zbwz$edabQYd2aW%pc&RhoC2sqL{0`Npq%)puGJF6vxaXL0yoWgZvq8SI3xNxTtYP9NPg-s4AA%P=ekzKSOeH~wCR-3X?pBfteC z_U${Xj*I^OgR+7Q5G90c5(acI!=y z-I$jNwvfTB##kTf7OGAo1qDq@cI@}B^5CG)3K3ITX8TXt*w2v9PY<3JYQ}EGeNQCk zMXBb1bbySJgUXtQ)%lKyF;q^|5lwN|l3=Ux&#qQi(<$vR=Sj(q*PkG}F9LPJ)1WIc z_v|8Y?A!KU=@Za^Iss?rZBtqB8clFu>SWbW0~>N4s%eaG+dv^PG+wh zGUhkY!tYFQ4eUyYA|md^|LI)w?n49lF#J)#cr03!W2>ccTF`EC!9L?|JOM2@iNdPy91CEeoy*iPeRo%C15~(B zPp$jsmQ8Y#=$Ocs@8nLbs0`M{by z03KIXs9G$XY3AZGT!+%81Z!U?1Qts-FdE-i>MBkA`_V&lF z5oda+i8@9w6*qAXDfax$anecBut`S{0h zuY@wpo)m7;ob=goq3qUT42foc=96$&;CME^PwK!&_nNQd>2F0*3B+d}c%!nxDi%z| z)g^7IF1ZxkgNHm((;Yr2Ct446va*~BkasblS$UA57OJz2jeU!pAkdCQ*vcmv7~h0V zTAF0zhOPMYl&8WEefgzSZy%H_Of$~8kmfMZ9-}d!jBCB!m49wZ%X>#!XRSxN*0@F) zM0eW`!`tBV1Jp_u$FK-Q?ugF88_f`}+^b*mLqD+i|3bO{^6#G|O#hkB^H;EvpSxYs z|H#kz8@KDXLA(Er-}QeQc`3|jUwx>s1?c1D`md2x|EkRV!P{kIq-Xh~WB4nQ>L)1N zzjJw2BlJE3#gRRM-|(PVXgUjI3EevG#>OaTJlfCn?-(~rk|_cC3{mU#M&0W7uw+Od z(iJcn40aYL*Tmg^LAbr$>F?-#=S^FUqeZea=@sMt$LCQOc_{iqF7|T$!}b5$miXiP z=@}UR06F^Yx%{LZ=kM09DP}4~M<3_s!NaF4?1vg2=uWT}UPYVGi_ie6WX&BTSiz)* zMd~>9_AdGrT%QYmB_7m+CtROB#&?Y}YCeuG63qp_V}Tl-Kd$@#2lxMvaOYoZ2fr=% zZ~9Pvi1QM3E%5b*{5sh7A4VfT{QtiyW&9@2Ps$m8r~Ql94fj7p{d2p_|8|D_XT|-q zGvq~CfUWx5N8}%!AwP8WuK@KW9JSRm_+VyZP5bMU`$d3HfSB~<`cqu`a~1qgP_n7Wwr?)EWDTRp8&K593vE(isyS;%lGV%iN*X-A`-!+`2Sd-yX`$TpGklhs{_kE1u87ZrJSEB!O4TPPbKR z;^DRQ8Q4z-A90%d$a0Geqm_h!sCC6+u8E3SfkaGi_wxtfUG5^cCrQfJBBVwYr15F6 zzC4#{(=Tyj;B%NmSJE9{~6 z(2;>6p!M5j@$7-TV(~lZc7{VI$D+N6&z~7kw&Q|?2&&Q$$9LeQ3sf=KRWsMl%vX}R zHK=b``E0l-r#rUG{q-Y`S#@pE^`bNa+Bz78TETn3cb(KFzlE>ztl+~LJk5hb!(^tS z4HtH)-i0C9*z7}}qfvdOGiv?zZ5+&vHFz8AWU78^OMXR}aiUfh5H?ogj6T$DED~Tc z2pvj#n^apI1=TdQJnp1F-)-nTJxtxGb-&r_ zX=e+{3n3E9dpr_}8;8P!`v&KEdv&QNT*RKY=MZW7ygM9GZsR(X`{w#_=KjR%RVOZw zJ!+gW-K|+`dVF7xsKG84uOrB!*K?^(G>XE^?p8W2OWW-@NR-0*4YE|vAQ_6auqGmt zKJk{Mwb=?G7yc%anzS8H! z8Y=K|MKCP+HB`@WX!eH$5`_myMua2_HeRV{;W+B@{elFfu%5c=oxYFe<0g`byc{qL zve-;JDNEL6CBZeDZnFDcl<2ajo7(uEtNKO?rs@HHR z&vlKA4PUi;xCghi8`Lsu5~#}FoJ&`l8<++-o_#&xI>nbhtLdHw5#R`6MFC}s$Y_rU zd6O8DV5>ab=nforWl6M-*KG|nAhq1o!(~E~vr_7;x(a3^*DUXtM=v(q+*83mA}beYy8mhe>`5}U1kr*gjS96lB1wx- zGwY~QTxo3yLXgI*0elVZlYW~CFx3c%FlHpR668vk-%Gz70;kc$#NO~g5o-x}fDkW$ z2@KR?t=EO{t8xpkj2{ALZD@v$7bnI+0y6(%-)8F-p~kinTtjFn4uLk!hb*#XX1g4h zgd(8zEqb}J9`MJ!h5=YWj5LGGGm;FGwewc<{KayWrI;3frHC!mm*_Fq7cRiXhsGPK_rC-Udl$DV=vkukAfyc$bH9F zuErf-q`V{w%nH3Rnn;|=iuzh`nzYvLn%_H_=Rj6p1aW16`NM!l9T zcSMn4mh)bsk_W8^?Xf*Cs$jM8hGD+bT_a1A;K!022dWFn?yq~X@ftO&*4Uu0C|u{* zL;JfpxA9ej7{({=o(Wdv*e#fEpxr?;+a~dVxNVez5LJ5H_Y~2WA7VD2gEU zN`D2%oP1rpH|2PHw#LI$YSbT+Et4C%0%XGBY>gm3q1z;j*<#}-4{8xjnk3LilnNjmajK;Y zBse&4vGH%e3tt7jy(|&MlddbO)yQ}9fL-uZ7dbkr-|3udl=7sKUV=C}jlnrbbLV4m z5g{DrG8tGlihs~Y19ZlIpdh<3seXY1l8Bem{Q?DiP15|P(b&+YtN1r4U|-M+6p)he zQD355poctF%*j!0;Bdzp4?uP#3)C-v0CWa^<8+Yd3q8EBiRu)SX2lt$^bnLeMB{$X z@V@EY+}kFL@lOz=;PaNNu2%w-rqS1V(wLR-%~~$zb}5?AT)9kP4qj2TF33_URVYVN z;zMjSnq?uxN8fyqv;DceIp89E`(cD8Ds<{URDN;Xk=o^hHV)X+$4)S>(jj)DDOnP- z%BeStw#Z`<(fPnlDETe~N9~y1I=X;DTkFGC9MQl8Dr%A!!QAE8kockfBrxq{P*wl5bh+-&n(3J;VIWWbn zxV>CRdzFp7pQ+enImtF?r#Abx@J@W*VaMn=Yep$q+Gp4lUYeZV3pD%%3s@w#*|DIV zd3rv)v}F>$L}yLwS`$9g^m%{(^ZD&@=>K5>e-v0;oH{-50S@fT^U_d3Hdp!_>{8V@s2SC@104xNp%{|Ye8MUA5QK|WZa5nM-3=O{4&!0e-V?N zotg2k3nV|`F8Ec^NllY-m{#il3 z`|J%;OGU#c$o4T$Gu~YH*KU{&P_|7Ap{lpHcuE9ZHf=D`A;c}KXoeq8UX&L(w~U&cN=qnQ-tn^ zsDCap{*T$NpG1nk$@({s|A#y;HvM}e`G@lSA$0yXoBsE+=WmSqMV=o$=U4&z>yH2c zCkWtl{Aam@KZx^E_58if{wB>|h4M{GQZ^HOh;Yi%+;-n0B1Zek;6I7w0YxfEh)KMY z5J&Nuqzq=gB3^bpta6+&%cMkZ$uhYCS*#=Q1mX=usc=lG#e>>KZzfK2xnCY!SbCr7 zcGIe!144nbb%!|0bU!)W0eO&5)AoA2Lc&oLP=Zsia(&nX$yLVVJkB?pw(hb~6GaZ^ftEK`zV!wk~Of5fh= z`ndc}OCS|Dw63}!kgY@}_pIi?yjm0iY;Sof5lt>n%`WSxVh;#$V|XswTQ~tn7T1qa zIk9#Q$F&3LCl-Vxg}{N0Rw+FJYuo%u2-p2j-d2$4AT)R%E!aQF5zgl((F+k$fDeL% z-{AX@>vS#>)fyqX8mikJf-bV_aAs&3_!lS)BlE+X-ymy;sv`{>jt7kn@(}bfY^2B* zzspJR#@`@GEP4lnOl5m8w-Cqgl$X`k9{=51M$tI0U04e9%PDnY+8CxCs30L2M7zLx zqGYRprdXKA_8qZ9OoBP2#FaP4Dx#Aw`Q5JPjGMM>|M8 zN1J9P%T80CZ1AeDzS&-7I^SGAj%2PZ9pj~QKU~e;DO6Gq6sLG8Q!1ZE8}LZxTP_d8 z4wLx%-R~EEpjRnAp!4+zo1IeGnwkMCaKP2}RP%h_MF zP&PGJXfu-(aG|Cly>TgQvNOc4j2}I#ZS`nHdwKyZmaDc>lGqvFR>I;|zIU$Oq%=U& z(L;$E+&3b``waZY;{fn>|Lyb7@>qW`zn`B5zu4bTPJ$mNz{`nb?t0KZ2*_h$GX7J< z>c5Hb$2``b)9=5C@{`p25*7gbAjVNPii2oUDF7MkzDCef#*{dx#JDdPb*~t^giM=)89+%_PLuu*I+Noh7+*u_FbEtFo zX?aCurNDjuS-;{`>#clwQ7Xat?us@qOGl44%XLyPhkV_TvVtt0*R!H{e1+idJ=a4> zz+^T-=*ph=TS&qwt>pF{0t*D9_!>Apo%%tLRsi0qAlWUQmjo3gUjWyRf_5u(ouC2I z?Axnx=o1%(cewaDz_-la@kDzzcyr?Yt8bXM)4@dD`yfNZJ{0R|iQT=rnjXVxV@vp(6$HAUo z?|&l6oQZatku-yBCZt8zgWKEH{i4Q@J85^QFhn0RjdrkHNMA$)cTS(StwHc{<1I`q ztwlZM5e$7#rB|u7!o4O=cmgC{osOYAHWZJC>MIQ)LeNeun0S@@b1v($)npLYHQBNh zOI8VT@LOvzV3k*-bwZGf!wR5-YpMi~(!o#;a4Y_GEnAQw&4m0F~QyFFP*vP9Sbo|eETJtRrp1%(HdL~Tv85zffydBzrS+X$o2BP zF1c#4d3t}D-12KT~cCbRjeRDjMA0IoCr?HFCi3W|7ve_^8c~+&e54JOWbyn$%GSUV%z4#wylZHiEZ1q zZQHhOb7CiNW}kibIcFcf@Av)DS$UGR?$y;#_r0p?s$bPmC>LveIGEX}6^c zKBBOnC0eYkg`dXbl8bT~{>s5fYU0n}M3(1(_rD6inRJHPNY{3M@Lv=*# zF^9J@zB&X?;MgP`e1hLrA>-wk;8()1`OzwXh6!7cs)D(YRa9fJtilLHpDT(tiDc_d z7<>}Rwzn0K$(2JWH`Z;QsvQ9yfMcwie;LWfhmQT(_(QZCq1G^Q+68y;Ij;u9h4xWv zyS?9)z2hC)%yyjwCtU>Hm=JvjGBT-G+bqI{1lIvRwB!LFI9be5xPJxK={fTOhOr60 zK7zX0F*b&l12@sj5MmiYyX{v7#t3AfU#)B-!bg~GTAN2Lz%gH1ark~b#uoY-3f2r0 z9<6&04{Hc4ac<-gtzq!={(9E;Wh*h2;ij$Y*Zk# zSjm>#m8ZHR>~f+53&=ST1sTk4f0H>2+&1Z$~F9sZVXa5u$W_We&-;JAZ0#<4kIZMupFOg8(YB zx~dec#ZRfSS$g%t6zLRl-&IQ>nDNiv4zB}WCvWQPl*Be5p~VNQGEGE6?I8h3BY1G` zl{ae90OQy!)5~;(|1Hu+@+;CNRDX4#r{nPSQ^!64zddmdYK9LrsMtlL#i9!mLkl<) zML%a(e!N}yvQ-}pU!23vMI&L2QbBehOa2CW)Id?vyHR~(L8b%Vks~J&1bK9VbXbdW z*#1Mc#jc+Gvv83?&-nXQi?$=cei@Dny&kOHa5;R_qJIy{H9*9qo<$#hSg)29unDk% z8g5T1=ci+1CQOaFd*;27fD_;!``_WFq~b>WjDLmN27tz6i~Og(c{q4GJg3AXz9cUr z7+TTu$}KQHa~}7NYU+?$p(#h}@wVwLun@`%RJTiJS?kf&#b><*{T6D&laKfBt}kvJ zQvRW$3)sq-TMi#o6QQ<@wdVQj`4! zF?xzf_b+eoJ$aT-x^0ds5T>V=Xzca;!gckG=B^(CSwCF z9mAurpSv&{H`>y&kKC3$ugTUhzHXJh;aOPA!XJnxUPmpG8K)hem?CkrP~z@~&{1MN z{`Wo)mU&mNiiiFzWOOy+=Ki!sQ1i?oka|PvTyLf(5wqb;JFBr_xfqMmH<^9|jFUGO zm8;(Pauedyk4W1>kXhB2k4PIJeAC0K4=WVG=NRPReBE{xGwrTZrsQZZ#8rs7J_edO z1FmqpUy(LbqI%;P`Xa^w<=WpOZMNdJwkx}~cokIs)+65f+7KXP--YkgztqUS+p2Q= zHA^p6zJi3|2M}VRlek3DgRc|B$1IM6+KdOJoiab=?rR?n^UfKxigXg zoZmCCme03PG#r@r3z8|uQaVPzk~gS_4*4#!a^89>>8w(7#6Ow32f*j`{K?U;E2+*r z4A(M)87=HWGfe{{L$jTWjf<0wjWhY}`|La5-#pvj;t(FOrhknQ{Ervi{~ROu<7)ZG z#ryv;NYMDMeEYY#{u!wMZ(jJHruwHuqqu6!$-jH zf6Mv)n-~6j_QBtI;U9ltX8p%lg0y0P&mPuT5H7AvDiAh3ZSU|e_&5~*{l z!_{3PzfzD*CTh>tN5$3FIPAq^l{SB{oPz(@1cXIqxgwCayAK_}a_R5DF}KfIoS9S5 zR8m;Tcx}Djm)XPr7Moq($_aV2lJi`Q1*3f1NR~vLt68RikGB0>!26Jz7EH7IggP@@ zkld{Bn__IKM9y)|@YenGx>*|oBI7gn;vU6Mh1-WYx9%U@dHcyBhoY^q>NR42QH&`t z{#7xC6u=OOS}B74p%_C!^5jW$9pP{K8{Al!H~864RE&r|J+9UXpOv z*CSDOTtgN1NrR*R45n_>%g)1Lln`C@>t|%~5u|ytZ1S|WIccmem4gyuay@k3hI>c=m#&t1v-%231a6+U6E*aPOvJ0iV z&HFR#h|P+a+6JYh7oRwedpTMdcc+Q+O%GR@$H_Z`j_YWn1Qr3dmS%a&eMlH#EQFB+ zl5cvPs6W;Y;HKOJ;m3D_?ZuOagr?P)B**Hald2ckO>GV+kj(E2%)X}YS*X!l?2Qh;el#M?FHo>{sSIECX=tAN_6v}flTj=e8A(CCc0;l z46$E`F5$o`P z%RX|86~33Qd?~rvl&Gy?wE|VTB&6uc+5+qHj!n1ARuJRFzab!pxJQ6F7rC2NmM4+u z=`!%nY?)%1CeRlWbVE`hFZj+0B<7Oe`x8cIMO8n`HorFhA;J`KKeeC-pKJh_G_Wi} z$)=CgKzbICy77Y^TZIcZ0}42#u&AT4t42+n|3uO=2Y;g=$l}pz9eGT>ZcxqwR3B7{ zHINj54TvJWWV-GAQo@RLn56K#aLj>=;5B`mbWd?yLy#Q=!@D#rS@5IYUzx}5Bm{kN2WbSRM2ONh&U@gVZM^s121VE4~7VNd_n4%6TO%su-X|OvIR8-56 zMr0U0Z3dJGURR@S{*7t3e3M;DLu)5U@n~(n79#(9Qu|sMI>-d*&Ea5ldSn*$kV>Rl37LHAr=RkK$ z=z!{r)WcL^e^genmHa&R%_;P+!ZEhx)ox_>e-e&Kk`%@y1WBU$x^0XCA+p*ISpJi6 zEFc8rZRtwE~uieezM5QQ{rAJe2KO?n&$wA0u#HWi$d z%2lc562tqYVW5c7zOImF&>IU^SPfM~8kECXEDr8vk;2;L595rmrIE#S++FYP?{LR2H^N;I?CJl~ z>-r}@;vbOVAFj;*SFh`>DoXJmT^=gBFQ^*7m%IJn(8Hg+u73lO{*$Lur!-=LK#NEj zrHotUOL-M6fCyg2qmqvZ#}h+le3(7FZQvTyc&od68H!f?2;dHQiEbA81RkecrPmjr zHIa*Jdv#+QI~ICobAEZ7W|MY)xGi&rN}$S^trZO9-r7!NfP`lZALcb3I~xSNwZ1_v zfQm(io5(8o3GvIQVGb-%1OJi8r_*ah^qItJK|u;c0Sn}CRBN=T{l{BOU9e@Eo>+DT z!(`M`bebrQBuExwx_zWZSOT2d81SDclw&$5wf;$ImB2bqu61t$nQ_7Wp}HyM39zVR zyS?LvY%QK2RYS6fjt*dsDkVw6argZhFxktf%@@T>@D~zy3*=P5P(vs@`uOnFRZa~K z)pid9!gV*T}iFU_(GPpGHd802B+dCQlX8~p79d&hD`}G|f2{)h5CkV8%!T5s` zO`YU9+DAa`8|F^TCoZstq7WPdj494kwy5_e;JLiBk*~y7i2)qv+Ri_(C&p3^sHNIn zU*C2wFElqSOsP#kmn!aTCm$z|((TSG{=P!!p9^09$4b%K*1AT<_WF8%0kF^?sjt7r zzfFJr6|?_?8GQUg@SpN;sf_fst!;JmwQc|M^j~3kh=c#|^j|NLSxj68*pDaw8r1*! z>6Vsy#`;uN)|OWK*0#p_Hvj8e;w?sY>5}P4?r-<7Kb!S0tEuZ+n%S9KXxi9n+x}jA z{1@W*j|Tm^-$|*oSNydB0>{5H=pUX-^?T`7O{%YD1eoY}U>wqr-1pvVC<9_!qufzW^g^v$^>vjIoc}bI&l9rB=j#=|hZ@`Z%W}{!_ z(f_z-{#uSd?VPhkeJS;=d63Pl#;+)=YawiXX{f7fYh}t}QQL~aE7ByYwT&%ut065g z7ILR&QM;+Bnm^qS<4Si$?#5N0N4_^*4+@)zh?07#008*P0eBLe?QH-6c=l%S@)v~i z0KSYUH)mlcF->jy4*wXm7Nd)bPZrmx^&i$?iYObgMd&fufx}gmPs`VM*3m}Mq5=&0=kC`BP-0RDsPuHa4ySjFZxhQwl%*wC};|88m2=!OPuMfzUgJzj5hCOBJKr=^-kTsDvfx1bfic2oirn4r1P zV$5}7q&Og4RA0`w3RF~V(5|HU@M$EYO57wgt!&GYc2?b)eo12aY8MU&*AajOQw_%}J=w8~ zMfB+hqroANLsAgUhH7)PR9&4pY%no1LNL$VBtNuQDs;>4b4a@$wXS5E+d5zo* zzmPR0)G})z%{UPEjq-_lg;YjWaQ{vrms;rnZT5*`)HSXi@tMj&1<>AHNld8`v-{53 zE(ljgf5UCTtWT_pmVvizpLnl3C|v!8ne(Xc zpf`pyN5?E7)~iGLwM113F(^NinvI@`y1Eb>d&^IM-#RtcB?Z1k9ZrEUiZ|iBL|&2# zk4j5)G^tp*OL}+(oLZq(apD#btNfocJUy#;RC-G7GGxYLN2 z32&#E#qv?wkQqVVXLgd9jZrY(@h&CmkRy;5UYgqo1fJ1mOUs5|=+IUDF+2v^yTs`} zqIKy^j_{rxSC7CMz43)uFe}(JFvVn?e)mg6p*UYRx*xTU*<-L_7HrOGc6eA2&W!ri zL40nbeQ@rWAo6}uWvqUbEy{Rf0pI%GCxg`{Mte+nLM1F&0{=>J4{s~TeCw#l&~4fXLIiqL?3G=DILKOr zAq71NQx!E~Z+W6>SZ^RivS~i&aQ_D0W1_{-Rbi-$@lI?aTu>TQLGxD^-8TWsb3jsJ z6$FPK^vYqi6azj?L_ZRnFe0^Jhl%oT-lT#b>L+Sl72k-Zb<)cO6T?ltc%@K1Mrr6a zP9_hk1|q3eG)NS@hQbN)ssO@@FV$YQTUo>+7k0Lk^O=qB=R;uUOmjX z3zxXY!o(uZ0(3XKqYm&!F>A4AOTM3tKRJ)08Rb-FeFGb!XHZ1{sdr5#daUOt){=H< zbCklE4>{r66WvMOQ%v?Fi2byr%wo{U>LVK-i$}zm!63^-xL9smEHw=v)lkt1NXP?? ztGrLJh38`5;hT0SY$1<6GiNOO@|n=*hqRfnx-Ae5->!&E_gn;|>f3uHc)+{@1kA$%+I)AivB zJU&N1>nSr&n+>+e48tWH6xuW;X7+jQuJz_bE@=lors2h7)ty*ou4BQvtyoh)utLEc zWUs7w&&lz?X+9DjmTD0kE!;`Gxx-fs1=sWK8I;tL{WQQmYy#`JD^Z-p-|C^r;#&Kf(4>nP< zi8EY)$TTbb_WJ@R{Z+-U-$fRsxQC}k*@-CuZ8i%`Z!#=kHs;3TK2M*`S*5#&rhtnT zE!;?bEuw-+P({q4ct!`P%?ula#fF_k$pIly47Q%VbdTTwv#Y0CaU}V^W6XZRz=73H zovGyjq>iD|AIK|fD;#)lJgxHeU`mZ0hV7JX1N0&eS}WJu2ynoDU zzuJwfFkmCVd1*t+hu+WzB>~#!y}2A&{~XV+_!^`3D_aH8f&Nz_R@=-uRb6GUCW%$| zLkOw+C!}TvO6Bs7v_lEfQd3!w4#BM&-gDJF1idrxq70=bqP|5%n+23<@Q2-ynt-cn zQwb(IVq8uiOX^X%tQN1h&3tki#khGiEEWqRMWZnY914&Uscx;u>=ShGoO*1u!va;! zQhvK}9DyIv_DDQJ2|D)PZR|_e=>vXp>ynf(7uOSv!Ebfol$3)#ffcZqr1?2H^6kp)rl*t zy6_1J-(pAy<~k}^gZtW5L@l2TCNwtZ^S;c|N$yoid>C{&*waFp6J1Gr8nQs@{A>yM z`An|av~!#e8u>lAbetAfC#@XHZc8jBL~f)V-YguxI?K;iZ??#3?!ctVt=5}#Qj69g z3G>cY&**{ezSJ0hlV$fTkP zXRa$B7{De5d5^_`rc0vUZipuuxdrQjlMBZd_#;TDG`H7E;k>y_FyvH?>4<~Dev6?W z#95ao*C1Tl{rv4B)8NT&?lJ68RI6^GrW5G051-h5OCp>NRI{VBofVvZejdThg+>HF z=V!|xu;L6LxAmch*Z_kb30ZY&N}Dn~*Z25}jUxS8c2pZJX{mCm%rRYOdEU`50k|k_8wcaA32)Im8{AvO>xRiE=%dP3gBguiY!$( zt0ubvm8Tt#bm&^M$x|o|GK=07s(|3@fHq5s(k5s$E47V{3UNzZ`2>$#%rn(^w#$tb zZ{kpTe`MepoPe=mz>+IHPg@CdOw3N}4o^@J zyu?rsMbRSCLo*bDNp6FH`&3JTG&+kMBd9vplUR5uc7I-9$|yR5J5)XnTQG4e$CB8P zM}4Le7u~Q{$_L+)CzXw16F&}lEP79;DL#6Qt27QiZ^NjuZZ583s;!~t=SzDYBPp&7 zbO3aBa&(N`o|M9CGVOrUyIL@zxfc5IW<2-(%g83Ax>P(n6ApaLgCcNBy1<5v=+hJ1=Txajn3u2Mq8FYM?Jed2==iun%L4 z9@vye!e^2&{36)b=dY{+gen4Go4D4by8!{dmAoJMN55xSlsx5^kMOZLSXCdBNE;85 zFBJvO^zhbC<3>>8QxJS9^WQ}=!~I&nwf~jVEDdu~_Hek|MRmKy>XK@TQRqk6ZGiFI zQQ&0^f9u8ekRF#n5hoG!`%@j~)8U!9&3nuFxlHP(o9*bQD!P=DH{gz(u?n)p6R zT59Cp+VDBi)I85ih(aBYxNNmHJ(J!WjC6iQudqEU zyI&gpiiMxf17-8R{-u*Y&o`2saM9?-;5~oHX074`LcJ5`TXb4j&fIfE8Z<(QPfRKq(rBk}?q$OVH{BzD|T zDA69V6$KTg+S&}w#p22PS@u}OU_0NiN)mmpi>H&lZ`v`BCr==Mn^`jt0tftS zq6X|jQ;k?1^~QCqlGHq9+zWqqtY0L5yN`uEXaP;XKK7}Xilhw>4K)u|5_g5pKiR-0 z?)WZ*nz;8!;wLck+UZxpl0L`x02FqDV(15_CyaEykIwEb(@g-|wzp4_n~q#GajHZo zD0=6r&Y}$zm$1CCB%yxKs@4|VPW~f?IC1c10p5?t*H;l__lP^H#uZN24Q zH;_CR4u?9kt}UlkF3Xd4bkx2={9AQ{>B(sLqc1hDNr) z7}EAoqFEwT0)`cD^CSG!*?Qt+AzRm8tSQdVwBu1cq4$^6jlGe6x=QV{+92WAwzYx7 z?_pj1TuhUjWs-XEuqT1ptLQT&Bg0mwVEpnN$v|;x!e1?EN<@*aOmvb+_l2X(?lPgw z4rs`Nan%pIDkLL1q8R|95{f9$8Y@SF$A6Cd^C-AaX2YfIV0xXhaAAOD8|4`za-U@L z5p*NJ-k>DN4xIamo11^}iqa&}ez6S>V?k3#TE@zxhL34fDQIiG5eS@JKm9hFY#-PZ zl;vFPOgMgfSd+1w7sMwpYCem*Nw(&I?sBD zUOm}%&bu)9!VJB^(@ED9Fktm(MFiqqP~}MtHw=?93_MTMn#~=d10d&Jk3*8Eom&Ev zGaDB?CAy3FdvhDST+MIj0W5wJJUTYa#yfk|Mr9+PRbvMjGv6+Z0|2}&Kz zcMouSIzZOxv}7y?hAZ8UEvtyy*YlvjOn9)Mw5$hpr<1EQ)EuKRL`d`u26+4&y9KOL z#wphV^v!sWHQ8qFXhXjQ5;={3SkpBOS=sLtgz`HZS=sIQvfn&@BnVs#6wTft`dRYxrUYM z6hc&t9W#T1iVWHqyIB)&jp z(sEe;JL{ET;5|>ZL4tv}VWu)@xH%IA$aTR93=a&4R$Wut!}y_9e9C0L!?+Rs6uSeV zYS(<9I{K~37A2-5U3~b_{Nz*B3W%j|5PcXm7uws7402Wcoo!(H>~JWD`i;(EK3*sq z<h-GMz zqnG1?;pT0ibCy}B`icmq%i|s}Nu-E;Bbk-LD&s(bAC38`?oPpc4Kif7;XL(BxuyH; zTdlE>hr(78#aG75FKX^i?by#T+-L!qWruHZ9JLH@KgtC8Kn0Jod!h4jM0a$(cYG0Y z<2~gb5L&)8YH*k>jjfGRo|P`0t*b26th7|SI5#;buBg_KH#Vy-sgVmOxv2Wsv(x!}dbwTD^mzS;AMY8R@7RCCBfrRx_@Sba`v-0c{TTB91EKjRK=L<={d-9$ zDlQ${r9!_r=|EW ztmu!W_{FgP1uy#RQWPp~FrV!4WJHi@3Mi&KpHVqck8SPxAI3q-&-t3*GhsY8m?&*Y zwL7NqI)Hx7@W9ew4t*uhVJ;&p>-1A&;wt>JT^Ex>O@bdbDgQs%Mf1VHXntd0O#hC7 z{jYYhG(QeJruY*U4Zp1+G%65}h+ELE4wv>QGq$9Jbe)~DO+i7BkCo@$6bEy`LDeU} zkQ&SsPeZ6btbtO|{^_fddlaq4`Y=cOWBf~n_CHX)5257WVZGl>`ZLe_eb;{ckCNs; zK|Q%UFuO~615W^A90UGt92_H0=uG%j?$OX(*e~pVH_2aTOCUzz;{W+<{hH;!J6klr zovn|;!oN4F{4ZzA>R~$ev7MpH=VaDfml{fqtzcDV?Kv3#z6-8CqCruUb7%iNPp)WE z4p!&hMa{3hdw6d^>3kMpqtEQD^;}ii^yxLfuTCgFSu@F1o#O&>U=<<{0nnm_NZoP~ zu637r00IRYJJ-_mk>QJ{(91fFJBLVk$ zg&`xc)czq1*#5~mK0y4|l`wcM|{5HE+P+hdr6+#24Zy_^7X zl5Nh!wGT|vFfpX-=^2@s=rIjV4z8}wCb|!{)`yDtiooW7mJ7C`C5;%;3D$;tT!ZUz zB%y=E)xr|*v>2B90X|A7LSqbz4zd8`OBs17&4fYGw}=-S00xK)vjYTg8n_R+Wku8N z!z<0}8H~&8$qg|c35r3&-wDJS4Ccf#qtiv_2lfUe3gSUU1fS*02FWeg5B-lY~U1x(m;K^o58TI z2=T7+fe&)`Z{W=OpHn>XH79gr&ozNTd(0jwA@kYX;L3Y_RP(gD`TO`06xll|!FJ>h zVM!-(1M=WM$$n-3LJS)P#A|MA5-kBN*4d+_Ou}bH7{xa5Bwh(9G&-={M-Wb>0IwM+ zlRR_P;v)p{D1sa3C-{SnKM)qI!pR)R9~@43wX|K{pPQr!qp#>T5}-ee8yJp{{c+i< zMoy}xOitFN`Q1`fIM)FmiH#NxFEO0W;d?#Xbw|%+)i{YKsfAh_X-%TIB+R1 z&h$h|tIY~om5JSoY&%=_)!o^uZCvt0!-p@f|DyDR6 zo4^#&FEwE-I-UID+!Cjmuz+~^ZL7Xlu$#u^h+bYXPSqs8$k7srdW2S$c>h&+wb5 zF}#)1!dYVBtgP#LEJpy3sFO0=Reip6QHyDw;4c|?(m@JrV7szXxSJ$Eec-VxoG-%D zOJ{^L6}-YRmxI2aqDo2W{lKKlJ^O{wjd7Im;IeC1*l51C?SJ&uLqAO!BIfP;uE+Ui z{k}ALcoPq_FKV1qR11cS4UmM(8hT_!L=(w|3$Jt~2Ec&j3ojiA$eqq8tqvN%-iHHM zf3m%aM8_lmmgL2T7>iHDNlhTS?`=tq;}mI6mI}#I%t?*ul<5X@S?E(O_hPQ}r&TjCGMIDN9uUcJFrHV7JK;wSh7^6;;tOF%Wk-gWrprl*+Qfv>R;49VRqmto5R(O%^QNU%LF0AsrrV6z=~ z(0`ji4ibTHvH$>$TZ^H{Ccx42n<>(8oQ@{!CxWu}K)kniY2|gw+Svr7_$_+rYiapN z%^a+{&F7Pc&*1ByKdE`BG)m{%PQFlk_a###vBgG<-7Ds%(hvaK!Kj?cW`9aLRE$|G zN3lkuZbm<=e93~eM?(0FNmV{u&_L>=8jDUDajzI)eBx6w2RHQ%v%yw2Wp?VjuU+66 z(|08kw9-CX2VQ%ASzul*X(ewEKHzWAAH5zE8;jEV_2I#Z2-#qk18qkHmzwm`idLfm zu!HpQmI;F`_#i@jV43tj#Q*?t(9XF@OMuULbF(OU25KQ80;vK@ivVeY;>&uM+XLQo zuFHz!349$qEgkpcTdA;lrwrGt3ym>o>B-s{J;&+o7(brPuTKm`Bd@Dwi_1>#z3Bvt z)NLdn4XV}>khkwmdUu>rwF(*5Lkh<0c?45%y-!W{rPGK_SSvEeKxWzmx~5kh0nT3N2wmb8dgag!8T2Ips^Xlmed zBGq>$*(P0rN2LP7uV0H}V#G>q;J$I7EsK{dOVYh<^#=C?_Pt+7maZnN*n*|x8=o^{ zJT&6ETI+$rQx}=fK;HlditgLuA_hi+`w*P=Lg((WunWN&Fp*5v9~;NnBsU;=evM2b zMFdlvlKbIYIKCC7jb6wi#LmHL4|`_Av`b!}Z#fx}GBnis*`kw04@gAXSL}Iu%QPmW_@wbsO++N(@$msQExV@EqbRq|B zx+6edumF74Z0-eQ31+nLf+;}C$Z<>oq9x2Fn7Tu0-6RxH&q8*5VpN#%(3;Pp$>rlK zF2t`6WB%m8=Gv^4yb!C@fDyu0WGi>{{x^^X9%ppb^W)1s%l&;$Kt5Tr!&)~US0Zmh zZ=C)w#^O7Nx5ZZ({fNZc-dlr#Vx%o~{k>yV#)ru0Z0ywJB#%=A=?>o55$kbYoGU{@ zYZ(Fa3kQk`Y&8>4=flblU`6DWKL-!XRI1cl4n=2+iwllj8bKNY&`ga=X@x^Z+1^fe z4#4{j-P)o9gDlZvh&7oX@NdR~pByf}hiU<%(70!GCl>3p4*k&Y?k(kLkN;3&@33s# ztLO)(do`)8{v-zyim1NYNV3{mF|8q-kfTb+!ulo?r|n@sYY^9zqHv!;GvPovA1H>+ z>;nCbf5NDqkz8hON%5L!CJ?w(aiX`^t}z=wCfx9|T}JY;3hDH`o@d(0N=AmLgC!3=|fm ztz)O2TG(O+M#c_+Uh0EtpxS90?*+AB$VR{RV5N2+&n>?3#>sp>ny}V{ z4g}M?@cwB!IS+Nwd_IC%lGk$Owm8GgcsLm#RyRUa8i7?k7XO3xn#?e0xJQ3F!W-XE zoV@-7_#lK!p-E#uA`e25j1}1}kjo7Sx*BXb0|DN>IKv8U1`hIcrIH z*+^Bk!o-GEwtG3N(tyq)f0;d?;H<`Tz^~m&7FpGAb5UYKX}z$c&d=|!$jqiX8PCj{ zvAL{?(@41Ca@c8m4p)fqBOXCw;GghYvv|YUP(InsxXIf7;U;p^)#_)r7iY8_Naf!^ z7o|i|R+sej=AZAgGtXJ0uC_#d(SVGt$R>`ZU@s`~qWvt{(SO$ct;Li6>3%g_;>Xfe zj%Aany(jdg!34ps-Bw=lEZ+d3-o!+`9%|p zfC7gTWLVUqjl(e#=e;NEoE&FIk=rZUjLo6w^@MOX5yk>Rv3&cG3v5b^S!q`tD9&-w zj^pvF?B)ArP&r1QQnWJmC*}gqoC+Mz6&( zx@GCxa!g!?3lAOTOa0VhdqL(8jTwX^k<>0YGxP0HPFdflw4OUVGnUk-mi;c49mXU! za4IL79+7WTyK@eR7}SSj&*7;^&S={@x5+U4h&s{wRHB&1SK6jzLXM}(X$y9)=I&VL zOK8)i9!qJ**mD`J>xPqeN=W07yaBt7d@zY^bK6l~OU9!V9@g{MWNl$3xz zuC%jm(&s45>joy*U@_xli{UZVI7C>xBf-bm-E2Knk}vJ-j34Nq3QJ|cI}o}knA~^m z_>W?C;3-h&``ggsF43DB+z%`*g}P*{b0&5f)ojsRxtu@|Gg>Z&WxX)R?C4A+aUf90 z&uP=tPSmRwnk=+dntZQveAs>Kf_t`A_&z>VoBV7YE+K0VT(eENT()vEvZN;O;cDeF za&Cs0Y-6A{O>SSFA@89xde}C=M2*!3(^7Tn{*-K*GJ$dRY}L3d)OtLo?8At>-oBTT zLct{ixBhUbxb?c-&FBZksp!B(sd{)TnxOP#CesBrwmSWoP8cdhK2V_9#=Jj&c9WKa z<>c}$90T3s(^^b~IT?rAopinJY+PQXrbNSR5I%10`y9bq-hyjb$p|(-1XG_Sy_0&h z1IH$>?~vQXwBfplO7Tp&CRLZyE1^OIMHvOro716|Sz}($&f;yuae8ZQ&FYfS&l{1n zC8Uk%WRUxUm;uLwgxNTWy>ls+^6>n<`NlMQM|*-}?cSCi9w!AENTl=&bN9WSm+4fg zbumM(rs$6*Iei1E-5@>m7CRhE{NlVDd*a62L4vAfx}DG#yb@5slvIIC?|S|1;n($I|%L~f-h za^_+t?sK(~+;KfWYGQMFky&+!QU#c~5mj+L&MQ=!PVA@6+A|HXTj+w2O=C`PuD!@> zPO$MAl0crwY*vfFGc*x`4o)tjownR;K6gd#x>yT?C}t||y>fSmC`KVlGW0JZeEj^S z7+MKZVpe#pM-(1t@cd0C@H;bo7ZalQz9&$Aza3Kx-hp=26N%CYLAcqNBcgJKz zjP)AnPI(yq#&n;${<>8R=gcu*K&XAkk|w#qlhiiixi3A>6!SqD=XNx5mAlqdUdcse z%ky^}G69xi4jC?@0PR+Vz3ioP>>M)I*S3mAI{R-2p&gS0CfW@=8L8V5f{RHTPq#7- zV{*j@E^5vR1o5R2w<~)?toUE;^}YjDscu_%qhVAQr1Pc*{1yb{-kxTv(Y0~4^dojm6bS+QuZxC;H6MKzVoP8C=rV=3EfR&?}nK`$yaQ>TN)J#qOblG?j(t8 zO|&{@D^VF9e-6WOa-33Scev)=CCIitKe2wqaD^y0$Taf8>Qoi4=w2zAcRtqiQP;N!&dOd(!O0LTd`r zZlO$J@|k#HcFi=J7nA z7)_F3FU(kdndE*Iuh*JjkbO>HJuCRvxMWGU`x&Ql)9;xSv^d-q>Q1AXC>hc z@xY_w$Y3ifMUbz?NBk0-t5Z`q>Wn3CB-UDcJRwVF1WJr+&)sL(NR(^UyB%z+thYux zJ?|&JN?c5*>rGp`)oIIgTzX9OcWp8cjYcaH<8=A4B5ZbQ?w4RS;1Vreg)VJI?K`0j z0`*mzZt0_sjxy(A-EM4g%jiWT*l}fzN_vUc&v|Eg4J++-?q#XZLL$d%Hl?e(yDkzh zxCl!M>8VSN6a7@W+fV0GO2m)V$aXJM2vp9i(eGY+HGRNM;~Y=e5vAqnR8H3>B84&r znj;%h3!Ln)(x$^myX0ef7#aOkZMAg-ZRRIw$!8fH6z7SRM5RZKE3E;SLK&szCNr8P zqo5oKU*y)pnvYKwTGb1B%B?BP_P$BYIgCf6rl2jXt7Gl5I7CRD7kwuDY$ydRrG9dC zf00^)bnlK$#Y)SnvyjY+e#3Y$&hD{!7&OY1bwduFHllJ~yK-4vDR~!>c02a7KQBL} zxrVAPXmx9Oy)4P<_>?0fx~XpOZK_|*?ch9VQ^8ShO0~;xScEhInV{nCgze5)A-~@) zZP3ZM0O>`Ki?wV;>u%)HFb@LhxNnz#mJxf!k)<$0>F6Fr@0#iKEGs%_f{C*V#WiF zl>W?RLxyPJ7&zj_qoyX-LR3@;ggI6*`KQELs{5@Wf4uEPUTAb*3sSo9n2U;)5(mBW z`D5{r-ByZ{zq*lg^5*zeI9Js5=IEJaQtJJg+AY!Sd5Y)5fp z^+n0;!Am<^L)-ldpMKMhU4Z4AItk|gA?=-_WbGgA+_v4l8mn#Fwr$(CZF9A4+gxqi zwrwl@?Xz=|y-)H_a&l4c7q6O)l5E?RlTQIMne^Tn*CI@k>5o)gK#T$8~aMIy&k} z984dhaWThjR}K=NA1AlYNk6@{Ka?>l`HahLIRW|g%jKJAqZL{H4*t6gH;AKX2+gQNc?Ndt9-s4$JlqJ zOs+bvX}UePyIdJoxB5ev)A&EtKcK$I{bYS_ND%RbQir4v`-X(Dis2Li?nkem(1-ZrK_LJg^?+QGd?TRsS;Pav1Dy5kCWW2a z4B*P4L&NE`d4~Ai{Ne)Z0K^B>oRP55+^w3Cco1MI`e#0J0Odt6BL70TRI>+f!J`Tj0M`wHBd0ndgV222JdFYZT=~uE! z00JhHjgRV}H@1qM58_gv4#SHG62R!WK#K6ICm$lL&`&QOBIZSm%*7xC0-^_c0AVZ* zACv%PAW*#38K9awqX7hqC^HJkT^ESUkO1Q7mdr_y|%>YO&-FW6(`LOFM-*Z1QN~I(cH`F4_H93 z0FFdQ7n;rL_Z=%fIBMxy>zw%e1Py;QM6F!ggS145txmNTq?j@AAETRG%pXp(ZEL8 z-OL{ON1xLX;7i9aQS3zTG1KAx9e6rj1o0U>xdu#7HU5jf#z_Mk%cXX>I~ z?lfL}wQ)u{#7d=(WwMB;Sob~{r-cs~K_9JqO|nvc4R4R!YKOBsO#nnhCK64YOC=3M0qR(OxNs#6SxBYVpu#gDdrr_;N{$+u_T}^E413 zOdSz46T=S;QsssiaVoy6j21zmSKBCg0nT4+vOaR~2$L z?17MbdwZ-FM#KT^<+u*KH@HEpqTr{Luv&NeU6IDC-d3!6_2%%bX~^9k=*{DfPh-Fw zAu08?cvd{@5&bw2xg~@^8TsfWo&j+DjlC%`=7|_**nn&hPOL%E@oQueCi>&6PaINm zxV?97dKH^%m)i*1Hq;0BL6uJD`<)h401P5HVCmCgGDQn@{boylW0*;ZVl}DAn5FjT z9eLBqw!fC3L$N`dC(D698V^etqOTRNcBg1)V#pC@uJ%9gC^t}Ivo~vPbBABR<&32ugCOC35 z9|ioHxlotGQxldQJk8^X6b!M>y}-J?(OczfoOZ_N^(XcY^x5E-Iij`$9>NC}3nQ>v zOR1Kj602zN;+PYbNe%i~%xk&7Qc%ja*fKyS(QQE?4mfefFpm0Zvlh< z^Pw8eui(rIlJ>fG$tP-F;v@=dDSyVp9Sf47C=E!L43P$R2iDV)D(%GRA5CKzTM7KXzfxI|=zA1LYY<7uYohOA}vYsuYm84I1~*p9fY9N^0wJrcf7bzOmVk>yA-I{wqaeu`SSJR%0@cP+9^*MJ*I#N_4GHQlMqCqXr7=hn!{Bb+BKXpcDht{ zaR&dPu2?-A(S*lCY&P5ukYi7;Nb)wHAw6ciJC&GF_sA(g4a_)$l`EJdDel>5nnc)c z>4TY%?3U%SX!8#~Kl%rsKLktHLLftz{X0Hy_`&Dx2+%J`FJ0amd068c!uYUSiFrXz(nq${J?_+wvL81dP$2+tL$v~}Kbaf*b*2Tauojm{zj?$#Z; zacQ>&faM98^^vs)qpvUEXPSeuFl{fW-?E-OJ9x8#k;^vlZB^!{d*-LpZ7mYS*F}eP zt}!dEe0GoHyaE~H$8cPV+s#GRSNrc!QMVB=^ZtOI7l9$%uYapISZ7!Dl`l3rGv>dR z&^(W>9DEl~y4Y3g*QfN2&zG;LOENqKv;tOoVJc4$Y06#PA6DR#lBl|I}l+oP zbMBQ;R%sBG32ZB(xr^S!t8au`zP9LDsi+^kv;Sy|g4+GHwRzBJwCoOW+9IO!Er$6p=55u@n?T9xhD!z7I*hZ$mS*PXVvxnd(#G3mbX2#CQBvy zShrXHL#gy*AOA5d`fukA;H%hk*=}uK*%FV*D|*=C#j?M^Xr4IM6kb{eU)9cZhEKcU z*~pT8LqXR#f(KRQs?nz{N0rKQg93(^hr82;TCq3R(&7*jt}9)UC12J{nBl9_$Db|K z?mKgRYu#t+BJLqqd4&yaEjh$fZWViNJ6jKXf(6$cm6`i41GNN`s+{7pBg|f7i|lF0 zjRUL&7MrRiVZJTo9yn$9cOPM10_5ynZL{m&Zlj0H)6#|vi<1Iuivo&Yn8I%MSOF!4Z?3qLs|`@r6&0I@u_!pY9w5d&>Rw zt#8X`8>YnPDaW?-Ym?Rd4hG%u8}FYK?~ADIb*s|Cx3q~?rcAa^4+lYy-8xwg%Jaxc{oqklP#@nb3ZzwI|E!R%6+!o|?KkfPX{vpYEh7i|M@MMx}W~wxf=7gi`7aj{2R>(W&HbxS_UF$8U8rMY2P)$Y1a zRe`YdXf!Wtw%K@ijh1d^$H0VQA>%NCh2dM8#ynD!#Ny={HS%`X#{0(W%{^88sj9MO z@^o8fht&nVY&SEEltgF8!%t*Cwb|WV^~`siFQ=V`&i#^p%o!k(k#hR7rZ~W`ATpM= zbK>EZp@R#Gz~N0A1M9`3_3{u#6t-KeJ5TxzQ~0;Z5Vb`oeID%Kdu$Uo6E^5~e({rV zLb<_~-z4xw;CX?z$S)$^rF4Yy9M!!8FPG)Z#Uf(&$v@*(uLVb0?=C@R>1Vx#NjjZ| z1$`t{8P4;vhc%hV;PBD#n>?+1C0hEgjWD2->tAh#x=rj_N9kQWd)rEk_77T*cr}r$ zXsXIM?e%l3B;t!~E(byW9Ia>h@*y}GNsZm_5v?N_1Ijv>UexUOd%wF2Q*H&IhX+Kf zWpukEwa^cFpXPd7Bs;Osj%7XcgCnO!E~l8<3=Gk~{2OT1Wd%tV!{>uD$#NTmr@XA)7sKQ`+&+LX`W~6k{Mb8xLEi8+~JF zRFAT{PvKpaUQ#*bT{zmxv}miCMw3_*SIafLwHhd$86$V z{#Rb^CZ>{fUB-Kx-U>|JjS`LeF#S=2_Y9FFT#w&^23^iA81n2uZu!a?>3Qn+f?sqW5p=H0pKjfYDUuL80HP{+Gw8PAvFV(7;b#R+WJ zYmMC8ScuBmdfkaCpO`Wijgs~@lO}#5n?slRo8VW)Nw=sfVyvr-A@XP%PK0*5YY~|T zlN9+D9On0-rmI!m=MiS>%Y5Sy4w*Jx^Ky6V$0rQlBWx3eo5sCw)Vkuq<%^3=QSfA3 zmxh%Euflhg^f%Zk$n5E6NHL68F*hNiI@@q|?eRZqE<2WU%KN8-=2wfD>6T~5_e3%| z>&~9~i>C#cMk(IUE56$jr)&3fvYQyJFXI%OZxn5pKw8_9k}I`Q(y8A<~b<$67*X`X>S=B zp0w5(ejE zj|EXJZ&OWaXa9|tAER+SWpEx({F%A(bB@ojnqp!1Im#)d%x=Am@NPN3GWR+I%Dj{`6j7-yc;IOgCyD_DAK6xrZ`I_>w8z z1kLvA=9>K=Z>U>nrD@FPv&-TKD^1NDbt#X|FBY-IlF)o9ppQ+t0IpR^92>hn`#WXY$Q? zNhn`ql#Q_$o{qMQ`_YCEV`JlQm4$&%snwyp>^<$S0^u*|?Zd{ymMRvH49mgL!_oBR z(b3fB>n+aJnwP$F(Vuc{w{jU8?3=sspG!<+bIo8Fb>rDw&R;pUSJU6mp}z6d_v2kN zuyFYNxz$+-5v;>XHvY?*NVF)-=&_ur)A|1Bn;MhFP&i3R{5Ncw+lQTw;L#lLQl z{J-J(|3NF}AB$S%cQe=y20B(mM?YVDavz_6Z&7PU$2^0ey?LahKipOIpf)&EW2<6i z>yKKE=J4U7>yP){H^{MYjS;cz9nrH#i^K7=go37mgnBse&u}yV0LoqLH{g%L?+b=3 zkp~I+#+Hy=v?k4%>bbXSg5-f{^T&X4WIOy{BDdxGU#={utcyCnW(Tz(e{0-*d^=U&cK9hRj2BV zwA;LskCE7+N%OOQph3p%TpIH%M{;0d9><;n3a+-lkkDUQbDqw&+NV%gCxB@`#oNLS zKgc=M)#+zs&6h9&kATqgj`~r%Rm~`zTl)xjy^cm56e2TNn_;1-DfJxdp7euFsm^g> z=~Xxxi2xyivWT{33dW9_SM|*d1Z1;5d(k0IdiTvPdV9pV-~2j}C*&W%iT(h`$Lk~DY-iH*Jb_exHL0bCSt}#+&QRX5GuUiY5<34X@8M1r zvU%`itSx5HDqd`2+{BL1H+j1%&Ks+FbswK>B;S>B9+h_)@kAO%VfD01W>rF}1BW7Y zKlv5B1@z>AsUwvW$ecXn6Gjrw`*uOwRjewt_M9|&Qe#=a@S-tfRWz#}t^xbE!aO8f zmojA1#@oPG+%d#LqLHK{-D*VLMl5opRqAV};fb%JUhZ~x{PtF@@l$ie5*IT-i~xk% zoQNfWW#ve;>LU`Em!fI6uw^Oe2A+AaSB1ElmtJIQaosN!)gzJ97Sk63tynUziQuEi zPx&N$hZEL%YuQXOL2!FDy7o?R-NDQR#d!n@leB(Yn`X18zwu6xCz9RUBWPwN3y%VK z(4cVUF5W7#fK}F>>vMoHE-dLuBZ~#s*{ZR$tJCC#=5NmRWsqapxODPO?3P3)&uVC* zy}i*H2aXOghZ3KLtf9MR{GLF*P{Y}j0JqFX66ol}`5HvbrplG7S80wNe${(`di98l zJdklC;DetM&W3K%ORDf!wsd*FXOc=`tXt*mHbo~sZXEJdP*P^w`57-AQcbHAu8;S| zhK+hTI;Xodt{=AqCMlp*8!-nOvOl|0Bvdzc;qa=sWRt#oh}XsAo*;AZHmc3xfak(U zu$udRK>du>N9*hhH_PT~SikK6)!U(F1Ny;Ys2{|hk2ZzVf__O?E@ZA{`bhJBsW0-)nvfJYVU{*TzLwV-6#W`3fWljm<=&^ zt*yf*mFh>>oILEQSI9gDuE8dGkG=4@_)}aovbWdw`2_bhl188adbdD+iMhU71XPdT zI{CUJRrZ+mtA)UMrK3YvfOF{4WPQ?mloI(?2J3u|zN5K$q>uL`i2QsM zox7>6Pp+(4*!tR*%V;_Q&!q70-hWnO`&ySxyK8*eT=zt4l!AM4uf11TU~D-CP||!& zBUoJz6rM-1N*3|OgH1sU3x1fqEYO_iuhOswt1EVdu3}p`M*Cp){3Q}z6_2mAH!_Al z>f$^F2P|;$lDvbFMe+1re?I}zl2Q%bRPPnk$Kw57NbR4tz*<_*Bk9kSqaR9%Pf>Rs9rnv z2;F8O$GB8&N8wif0)0~f`e0Wt`B>LD5!IY1N_c9m1q6i(el5kgr5)wCE#_O-&AGwn zhTPUz@}LR7#6i)cgexVRhC$Qc{-vXp603)Z02S5EDJVy%qgQBZ=eKD}N_;LD9x*&A;OCLz5; z30kI>ze?HI9I?&B)~gk8e>ER(73>_-!W*D|V_+>x(6HVs=;SOC3=Ox{D;M zGlc&Io3_Dq@AQn9GhxN-Rarb%p(iRe53;90dmeZX&5Ch4>Y`EG)UgNMgyx(8VoE9O znpr9B-R@0g8rkYZf`=hQe5si z&SVYODAwVmN)d-HKorKAH{pB1=>JEc}Rw~4R(lvdX4ZX~!=)cD~yRZJn) z!&1m-O-z%wxfYAmN;6i$TdrbKL&zQER=aN6LV<=2rdd5!R!U4tCzpS=^!+>M5VdQz z!oe<-Aoh4m4t0q~)nYE}rcI2Ue5a|w&2e!v!8%x)j2)TtDASXNaV#BimLF9QN^PIE z8aPuUkf&x+{s3Cf(2`{wlnYwKJVQxJ`e-H^^Hq8TxgH9${qe1y3^vs(SmQiM*hWcH#wfB>`AM$ZsWw z)k5u7w<>rpA`6wA zps$ID;ByVsUfUZoiR#v`h|jVR94!Nc=17XbHCZ#mE^Vf**{K(-V~}yhR~!t-))1Gj ze|n)e&Wx&y%7fWNPReP%qi0$s&J90T<|Z{hp<$}eQw519x->o)UUjoMP#`??Au>wV zM%IIZDqM(J4(BRwfdye8IK}7pT<7%1w=Yusi55(hL$B(8v$Io-bs!%gUE^sveKZXs z2PG#ul$UV|EmIBVDTX(TrepO*O3|SNmAb!W+JO~RYZ6o|I$S&@-5yP)zwygoQ+h59 z4%~wJN{N%00FE&Ri(pa&##)E^hgcYv6A)KFn&QJyxn}r^9@DuuT3w$T|7-A4@*Vq+t~MpJ zqf*KP*kl9CL=#7Dth80qV5{L+{5hJlWI9i|?$4RpV5Ezd_+d;X7^+3)d5GH0;+bNa z-%O6}z--6{cF*=xg=SIki}(^uzZt%5p_Uq8$TC6Dy*8nOJdB>LMLq-tzM*)e#v3z?;s#ZiLf%>sDl1(V+PtcS(BMP+C(7s4$trfZ8P>F3znMogNF`)0@ zs67nDT3ao$-J4*oA^Cu|$FFdU8MILIZ`*@EBG@u>0XSiZvlVS&qg)axbCo5PQjb;Z zPl(1&7eolO&tX%@@*R~W?D>LEs{-6`iF9K%{NNdJ{^meZYFgp*UiT86g_TKyNGFVv zn^m3mt9B1-zv%Vr6+Fs`LP%L8bRGZB9jpj|ujgx!q&(D_5$Uk`KSUvlUX#podNOE4 zjso-BCN#Gm_cDYY59*h3@fuqM8esDM5fiApaw3@nVA-$a&n-@ujk7{fr^ao6m;77h z3f^zCG)_z4BD2^V{m!mxQpkejfeQ9uJLYEo!3VxDCfI{oNiT@6(FsZ;ufX0*Q9nmx zvEojz;!2P6cA|tu{Opvpwc~MT^0U7?{Tgki$)lttdY8m%b4X z@g&2lwQ-~<-y=ZSm~#TeakBZI(`{oSv$fb zPHaRQ9a3`Cq>85raUD-@1iBOD+&iwn7sxCiT*s-l-5t4Df~)AjDx$RG)EAmPu1x{? zFKHUN)pgN$Ddw=uUv9N_bn9h9oxVRA&iuLV(p0~lAY9Bp`C&Y{L{QBV_=7Y|u3fh; zMKgM!LXaP-@(ON6HD$%cu&?)JU#0^;Iy?ZG?@J|>qpuz!8WX35(99@l2p*a3Gge8$ zg}ODC3qr)b7e!~nqw2fT`JPF;X7R_1yGQ*;t8nl^5>`S&KlhEF;g|2ikDTD?ZWh7U_V~y*;!FYIB15e>*$OS#=YWHfRs@Ep>Ojb0CiAf3{d4&Uh`CZ0DzS@T=P`- zZ)r^ibbspyWSx95Bt1Af6F?%v!r(YPEZTF#0YxgUh#S@#Q?Pedp3mkcWWLEHbE@1( z-Td3s$}IMwRVfeYVf)8JJRinl6x3lS{3<=r!El8@^i-}BsqfUK(=r8X@7?8zr>%>@ zK$Zmy&%0gwGqah{^0x=-h2Q=iP88<3GT$qRjCIZk$F(J8229HX@%Kde+v=RpE3T=A z1-_9Bp>A}0!J{sGaHQ1ykoxKj8L}z#Mw6yLb(AEkSuuYmFl~hNhte<|1E*m$;*fhD zr_-dBCwf;#)we3s@f-YNMbkF{bDCA^X+WWS$+PgUQEJKrRQpeXrr%y={i3eKxDv5HO@ERUji$iMLXPnZO(?_mvq3^eN;OP1_dSC zyKv|2-+XCB)?Qc~5n!t8Y3Rfg)VN)_2tc^ zgF<~&=|i=5D>y zffvnEb2SQ7q~=1j`7|^VQnJ4ROKC>g#KRIE^@2?&%)dIuT7d5+k1uj96pb%kmd32J zwt)iwOdmE1@-n-Mk98$LFODU4I>>;-@t5|LA!;cCQRb>=Wx}Ge;oF<~AYsoAo*Wn5O+eZcdmQT z<5U>w%wjgV+~eTbMvkWwsn(l?tdKLpg(me&Xq~8CRra_VKXucV_zvSB#2qz|n;|w= z14Gf{l4KALvUBKlWcR2phoXMH6g-`8ji%>`=h%&aP@|F*&%`|1rsv+mjF0s)v)LP1 zr%D%>$WKHeB7MOXmzyN#_LL6B8h2e@vIuecsLn@k#CQ~@@pFA)HBPu)Nl z(liQe6)<;SO5wB~a=W*En^}>O>&r06-PLM5Z~hUbNq)P*qvi#anux}+3w$v)+7LOG z76$Y#N*F|<-6>1Vdm)K(mLKcp6Lo#3UrMzwgb zcpR>5I0UD%D96{5?=Rhkj?If_lIflx>xH>2mE|_~(A#*<4F5=XrTUH@nwgfMFyy2T zh+)>!ZPf6^X*FN-Y`;;fSk-}OmcNN8_q(*)0}fpxWLFbmW0&GpS)!qCdP>%tB|5Sv z0BxioscdMGx*05n+)|`Ia+@Xxvb>R+cplw?UL>?jZX`@M5L5{`I{wMi)-P_F(i{pJ zHNn?95IzP&ScAgiud@CE(cTa|BE#y9FY~!anr&&8Iq|w8=@fk}wr17JBl7^{fNH)U z?N{PND%|l_j1j7ONnTF8UnLK7BL?2sFcken**q>s;OI@)K)lklhaW#}MDh-HYH3`m z251uNY*|9b9i>#_stCu}3(Z1*oltCDtl+%$Z4P<^dWS=L@|3u2bqT>dZLgK(Wfc5y z=3>*+_jEg$?MBx)h?DG^`JG*k-hI>fd}Ly{$#qjxTJ%{hG_~IF7H#fU;!SYLuk%}@ ze3;Tkp|;2HB(H4+xmJEF?`CiGS{oTI7VdLkSH)yA1{DzTB-+Kq&x9!7k?r-#aw&*D zp=BH)xoHR12Ut?je(7>LK?BSyPB4|j_Yp45A$e1}%((dc7FomPUP_>;k@7s-oTPoM z_b}r;lf8$0G+qNNa>ZM>raiW_wWc8Tp+clCgY}DL8RJ-gQAq*bwA? z_7|$B%gGe>hj`R_7WPp=_gMmU-x1Oa6ogZTbfUiocnXj0ldPt9cZqM1qm@H)+=%_8 zA|e$1XmZG0jJ*{{$Qk!?=V?@J9e#nhr(UXKOq2533=$|;4pjI6W`HVm`w8-*+DfD? z+;a)1;4)Wzh$fknJ7pd8l5Fw1S}YW8uf)E*Ub7@&#(PaOQJ~evDD6F=9Pfp*g2f-1 zmvc#K84dtKlfY$^PV-rhziLgAQJU;%3!BB9BG`pxN_H;I5k@m4&7EYFmT7w>riY)y<{z~>M>aLiM%2xuXrNzhJ^o|TO*pCfg%(e73n`7`8x;&V{hT%lHh{TLXO{TMPU(AX0$e;;Sn0G*oR(NHyk~_c3F+H^6U@dLEDH zAKJ}&5jZF?=w&CJ;%}2l>iGFV#->}5I%504Q~F#_5432ZR#ooXS{3mPCe|07uEV+UIM`s*pLPuW|{PZPM(cm)j$#w()m+^m;Lq=$T)SNVhNCp$Owoj0c4S3 zlo4`gI#n8ji({5S-0#-`}UZS-_-eZP$N0ysrOx zUXMfM;K5%(pmJxj2Krj;$RdA~F;1&}R#I18R#H`2S6y6ERbgRSRi&j|IEGs>@JG~Q zud?+G+X1czQ2P%qxZ=Vf=Z4`5^Z}&?0V{1IDWuBA>$x$3gXIK6h>{`aVzI#uvl&SF zg#qLN>;ogIhV4Ql+F=p+?a3>IN#H4j{m6z8;A2qm^#o8N!azXQ?N#yxg6#0A1=(10Laocah$5d}5XCA%JZWJtu=@?; zf}Z4;)2!gMJjZr8B`c6&1gEGnkBz8uQHEYeEr$hGjVm{&tIpbdA3i|@H ze`G@<#Dz$dTsIJX`DRXGzfR)%6@meapyK)B!qfosR5-|uk^2>H7~oJx5d?$}V(P9y zQFkQrYgy>Ua~+ij(DWORqCWh0k_3Dt#FX>o`+yPgi3rTnF~jurq9h=G>JmE!sX)jeq6Q2|3IUORdf|RLVRD-( ziJ%ZcO24+GgY*sbej)X8vbqm9E9)02Z3NyiR@o6fCci=G02lc~5#&qLG6BzW;>*{S zW}GQ3c!j|OrIarzwc?AZQGJ?1+Y|eR2L&ZwKP;UfU)koG2!hm0cSLjeWZXUn{ak+T|tiI4L`TM;bP>%(O)rtJlj>5l9aBr*Lsjw0lSE-zCf((ZAJ z3}K$O4v-5H^eAv#H+^kjI81l^+9X!wicxPDpz#=qLg^c_#9;Txf_S1_PdH@tosa$TDf!$~jVEIA(Z) z+#dK>^SpEjV6FmQ^U3Pr@$n~H4H3ENFw`m?F+iC%LlF{oH|i2?7TRWlCk5iYWLdsFMHIB9?L%P!?hX!lxUd8`Z9Dw1c zEWrjsnr4N@2VWU&sDP!6*HFNPmOby1{cdYtD17fdXXNTO<=McwvUhFb>>=+eCDM|l zv62NRFG25)4ru_BLr^4#TC$RKXpVZwRxcoU@=D`yAUYD7hfq@BVQ@!M1SsKhfPn-TJ#dMKtw6Z7lN)mN&P($I zM@-bzAdGmPnq){52JL%NidmGo&DMo`SddVA9(+;L>~f_8N$x$JHUuSmsBQqraj-X_s*v35@Kr0t zCdigx->H0b52pt_qXHick)X7XG1?=F0iT-n99&ch0D`CI{g{RnzyUY!i0~-e-f$>b z1j`(Gb!PuD4HY;Aw2ALFxx;RF9lJ@;my5!Zl_?LDLM)i`Dd^p)gpxR7Ow0f^Fq$c# z6!ui@IOQRy#!Oz3FE?E_bTl#l`0V-S=>qu3sTkM^O2ZIF5=26Of#2CCq?o}PLx)Hu z6({j4(-O4-L77;;4bK7D`GfB070lV@)FUmoe%90^MPQWU2L zO2VDY8sIMCi6Bq>4OK$tnsf)9VTScb{Pd?O0r?{uy6?pyVnaatX-a6XQs;`L`r-H0 zqNg?PuhqE+@Im?Gzg}*ajS)xkBcUq-oaejEtC-t2tnnlRnMd`x_3@5jraPLmI=d}X znv-zap3A&lr$@^{tW7xRfx&Iig4U4Pm9I4;3x*KWce~K*T<(rJI^Snm!2Q)Ss$1aN z9s_^55|HP@%Fd(yHlL`!5LX}8exrKQ%!;AKY51v&(XF|EI#Vw+tKu0kY=2hvriKf? z{ocB$s(EU==^1^cj1;B4SvD-Hr}`uLfLmFuRkbgN>2rKLav$vfwU@^)Ow-0*jNiJs z!A&^#p1~u5D@X*tpwoN`!F2a9()9X?OkYS>)!I97B{y_Syseetbh`LN=*yGYQtvv{ zleY}weXP=b#%XwdMTrCIYn_F4<>c!@@GVS>EHgFqw;4nJoyXG77tzGw$aVQr4p;Qq z5RxiInswT(oZiBY+XU?8{l0YjF;$+#tgKG;&(KF$@aWf+w?LOip^fL_eyD1hbLEO+ z$lFk*N@i5)H8uMxv;CZ2860hO4e2v7EnVHQF4a<404j15IIDCeB;;`U<>Mz>uSgm! zT|2~I2La9fAn!7&ph}mTDq3V*#>ewxCtBy&B0B1d5!3Nfzm?A!JAMV}{l#Ts(~v=T zS#lI5lqI&PE;{$UOHxPFiCEsV2d=!^sXsv9{I=idOdB64x>ZLtdoPl_hp7Wxjkxlv zIy5|~@9wc^(f%g-$4<+(`&w`PyB%3J&v}g#j>}+EvKxzb34x!fyN+`E*^hRYwVqU0 z32DSQXwGLU>Do9ZC-JFm#5Dw*lx4#^^6s?H6FzDVP7x*Y0jg&ah@;TBm8FUmZ%p_b zTMb_+;fkGt_ZNLFF@L?~4~F1#5W4rK;(;Zh3A*RmH)xq(76!uymzyFRx*QDIG;GT) zx=ef&oVF^e*J@5w1IL0Z&geUgYtXD%K3IIWnComl@4y8go??Z|uUdttnRs~8x!Z^3 zmLI4ln~*)dl6-ZRGs9KREL(iIruTU&+96M$~m~Tl-3M-|5`- zjR1ptAN;66FD>B>XMYV^dk+>)ZDup6Oy;|}sQ^00~3GQJ5wxc zlP!j8XhIS%*Yy{xV`x@ zAG$BOwwTi1(4HUR&oR-k52|4i>Nm{u#1-}{-u!6^XaTK{tItTSIdo&|zD@LbmKmB3 z$6Uv?Us1z?4T%NX5Y1CGE|2-H5%q3gM$7M<)88G3<74;UPbZcyL_SvPPcEbOS9#{V zlWer9Wj+9K_L~i_kri{xF`o6dXVF*=RD~mCdbO8?Pg28aOZ%#-mz*Yc=^4eO^UyR; z1PzpMM=VlxXE~T^T~UU0d=K82Rv*<&X0hKd%+-;%la4W2@asc7aYkBYq3EcxWe_kw1d_t(o z2b8Q^RjhfFNnjM@ruE}^Qb+eXCHw)DQIe0un1+wc2n9#)2OQBO_q zwyvsgJt0~#;-6aP-eGdAf`Fmbb60C+s z-A%{S$6_aR$MU|O)Z~ullaR&T{-Hcr1uN1ON~L4j&hwPv>~ib@TD5j)+o0&(|5(d7 zzjbHio>y}Gl<_HTvDm_yMHyj}l=WBp&5hrAt%x*@bh8@5r*liyRcA?A`nvUkeZJzH zIcuSkvfB4xNcOU5ma^DqcMW{-izG=@!13?bo?UkIeA7XmYcdxFYeYwTf!WE_P%Y15 zony-J;pc4X6P12HrsMGOsJ?FH&azMSu;woJoy_51q~lRnRsGX$T(#3>WLaGVW`J*E zP|y6=1$NQw1D%~32{7{uwk!>eG{;9z+J!BikOR2>FBqc)=IbLg6Fd3T8V$Usj|OS_X^TEMk;yH+oE~*Y z&$8o6M4K(+!{4Lok^`-m$KkCoPyTDw6^JTg_Pe`{GuW6k85o3&CiO7KNmPa}D$`b(key_`0gUJLeQ^;Q}NZR?tnrhAIxiy%d3N?KBrL%tjM|E-QvtIsz z!~OYcvmYEyDm9Z=#=GqVhxegX=fAM_7GQBS+5Ye#!7T&{L4yW&cY*|WcXxMp2<{Nv z-66QU1$TFMclZz4eRtp8ySsPq^L@=zLw8TrsZ-X|b$&;dNUz3IU~$nU!f;V6Uagdt zp0vNFAKq|O-Y2MJe%-G;Sl5}X^R4Ypqb{*_`66oHa8?<3AD@+P|A@ZibZK_S<@!~_ z`zdgs89+v)8J9Yo z0TBFerBpPOKzYgfcM`s`wtQ|8cHTg#($Qo z(*8Lx^_x^L=zsEg9B(KUGykbilLtA8Pw56(I{p(nat_oNB8sS$WT-C`%nZT&UY}%Q6!S`q0Dw+DiD&v#A&I6lj zvc{Fw?iJULAQ7cq5ql`mT-Q4C&rbd@8Ikwpc8pFS z&tLhz?MSLGmfM3&U?ACErfiM9B zIA>EPgLB^FGdb%aGgoSwM6Mx*>E5^8h$BU&le0)DmHfbcA8G;heJa5;*iattBq^>l zRHHqHYEeoN9IU=8*>M4%fV>}9`-8DLC>ZOf3(FA!j~`K^-*K2(UT?lm9F!MRV+vMa zZtvo!EzhvA?Kknd&zi$Q_=+~?g$sTve@Wh!WB)nFwc5|Y45ed-c+$_L&Y+=o5Gfyi z5S$xt2_ciRvCm%Crjnv+zwnV5Q>m0|XsO`RXcx*=0{pb4%GsIm_zJ`VuUdw;Be_Z< zgtzJwy3Qd<0t$3DYMg*|oGgx|!sl#r!pk62JvE5Dtb2bO3*DoOy?J0jQ$`IUtsC_R zi*G|~b4LR??DwP3Di6#qf@HX*PML%ac7B6oKq<~JpIIAk z{VB4swx`Oxfgj+FUpW427?W5@G$64@98-L>4O=hl57P}_z2jM zpQqabtuehw=Sj449IS#Cg`yVmd#*T7gP$ZL2~aj6X#GoJrlw-J5X&Ce2-e;GR1=5Y zB{d}d{7{&VT2qcT9#Yj64FgmPhjHjDssfARt<*Zho{LB^C&+>Ytne)qaB&w!5zib3 z=-mW)bv{TQF4gUPV~Oe>;OaSN&yA2J?fOCfv2xt9T_+8$q!)x4f^BIog(wa)0jqwD z9ZhN`u-u|p#b2e2$WU(5aI6BXaAuY8;^sue>GQBlkkB`gV;E`6nI0?+y}Lw7uc#Q4 z8**au&X{1@9l8%%)rZ4P&xq1v+=)ug6Uuz3aOqcz+(6E`D8IMI!UcVvo3C#C0>6eYG3^Ck_735j0=CH zsTa13<^Q2Xo#`vb_qrpk^Ft`hdgh}_RqQ>cND6t==3{1Aql1&AJKvt=kvXNY;i#QO z?@dTgifo#Ev>R{x&x%8RjG_4m^KaV+QmQw$yU646r%rlhtFE=gGIwEB4WgH+CRsnPS&sJf?=EqGi|- zf6~Z=uKvv;$k=A#ds-Xd`*z>JlO3OW^~nI#2b4A6zKAMTM0<%6H5N%ncf4$mR8xy( zn~i{1(GPv!y;YZM<*&fOTxM&0S=7&UT_PN4_43{|f|X~OJDP4rcuH9s4&Cd;YKHCG za;Qr3mxbVjc*CcRn6ToQj0__F!us6oiXJ69r-ENsTX&LmR7^8`B}%A#d(_P%A`S3Z z0#DAiGTXKa*RSv_GX7dRNNo)Rhp)W4Qt4(qi|t!`$toPKWag+ajUp2c$Ei6RRFs#M zXI>-oZBy~{@`MlOitmi{JM3aZ`{Hl;#C@^Nc%{*8WL8^YU*l@w`{hR|sE2nAIqky) z=sJPw&sF1B>*Mo}*;V_LeH*wc3yaudO@`<^M8ykyh2 zd-^tJ+>%+juHj1bqsHJoFD9v{NU;lUv;b&b0evfWX~UT(C(=^C@*`_kU5yT`FV^_r zSm@L9l5O#$^npWM#Yt95o?2rWn+$zU95KqZ7Rb4@{MGQs9@ssh34VC|YBABlq<|4v zB+Tg0P$x;qzG3pOCl={x{)0sl_>@|OYDg%VbIBE=;ykMDJOCmsuzG;d?fO|qk_X8% zC=GAYk4f*(NuODUE3B|qs9C;ELIlFXL08dU!opVF()3|-?+4y)Qf9uc%Xb7K;&Qp& zovnBM_2jgamEr$@ss0n94t)1}i@ik$4A?}^Oi9oB z8&iG%uisUHn5xo;*&9Tk?+Y+gP^f;+Ae1UROz*6D6@oyk$e}g}6CSz#+6e*zk+;-a%@#ua>M8BJM z$-et^(r~+Ew1KhJOQ$W>W%2G#2>!{~tBva_y`vhO_&qf+;Jp}hX9uBL9;+-k?7(T- z$6kT=mVRChG$4<~tvx+7fhZk>QSmf)zl136WvEP``@YeLe(n_j=ZOrl_WDlGe+YCs z2eHmq`K3ch0I5R=&)?t+k^w9djdvhEDAkBvN^*u5Qb#@wzZFd;5)nR>+_%ATr(HS$ z84zPg62gKn(HLbF-;xBKD&%-*u!Y|zIx9X{5vusPBm4i1crX3!iA0(tfCyY9;|G)* z{#`mh5h5zy`>E=-Eb(bxBI~Wo;Y7Iy^mf{cTp)%@@AZNwJI*^S^F=!gpKl?h%_XogR!2Cos2g)ycQaU9G@mw2Ahu}NZ#^CN? zr~Yz)a^hD-G5+=fs6eT;Kz^tgJs&M5Vj?`E1}i8uYtpav2aSXTUIcz*PUs*3(7bX` z{1dkQO%bKRWVxFbdTFGA^INJH1WxcN5L8fS64hhj{J1^*{?h33vRiK7!C(@KX7w66 zpq0M7miF(B}SM@C5U-iA?@Uy)M&lszQEvVfxZ^Q$w^N`gRxGjLJwZ*BoPsyW&doPU$=3rJ5m*e_d%aL=%>y!cX3 z?Bdp9?4g6Gd45-WZag;* znIl`g927^yarhWeT~6IC8Jl!q@RSDESgNhoGmsE}lt&6t|AmA`|8KKD(r70H_UXNz z>bp$qQUI>@qv4S-SW0KO%Fh;1kf@zLh&&>&9)cCn*oZVXvfehqlYn+x+V+5r z92p^1a$u+y3_LB))<-{ZAKQ55!6+@<3W#sYu%C&ZUe_y{RgF%m#{)7Ui{6n*MB;AZ zFs@%d7Eqj-lkmy;!Q}FNkCw%{A*}d*SUKa!a`YBpV2U?_z*co(MX7`YtmPkujihn70-+o8ZLW1iW(v9yoTZUvQ>lZt&hie(>?}_eF852L_ZYiC}yL zF;98gk{I6{t+hVIJ%8097eXf$c}+Lpep-~X1BNti-nsKjjpXhma8<4_`eqr2wm0)# z<+~k-0fAnXaEHYPk6?JuH9x^U)tBjDhoJBrQl6H_-g@kiA7>w1Q=pZiOkY z47_+u{?%tCK5cx%2axAR@suqrC%|STC86MZgj7*aYBaqWob(2_Xmm7D4|xr2Y2lAw zRWdAwQHX_?p>qTh1$<=04CZxNwhp*CK7mr)crt#>nR@iblXMwm87r3ai6jCqu`J@o zKSh>0U|lW%+nnk}@FC)*;5mkTinsZG%eQ6#%^EYz9=JfacKECBq^$8D=sq9SN zW0Sm{<%*FUDm5oYMhkjap&JE6KV5YB-CAVLk~?$Uk4(McaCtlKi*Jr~sAGzu-l;EK z6CT_3A+#445Z|H|{0exyHUOkh&g^JWU-mVHjg*lblEi}vh`*8l+LlR{yw!G0cwA^z z+-FM)o>Ss$ee@CxLV>;S0CYL zyIMtMln}{UWvuTPy55*E<`na-^{r*)LYB)8i0C5Xs5uwL5z*{@=>E;0DqU6tWjB>GB6t{m+| zCg^kov9|Tv-lIKEl@<(B1o;YN{c2vQRM}$Q>$u|?a+U5d?M0B>DKAjfbvVk>jvDK2 zBD@Ve+Kz`$4zK?-B-T_u(@$qf>>OR&%msLP| z>Sxa%*lZTgQNeuf0X78~q$uH*vIS*s?vL%t_dHEVpD1ZS|4ZZc(NK=%r~YZ zi+^}Gh1YtT?0g{@b?RLulL!@UJifd1YL{gt7#gJ?|0#6u{DOI?gFBN>iP$%EO;rys z$>5Ry%--qhyjE&g=gs?)`nb9v`QSM6aG0MA-f*)Sx>94|{Asn`Wpdj$Tp1(RxanLp zLX%}?>yvu&-r897EknV%^%G@F)1%%j4jboEi?wu^%5fB`uL7$-=TtcN72c-eCQ0#m zn_V|rwno~iOf&0s*@jCdP6SN;Iy6XaukwdWl;T`K(Rvs34`zOR#zvORjR zU>yzzIx3s#N^iR#y44;AOTCt$^jyo8rJnUk-RP6fhQ0B2Vl;if(EK?0ExaV7>4DVWLC zbHu<4*W9dVd8dYZ=;4HZ^TKSRa-ZXFSkG;4ujI#^<)`7PuBeUo+A6!FN$J0~Jn0+9 z2>Lj+EKf96TZVS52cN3iR^7IXAyrz%nhq|>9j9_qqUE@1b~HxDBpDgG465*xU1t?1 zXDoNf7gfqLN@i=B(OluwIS6}n&jdjZ=M=xAzlyE;XV0HpEdvL17G4}eBMzM5S&%NKj1M~_+{Q_ zdxDmoph5e{kQZNqWh_0PWl+PGOJ^V>_o`l;xgkH9d9?2;v`UxAExCNAzBlM?QmA96 zY@LH@8Qse~On)KxvdMRG%nGLa(;jY&*46x;7tvIEmi(DGoj&Wzi!lI**xB=7t}aBFiD_KL_esB_~8s#aQ#dtu;3m_Cp=+b<-dGPQI>KKX%QC%&WGlkw}AWs z6FiAy9OJe~@sX%1n%t`qd!@}}?y5o+!s)u=lp%WPjqAPsUV!QLv=D6iY2&^#4*lSd z+NTHqd*|JGDO-=3%HTf3gwE$SQ`8mOdFIRB)MtWG#Up`ayIRVYBsb;9h8$l>Y+~07TdXDyP-;tH-(iB2+lCeA0zi=P+3WOj->HTlbr{r zqiJR5s4}@%)jdht_XNbcy7J74!siRtI*zN_^@&|NyUPQg>!qw{A2K8@beAlD)kt?N z{t~etVQ?A>Z!%C#DGGYA$uGZw0?Q5yv{2_C8$4A!X>+b}ISJ~$?h37P2;i1U$9hzA z950H61+9JOYdBROdNG-(Rg5pMta+9QM8xk4++#nfDScHlVM60`-H zS+-h-#Yjx{QtytA&z$WgA0MSUb~RLKV!4gPd6bcU4)uc%0v1I1PBE>E(CHp_OSa=x z_{uo@pDxZd>89k%%P|~GUlISl<9~yEAf0BmB!JkI7BJENd&r0CKZl(CH_F@pO~~2G z3OWY)e;so6T>!+t4m$e=tie((7cjvB01%k}7p&?(;A+3+6#oWOJD%Bip3#0Q@9)rgk|gcO4B6gwxt3n~*MDjdZabb?n-B@*8o3QFWX$Ub%)m4LBO5lF6W z!){8VZExF}#~O3B!Obs~OE;&RCd1?8!QMf#1rk3QSLY68Lr)GU0W@dya4EV)9dRUgeAx z4O?cdVm(?O(lv#vFUz;v)UG}wg*`e^T1LfJ^kDh}gRB23L&xUf_)oI@+;)-S0JpTwxwmrI>KIdo$Ml}U-f`q~_@;p83|em`K5McEVq==M7*iig7ZyBL z`m$%jQ8}Akq$ks^)552tpBxvfNdz&k`3*INF3Sqm#9C6L+PMw-@$8)viA+cUMTJ_b zFpe|d9C*n+mnQCn!Q2Q65vvsG)1w2E*$F%IpKj}zGVJ4HNF};Gjn-faA#te|7edoz zPjj1(Tdt=0e2Md7uz5M3Mwd#Iir;~e#f%~Nt1~-fMX}w>@m!+TUuqnL_h0ID9N(UL zM?kjwa}b%h7l(A}$0Q15A2004UNqx3s3j#$a#bn|ACDObe2%15@&8c|33}2`kIP)h z0>$KTInY25DS!H7^@5z2a4GAoOoeK#zz!Pj5svkA-K`bWgux5L%e?sHlkr$1`k098L5HF_Rutvb^a55TJdIo939R^y|t6lo8{ZMS$hp$DzNu4i%J+|I*82}PR!8wV3h*n`((CmG%{}7T^9b%Gj zO;QMcvYH%1l_KwyLA^jqJC)!*@cNlKP7q-)bnV(HcPNN=OliOZ2*}GB- z9DfE={b)zc!i5-#b(v4xKXi4}=>g5}zzYT%wC-YCkjXS9mej^Ee*XDUv9=s+vG^)k zrLc!}&D>TrA?%Rcv42j01)y@&B8gdSXTlrjaQ)?S%iW|xJAj28`Y;3b42;G{x{U>QkdhzHw|9p#Afi>Zk2qZsSR`+PRQ#1vr-7T0w5n zg(CDU;R#ss(V(eg&6?XkvG%>NyO5yF|$*gAWE#-cyN(gXtgZWNgE zf%FXI%ot2019}Nkn2e`YN3X zLD9(Xt?uA0)Sm(fn>EyEReG!vw-HYw-qjRAWA<-xY5bZ2-4~zhwIiqd5y3Ro*F1t~ zcq_6}sUyMa(Mq8>qIfPl_uM|IH@1ROPO8^}Mg40%&@QQPYaGXuQ%4f!cie;;;Y04WO>|TC)*Y1X zk#t-R2jALRH;S1vXFXS-_HiXSM-LUOhnOfu$_%MONf+4>`Roa}gVQNQimSnmSt!Ju%>EKO$E)Uv2h%454M?6?DsR52wHLeV{6}U=dexP{`?aPOkqE z{rUAXF(`LRP$aj>Nt6l$^C6bLa6HU-RP*6a3VqGEuW){P>rPYkjl^|Huy?)NGHpLR z892g)KIZCUTh4AH2<%4hVl2twQl^Z?ZhfWTmNGI`3;9mvw~wx}KhyH7ENsuqwTkXu z3Nnv=hjcM4*ST}IL?hUj;UnGf(_U%@;^mO2nEsf;;eV>`zNo_$YBm@<@i1z{>x@xNpndf6RDvKkCKIv^#k-tn7!7#c! z3PK%;D++Kx6F-rKUrUi0ZjQEm0A{V<#zibvx%<06i0FZ@lj%T*-crTKGo!JPge$ zmv4Y&*EA(t64_!(To9w<4|T_w!__4TTbjpvgL4tF=_TC zPQ73KSe2ox8Ap{Q(Ulf683a6oDX!4#8&Z{xTny;M!aIcrP884F<4n_@XLmTu^g_Mg zfp#NNl0=oNPj4KeJ|p8d#6Q6d*0G@!9fhN$lA>7BxU^ZuwPZmkDhYhrMe-L3MMB}A zwFy*~akk001j*vpi`~7qs<9U@_popb2x-6XnzKyUJxn8{UnmvzTRo7sn+Z6tlYF{z zjo!>kJA)fI08oSmDet3gPMbXUGSfp4`DYh?tLL->4oKtt%NZQG+T7qr^v9+!RI<3>JW z=a|@Xtrb!CuR!a5($LLiNyL_hm@SrJE>qnWd_W90r;4$4Xs^lPA}O2Dwrl&?`Qbc` z8fMm_%V0HH@`r&)~2@Vb|le$NKliHs8JJW^1qv>fz#R= z&C?lkfy_RHGqV=Y@C`O(GU9DD)*E&s*al`YbC`HIMbdry&Z#L_A5^1wDCPX*t) zI8<)K_E{D7g#9)3Ag=wP$SV>6$F!#Sku*SW41Brw%1OQ$N7aodPC^B2BM&hB zA2j~{gBIQ28u@QD>HcWs+Bx!LWliqxt7rItY2MsQY7SAoUft2|^(0d%S!uifvu98< z$e*}{1aj;1zO)$+$f*=C@%@KU`7Z{A7Kr!)|9&!PVum8f+>mQvd6($1Zs+ly`VQ&%Zf$!zQ7E@Z3RU^oQp%^Py?wDs?syP%s#EKr;YeA+ z`0O#cu|ObN{S}?Wy!l|yn8mW={-VW#JdEI_StF5E((|_(CX3{q8Qbr6|HVxEZA`{& z2?_qNsuubWWAfjul)nzj-&*}|2Idbl>*mt;K5xtRFU-6d+>_kl+P`_=eM>lhV&<UPl~C3)Zxb}7L``GErZ0(68rbM-j)uqioYH2fOXN) z&@xf~o=-?XaxlC9gXF(24)sTI8v6f|!ap0x!~Tf^E#qH^yv>*Ym5q#oiTOWk17mpQnpH7J!=SScuDp!BYCrP(#|;B8)SLuf*2>BsKP?)dSnkz+GGG z!(?0L4X?Ym9u!KQ9B7uy$AcNX5@XKq@A*uIz#+kpAT*ksV?0Sr37<23eEWQsIq=Bk zoaTAKb$ZEh>TYnID=1AWflS@-jRGmo$LfJB4vBV9R@IaSLktzocfg32+oA#`5oORG z7Bq`P7NvVVt5l65o?DHuNQ9Aqk2DM-rpKseF4W*&-$&R)o3DHVy3)G-(8Sb~d1@97 zx{>#QT0W|in-Zy9)-M>nvq#k*T1Sma!gsPrn1xjuL>FR-uGDD3mpnKxwV~5B&!s1C zK;!Xjx$dI|l`Uo48PPEI5ad3jI+J0J;>Q|;&dgPzEGGrhGKwRV9;b3W`USH%gVgzL z#ytv+Iw3{N<^;cMs7aL(FP7i^UJ1!B{ddMN$R$*( z&H6})aMP%s2inZn*UJ%xja>Bh7xp7!#a%`U)2;8GcV|ZLAKD+uHUshlQ?5>{O%Pcf z%&>2JjA8B_WrA}z?yaXyCDrR)lvm0anq`eK@SJvEwyAr9OlGuVI*d3MwYV0Q-ao38 zi7x+QRXCOob|mW#M74eozxY#sPNqvrzj%8iEICF_$U^C-dEe1zNC|Sx z>m^kxRDr4=_DmturSzDGMHKde)uI|FYz+nnV#KKf2cg9^55$UAD{h5E_N)Wk zS`dQvNB$(Q71kjqOA+y6tqPoF6;eOy>^387nfOTKh2p))euB$UtIx6RP&IHl0F^wv z!DZQn@De!f2XETY_^aH+NO4HNmz))~YOHHEJ0D<>~ z9CAIm3^2W-JwIrsm^EcK3RK&#hBS@dKj*_1E0mLCFaI=t8hMl;eKj&ZJ_-pu_O!aJ zSTt2YnB$Prm_&ylHd8lNy*Hg*M^D4q{!ku0dleIZ{IphSdMkFZ+U9CIowk^UOlHSk zLmrPJ-Z0(bu|CRtJYVl}$Ub}Ga(imna_7`xcD!kd`!GwrgK}!3q4xZXo7>Hx+x{Z> zCGQ|s-}7-h9(s4RWZ)O7do#;3{=H%`3%hs_nY>c3S zZ&Qv_7|!PichbXCqUB?71q}r*gZ`;Qmrk($b@>+6=z_s)$|XUs2R7(z>op+;dn-jW z{q7PywY@;?sG?+FAFc?LDGa!P9|;$?5-cr`5A)Bb-0mBu+vv3d=cAt!Lcd6$^k_+r z)n$~{|Cr5OqE&7Nt8>+a#% z{?#YTMX$-5A-uEMI4Q1%^MGL(@xwr!j-p<=r~-eZTm<1TyDQOdaX1;=d8%)CC2 zlufSMURLRA!YOGGiY;EYXLo%lDf|?tlYV=^Q;+?6ve{oOwRSGsRMLeBuB|CNS=5;Z zdUq4-T@Fc2ydG34+In_NARu#^NH{Bf&*MI(Dh7ZIeRsU1@Hk z&g;$oCgNAqdlHF9_jq?L*q&}?BNw#02MQ%*$SOmYZQ17658sMr7*ftl_5uG= z&@pkAV%QXp-!MFMB$gf)oO2{yBqXbPYFznZtfXgl1D0X1TxCblFyi~j&BIf4^*Ac2 zlrpgqgujTTg*79Czsw-)Dh3=y=x_szba>|Ifac(QRe^A@1P0Q{b=GMxzPjYz@tV{H zTj9#=-c^$dTN@&S;0j{wCr)fzf>XXr{Lf76m~I|zE% zmua;Sj$)IBCz$%Ds6Wb_GqQ6=YnnuH_%zn%{M_36K`)=XF%}JzTU0=}E`uI-pmV(d zdVS(UFMoq#8J`dvI3rPsN9O=_eP!sOf`lLh|2XEY5vcACo>$(zC2#+kc{LwLaZSNe;0S}lm!p0uZ!T^*DJtC zlD*|_>&U7hQ$oR8ehy45t}PvjYY8i?RdRCiv83`uwC2ipkRIPMxk@5;7h~qq#H89a zg#yv^6Yc*`_(iH;3`ItGJpyAtEVcKHLpSR%(35PIS_cw%0ACW1# zYqr&4py0uEV3brXBz1i8q}8W@tk|ODGG z)8@%IPko{yHb_C41TSHncm3E)(r?|?AV(2Cl~U$gVpVH94ZFo~BeD4~_o-?zdhc>w zTj}nSnsmw()rQQKx$AMR89}k%65pBVW37KRyXJ91%zP8vnJ%65s;LY*ERn}>L9K@H zsR|O@Uh=YSE<}PsUu4gMRCXb!5GKa4(NIk$m8QpPEe4+C*#lu)zOw5`TuAmMG91FT zT~uh<6w}D<<96M!4MN!ULh)ctnwRQJB*y@Ov-;&T<$^7raD1#|f~J#QLNpwEfEDRL znQcORG{vHjGD)oJqf{0rdg;>Ew5cwv877=vj?oY|u{iQ-c@t08vXx|yANx=mGCx~m zn#@l}E9@W+?|XX>Qr;{cUCE$@##CM6eb$5>Aw^Y0)t~-B88zzQY7S2EeB;- zSRg1-iO#s++e2WBS5(JaOHd12AX21!&p?0%;Z7+dwTcuux&JkN=F3-3uME~2$`CeB zyfGw`z%9LXc}0r=&S-V{~K6SoLS+!v`W`lLl1 zuKS}%|LN&z#`vTI(zK$)>*?v4fP={W_1Bh6@Y*r*LKg-5?AyS26M%5RsYP$-ODO2u ze$dNf`gn)t_s;M+pLiJr04*l$2)x8=qH*Fek&1~-D7fRHn1TA81Naf$mskLZM3bGw z{^QPDW`7nKI6%L)o$cFAoAEyK{7P;ha4a0M1^pK_CDgs3sI(m>8f)>C{>TXr2j(y!?3cMj4<}nFsCDg*9Y_&Pcb2DB z=Oa+(A9#4VB`m4m(yigRFAZY?=oBp*p%F%Ye-kqDjag93QswUvG| z{v7r^|Ce%q`RrF*s>IlHRJ%{s5N2}y84ZYJ-}A9HQ&R(Tz7hZ6R=_dd$tb|fTJ>3Kd;GGAYTMckP4vag@ z80^snut2|F3Q#Ht7<;X{-r@pg01Rj!{qEIol(h^XfJEP7|2INh)5j< zavODmGz1{y7!jKl67PcsH?^lcx3C&^&6s-L1*kYAP2aH9q*dpy2 zme>IzZPW_?Xg2UU;zk7Mm%e_hnZ*zRIC;9W`++6Bu)5QL9tyo)`h2%Rt}w!P-p+gc zg#lhGS))9)TDRsJ>@P!f##_L=k<$-5dOB(N-XE7BAm3{BYVKf!2a@XgjHuVdftg{g z^jk0&NqJ!w(-L7N`X*0*TRP4(#_bUZ03MI^xwj`v6@O7++0qQ>a(xakKc1siAqW6( z_4+Z5m~9o#DlOyvE(9n}`j*)lBb#qDJJJ=4CmH2LD06+YLCDbFM1#0N_Q*!<%)VI} zOkG*$TBWc-@zC1!Cta@N2jzT8SEA%`gJOomUsUD1hUbl(jby%C6VBvnIKFbq+NEbR z&`G!q4L+AUqLhsRYjf{pD(2CRuuP{g;axW8pzSZ+Jw}xhN(VPu z9mJbdCAt0RW|c;`;wDaoyh}v2UnI$p4fj{vqT3R=2sGobqete5cbl*EXhl zVsnrL+r{(hN^YgSox1nM9CCWr1FEsDqL@)*dFsl^GTvdVQ=jYBa%KR!+FgCsNE_`l z?w0v0;q}=j!{>YI`m=LHQ^d_^SIe2_EH2M9oT@q4Nl7t0pk>~r9e6@A?ss=aiy}fU z_Q*=!-Q!J~o6yTA%kR^H;mZlh44rfL+_$&uGR(UJCtsQEz}cgE(y)IP`Bc6>TR#Mk zru9e94ba^5jxHDTUiFP8ti&8zd_;iX*aUmbW?jBe*+98B)WoXwx=X1oH_Rw-Qv`k0@A62yDn@JUHz{kay7v+-~3CeRd zg~;~toaM5s)g%HkBF}d&0%yf4AjnXEtK-?`GBXaM3Wwd;7me@4pi*OJ36L9-1y6-864{q>R~OVDw2)Cz$c7T=&vb!BqK|?vliz!zHC&HR1K=kC5Z$iR|dF9bN3i+EkE@PrETo z=+^u1;?}uhxZdfPiCW^#Z~+x0_Q}9V4;;5oTKz>wp4GG~n*~T&aJe@pR)xY{apIIU zQt*gC%jnTo!=2xKNWadP#gjh)qF>dbN z#=&zHRME7Iw@{IXW~mB>ttnG_>s!f%q1~TlR*d_Y}y&` z!P*H+1bb%LMi4ErZMQ$o>lSGEto%_ZM9Xh;xSPK>t7TzBHlKFG_Hk<>-%{I*{`G3N z)NBFv0*<25y72XQ;us3X)6F2Qb&m@%v8^}S8lA~r`Bq-D2%NDHFQk;6^BOUD1#2a? zybxZ6J$&&O5Q78)Pj4_26yObuA+I*Id3*hF`~UKwq-UgL{4=h>NJGoQ!a(;2uJN}A z<^KW(D&b#RxHhxiio9PehF)~2Zm=w%-9g=m4P}WsuZbMBYbY+4eX=Z#!35eCGnZ3R zHD9!O2HtOcfGiW@Odk5wHO;LrzKu{$MVska>~{h~mjgZE+2LPzD4etzt?IG9rcE?H zNWQo6i!`AznoZ)`&~t$to~f(V6dU-cJkLyN2%xDwZ?-@0KKR6gu=q4o z$+etDebkrTanR>(cogoFwh5O<>s4AL>te_(y#sw)g%`e^YES9cu+gU{p94Cxjc@~5 za;yEs`V$h|;JIyqKu6I-r}-V2hV==0*Of&~o!8hl@3|})v>BT7Ymybq_TAR5kZ2O` zJy%u4fKL}xH@peA^{xTp9kz!CI-M8CxV?%rH<@vmFlU9&hRu6l8t0r>b(iL*R_rxp zM1+&ns3<->7>sv8_v>6fkgeSx8Pagi*m;gfKf;B&Hy%D6aHo}$Yd?*K*m8WYRMo$* zmXFYEBcUEzm<*>V>yV)uaepKg{3O)ZQY6|MvoV{sEu7E$^ThpTwr~1i)?kPRS8U(x zKDeY;r5MAo;f8z5fssfX9$|5A^Oo06#;n&2c34UI$KBM6@S=t$XURLA5dLF%GBbN5 z{N84D^m2{QJs#(BjG}h5^1hDF{cZ@kqn_gI`0i;fSRkkr0&i*YS$#_b&4 zWgmOf5}AzYs?vN1<2``BpDyL&V`!KjhxJL|O-Lh9l?MmRG+}{7FTk0mQx3WYLh3tD2 zu8^f<&z@!MyRolrDupajB1=>#64?rE$X;YAyRwwE5S8VB-Wg`*9mb4)_y6Ae_L|--z}5v6Q*X|J@u}K{nG~OSItWG+{U((5ziuj>}RP_9<*O` zF+afXjWbI0!Y#JLdE;j)Mn4!zI6u^3qLG)6uI&ExS}O1tkNpVq)Mwr3HoG(Zc^og4 zKIh$QYttKTX~^n)mRj7Ib69CwQvILiI9@}qI)^WXTZ_!rU()1{J$3P5y7N#W;%cGS z_v9cy?gk~P$hdKhfB{=uEn(i_7WOApt+V2;pMNUad^e^KxxG)1GxDjpsA_1KJ4Kgm z!Gne9=(ga*RG<44QrE~byDRD6$pHyzyk{&MZZwrSmh^OXAD!4p5kcz&Fh z9QIePzSs9AgAcWuAV4x~x?a)`;-ymFpYQvUh>z%fUm~$_ux*2$IAZL?jL-rN0=Q)=30r~XNM{A^2 zfN8sG)5o0lKtXQg-NEW@S+HK4v?=z{kf`HtUre zA9#Rtr6@7_U^(x1#Lwr`ER5L=l`2dMc@H$o%B#z3YPcBVgteGUd?HW}g1m0c?EM~2 z>(TesBQ3i^Q+4#=pUCQH5<ySg*2St~4^-6T{Bij+fpf863wH3^x#;qj&Qi&#<6@5i@h3ST}9SDtqF zHZ4!9-*MGluxLa~`!ch#dSRA(tMbLvZ22DhC$@8aydTOQisg)pJEh3BKIPHo-gJFa z#u1bfeZjS_7wwdT2i|JWzU!El_fzCx^Lr^VN7Lhb?9iJp-)qz+?L!W=WL8i|DcPdr zt|uo3i=5!L{M_WL?V9Z(c~{d~_{lA^V5+o72RlSO)CV5-PX$dGh+&SP*Vuy-y|DveZwZ z<8PNH16tO&zeD?~+a_%yp02U9&WV2#LN$=Ev664@yKcVQB+cZ-81K_xD&i`N^~ZaZ z)pB>(Dd>4CM~zeRRT}3G>OS>)PWOAid6jn3i2CEArkcJ2R&PPh-@WgKsEZyYw0)8` zd-CzEjzDaAaJ#Nqa;=Vq*tcQb-5WmZ%G6s*`1BP=win*B?+R<9)T_Je`SBi6W`hjQQaf?UQPE5^>SRb-&6-6Hy9jptV>ZfYY5A+Hs@hU=3h1^@;=6zy=fPbx1x*tO*ivXAa|=rsO*uS zvr*R4XLE#(7i4!umD$e>UOxF-%KGIO=Cs5RyWIBa=Hx6BL9d4wHyC~m=kl@gahgmo z%pGjE@)_Sm(=;UPXMQ+(`u(s=affx`SZ4E&FMYRr-$#fQcldN~8+dhRWLA{u#0EVi0W|>$$;3`LIx#TI8Aegyul$tyF5?BWB@(`^()8IqMF7 z5i_oDxtOafX5&4)^>@;yhdoUO6Ac6Qz7{^CU#^QD?t5mdKT{oFU0>>LJ2Rc&m+^F9 z;+&>s>zAlp+qsVgb}y6kpB(!X!F_A=RB`P?4%WUAE{5?~mzrJ)`dP~nu7Rs9%*DSQ zr=$jcT;MOJ{W&JrIP4ri?ZogaO|W?1^teRh;G-jtX5D_F?6vq$8H{#z2xV`yST#3` zjA#4yLGG@yM4gpvo`84lSVqdP=&}w8)ShmojoM*9O0I*Gg8M}b42~3QT>DER#CGP) zSn24*kQgeI>i0(e8vYlOKU0{qB-Yap7-qWeemZh4=%1D+wbpk7hL4Naj-LCRI^o5U zP0>~CK3O9Dbcbn&$k?p)(_?;~&Q*pr9KMv_)aRUg&a638SpH9Kp`pswiTJz5+CK$s zholvq48Q&5H~GBa;n3Nf_||_!|EYV_#bQ*Kcaz4))IN=lU82}})=-zS&9t^SCTir= z6IAQf8m${cD5_cgl*6@Af)ozN?@*SRP{| zuaK3rKlzgJ6J$zn(l|=-G5ywiV>WvzZLT3}6w-5AvqDeZPW|{O>O*9laVE zc~(U`onW@M$Y^T?!M@}cwK|*L>FIqcoE^8hght|w6*7Bn9gm!D6;})RV8`Ec@8kEX zAG0ZzDL+M1A8je`P0IG|jqsIEX|TJWN_9ecATH8QxrTQlx1~;L$5WP)2s`;o!Q_2C zrl*|3RBf1w(D>!*7SWqUpHf7yGGipoNHwUlF8 z%MIVBXkOVy|E<{kjKSn1+1v5qdL`RvJ+eyG(y6X%UG%K@tyB5)c6awWLq|VdkC=OT zM^t?qbn{csWt7(KG7M7N?Nbu;I3w@qe)}=j>4dCey)Av*wL@RIwG6sjP_n{?nU9}l zxW~QF(tB6071L3ft9Z||{Dkti)6c{;*#hOdQG=Hnku_(x-aD*yBI5I#scX%_+S}-Z zpRMxJOuuvge*D3-2k~zhxPzRkqar3^tsZbcQ9kH;pX0SV$EA2`bAv|4AwSiOMh~s1 z<{0D}4>05^$h8do;sAbRj>9*&fL7?rOK^4oJ5APpM;ViMriU}?k%sLv& zR`9qm@0B*vk3BOjD#z(CcZ1@~wp!_t#C6De`rlJ$yC3|kt3A;YY&<#C_@i5VPG^!6 z8S9m+m=*Z<-y1e4d@_-(zT7yV|LL-oMtZ#pZHK(}^-jZvqPL;e995|~wpTrMUDnv2 z-(`P(v&!C=z84Gg4sz6LnuUbWYurB=y!BN^Ghgj(4u&qVArI|jnVgp|r`GXqmfLq# zZMMML<-S4M+g^J6ollHE8ze?f4bbw){N1sI`#^O?*3OTWgC1dA>?auA3bMW#fyb_0 zRcr{jP`1Wa$LR)_Layo^u?Jf;f^>YM--|g+Q}In%6)H_y<~U2AXpmH&P<+Z{fBa)Plb}a7<#B&GHvb9Qg)QhwIjEH-;NdX8vLlnwqdo zd@-WvAUKsuJzKiNr8K~x;l-5n?*hwyv52zMQi*kMqe}A_S2wL5-fgO1d47TErt9=lYeuuejGl5x_pqdLJr}z;ilVqqoB8ona?|hq6SFsq!dM(EudOcL zqi?BJuJD+q&1#=oFY~M^e*w)-{nW`grm0#wz4BF&Yvq5WhS^-<&_=o$K8|+Yr(7Y} zJ?kPoJa~69&ULC*CL-dD;<=;;>Z&ygDyr?~>!MX%xMeat<6h+5zu4}_{z;-(MRhIL z#<1^m*CXik*|f&rU(qh$NVa0=t8jffd_3rseXG&ir?tVY_Y|0}eAT?x-@)y$AlNClI67QPD=@RbTSe|m-t}#qO`=f+vDI%jheqmtE=s)hH*L9~5lTiU>7A$y z3roaERh3`CUFu)iTTvHf^2aDYl^DCd-NUb>rI%&*;J(nwn8wR*Ev4< zExCv9$s0@go+$)J)Ok81+1g#CdY>fT7JJJX&@F%ALbqAUrIX_A4%TP))`oHjZN2h+ z@Mn(v1!Mq)!u6E(=7D|DnT_(2bEerRUTQEJ4(v;^H*lAF=Uuko%- z_uHuKtr4r!TFgIm$&J<_MVk)ON?qb};NJc~aNWkYXPh+3cjzW@vo+LEYJ>M=wP9 z7e4CSafvqZ>|=eY6!#r8Ym)-MCI+60qxYuNmzLc<6fYd_`?;Ofb+%If7aOa& znHhB^RrxbM?oS`T-Mncjb2D&Ddht(%3#eq%TKaJ97{7BY+ax)!O4J92PUPJ+96mehcc#rtGZ9eFR-ewD4 z*D^wMz8uxz@|Qh7QZ>KNi~eD!UXTrFK6=ZM6M=Wn*3uYIZJeH5?7Sw=4^gf> zUxE;i?AEQZ2y>U;xzXQ@5+V6;^m1Bcp!|Fdf`j8mOPng>MLVT4B*$Jl7kG4Wuvw^2 z9WIbP+wm_2BL1?G4jTSMP*WDZuHLFTmYBu2f2-~Y(5SQvG1!WT-V|_p`cM%lLkRe` zv2b;eG-t+{7kALi0I0>m!gm+CWYIpw4q!+g?vmaGxGlM^X4}lVCY3wE4hUX2*lW zFIGj`vYf1Ay%{{?Q_)zb_`qG9f25ECA=GtAcQ(yU?I6cZaLDKm?;ZpluLF>zi_Czw zArSu7+aoD-C)rWp^Xp`{-JFelRrX*vV)Nnp{5N?+4M39EMrwA4S3^mu?6j5FsRKf4v;(RB#f4_fftb_i@`?`B;s zIjwYLYUdq##Pi_`_OzUsa)msmQh~F(OakaGWu}-;#cl?>`j`Au`gv<&1CCx_<#y(o zJQ@gSp)*Mt;kD0s^N*@H>eB=2sp#*YbCzqZ>QAYEOM!TEbG5&L({^AYBm8Ha zABjKWcXbAzRB!zCyx=O$Yp~)T-}dZay)>v|xIe^tvzcwHUgAbbuT%liYZ0Dhk(8Nq z%^RX^*}zxEypPy^2V0~-Rzw#{xy_$(HzJ^<9h4!2HvdRV%Nb5!14R=3o08WesM^}K zqN$49o{znIF*8|-%(76f+0nN@Eyo}|I(*Ym$hk7hhMsWtw6I^=w^_nH4=Goq-VbN8 ztq_XSv}{5~rKcAe<`-C3J~(LZ5vHXdtyEATA6%ECP>^U<=c=i6#qP)<_unZxK1CMA ztmbLoa$P5onc|G*7qxwF1jSWx?_j*;$RB5lyqzp}oHwo~Jcldzdoe{-T|vr6o9HLu z**4K06XRVYlp(TrTKr^-ecR~@y&Pj^C{8L;tYTP=IS{q@kP`4t^tBl=Bn5o{N0y+o zLTeDKmpH?-TvO{1_)SSHG=)CCwDQ(Az*;+l*Bbgn6?qN94>B#lLm-mQl)@V%d8}zU zd628ELqMewYnl&T?@f{Xc*XDwH5NaD`D82HoOv6AU`Y>66Zgf(VX} zZkcE0kViysQqG6~C_LJOKGB6w#0Xr}f5ijZV2N^ovHiv8;}(EHw*LqO6f6hA7!O$D zbS0q&5I3<2c!U3ei9sE3ku~#h0r>-_`eva6kP?xg91yPqT!I27MuL32!kj#&qN1Dv zpnQZ=gxAD~lTVO`2MG#wjQC73N9RavpuBjIKPSA)f=Egp|`y8(b<4e1s|Om zpP48RAFy}<0X`8<0TEL^P9s4caEMQo2jDZ|HReM?Zxse#fd57Qza7AH4;&V{zN@QL8r3lsHUX$kEBWN0Y6#z*uj$dpF234yhOJQYtv|CNHs@l5EL zE(k=}3qF)gNFxyQ^>7%sT;gahzFQKuwjIkqLM6?F)l!hXy7?UKU3a#vAYsN~4=6p#{QdOO53U(!%gDT=QHg zq^$#n|6pRH7$$;4XUnlh0zPbnwJ8Kr$OSv$GIDXpG`VqQP40p<=|k8gm$AJ)${FQk zG!J-%t?z@iUb{@|P=OE15@l&`H*cCNl(ZKniBo}q8oYj>58+#y{+XD~Ffo_##W~=a+`v3yy=7FK8Bo76MA!as~<`!f~ z#Y7ND>V|TgQOPLp`E@RD$M7JSDpNu``tO>f!3>3`|<9)-aB~UC@^t?qoA;Gf} zoKZ;A% z3=9*C$hWX`M%g=gU_u8O&9^?O30rVlc;{{{)jVG379Kva^3GvQ8KU;uy3W90;7g3) zE4VN5>MhZ^MLNavB>)F6Zjqx~$h7U~*`p5texVy4vjmKm0D#y11>G)+StJ|^>NZxP z-~-&0OP7+0>-K_BLe9zQE@55)AAk6baosg?GO@69MVgW|tPv)n@MeASnzcouj7*JC z^P?rv#0?jB@M(e%_2o-7i%ljBol)QbAqG7OZ9wQ>l?``7>tfIiA3gV(;7GxuClo`@ zoR?@CiE#x8MUg{$Um{qrx8+74v|v#tl^hgURGjF}b3zZ~LO(NxSCo@~(VHK6crkCO zMB5D+;B>)*38Bq|li=2bfonTQ4n@S!!5 zHZBG)Oc(wD4#u6JIJ7I~x*M1nfN`EM54n>{4jjDvoyf|Epa_gG@H9lU=19aW2-}M@ zJ>E-W3jsUO3n3AkMlO7TrHX5j>fo~*K_OAmm0`n;@t-DcRGwImKoo-HQUzvJUJuB@ zMgWZm>{;;efhQ003GfPmIfo7D#%?XEwNsqZSA525+{+S0j_x3m z@Utqofw$CI3!dZS&(q5TrQkrArTGf zIF;?Bf#(Huf5&GZKB$4BP5J6Zd;Da2@{}RYTgX3pDkV9eH!N>&K&Vz3;Tqc(G zE=>(Q?PYrZzf*&a1-59er9=18PO?Z^2%av(TXo`Wx|z>q=1M>t=z?fWA*Fc)8xIig{a$Qse3ykCU% zg1ft)_X~)QdgEMQg)|Ak&NU#KH&R2HbucbA2=r=hm;SKgnC9qhUNNWy96&D{BjXsC~ zPz?q=0`!uFi9mA%#?GVAO2Td=^hrm7%~x~a=I}^ZPZ9#agWFq# zO1619c|pmxupxm#gOT3?+Q1+u^gz@r^=n%A0$oE}pXxALzSEGVWf3Y|B!X7bpeLwd zc7%D+JHmokqaZPyL2Ui3Mp=TYJ@iUDytaFmmL#%8z(~-A1xAJ%N8Xm_gR)()k&YlJ zI&Y*~8L;{K2%twHVtQN&Ftwiwj~M`{li-HWgi`4L02rZVt+X@(GB25MnUST<>%?x7 zkiet|&*&kth?y4;+PuKB?fhm5QS-vh_fKoHtXqf&sxZUXVfX`Ngkh1K_~dqEQyWzzq*@8$zFT!QVGp9HPhp1%;=Z%oH2FKuL}C ze!^f*bAlA8MMUwSgE`S4uzWBl$ENAcNMYhy%+!Bo)5HT)HfdZivLFR63yU18Ae#c- zg}ruS4IQfQ&^BL!D)eg&$*SjrjsV)C-s>&I79nio&=z_s@y5wXGN2bbE?Mt^bCxxM z0fGm%A`qG42*gpCY*q`R9fQ9ikXV$Re={OE9s*4tE^0Pk_Bs`oz zCOqn+XYJ?|2OCw1PGvBxKpLs^A9^1E=EhOr^qv$9i-#9cENG?U>KK=K^Cp@S63ST* zV7SNi9zubw=2!tj<#=8o3Inbj0_z787%0Qy1MlYJ7v|&_MI#}4LK8;_rF-M&6F@Pi z!Oc5Q2xgeExlSC(^5Y8$0w9~%z}0qNXafSVmX1&lERjUT66j$WlR)>LggHQZfWKoF zzaFwNfhY{F6)6I1on$Bj{wi2}I7NxX;SkWWjCo?QkkAk`rG>xf6(5Vu9dfb2ZFJs5 z@r!DSPI+;04E|DAd>reNh~t2(h(sX6(n2%`!3?bsE%0}(;=@5ClM4q<3lE$ZEEuBP z;Kq6QyH@eR1mS_1PmGqDlatu#S^n__KY>T<`_F>nc+T89n3&{!*ScfKhy zA74N?a>E~A;EC7T!qXK^UN;4jL+0K<$4`wL*e<}J$*TU2kju*9{zN#Hnuzuf|#gEj(D{}l;X{~{5OArpf? zQXU`O+UIa0$VT@co&JZ=!Jp@hk8Xbn4BdYP4z1KB9SJ z>2oo&(iyX|7;_kK=yB=kbI~#CFw!wI(Xp}WFf%f1)`a>LoJ&n1oqq z250qhyIa^WQncGoX-!TOoy$pydN4^#H2>!tY7jX!Y-Uqm`Sq9DI_hLZ@KE;0Txn!@ z^{ATG6EQvJQI^rCganGwb17y#>*0Dj(eUqgeYJf8BpMEk%o=FXWN-Hq#j@Vk%3RrP zMkz!qOwz?|g-%VyOgTk;O1#{WXr_H-^f5e|Wt(A?zlmmsv^V&RdjF?|5)@Ex7=a3t zwHE)MR_J6!dLTA1*GX?%Z#qFQ%rx zkwon*hQ)pL&2C_s3}&&$>Boi6kpUP!3TyoP-V}k9g?e(QW#0#si8mZS>=B%Go;)To zVrg1an|M4HvTS^^bSjQ)&GSG6xy@$oUcw(+RcHsPJsdrHPP=HB#Kh=)*0xHem&31c zS#7rWS#(y}ccPOYYEggKFMqxhNCVgJ+ZvUfs+xRfjz1-vM2DHht;tsB|mYHB932izYph~)DJU`rgv~pT2#}m#4mZ74@lMj26vm?)xSOW+>Rks%i!od578xcr3Sq9Rp(^y{$V1}W)PeZXH24|bJjgONOJw@>C|Lr z3todT7bDB3>4f8*3EB!@gK1m#`v;cFnJ3Oo@HCvb_*=jZK! zBl5vzYZ$7;|X}CV7wz@r3j(?Vo^z!=Q=rIG5SwAM#+RgwipoTm#Ecuv-=r}Fk zk%|2PXH!r}%HIaOD_YdDC$l9f;iHkYj6ia&vTk^MXyH&1yQbfXLo1ITA0i(ZJ=kJ7 z@jP=@ZBx!2j9KAZwd>vu(ZRnea1uXb7*tt(h;%Yqd|)7N@*cVcpWe}$czewydH|;c z;=`^7xF@`?@=ytCHJvr zgJ(u;N*JCt!&4P}NtI`>Iu!Zp-t9zFc-u6uI@?rS=4j2-Pu`|*oj&Hx>|XJE4I?efSX4~6 zJY*RFi_^SMYhPozc$qA}JhL8JTh~zw5EGanpY*!&7!clalJ!=`1J3$C{X7`j@?qHX z>5Thv_UR})AiSlXc4XE&f{5qZY>|7xChF~x&gsBOhZl@@alR|_1Vh_XssKmsC1OsN zhLsh`tP*o7>qyHXz(Qx9yEXQS_;9h=x-+mhmHU3gPamC_i1&2K2mzOtWXZjWdVrzr zvou30ca!PT9vAxRf|9%LL;iNWl6SIYq)+V)Q1o%RuWfg=AQ6{GMdflgIN?*gcGX*Q z?gY>Zd;oC}(wsjs^?8vo8Xhsu>-i5h9D}LO98%inlj8|TaMt!0=Mw~I&GiQ(^IF9_ zT(w^%2N#m9wU%`|MXFc?npN?Mbkq%BZ~Q`ED_AyUy+?_68r>P z65aXgVw#}OJIJlyt>3Bmq-_4Di@CSEp7Mv5;lsXBadr(6YzJd=<;3|YYZ|HY`6Fx% ziNwA*9n+6gQlov$CPrL`#**rZ!}bZB_T(Du51 z;!OqXbkKsQgHn9X#$Tq`!W*2TXhaFo9+lU3;Taq&>>r|EXsH{z#3V*XM^_InbR6Y~ zg5B&*wN1ebEb@V(qokSAR~PsC<1Cz}(R_UMsuzw63x=}jwAg$p#q3=CcZZ9gWW}h( z?nO#HJ|%fRPR-Q6WGL)!I9U#1t0_$eZ_qY-0Hu#btw6b`=jde*6l^y~2cpvUU03`A z3E^|&c|B@q@mJ(UKDp*0=C#f~E5p&bj#qRCZ2jCHZ(7~Iy%*4;pOQS7c$uPEP^7Jm zPjB3-yylR*vP?t~$XjneS|{ORuTt5388l+B-@>X34hb1J9|#F13Yrl2tN0wXQm^cuU?3kP znUs-5;YT^8WIozBd*;iocJeXRl!lIhn!!FcFb9ORKug4{h8&U{-t;P|4EAM^yl0hs zR)xLBnaI0ZOp;iYgG6~A2}`XUn#B;J5gH%eOyNd$c`ps~*(&>1RKB7G%A_5h9-P=M zE~{-eokoeJG7TPy!xlv_&AVZ(ql%`DH+hx$wB_f0z1+CBg`Gj<1?f}UX;Ulv)IYh- z%`F$pOaDrJ)r8j};?Ev_NBC~r%~_+Kto%qkuYy}Sv_)UCF-wqe{f_9WsX{+#GGXpR zXs6ZfO2Pg92$l*<{G>AX%4>V47>^Lm4Oyoq%-2mOP5HE=$F|%9M9v>cl61wdb3)7- z-diVqCw@6i{aSZ0itM8N_J;exJH@PlYT$+L-d{f@7_?N%JmrOSjL6>v)SKMuW0$3+ z(7Vixvh?3X?FY(~BxWnp>1HVducYc){PA3*j zo=gNS<;tq#RgtATt0gCL%flvzTJ2Y8Kh0f&Z2KMA7gP|RYFqIilVzSh{OB`ve_s`y zRO%=Z_14*T!cfeuGxn`B-!luWwVx#NXm_c=ZIjNolOLF}!o|fy10}{TiVoIO)2r^n zm~t5waXlw#SEJ?@0Tyr~ckhF)9#BEL{Z-i|_E8w`UJfmq6_tmJ?9p_H+LN(gyFG~d zmF*R=Un|+Q^k8e42(EG)F>;opKDH*g7AX;8hK#V&E{(CbV{<+J$neRL?Iq(iuI zyYy^FUl--Yzyud725Y9i&6CKrN@)^p5M!#YlYcNy5oh+AA5TwkFvspDex^x6I*0ad zZso08zM;F+6+#g)>9pzgZR>$ZqM4hCpYvm(mU6i2S`6T21l3ENONH<`yO-r*jnmgX zI&6Cm+z)6e;-go?fnJk_IiGLKrrCokC89a!WX%#R8n3%;V@>sokD`u}l<#&HB>sA; zUcg0R5ed!dmk3ouTER_q`$FDP{MB-GXo*8h<;`{3^aoe9pr5i~KZ`6%p0i!)smxlg zZmn=^{#`a#ETS5dDe_}kYf{T~=dF2YfrLZS^72J-`NQ!P%)PxrG3oM(3|F}b-2 ziwjxGj7Hv@C^gjq`-8^_t5^Xi%pyoTt?Yt7PhxdZ6k>H9Jly@JQyp%EMvlzXy~EmF zur?d3K7mJVtbD&aFpKtSnfF;W!}?Dh8pXOIf|-ecW~W+sH}vr_-ybdDzIuK@;u4TX zavL#yQR&T=ZLFRDp;4?=n_@zmCKLhO9d|b%DKg%KEO%?3b8%$00cx*$QuiYSTKiN} z#G3~)-3mpD?Qg{b*)LzoERNJry-Sgfz6+%Kcp~_&07I@i+r;TsKJp{gfJeq|@c>AB&K?_f*G$qU;j4sLdDUxs(sI z3MP@Ac&i2OZPFFXDhCqOG(XlTYr1VLd4}^HPkUfKfC)$EX7i;?yMPCS^yQg9`_`-C z@eo5(c0Qsoe!{+4f)8VhlXB+n9%TL`)bF~f+KC`=(9lJC#DyEy&^}g^QsGk0Wt=MP|Vp;Y1xeLU_?P0e$I~DvIPQqCGJCZyNjFOenyq z$PqE)=YL79Si%}AR)JSwZe|-we3|Y63KM9VNjtIUBC2_{8QN$penylVgB{Bw5gn}c zMklJm$@437hxplYQAnRlgJ!pc07XTCmfzmaQTR1`=i_!<@@i<`FhL4!w~Iq;xNJob zU(%X*BICSvOt$sb3i)GyeB+gZRDXXzo;hA(w8dzHlc$W#(*|L;^g-G;34bZ0{zw+; zyzEnFNi@hq?w_FY-ufrapTxi3Rg6X2syVc`)mga+Nda}=&)Q3&#lAf(g*`O6Pbx^G z+^~@Jm(M5PAh-d56+VkT6&_jx8q#6lz2%V1@$nCl0dyau590QS0O{U!WC)NlQ}%1l zXM#UZe*ysU{s9j#biTWwuVzAnGuhTmb@EOT)|w@7P#I&d&e~PwY*ko&$EGs*=@0=` zf^-|803Ts&fNyS*sFZRdu?z~dB1EYV-KaT$F-2PyG%l2&xAlzbC5%^0e>f$uKkb;|0_Rzzgo{*8%{53i_%bz*OQ| z1|2mR0X{9`m+5r40xpQqN|3;6##@31pvS2_ge20r)-hl;(Y%+D0PS6Q3eR7%ibAW< zdv~J%1RXl}{&lz^1-|4)ivJk$bfb;g8EyC5 z5Hc3KpCO*mK5GwyDv`dgJ65-FtwL$1boKym`wsUl3_vhC*IT1gloBEu5)L|KtNxDp zB6KqxLaZt!l->Vj+^_60TvtLI04sNc`roynSJQ#(864{>DE5>7ZBxw_Ft&8M~WO*sM;3CL~Q9?~2#_EJ*tzH;pk0>c1`a=lXzP5|BHwXO=ODZR;PeJBiI zt#D086pmHs2(^8e0@@rJNX6^4umw#w)tDiPYP&|fU%9_7=|N`oqsG4gEoenj?{^4* z=c2s3@PM&rXN;n(%q8F$ZojYDZ{ zAZ|U>`|^#@E)Ti2JN($;H<@@8(z-oVKzjS>8S(+}X|d6RFz+)XAicB>=8ZTe6=nJ< zHOAr`J!QHdvfbyt5zZIw)}%k5mv-S%QJMw{7A=48Ev#fu+g!p4M0v42 z=~YM*+@?uuW!8gdf@q;x8CQ^T-U}do-A2zh#mGDXZ3hT#(=N9tXZjVd`y$BwE7JBc znjW@J2~FkK7-M#M!_%vL&UtW{oqok>l^lK7s=RQp=3`?`a*i7t!-ToEiJ3NQ(OB0Hj@Nfb z2Rp!#=_6|NqRisShGYFi6SZh|Bh_3rocN;5g1DPPDRKq@*2ReU<=qQUo`(r5ff5d! zNn7lSN=C0K?b6j=Gs|;1BR%EbOuEV@4Qwa&5G5{+o?H}Jq4?24+{K-<2NE#x+x}Lc zsM7}-(7`&#ynT z2fg{C;pDWB_yv3DMQqd)8GU%ICRp6TshQ#DfqkIl-c0V#BecHn{z1J#&*L-!WLJ8v z!&4#ZDFS(c$q2+$_Ok1%_FnhNR89X>8;aMu7VK9zK^(#p6#!8EH?{v?0k;1@lm8pg zR+3)ASXWn9|2uCwxJM^3N4T(`smA@d7Cv#+5kK56A!Xk=>WE|G{?WZ#EG5Y6W%N~( z_s0I2B%k?a;!(U0_@kT4SKjT*MlEMfTPK=b_lSgZ`N^+at>69Lm1gt3QDG=>Z5%68 z$Mrx5vpZOtR|x;eLa!)D6~b{TxvcvNa6!X`W?IA`Le_c`)tsWshZI7WlqunCGtOCMf~gr8a4_(`2^%s!gYnw#N6N(x|mV z){twniHacdCec}o$V5PtkGuz6Xhyn7&+>CZqm&7Sgg@D!MuJXF+W~mb*XFbi-LmRO zwk_L>uGF&T5|Vf=qNvo9?>8*gtL}1SEuS&s-G_0;?;>wPBxCN^t*b4gw3~b98q2)6 zcxgrO2%Mu|`3xE>*SF8{i>2z<2@mj=XB;00q2adP*C5^H6-i!?PveztjSKDw-1e!9 z2lE7WOVe=i1bvFnI-=NTu~1KRktp7w-wA>JR-ePJbJv>O?@8e>nfMe zvHAMN9gwvs2iI=rVj=pw7pgi7<3+}%F5X9aN2cn@_@_t=v@*;(0$oed8J&SGf0H5n zq@2173dx}rL<16#haZbpiVuE6EVi9A(&I(bAg_l14QXJ?51KNe!bdn+nxn`YpCp4% z16SaCH0!?m`gQj6_f-#gR4*-ew5&ZNUV>{pHX9>v)bQMT3#G2AmZ)l~M&fCQJ_n7C z45$SZTYDePUab(Vr>?7guKjp6dCq%c|DuGtmk~pYKC#Rku3j&N$zk_{{1geV{0fS} zXy|=O7;UX~Yx(=7oOQIj&XiUE23*%Wxt<@6Fgchcl8Tt0bko z)?VYdzNp;21U*#ty2HfM(i}7coTalHxJ)pSQ!*@u!)BTjo?3HaLhdX6T9>J0`2$%;p3*N^aUxqyYCbsEP0*kBRi9j=Q zHS&%6b+KNX>+ZdXe&?f$n<;ImRry^LaYgdBigdA?jMJo1%zZy78w{L|)rt5rcZJ%W zbnbK9`y3HhCdvb;RQ5|12utvy)N+xkt1ddW70+f71@8&xMc-RxI_4JFffyta6gNI? z@z7gm7l-%yQ}eRYGJfSY+H_(;NmDLL+mr0s*}ikHj>v;a7xql)>Ie;wqU!M|zRx*| zZ^FU7=5bDwKI?slXioA0D6kkHEi=3VvxMD}?K%H4i8p`gOSw~NGY%kNvTJ%)&~g2) z8(wu-#De91{lZh4k(TY@(?Qz{c2QeznmR`TWDymed{wtqL0$D*W2C19?m9 zTqwWTItWh%p{=01H=IVz8_rh%z?+)?Ep5d_$IMK}%<(sE#mLOc#Lmje!pO?T@?U5x zD3|8=fAVQZ%P5`-N)XyAiM=F85N6$9v5+~|Ir25zXid8=C45im9)PC!PLFBJPN5U5FzHh{|zU7X;vv2kd}E5?MoSqnuutE9-*jUckclsJhu%|=5>n%6^naglb3hr!I( z9D~)StG))ZSTPrGny1ee2Pe4BN1zk_z53mAq{W(+yD0BXm(i$!;oAm{r=2up?RxJ{ zNeq+K%he|?!`;sWr39CUT`DVF7ab}$!r>Emqa z0rU$SY#Cd=)?Z^z8uD&XI^4@Ck$n$i*UTxK>wApYX z+j(3c{wn=Yv`S@pa5b0f&8w#%U{^blTT0?;Sjx}GamloiGr2AB@ys>n<~#&D0I@7d zN4MqfwgOew)Y~Jv-PV~lQ>eT!YoZTWt9!LoP+2;GBbBPH;-hBo)P%iDxEuZ+59`BM?$(2@@kum1E>#J0ZAS!<;jL-X;rDPw_xf?rVH zO}VvpXEVWy({VC6n)k+<*vK^n_0eu=$^C)wF2`NkQijLH4~NkovfyK%HM-q2WgphM z5MATZWu}8|6oIIa2gBo(RG49~sNctwM0_dFk%5Xd_N7CQp?y!i9?W5j-ed)3rPg|v z>&g*VVzLvjO#8Ja0q&65rB>(RZW7MCd*|V3t!L*8=RV0*R@#Pmhc_!LpUYN2oxc?v zt)KA<8+Utpy}PiDWP_m1e5AEo1;c?g z+hcxWY9~K1a~p3%=wIkDQ>K1t)_x$Qx3DO%%B`p;IgMks0@fi4X5U)9e|<&$%JpDM zX3;j5w5i;C5FONrvR0;&a_Q+Rv)=Mrc=Y8!L~Pmd8(-P0lIazd7gPtXhf1uzF$H|j z*2Ix?LmZ}*P6VC~Q-wtZW{dlL$|UNql^BLJ9o;UHyjzcyvahEK_%MAwN}Sp7JFT8& z)sCiB?)2d8J|A=*TFTdaoNV;*yomR~xyh8RZVrw~nuPYa|MB6m>&hTU&kWx^I@MpI*>b5W;}kGbwp z^*xgkqAtcW+gm+5o#r!hxV`SD1GA-8_+LFX?qZvIXZ)Mq0OS>i2|-k$EJ?>MxLQV50^J z%*~Oj8r0#W<-fg1;>etA!Z!QlK*@i8crC?MDL1#<@If6rEhl6&>dv_2^Ne7=n)h@A zDZhuRXN|y%7|r2h)z1sa?*&NZ-6e|}&Gg>pG=a3%$0oXxC1dPEJ!Kx{OJ|Hwa;{^( z>}fMRmNM1=gPQ2V--5-%J{zlyEhpadPZ4_f#7!Lne-L=!B;cTwcTZzJM}VWlB72qR z+L>`hCH&=e8PA@%Yyvl1l)Cn7(3HMe(A3fas9qx{`2CkA2^3MT0$I?2aA&1dX*S!9WlDA}>tn7G(fTs1XCay05{ zEG%DWE%(aGj`6}2tMd5QChv&a&Kp^!_WHy*bG5$6jXFC}7W)LBzU&z)*_ z`pt-%`PCWdFtP|FvWsKkiHxYG7EM2LU>=~EZO4T3A171v*qI9UQt#=L1@+jAXWgl# zCNJ0_YiWEFeduos*H_?ZF%9xScA`Mu{)!gPDSI2Jp!d&F=azxo(=*s)R=?`ouddd~OZ<=q++j71ue%DJ5JlELU*E_b)Ta^%`}&KFTk z7so{;uVX*BBsH;)xFmO+MG8?hwkVNf_s)3sKGLX<1>i4hxq&>z7|`NgsQ}ke164|!~sXWC(|pT(ym&q#|IduGYZ)3Dz|vvqAEnV|Zrh%1FhZQK%P zFRssw@Z-a67d|LRY5#oOTJ_?5)YP%oVoNccz5yKMR#FhR+69u|s}ZeC%0wCHCrshK zMoQRYPjq@GWF{xjWL2iPt^Ep3soE7sT-y5K1d*O0f;Dj)W5A#}1$4F@snv4da>OW@ zx`r;=tlkZuRKFmz2qHpfP)$=17%*2TB<@dd}pszyj`Oqx@OrrKJr8^cxdQNsea~1tWT1%NuP_!6gGl|pqTd`5uoV?*9%o7fSlhA{gJP%O**J+{ z1?m362Bf>-9-{ZboONO}Al1`lSu=5JX`L){s{exeNpbA-rLdN;P)Yf=r{?Vef@lLl zoh05+3SuZ0BPDjI)BLy6V`X(2m;h#Ym+kb`F(SD(As7Iq0XnoWeU$N(7a)j)122mp zxj&RS3u{R5!R?n7ReGin?qz~_w)>WV=*=&kClJv=Z~)sKTX)Kys+^^m^DSaK`6OUfn@UBmfE!e12EM!PnC^S2o-gUBYOr5_--}#3#j}h z`|k^B){r94pLBgt-uWo$$TPT?e=&!oUe0Ofo{3&DSl2 zP7X67PZ&#>PDU4A_a(}j@4DX*(-=&#!XJ~>VtZ@*+F`$5xAj1%%*z`qk7mjZo76)49*b@nPMixwR|K&~Q znKXcGl;fv<*y661_H$c+y+YOkO#Y5y3s(RHdQN_MPmTYbbeaA6$S-1$@snO@G;`fC#Y1Zxn{K-r^iV0ca%^|1t>Rn_A#J$>~xACpc250-3^t zEQaR*GrtxH>G2Sdg!0Qb1o1cPA*4g1?gcdfpxBNMh2IUIkEC4il>PwtQggC>F79wR zZ|=B4|E-}KnT0>o(3%EfR6yNEB9z^5_gjpC%E}HXu3!08OI(HbH{&p3y2pj|UAZog zKyY3oGiGuWpiPVZmvG3U)!c%xaUm!WjUmg0*qQ{o_GE|+zWnxfUw2nn$Z_U( zyb17tKIc#7s4p0mvVQ2wV^fNj1wj)$sU7W)B;2aT4uE)NJkLrNx@D5 z>HF236vUkGAM9NMIY}wb=Aa|(+tPz^3-Ar?$)XX|(K~%lijy)BH+pFXZDe9cjRx@5 zUT^xnR>x3ySD!&AwoWUZKobGM*RU#NS$7@U0VK>vXrzA6AxCz%dr!HAB3_y+SV=?c zCajkR#5YU$PryO+yYgJk`|@;Ud?URhdH0cCH#jT{BAS?e%(J+z%)-uYNXFsh_bva?cZ9?vX3Bp=s5wIg@st70|C1`m>gYrXn z!$TsC*T!y+DzsRc0UVB0?!DIX;E2cj*NhVFs1`pDQjg81VHY0xE@Aa(+Z=X;L4<0t zWi(7V*V{+qwTbiF13fy!N6n-J^XsLWH^Uq_2|7D#JusQT)!S3X>1Eiw&mT{Y1Uj}z zybS5yzI#mdI+jBiXD>h5F^I-VIU%`{mn|mj@nO_!KV+y!+#a7MO41FALfGUzIF3q; z3v3U4t)lWG%i4LSaJPoy!9}G9hYj}IrW^Nue^~V{ws+Sn7N1%Pa4tE%%+cI|7b6Rwf-Z-#4O>(T zMkNisd$er0VjMTjZ+vE+CHd41E#96uJZn(ZI-7OqyQfLTAFZjD#L^ZwU5(P*OM-oM z=X6>wx|c|gYUorjyjZ*_91$3Oke3^qoI>iJ647Sm$aSKqBWmB-V6n?&20`()Ik%}p zkCZRLbiAQYR6cOyiD;Ju&lAes2b)~_Jkc%Atm+SR=6Ttq`!~%Zo3NLDKBJ*k zU^=*ZQFeLPzTcf+3KOp-dZ|~;=saReDk+uZKiFhDLbkDwJW`sjoev6AB1%~QE)v*w@#c)M26OH6Y|7I;Kx%k zG#cNt!r5#+0;N>&e$G1`js)G1)M$cR@i~-Ub=eFl7LZ~RLz#$xat9iIfv%j96gmrL z_Oq;4ah#LL%M*AFB8(JD)TC-daXD6 z7_U9vHnn_g@8%Xub{o+{QGA^yRQO%7=_B5?I*p2%173E% zJ}U(8$e!uCp$|RHj}4So=&ymf)0y(vGmMX;UI}j)u8=I>^c0T;5!2qC3kOU8<)8rr z-2fqf;~?RGd}D^=|HT_K91Q=@-ywxpGw3;W5C^F?vNXVYw87DV;X%Q*fJ{ugNbb7sYsP^*>@a3iJ z&rA(fM;M&Ot%9+1?`HfKW95O@vSI zv+hW_tt#=&330Wbu4>ty+KwM`t(}^3YZ_W=?9`fStkpiNVP$lW7HCrnL>n&VN+&Jw zu*ZID!FoE{9dKxWdQCW=7(_hqp$wfaJk%_^I=7l!-PXBD@UBw!_VmVRchy#D{wAc< zTu1Tjs6v#(^RD1Ljk?jOeagxrYkh14OL}$gdq}eY;fn zG5h$B!0KtXw+3NBhQd~b^y8_PWnuJz95X!@Ne7$dhRnnf@RLsEBGOHCGpXV|3I&Qz zO{LsS)YW~<9BH>B=eeaTjEr|JJY@ps8Q|sfvlfl$r?abQiq*@`{Rl_i`)P{Qc`k5I z>8Pd?u*xL|NArAP0Nv;5m_eY*=H|rtWYIz6sR5CM$0ofXkKyWD8r!4u-F}F(XW>@n zgon#`9h*A^-Nr`b@248(kw?61u$-U znq2>ECQ()TbaIY6*Feci`od~4m6+3>4xZ)|cj#b@fI+p~fPI<9y9U@)tCY-TCU3g9 z?IDA=6e39`>7bF$TM0j4(`tKi7!e?x5m}&%iur?w+mG}rQ3=ea8f5|Mx7~VdPpQ1` zUg5GyvU6){ay%&(^I7G83SPWD_MA^EYd%i!;&`&S7z#u8ayZCIOSnowX;4>TsJqG{ z((%6Emu+?09C7f`+WZ1^_Fq%#{z3V^fP$b9PN$--=A8o zTM$!foy8eAZnm(ii3-zQLN^ol}jzuoYvRMNI!onLvLY{ zBy%96=`HnLYpc3GcCs)4&vyHLZaPj)Wk^w{-0SyBDR9)GC63sYMrcPFAz=vev-*Y- zx72*m(N62_HDF;ZMHAB_u;<8c<_eNsHbqmWDSy70WX!EShP)ppCQ&fN`N+5HNKP2*{T)9gcr z#qP+thc~EQfbY!p+9uOug^_S?_~x_sqwUl1YV%{OAGuHR3g}9*6G9hjR(f~wxEd!u5JcHd()uL|w?^5*zR zlHT>U0FgoCvJsrjc9VpD!n5gwUgoj=l-lU#zV|L^_u+LSkolZ!w*@0fR4-M~Ff4WLVUDQtT%CjL!9{5Va1Aeit!$p2iV|{aoAy98T z2`s%mzpc+0DQOmemht? zr>RZpxbHHGyrR3`niA_Z=vLN)goTW-efue(@nZ(bl!%`X8tzAHvql48t5{-#?G%TgRu;6zY;8NgH zn)%wN+IBKF@PxaMQndDGQbp)>_dDb+wxli|KZ}58q=t1TYp*V~qgFvxLUhJ)C@YG< zwj7N~VF#H3VpD(Z*zFH@LxWTa2Uh#xBdIf`U2S`_&3yH<(UW`ykrPWrQfl!%T>q&mZ3} zaAa2HvH6&%e%FYhrG=9BjL%sj_o8Y%M=Pi$+)>Wm-SGVw&6gejS_u;$3+m-qV(~m7 zEF*`V;dud%Pd80{Q(R8lMmY}1)|cXDqH5i2@32}a)f87ezn0PO(heUcMMf>Hb352- zT>zun9fcf_U8*ekIpP#~iO>$%Nv_~`lieZ@!=r*OLnb0s;cuVg;IRLAccp-y0?rX!5-}sWpaOqG63_rnM=X!9OzaMBvSaT24)Nyoa{eweq zprxNj2i6H4(m*2m`I}rpU!EOQL|ooOL$alprPtygZ6FDK6ZE!q+eDjv2{iW7Ie!_k^S#-*o@Urq z?Oj^}Viix6%};V-&dM9cQeMzvvz^|`2(Ke8$pO7e%(Ly`0J}3eU9y!65FD(b3oLVS;tAqddh%@X1m7KFZaWw<9!Jv?>XfK{|G%U`O~r{aZ^$yq+88axgT>)YlQ3s z58 z8}G5&Wk}G>*+{lD1zJnM%7_g1R#==tgY2~pnKbz`o^AWy+GJSq3>4sz5F`QZ}T*c^=CeLGJdOy@;AP556`2Is&-JltsQ8e?w>#LyxGjehL# zzeH~s?rO=S+&UGe=W1?wR`C^8*~vp3D_HVhD)!ON&XZh#2DY8vU#3gQX)AEHAGUdY zeMM+a6R*4JAn~)ojva`MK${cufBthX$?b2+fLdm@u)iUzgQAf$eMy`yy z(y@xw;n{Oh9qu_14HX+37h^)?LL--DU0j)*ECuDKOAB@FW@jZJa3f<+(*2IR@Krm{ zVshz16W_F2tQKXh)CC#G2p`2}R9swKY&DIBa!%rmOI6jBORZt?$m3#a!Pt)kiCRn3 zeX~{7MOeZ-Wh@?;p4m@_5SZt82+yqUmVw4yS2Sm#IGtGJ?#}PBzi$j<1NsnHE_CK! zoBeBz;6Avi-=HCY^K~(`^FO!%#$d5l0AN!?;qVz}3up5)SAOnb2x#~xgl>GW2LL!< z)&TP6G_jo1Kij#d46@~Ymm}f z_NQA&i#C+q_kSajZAk4Tq5!}Qq@r)R=QJCn(6;06P62B_>z?o-89(Mc(?4(icxv*? zD$Rl@CuQ3MJxmVp#cA`Y_%kMi3`_g%A{h*D0L*#lxHgHCulXU&{#F1|I~+x92W=ba zI=}_s8`OXF4WdO*cOHb<0FeLuJylSlf5H#J;PWGzmUAew>AC`)Ur1zaZRsVTmtM!A z|Jmd}X3ZBPb+iuJj3Vw|<&Zx|KZ9R;LEdwo?S=UZrw8~Ngq5{JhZDCl7h*>;Bd-?p zh`%pV=OiIu?b&()L@9u6-4X-j*}3g2BU4K)1OQ(9u|}l>6%Xlvm4xdL^?KQnT;xw&A((Za<<)FIY1(MQ!8 zdqp+mw>4I>=v@tWJ164cOP8ywp|aA#a22Idou!)QN*XqS@wpgt8%rB=b>&IrH0!c5 zof=xKvWQuZ9N*m$m9IEPzKH;#=)`{w)(35r^_d}e}cMDOe4&j z?1ekT_x=x17XnB8U*ZG*@jq((1JwNw2eilHoV?jSHQTq3qf}_%341x0ol>ziELv_8 zxHg>a%(@}kVA7{3Z8sTOzjO3F)*pqnTE+i2+FY(?9_ zR5kwXGQ2=(s?ysHoKV>ni5)PBkg;uP>r({2_3N(~7$pjkH*Te)Hj|y&g=>88wPF&- zOAVe9@U0VK%gB=8M;hq)>pgOtKjqZBJ-!(gQU+p|@4U*GDqcVuWCEERempcZub)2YsFK2MT)RH= zs6xhg;qA6>jvuO3>eC0OW^F(H=?ZgGNEEkK$61Dm$=Lc~87L{qp+W0?AE(F{KN>(Q zxU(8`?f#+%t>Iw_Gi@aaBW`^tB#|K#j8UPX#+C@KaG0`q;n7`?2X`W}V`i);aY*^(Z1~@6g)*pYzkbR1fZOPoVBW1fE%pP%u z?MbbrUUq@Njw0I*hQ%hY4gw@)TqbS{7@ZGF+!DKf^d{RQY0Fl69Z+`WvDD$6;mzX5xx2# zvwOnBz=bz7?NM?;_s+oLp`owSObDdoEkmi2m`3~tpsx^oS^7}bO>x2lUPJZ-2@Sqg zD_BR-o;;hc?-~0FY%^%F1nfJt*nZpBK^E~e(qM^*98 zz<*@x(`Y;Vk$n2&SckFyn9lFvTc3B^7S72PZSh)1v(7WMADDCampfUlEV6@&j_nhc zsVA4Edn$wIC@j&YjzM9!~G7o+`G0 zM{Euf8YjHrai&p*?Pn%(3|bC1#|3&;p{VHTpL#W}7bqpG%hzMqH9fcRr;h7esZ}kq z$HS!dz9nUnCSpx1OE;|9|6%HEALS z6Z!Wd8|XjuZca8kQKLq z)Sq~P@@-USyECgwcW6M#Pcb8XD9NYKx^n}m%hYARWO-w1WPR`PTJESl)N*?OgB3T| z@w?1fMrtRXv_A}Dq&Jr6{}Q}{J5c|fVU~tA@RdYaTa~l&>iNNPfn__b1hghh&=Rer ztoIq}s*nKi+6QYCkTtP=Ea_>bH2iI~uTMfZjA)ers72TJhw&$qF!i`nGg&!0u za`TXE{EFAoGTeAO((rSX(A87x`X0(qVy&y=OxvmOy6D&>t7))Ff9p)Uxg7u5RF)-V z##1;A(jbutHXJbxGnC6|s54sV!&I3&8_M(}*L8coy-qbe*YJ9d-TqZcZ#xuPA#J6h za8XcV;q2~%eHETMH!_Xh)APsGX3WaMJ5D?KP}}44G7V#QDLAPi^jI(pGETVeVeC zbzFmhfo?oufAq8UZdw=eE3dPO+7VUj$OiKL{xzG$XR4j6haRZpJwbDL2bExp<%yyG zz6Z>zACI@8zd>_S4i5f+7G}@GSGi*2`L-_#xYk0u3zcQQ3joHo7EszsaS{%u3$%EX`uqli0SGkj(RJpv18qO z*q0)^gP>h-(jk~-XQDpapr&HKVD$B8f^b2gtF1yWfeT_WSTom#uWoszcG~+m4eYt$ z;!|@Pf7S>ok=8shG~X?~hpFD;Q3uwdCqD3?p&MLDc$W05~ zI5XLVMP>Fo4_X))hn5fD!jH<~ZNBrcYbc?rxQ}3sr&TUCiPaoWiRLk0UB%j7-cgaS zX^9-Z=6469-P{!E7jLc|P6b_*t}Imq9nH>~4?vtgF3Y$IUj-{l6c^c6JYm^fT!JZB zq=*C;$T)2u2YRjT`2C&)|JW7jCl|nK-MUmF$M;G_g<32zv=V+fbhZFnQu*w|Wq%!Y zqoC^Mxd9>+sONGU-};HW7ORAEm*XYq(cAL)$DWsGXR6M@DPoaiyhd|YSZzD0YgU$W z0QUvwf)U@Vx{s6xPb?Fx`c*VkzGTu1Fo$pB^c+26z<7R)kEevMs>~DoSw)rSs9vNc zStm_sfTv<2iE9+fQs}ITf!#;%Uvs3b19_2KTlS8f;c$&F7%p6pJoX?aM#Qb{!I+;n zh{8%TwMY9!D~D4+?f1Ti!coKe!n1n@AijB?-qj-KHC6n}FKb2@b#ZpH)aQy+^pzT;;(RWyK2lm)RaZOiqD%{e8dXLtxV zuG`wxXxSJ}63jpz$B#}m?>ASC_4vB*$GcC`LV}hclS!v_{B96n`&LNZdU)L2Rggw~ zcAU#>5bczTX#>o$$N5MFcoCX993^GCYpqznJrUmF&F2-N0WSf@p`Safec&7$k6gCZ zOqEtTJHHOH8&ZSA)6dpu_%oH0F#Ln9$8}(hY~|7WQY>6J8z%?{ZgSitXPmvRCg00j%Qy)qH0HD5|! zn^@j^!alliG_#v#QK+I(uw9QX^C9QeH`(Y!8iJ%_D{BSP-!B9JI`#L^?onI10L4OO z=BC_%KqH_l(VM7gQwbquCxBOJQ)*7$iTCpG+g)pOtw|~pqLCjZ4>I!g{&O7C0*T9>_8c7+uZW&n`&7v!SZkonqN0BJVEijr`XEDO}#FA z;#j3DI<4~BRyb6*qB|NjKxL`ixgLj(QjC9a;{fJ+O#_xwh%B?7q;==n19s(2K0u4j z+KH!Q@y+E+97*o z!f?2TCQmz<=NquWnv+gSI>Wl_#b*9Q+=imoP#d4jcn&t@Hh-Lzva>cFmv~r|iGXE@ zOHiz%pSrpk9@AJdX^k{sC((3gXv89%b9xn*W+?kDoFHXsHVAutEX48TwN)x?XHgk>3)xQ7Wz55`Gua8(CVxC zFKZJCQylla(lvK{WmNO-c}ZYelYeq?-S!jjaZf87dX`VhFl=qW^=bVJlVUdkMVhKG zbDc(i6bT<2B}I!WE0b1z+4!@3k`DKYM~@>Z{VS6L&A6r-%GG??Q^dR|g>X)u4$ zsH8Ejml)+yIjtaEFKI8U=`et_c2;Tkoi{!Ogr__vRBX-^1z%f8Ak-5qvBsz`KB=Wq z7c8^6=h?r#8eh*0L#dRT8y}^dA(wxWP?<>Sp2sTUm@0m-ugB5G&l8ye zi2X_%PaC%};^0+WOcS7EA-(^Qqsd?6;{(as`|V6>U!ym$J-jMQZi{LX7e0`1G%*$| zDh=_H4=G41jwieY8cMJvdHZ~wXaA<25!cSWLB5*TA}v)ZSgZXKV3SF!nYKG&4%~Dh zzF-PSWOO$VF}fTz>XYSAcZ77(3c_3^Q0=kcN0`W?Jix4inIY7tld0=^)bv zIU!c7IIB#hF`ON#bDZ#Eaj>$2q9{vf#9>4_G^AQRwAwYGX1C`uxNR|>m8L*LEzTjy zOoh_sahv~y!)^iVj5o(jB}JL`j&ecd%J=8<#{*bQAIEz%i-Fa166onO?fGce8YL%1 zqnpkrGdAQlm~zk^9%leEl@4geKJdZVU7taS8M$}I@puh^qtOsE$p(hUY^&b8IxCzf zTNXPms}o-7^rew%D@Upxv?xhCEn_rn4zjp%NfXWzue|!Fs28VnKG!}oGP*eT%@8pW ziFod8Z&6fAj9uAle2m9*fNtHdcPTp-F+SP6Ic%nSFv9V>E64T!D7m6VV$NRQ_o4HNuElRlIPOxe`YZ2wTGkaWLr$A2vb7%<&H^)h zDS*IaQglxguiaOA@H%o*ws#O5Tm?(>M}vE#WTYYmqjG@uxX_b1`j$-=E?yNwy+pGY z?D_cD`R~mrh`wz$ad&r}%DMtt4uU+$BG>Nfx9maq~Xj{0VWvAw7@`R84EXIzK&2mzR zTFvLyK{;A3mvY@U1lo($W7O&%{zMiI^#(mO=Z>_}#}6EUG0od*_*bVv6!~4o#>T6# zRb$+SC1T6&?$2YfdBr1wss-plqmf`MEp_b^CYbRr_4X!igy}gU`CBbD(e|Gd{nt?Y zgS{Un$v)l7f8zK*D>2h>{4D*M@mDmMO~3Thd|x;u_{Niv@fCUVUj~0qF;nwHf2j2D z`||^Doeq5s4hi|w{m|cSf25gJ_$MeZTs58|uk7rJNTC>a2KRM-9tUnWUe&L zJZcfR&2r*sSikL$ca8EwCOPQuwtv<7Gi8_lJp6W08y3GN=2u}HhPVFhfl8}t8_xju zVY2B>({?8Dz-5vpR?l0z4T9^dACvT}+CSq3t0mTxA6Z|0UCr2$DQ2xJ)0!VST8*L| zwfAxrpCIQ68RBTXk#Eo6hK=7-%RWmzZXvtFYohgwz5ZEt8+|*6{sPl~<~PDVbhzA} zGAty2`!(0d!spBOUv6z0)tf;`mOiE3m&LAu@Me8}g@uo-FXu>qUER4;qv7_e+s9)3 zk8kE-o$dN-QF;94y;sQx+s_AI#Qn8QcFv=%Kepz2>Civ6-a_xbuMV=V6PAzeU*^b+ zWDeXL!v#++W@NYTrS7pg{_5OrXgNLl=E9rXd+ZiQye(QNT#f@HTY2x8#Xm>NO@*&w zdv~F^1!VNA=8t%R|K&}SKf{6*A>psv#0BTxV))l3DYYFDM87xR*$Nz+izd? zy9FvcoL}7uxRc@cFyw0F^OGG60=%XeZmPTY?@N9B@*dcJnaazw8C+1XWmcc~6NmrN zi`YOJ6EdOYN9%ZdprWTvHgGe71$WJH{x3;qw__nmGyPHiUP{^!xnmG&6yz2(LbZ{ zR-Y|0P!5xs&jkpcnh%XnSCB5z^d&h*b*P1=PwV8{2q6?^SJOMEtdc$#U&w2NM5bEn zw`3H!>I2K5bd3LjO}!WH!1#wgx$s6 z!4q*Zspx>16Jg>O&Q{6-|8z^}{I&D8Y~&SOip;YVKy%vGu9HPkVJ_V0UQP^!=!#Vn6|oG;M5^SwUiJ)*s)!ug>PTfqD&Z6iekkPpoh=PrRRnw@gR-6O=o6F*YD& z-oyRD-yYX9Z+Bu_9WtozbzmEcmF2I2KK!1$1k5SW+fkovT3U&xfPhQ{DLBs5A66W6 z)w0e_aI4y3Z@S)ywY|Oc$^?jaa0+dfiGEH!cIjH*4xsm$sJ6Wo7XN^!ja5yp!|AGl!;DW%`l4W$ zdp`@sMw&U`Aa=uG#}7d36X)OlaUxsAVpk=}qZ8w7nJ^vm3v2+ReC&&5?04-Nff|D4 zmtZ+UL6_!t>v}|jCG*6_YsY!4KYi^66avt!GrBFuKjQI8qVQcSnRF&EQ7lD%eJgSaAMA37o-bd94Ba<9x+MYTNaR<#@ez zOs!p4-ZYs5ZHL#mM&Gtw7h$dAqLrf`5HCVo)P_X|wOCmDly1c!3c;lPv0TRN+`4_p zGE#)QZ%q>W(Z^C~p|{$Dzv;<0Cyh)e?Y4K2!Yto^3bX%P2aW%4uKRaqF`clL8<4L1 z>9TP)$Tza+M2m1K^%QBW-)O!~XR=!uTE`~uSXOivPL!S&pNg*d8i4d1qpmWvnl5+x ztw3-98#jiw`{;YXb1UN*WqftS_7UAeTANT+Cne=lEt^D?IkwuhgxMvW#LRYW4hG-5 z>gpwpmu4=go1w_Y^uCjf#)=!si;jurmb@0D zkH!-`s#3>`DSKwxvwFBCMvizfLZ2J>%G{yG-o|z}gph4(%QQ=Z<@qI<1m?iudkKtA z=vA?`OFn0Kv}wXm$}=qCtDkoL%CDj0iI^qNZCyRcXER}kri|4H$sBqO#!sJBsT$_< zs})B=Y}5{fOV0fsLr?WJEor7fwKyV?S|#iPJPY@+%q-0>m(%KJSQa%B**z_XqQyU# zTxcgox&7wxky4jh2zH`-b|O(lJ|h60Q90|(g|ER67v~=h@UgGYUTlKa@qoQ43DofY z9a{^=UDo1g$j+66mr64M8%Vo)X(%DCuF`%ax`pG+2S1+6)?h@er(ymmdy}Z=q9B99 zn;Eg#6`FP?MbK;%AqYPZ2CT8{HEd>=_E-8Q>1rSjCc@_})sMD(YddtVj3?P*Cz@xj z`*s;duteP|>`(V4P_IucTO+boM(Mam~F3B*~xFN5E&OCKoHr9ZKs~a%@$OEGvbaW%Z?Mv&(V&Nj{bkUkZ#H;OQ{Ax{twHUoX;|k#>ib%Cf}<@+%T-Y` zN9`j3n`mCB>3&>i+Dt{y6MRGl0CN|%(NCz#c)eKG6ba_3+WypR+9T`MSFVX*pc1?m zX5pVtu54xBYWG9m>>$^`67!Wm9KjSg6`=2?~%SD=BntYibV z7%>4i$y8&b`Vu|950*@n;$dF|;YyVxXh|jPMSbgw<;U4GdE5-5bj~HJ@;ezjWge2z z8XhSf1?QfE{!0$7Z_6G0TQt|{9j63vvzcp06!%IYHQjZm4&k8QO-Hw_I)JS3LVf!votvSdmQ%; zb1q*KJh#S<2qoCy zauK2?V17K(Uy?SPr96OGJzpb4b$0$%8NyRV2j82UB0?1MYO$*zs!V$#NBxg2Mj(p@ z{?(A#tHC#pu6ye@N~|NX&868ccxx<)QbZWA)k9Umlyb2ZfT39(cU+T!U~do^MlJM$ z2pEq8t4lK`-&sdifFsaUj?f)a)Q6=Ejy=Qa6a1shlW*B$BNK(dSjNr0cl%TQ>Mz4_w3Boh zzt;k6$}N(oG<`IQaq{k0SCXnVbjqRW|^I+mk}Ph_B)x6M;5$SiM_KWAjLH03Bn z;G$?elgaPUBFczwl&>;0Y=3XG7T9PsXM$B^-Q%GdYWm7h-71l4^w|V~-1D-@I;f{x zhLL6**|zZ2v{15OTOEZ6d&RDJxdu(j&X+7T#)ZO3i|xGNXL#^OJIWu&8bmaAxBWZM zassU9A@sRxm@+a0TqLR|W3zfXw4Bbm)Ae3OrTQv`&tf^7M(F z0eI;)jWYC>Rr;<YcbB$uB_U?B^su~ATD~rAp*Rh% zd@^NI>H)KX9*-0+=D?xa7d%kuifF{^zaTsk5IKPFG^|W6f z{Q;!%THI*f*DLw{?H%!C03>3Yg*Cmu1jUodFS8^Sq7$dr$~fhBPg{F}o$v8u7R|Sx zaxR+FaB(j_>aa$|uPAQwm)|S$%LLLK1&86tk*}pjzAt^ULx%=A$nunYBZ?YDi_63- z8YD`G<3E1!#<01wlV+R}YkqKzI~FQVYEI5a!W8O_C#2!KuYP96l(9acMx|@#Z2d}R z@cD#Og=oaljF6SX@$(@=`--7D4B`y&tNgQI;f;#@)(w<$|L|PDBRA_nT3aEsJ3Gu1zM&)vXaGpRTA=oHlKR4 zcSMkfW83&lZ|_+P`UJb;CFm&k@^ka-t_RknVv{X}uT?mdW(aV0gMigcc~gX;lZbao7azk15ck5aw=kI!4d0tQ_~qf4JB1${~d6lXY?=uVJoNqOspHj}91@^?` zom*Y)J$5_bR+r=sb+!3RYx`w-=Y5n?%#H2)u&tal70Pq8}G@|c_;tW zxqevjE)20CRi_P_XSMe>GzfHlYYhDM@A2K)cu(AQBci1SF#J^HoEQ{;cCU>S9`JUc zHT9QQjvuWiznrBz7 z4DAOWqZ?rQSJs-*PFY z{oGYQ>TMjHhun^LL;c@t@QSNR7}*r!)tPH5J9*omx6+vK`3J;~6kqjkcF3JU!+ocp z2Mtcd{C#)tyeIo;iuXgzwyUes#kuKekjW_=%5;X4ZJK3*P9ZHQNh(Eqg(*>u#Hv6Qb<-S+N*>p?C6wa zOo&VxpG7|hp7MsP9>6>?+E(`o)=sq``g{N5KNVeva($_T$Uk8U>RFXuh@k_|ePtem z8HkyeX6mW!TKP1pE))QGyQN4a{RWgZRkoDsi-Q!}RCRJ(y~xH`yVH4N9MVe0+4vFf zQk5fRufy5hl@p!RS69OjsQ<*?9XPVIZ|ML3D<$@S_>knEO6>oDy)R&)GvoRA4DKYh z5tjnM&B0Z{);ud9GcTA!a5lS4m#?~Yby>Kw_Dc1)H1nA7P?$7N$>;t+q2upb{0)Xf zDK3TNgo?p7H?B<5aTqT9T(1C$V+x)Y{kx2sVN$Y%VvLDIdv_K?68N|jkW%b-59DXU z1Yx8lx?UekPiW&o7@vp}@F28{QE6W<7oTt*RBsEzP@C&JC-|N04U~K0)A-ulyf8(hxv2?t85yT1>c)5 zptmFxcr8|}p3~2vykVeTCYAgxLQrlgyS&W)}*Wfhh7ej?r0`TIL6{#RA5fktd}TW2V$^h;5QOoC7@U1 zw3U3P&^RI+u_se?tixaKR)1ODY@l@*o}H@gTS`u72fF=??9H#TTgTKKJp^G?NKuyi zKSkO9tpkZak>LNrG4;-$qOAFdS`WDSlhQ|nZ1uLBXK>vedTYOXs1sCMM~i=au8@!% zub{Jj5-lP>+5P1BE~TB8XHQK)P*6~}Aug`G=eMeof|%8VPkw$(7oQ$P1BOIAU1-cr zrtXtPzd1kpQ7Ch3+eFXi;$3~~_SG}9PJgTcUxwP@WAbx}xIRt^ih6!*8g2&? z9++DgiK&iFL*`S^*BA0-wXq?wB0$D2ffY)(RY#Vc4U?Eq%4m7cO-M;${0se$B^Dp$ zrxv)63+XDlqfn|gR`D2VDoNOkQne zJ;C!mZh6tNYxw3Mj9_JUX&zOsST2n4-Q2M}h`dd$06LtI3AWTH@y zB?=Js$>t6%krd!rMc^!4X(1tX57=^*rnT1KY}F$mnfi8;UYDiBI`PuWVKj9{Ieo9I zZr~5nq=!e5*+M7XGMnQ~+VRg>I6DW3R|a(W_bXWG6^HrAPm&j3P#5(i!S-;^(B&E& z*Y^;0!1)Ft+u9{MIrrV+>L_Nis-vMMu-V0xE~Xp_LT{)E6OVuVitQ{u&Yh)LQ($lP z;L;m$Iv+mg+J}xR3aE1FrpNTEa8tR_NpCU#g`2^qMq;gNvS)Jg@)S8GTD$UF&^l(6t^ltTw-6X1I zMbX^3y5ljQISvaclf&7$rc0sP)h~-Y>m&ZTsgu6^ugI7Og!XOuuq0c9Io;^%v1vxV z1@aa(brobMv6sm@zcld0m%d<%?s{(8Bj&OP4-Yng%7za-g>x^=%GG>|)S`^*f8OH?tUWgF@JTc7Q!4Iwq{7t zJ*s93gw`C;VJ}mHwWCF-d~XWYv8qe!V=0HQ=nk@!)Nx?i0q!ole=aJ0qaXzVMRy+1E zq0`VX^XxPg*~{G|OwEV*B{PRhqhRoNz;?oD~> zZ4aR*`5a!(^It9b=(pQ4$=MjKP5W6N>=dY|=XG!Xl>T?=(2n6s)@ zc+sQN?XbtzyNZ#T_j-hQe2mDH@Q=M}pN!2FVJU~i-{Dh*;9}WO9)5a=QXOp!VfLGr zQ4q5Qshsdi$0@k1^Rc6!_ViZq+;zLu^4hUmuhkd&DAN;6%$EDC)aQHTVy`teV^x?NsEJym1usK6_ZUZe>xwb`9*Uh1Sl;l&gHL)q-M1-;+6?|9Gz~cS-iE& zAO?nAk?ICXkJQ?4jeDoAjyu(Dd&ycQpuzRR=a2j79a7vdK$(quh0k&Nf@a)K4Qy?) zXXA9`<)amop$x}cmr;?+c59dpD;N%Vm2T;lFSDLPHW{6Hk9HnlfCf_68BQk{lKlj0 zb2}M$SBg7Jdg&^Ii@vX~O&>kfVs-Ay`#cE!ePo>qa^j{%a##DHa6(SPI8(AmENB3s zzMmVKB_l}x7j#E~2*2*o} zi#WPE6Si>|$-Axy_keeIOsnM9yQP^+y(w3Ki^^?6q@eR-Iqn^cX<-*OQFQHQWN#8H9`i@6SY&(Fu-nDa@FK;)I_DB6ZQXu$v0h5o+NP)&0O2?%zG@ANU z>>tdaTkE1)zA1^S=O8QMJeJIR2&kP?-o?Csb&_D7xprKkF}598oL|PQmNu!G*FBtH zgZeRM?|ngEjrj%O1YLtF3n(i77+s;x#odxfi_FEBiw&UkN_|GAx=g*sqb|k(Qx3_M zpN#FjddeoE^+~a;osL|%$i2cE*AAlWCQ`ue_TG$IFdtt!g3_TCe!?8iwP3yP{;ZsJ zHqkpP-1>9XRZh-2u}t75tAA{=8D}CnYdnbftzuFa(ef6On^mtl{4^!8sowqet?ci8 zg>8kCoR`-PE3?N1Swnu9;zGl{%kw#;>_fL%N&BEldm-Px=0(@B=-}6hC6>*t+1qj~#G)x;0p$Yh0 zSua{(e4f`YS=NRko48}WQ73LNGSIBkHnFQ2J$t=)2R4u0S}IrC9k`j$Z3F2q+}cYu z)1ks}4%$r!E}W8EoB$21k+76eD8utR+7?nzwVE29Sl#d9Mh7ci6_Q>dty$q z&z54)$btyuT}Z02@GY8VMRiwH)s(VWe{@VOTZ)-jqJ(l$wznnt`|kd@R7Bzl*^xhr zytU?RAHB1%84+Qsa{KSql4!7VWrr~Ch?9U%kF+~R?SN~5rMj^e&P;!0uH8}GHoR6X zGO9^Y>dBmfm6u(WluVI~o`Fi`9P#sP0ld>SsVZssXq#O-xw+x-j{I3lvomzb*o|yfjGFlV zVEr7L>F7D;gqO8Lk=BwRpjzP+y1z4i22HRA5!EDuzKg7^Ky#ccvJf7^kbKBg>g8oU zpk**-V^G0IOl^FO#Ai)cf})AXqKS)G6-6r@@*^EG$Hs>t`LhJ(E>e=+$bJ-oFTp2awd2hiZY zr4Ud_@e!+P0B1LvPPMfnbm4f}svc`|+sf^$k*0@wq4?G@8~<0!x5U&6E1Yt*XCoam z8Le6`#V1d3;cZLrdI?69ISO0&RoEGxoCPeUO*ovC3(m|(+i9Y95k|Tdw^>yr^v<6_ znKbS*);B|@tj&GwzyP-O3uzfq>(sN{`;D1Xfvc50qQ!)eB0j^Av!We62N$n*UciE z&gG9MiV-hRH+6j46kTqG`Z2Fy(-&B)lB_2VV7j#TnA{5*Tx^sV81KIvVuf>{%TtWS zUfkK0-5ru2mE2Xz^-J1Pw`$T7}|n=?$R)2Gwyk^$(5-a%p!jGDEL}^j<6gAV$*ULfU1=$qMVX`{Tx{ zs3Nsi#@1394t^1g@Ma47-ePdSED)Bz1Ll26M1UX5Y zheFrt6n;rOH}Qhi3rZvw*v};ViVcDKh<6{r|8>!oESMe64ro9^SB~{c=<{&C|~X z|1@Ma`_=W+>o;KLR%pqc_1_y(kc zHOlRQomx3HRNpGu@B7;!$v+Aj19`%|v2u~H<$u}OugqkI?w{;z;=kvt(a)D$?N*69 zQ1)(T!w^^|ypaZzNlbRReO({@RN-VFfxZ8s82)PdHn!tEhywEK7q(e`U!vEEK= zzdmw`J}lGSV!ZDkV$a{XnYo*gd;`%wyn$$Cdw(tUZON~n@AlXK4be{i1<{85VuNqY zpRfK2(V92MAt74Q8;G`&@b443kj%>d_qpQLRR1?bYs%)GUX*zY(JnuK$NG1O)?KOX zhsbGK`s5$`V_{`K^an)y82m4Ymh;DMSg_)vL3aE8;ol+Jxt|cNEUA#ipAsu>f?(Zf{ZQpQ5}S9wGKGY!+Fk=27TRbo>3sG{t-SKD#ruVI%Jz zqv)yD{rGNY<@@FcIjaum*LMScqhEMHsT)S|arV9blRB&c@w*S&hMzuTR;ioM9SJ*z zIGP|K+y$fhao{bP(=@y}lLn5>~e?w@kZxPzfX=isA zLZPhnkqg7*Ir1?4?eAzXnnI{^oIh8K`$o{abqdSRWd(Ny5;JKQpXFk`FeR0HH<)|#pTz|P>y&-6kWB)Texc#0a=dJVA8-EJZ|M}^&|GxU~ zzpB4r`>TZ;sTSb+@Xp=)C&(%O@#Or~!hhofmXih90xQS=oe$Vwjr@P&1GX{k3uGH9 z`n{|A^PTu#{k=3XHZnFbF<|0(!L854Zpfq0#BIQBh;;dC%x0p`WoXQ8$cpp<`-16z zz>fb^`EJ(XUo*Hf`_L6$hk=LL{48%B=@hU?Q(q^Q zr!P4+L0?v?PkeygG}{zCgcvXCAd;siIgyZpT0*Mh82^E>L|6=Jmp>l=k*t_63DeTs z&WI`T2QW7`H)ZTU);F_0ohgLtuIQ}yJcgmA@Y_P54S%SPM}90B6Y#ta2zb(CA{O$Z z&zUiJVq-#$?NY6^G(g#lFI0vkMQPIQcmvh6HB4;`lm@8b@`w?vrVc&eWZTv3qhP7YJ%cpMlaUwIjX9g4K1Eqt*XR2HP%-xs<8quvnritY9t)S>%4-yxpZ_gYIk$C zde5=zO?7!~+jDiFd5(`UpIV$y2 zr9HMCYboEoYl;R+r2tEleqAVT@mjT~8wcN-9K>fcK|;C$?u1ZGb|_K&6J{Qo>s{NS zJ?&xT%1#g2IS|{0G%5qj8_5Q71n{M5w3{?-$za#Epc~T*`SS}V){(vl1weZt$UqC% zZ8plhEDqU^#PBNK@LINZoby>J|_PHkN?YLBi(&!E%8cejs3^$1PeEHVfY_5qmb5{9znK!NmVt;=wLce;A+=uyW zn`67-3-pFQ1u5$?!gi+3r-MORjP>gVJ#Ze}gK3Q7d9Do>UD>W;ErS4t=?RyC5rIey z>492v>GhT#PJp-5M)JN4>D$?ar5%uUm~p?sw2N)sRFQat$Y=^P+v(METkY8yi?_*q z&JO&cPQ=ToGXhh&;Hry=kJ*ZQQ&m2I)k><5Ot94*AqU^Cg^Sp8%>6U zd7f(YO3MSTi6#x(#m;Otxpo%MbBcJqRwE9t{K+pw@$H=c*l}}*9Sd$9S!|J8W zVU6Ft4l)!%U|z5`^b%D4rM&Q1Oa!c9`NN6yq#+f{e4)nBI|NBMOyulTl9b&dHGx=}u?;M+>r5vof&$(rTP$7G_cS z?zE=kQN!*yjLSs#EE~qK4KWXgPVa9MZIgz#x|Ak+B}OSixyzun+T#df$1}{yJgA10 z?A~?4F3fjaG2Jk2D)unqM9n&F-7uG|^7PXC-R?DTx+uw~{Bpm;5TtgbD8oklWi8>F z3RI_3J2C}08i|{r1zQ%?ocdrCdai13-;*oshbqDn0lVv})+vIgY2)GKS7)DVRmdtO zm*oV&hU-E_#5A&V)9=}tVP%2gkIpA*EP@PmH00&SL#3+G^-M#kt0z3IzzQ$v+XHA&PymiYp6b6QI#rRU~ zETQH7tWA2b14Bm_kq6IQo2CqBnp`+lp&_0sTuc=-t0XPD)X+IKMN-1Y*HK)ld^Ilm zbLi#J+Eatge${qCfb2nSP~3&>Jb%TD?;O>Ko1lW84Dvx`uo8zr*6C++C2 za?z;7^XxTL{WXRf1td8M`L4bxHLH+fxq)tZUqq_D6p$J}D?69Q6vt?lR?{lsL07`s zn_vNf9usG71rb*N*+`4hN78b#AfDbX!my9cD^}&Ru{)g=xQV;>*vem$!V`Xe2 zI?#ksMvUokU3E4qF$AxrpCozw7#Lt7x*jLgUvDxT&5g*?DOXEg3P1?3J-T?n>z1LT zKr6SoV$C~ny!Ot``kU#6IO^nLf)?3o(v_;1Tb-$c7_FNhX#YcT16F~77qv+{jK-zA zp$pgD&7W(rYwvoq5AAx>PvB}qljWdDV{(M68W?Bm5H`f6KHI@nT~%TmWkS=RBO^0{W`Z)XJTlldNLlE5 zmbaK#Kyvf|i-)c54+Fl+s_O3{rl6zD;~w~NlUleUxFY=utjetNn$OxbTjiCk2AF^_ zuBVLhL16L8jKe=PB@z@gg2HnbzqCXjw~Ro%ELo+o>dbQhKA zGK&VUH}d8-r9~NghNgL!_NX4_={|Xv5oNvgldIL#B5iFVtAVAN42F+TfI>rz^maTw z*vru(64xTyK`9R^mu7gk+btktc>AIOC!)VUd62OKnwDa4cxa7#+A5nF95XqTafxe* zH}$ZUtf56GUQ@?_fmt>Li_~6e*mbp}RtZv*9b{dM<}6RIvd1792)6L2Q#cQEm3OmA zub83>m<^8YaRmSJ@)>$>P51PBl$A-E>EyL$+30fJj_cXtUP zxVuYmcXx;2?(Xgmw;?lY&ArY#Yu|m&{c-Dg7&W@P>MQB$8qNED;((mZ8tk<_>FpOL zGElbjZJ>~E)O6TF$3N;KJkL68TL|j$fV-buaz1~X?Or00#3XLKC1ro1eAF>#(IDz2 zD_W%D%GbSW6`c36QG(Z1?~gJHG9P)f)UiLn#BupBv>`_Ai!jGNH`|k~csOh}?%R=BKq~^ii34~C^ymGUB>fRbD0Gxy z^cRb|kR3vXcitH3v2JH9inYwjbyq}Os%}X2DU@F|EL1li;i$wk>g4EPj>qQi8N0P+ z>h>rLVX-brE_OO3bb|`?{UunhnnEgbx70!0(joQumgx8v`PsL7K843>xg2W`tBM~E z^E?(Y(J_%Twxv!y@On8!T|Eci?pMD)E-Cc9VUds5iVoANiqo?4r2bK*KB+yGKr_W1 zaS>b;`5LH=9z=qkBqU2mu_r~%m$+G6-1YTQe~$Z;Jl)HIaore`H|P(Ax>Jt6nb1c; z?>WSqJbiCY^_ouD)>tzGUh|W87uEdN_q- zRBWEZ>rs;nb4ekZo#~6te?aRAw@kEmZ4iTU0?J@NSiK?C0mP72k&$j@k&|wseajr< z5FO4PppA<20pzl|GP?KKLRdcS-9bRU0QwA0$&YpXtU=92IXE0C_nb zdizCKdzb^}?Y-Aq7d3CRgLsrn#_r*R8+kXH>`b&QJ0Cicwt@uxoK2M(N4Z$a4+Z@p zxBj|hv|iGT9p;#oPrfTxrb!YE9kz9SvC?WZ!)luT>XQb! zQU>LL-vSOS;%R%bUrV=Z4H|^SER&R?qDEmt&2x;`QFK?!MOZ=APAjn0R4Z^$puQ0y zMl%zpLt%q$7txGDgP0;hl}Cdj<LG3Xwgsi%A0Ls5)oRgKB`Z<_GWJ+ z5g4wPpfN;pgd}}=b&F*c|V!gutkLglHGb$U1O3Z;4#2It>M`}bI1%E*sG=1nRl z=m^;qtkIk5G*B%kfI}IVG%X_pD1K3TgD6dz1NXVe1R)@e>WtZD=Eu5D{|R^2X=#?Sp3g)xzB1do zXTTdr?Dn`?bbl?OS28bWm|)w-K+2`-mzUjoAVO;4XGz*SbF3kPfZw6M0v)QpXB|bd zpFony1eX>|UEU=KciT^G*%?sFb_rSbZn14UE<_V)&^V1#XJxH5?VSzbCEGDpaGwkV}Nnc*W40^S)vjHGuprKkCd!E%4qEywi6>z2c4}TC0eU`#c5bnCP^+4Hw zrXioJcW9s??_od#5d_lruQtQ>#F(AKG*7zH5eI>0AP(Mur3hC+ln@)~|E_`SUdEUUh}Y+m|!@t3_Rp zTyDbxPbAE<_{|_E{}kj^6FCss4IN0|LOR4Db+7aeE&>kSH@d2y2KZNU zuk^yrH!`ZO1P8gx`Nad$25`9#0CxlKw*FOsy?OuP$8&)@5SC>&Bxs1reElzAc80id zJwVR4%6j;9HW2-f;g!5fh5YNd-$LDCjr6U|{kLOep2I;eJc^;2mnS!}H5;0st zmcNFphk|taqxS2Hf%_D^?l}!r9?KE|y(enr2UcJ?y0auU&uyzXJb=ZE4mZaKBqDGL z3Ep-p7a$B+7Z`IC`#xY%r4X*J02AV#IjKZ_@LY)|;Qm?qqsifcMFM0R z5bkmX(uY4yh!^a;JW~*o*)u5H4j~uh7WK*&+=JhoAgF_rBt=E;1!0{qk(OKXh?1i( za`56WC;C~!(U(i+TYwUn-pAHcc$lcud^{Zxq*T8~mmQy5u#*>#Y0(F`v z8eTnd0~Xu!K`2J(8-P+B$S28~iq2GlVUFx+d_YS15%KByOS96z$LB{iKC~AHZQap4 z@Y}cN3n2FvdvrE;-ZBn2I&fuT;kM+dDCUoWW# zSb{)xsIH^>=O<6~o7FWb@O@}?t3A+|BVRYY1)hSJZ;Ib>cFq|HS(BQ-8WRW3JSXrh z=ND7>k>85-9;`viSu($YaZ}-jw>*-G-{C<#{aB#0u=MVyPXAg11MMY$TlYvTi311i zMRyW_CUkz`p=6pt?QfXG;sAa4WWwChE)lu}GrI(cZ~nSv0K{>na6qi{Xh>0v0vg3b z0T-RNYme#$E*HPCp#i%Z2K%az4=@*qz9v9i!U;|WN?9OmX6iV|J$3e0EWG6s#DR@H z?^310ays{?HD7c%9T?}mXo}uNSBj!`ZM2;jdUc__|zcMR@HWW zXJ=Mj-p(X8a-Bl*Bv&+CSf05@ct%cG{)_m`fSgpa#UvSxR`t|w>AHC^Dc>b@lv!A3JWzHiL9l{U9?utsxvhm_&wqa4?^hG#}k2Z%b z`qsCDTbse`>CA%*x=qgk!+vX5K7qw(10C%}z@hs81@8LaC>j4JlXn1jb^dMgt}J3- z&Ouy`hWH}TU0hfcma^3Aq_)_T{GhOpY$nq{dPO`w4n+!#8pdDcs|7M5ZugB8iaG_V z>M)Ef%m}-f%T%1OFWiu$fo|9q;5gttiZGo zOp{bJN^n3BPJ`t@!WLh;Af{l^7~Jlj>0y$?eddLzo}G}PPW;%g!idvcq89F`5N$Gc zT-k)Ytw96_wY010Lu{~;H)Y){Uo32TSVd@6;h^K)Oc3Xf{lP6&rmeL}Nvh;Wf3J+k z2iI=(V3&sst>%KFa)*cDVdkH|0uzzBxh&49)oVE~ri~WHGqWl=8p#DV(RCkAd$;Ov zlxB>2GT}{7Z2im^A8z;(vD44eZDR%#?{rW5?8F?eYFqi+f>6brcO@K_T2C_gm2WnT z4yrRKC=av;S?5s~9;fAxlhsY!X3ty3M!6HPl&cLl(zWCW&z$JV4feKO%(yrK_{(Kx zZM>+;;Ia;SnwT=#H2K(1BPUct>9Ni&$#LP!av4^9! zDN&A(QYTs%4v_*$*Gsda>3knLjAxMPNApdjq8M%;>WS^T_GYbDjqPS?;i^zoZnJQp zHcfHaciT1x~f{*Z0GY;@kKwZA9FVza-6qC9WvrNu>B0GTbnr# z`O!;FIorsl>tk=uJ3oP&KxgBkfmZ{| zDBaoQuNHi^u7*UI92kPDWPcT0(#Z6)V9uL~iz7FSiLCE#b#R@FQ+Lph<#Z{|CQHt( z;_g(2p3B@s-0`Nda2r7vrOSl7zQp-x2499Ag<{fX=qgz7exb z`>G!0u<~7?UT)nW;NajiN>S3xz9C$uQJf*k_VV!2x*zyoN`<_4v7e>-ijlOxe33>^ z;LL}=GAnDLZhfg1V)jyu`)HVW(%6}G1*NSon9FUYQMd%kFH@)Scz@H(+McI8vB|Kg z_Hp%bdPZ-p&B6v?#QL<>1aVxqpv1d}OQy@pDr7ilPfL}5Qp6X!?TGhJK#6AtweVRV z)vO*?<*gtV1cfjyx+HUspc2SF)YKmwKlZ)ywfJl;?@}3vX*83l%j8B|$|1qvcPxo) zQ{cev^R|vlg6k$)k@(|m>`6f2b{*xhqk7Epj}^ikanF6d8&Lz(?QT+3pT?X)qL3K%o@l-C_4*7S@smgua#T+N` z0r5dGO(o9BC-&DH>bRwI4J3uCj^}L&iME$j>sgly4>Ybf*8yv)QFmpxuDUMA#p^0- zUV4?pF2>gf_C=`d66}WCR=P*W-!gC&7H;<&)18QoE7k}wakF`2kncN`$}&*A8%G7Cv6osxJsz9COw%cGSuPcnXFp5DV^LNi~27cEyq5Hvt6zPB|erL zCvmLvz_r@184*$MG)+p=jZ3M~oMHBPnl!Px?hJwT?Xl0ep0;hJMZLN(goMIS!7wSOpYe<$F$MY4G_h1j_tJl7iiGpg<;D+8W)66 z_U9Rtu<3l4OSp9tQ9-K3Yt8JJzML`KXSJu%XkWAI&H7U5OoZGenK&c#gn{iE9RiB%Vl&NmP=TPxoy z3roZ@7+PyYFc?S(s825*2=uzK6<>Xi(qLx2%GsP5oG6NAsOz)IWIavKC~YScCC4!K z^0D0*n+WSZDk0du?D}BqpQPk>i1-F~k~h|ZkeHmla4e$S`dGc%#`05rp(Z#h0c~G` zPNC*K&$xF(zg(C57o*Z~ZBg_PW04BDU$j8DE8|QCE7Mw+W}dw1Wg(Pltc$a?vol_B zG4Oo&GSYF4K1Q9GeYtS_{y3vb1QE=bjczIAS+jmoL552}oCg=q11@dR-uzVIhsYhDpctZ&0L7mQvyDZN@c2`y|{ zU!Q*we{hHsuz+8zus~mYrBKwo;;ZICw!cs5M21pD+>I2;fBY%IA9J*HYL((_%EU5^ zpi11wXE%azdVj!Xrp)-@&RNewu7rLu2`}(eA|~Q&AacZi&km(}9G&fk-sj_d(p>&y zW#@5dA?n0y zN;7^bBz!7cM?${1ZOQ8YmMvI$h&Oo)TjGl_Q!qP2PGX-)S#2rRRHc?I`W@j*x=%i@ z3-;}%Vg1T13py^mDpyr{cNA!E{3xyVnT}sCnhf@n^;XrYKo@kd%&}R7olHjO-G8r9 zk&{ePUE3kKM5$?4KeQ+P_%f)vjD&phpcHl1icr1QQ##Qm03% zR$=~QplMMj%Y63u4|KM+{1IW7+(=13$uGJeEWPOR$sdJk*9|Q7=3&wk@X`Ai7jlI~ z##@;T&E}){cSIOI9ENyECm-=yBjeZFXY?#=d->dm#9?K{TLtcg21tH_O!s!ku-9_M z%pebZ#~K+kj**$mypdy6E}k>IZL~Iy(W5XyXCIJjZ{wK9D3d;$^QI=yDeZjc&F2OR zcyVM>4R`^Exct$AJ@VkF0AL;MBHo-_eh(gs6z~b?j7&=msF@=b4x4IW&zkZ#Dk3P^ zg)5fRNV*v99d#`)s^|DfXKIihoxXyUR(_o8I(}=tm`6V9qpjs&Bp7#2z6+4*BV1Ecnt8@s zg!_T)Gvg9`)Urw6!ih3c{Rb2s;GHF_-SAEX8YHX4@xuc9*ta@Hm*V;4#7NuT)hLse zG6+kA1~{=n_?-o6F07#FN1R+()8QOsN1!Iy=IW+v^m67$+?s@OsL&OLfUnC1d|&Xy z9p19l|CDwdIUv%KJDux@ftk5=Dl%+zLq6ClmZ)aBZv@>!JFa{)x-g=*vrX{Cn zWb5N(F2d;w?+Q)wUQu}^w(3XCYS=Hdw$|ux6Ud-82SoygDt6HFBG8KBBGLL*z-jNa zn#fv9uim#z+nU1^etM;v7gVh1d-S;QO0ZVW&%A{TYnzRLMFU}ACGu6O(fFGS(l-+X zZe@^o%gtJZLWQjvnwE_W|F3z{_4nezJjK1I*svXMEu8q#nm!B%BkI4`h;P4eyC|us#HjSdedO zE{xLIu#?M+C?$|xh6gw~{UGmQ&pC@KS4lth5fhQ}2(MZ~Pplmp6-hPAAY*UhPe>JC5BJWUT zX`B!V^>A3$iA0Gxcqu~bGr%~Q0dvx|NS^<8ntf;(~&v~ zpJoV>e!Vtpt$I&`t_GGWDn~Q>wub_hns5jmicY+4iHE???Ia-$;Xcw#XURR*89P@) zV7;~8$SAc~$RQaxhh$FzF9p_1vnU?S*=8uR{4>G1E8qk5r-((VG#wZX%TM({C4G} zJ_Ysd56IuJhW5`g`j~P};!HZOnVEHCaoc{H=+`AZAgdy#?jgG{F6A^hiBQoCv%_Cx zvhY{mn^C7_q^GyC-#PmArUspPLq`|>*Wj0c;|CiO^pQwHrwy-PFS?4P>A}W$)MTNx zc1uy7QHZe|GRYs$kWDsw+iu+p9yI)&Ish$5vCueipWy`zMLxAbjoI@*7Y05)TdDZM zfX+3)xXAxG^>o(%88-Ez{q-bD-w%LITjM@2B3Fyul>*3E|BFi>(B~|SJ_ktRo9AjN z7Dg0@t)S-hQZs-HmBAGf|f59vvuJEV!di+oDzN|^*m{tM^g z4KgT>qLJZ&|2bqXgTm9LYU?@Wmw{+zWiO62DZt$!e-u7LdG59Cu`K3<$FRHlR9ANZB9-!H zwZLP;&w%5p5`fs%#m(Ckfz71{*ExXmw8uBJ~;mjyz@HpcH}XmE0Q5k-soQ6~(8U{-+TJ2$eG!2Fm)(c$CI4Pss0!=|*^0 zlmLc;fu8y7IQH|TJvZi`(ocCmvYj#H4%XsbfjZ;f#TFr0%J$b+osz3{5FoOiyh9j< z*NgyaI*9RCojV4um_Mzh7_8=r0q1SJ{AqwSLwA+L=DzVg9t(i`ywkjy(d)3BLxMH} ztGL{~11Cv9Dr~wppdkhtVgS|?hh3!qEs+Pj<2#BUTfH|nXlTwH?)45LuE-1hS43lW zPZRj!88z*ueU35s7i!uW!vg~C{3~jj2B4-<|3FP4{)U=rJ)x%b{{}V9`3p70ct%Zg z(%hcb;!yMYxE+K$QDMmg9;TTAaAv7@70{S=pJ|PG@de!M8d#SM+A8%x_W2THx5EV} z0W$Qi;eN_fY%MaOLHaCLm59L6XQaD!N~+wRM9pIf;28q#ATUF|1dY7{e}XODl{)k? zn%!pVoaQrANkGktOK^z1n%&KyZ{(O=zVtL8O?T+y7ol#VFHP`1zbtB}r)!>)`zSI? zFi_DXsYsdF-rLaFxS8;b{;Q-Hzn-d@R0ZVnC^$|IYm&F!O#p4QlqJ9d>UpoL{`kG! zdQbZMAt{a=EqofSdwfX*C*8>T5(X61Uk}{3cpklXq=D;ZGXO2SUjV1=K4{aNz*VLEN+eBa*QjoAS z(|%OjV!V}w!2)*(2U>kf&CSBeAb6n-JggXQ?B;=-FVD@pvxgPE9FoZbhJvRD4|430 z(gtn`-vWT1VkH6SX+sIHVPHkI5`Zdo2JqggEyB<2_tC)t=;=!n?w%*~^sR|GApI-N z7z1dCy^`bAd^66Fu_z!M(a(c&`tp;g4sf~pjROt#Q95{Y@l+jvt0LC9T8~dO7>VS9 zKsS6jU?4ubBW7`;Mf@X82G=8thC=!N6JwjkIfijZ8ES6ErcTL~H1RBc=}+UQ48ED; z`2i;%^3o^@Pl*zdEoO4q#SVr`?Z=& z6{j7=Fv?Q2TWWu?HQ3gbK0MEz?$mS`ywG$PatHBG4-HS6zQm)F1VO+S;S8Yr3V&18 z1r%+m{=ZUM{fCE!C#6-Z|5b5SluAVR>({TEpBv_+FYO2S?Ygj$xG3I2>nUIC#I8gH zHewR(h=@%zoEj~XR+WXLkD=f)BsG6#xA{iAdcTvLh~YPii>7tsqNm@!FEyL=(u}UQdJp$I;z7oUvY%ej2>7wXVi8h@9u!@r zuM0D7CzsQgU8a(op<^j#K~C+)<4tsmsj=$;p9<7j%RR`_DUF;kbefBSdw=qeU znx|f7lf*&ib~;=qdKBxnf`4jeywZ z+8|5ynfp|qTr^QjyXtP$l6af6h!F;lb<*zXEb7~ts+2zYhWm;dR(%5mT8*B1Ywo#u zrg$c_Wf^654F{T91%4S3Jf@13>3TO(;qYLb1@nhlBEPo_>}=ig2c^WIKS1TV@haG`2h$O1z_ns)K1O-!`;r?34|r?0 z^#SpiexRx)RNr9uaNb`_*MZOZCHvAA)oPWZlU^M5RwjB3iu~vUNuiaXK+vaQpN2> zh+b8^K{Z>>OBN;(wkUr8`Da2P&d}DP0 zw>_?_FEj}CVB=<%aLdkH zZ7f@6WAuQ6a=UR2a|I=5MfXApo6+^y?z{AW$(!P7o9!fgdMZ0HzY5o&U%_#zDy1Lk_nrOzfXBD1>rngH0-pAcvX#@l}V`4Id};mPeq- z+SH*IqQ8(91*Q=63m1&lN%}cn&U)+~_n4VxL-Z*pC|Sf}UDod&L0d`5K~8=%&CNWj zTu*z4Z>}i6vDE1yYUUlfX@KZ*XEaQ%eVl`9z2tCe(9e3%c(KkMx<32ONZy!&RzW=A z_J(L5a`HMgVA`ytr^~RxSs`nJv#7zalT2g)Px%EEzJ}*gil6d1Ij*Bb{_PY3tY+F z7xb<(lw9GYFHYd6e(64K7cIAXc@L6!(-quvvsk3Fgac{*< zj)m9YCvl03;FhF4VS7a!8^xhI&O`{VT5oT-poHec#eMrN?J=sw8>K$-U)(9{FU)QS z{NL`6{|&kHkNxq#@nFLecl%%=u^_#C=H-j9OO!k_k7qccUbfRXO0_;`!tC~vuA|^^ zosM}b#=&LdCp{a*{a_|azUtB8ufuX`4-pvFrb{|&XBS)6&G(Cyre)xwPZ3;~TH`j; zt<8H}YHPjR97nycw^lZ>Dh4TTodxN}R+5l>nht(KlXJ0$l%#qb)1l2qm$Z8*xNOKY#ezZixbK{DyF zVScSn;R<7W=#ib<{a{_`I?+QB%zJmrdptqIs7K#8fTu0d9x&pzrI{y|SMzmG6-8OX zLqBoL-3;%h$(@IY_268>cYJQxf#6NM8WjZ+4ALT|N)C(1VI*3uW3YM?G5)C{+Pj*g z*kpy;iZ}6BQTH(17Tb#07EADS$-G)Eo6Rmdo^dlRF4bqoH3<(l?cuW*G!LY9^L%)5 zjQl_0lWz}{^}D}~_A&J1!SdD-Zk-@?EZ``+4c>pC)!1I{cY(#Bx`o|blIb(J%|%f) z-sk2;Ha7hBvyeqPj7YqL3q7jB+eHv%HnYwgE+BF8R0=k}F}ca_TElo&ol!BTYQv!Q zDpB70+P8$=O2lC(lDpQg>(PQs0EIbRR;-D-0WTVj{@05>b;RAtRl&3a&(+b2$A>U- zyW48%Ph{(+SfhM$_y@6PV9^ii+h2NX?xHK?QzjaD&-iT zW@#w9jEG+M`DQ(K5xG)m=)zKQw$h`w9IH~2zw;37KSnS@5Pf-y+zM=wzX}ox!L&pz zifTY<%x(xyzJ&*cGwGJvji&OWs>ZgykvCtr_&YUFC7GM2kfUu!s;b;dGj&LdGwt;#pK+wl!#uTFy97`TuxZQ;zXbEV~WVE zgA)4z<0~6KZZ(gbdBszSRJ?~2H>UddhQgfl_Q|t5 zRYMa2L_P1>uU2T1ec?(7^aGlUcg)N+)EuwNq((-V)qEMDjZ@pJsjIXHaS-nrdtc%2 zyC3al>rbPFwjmi)C$-Jsyoadk@9qN)w>bB;IP95S?v&Bg_~fraYWCCKF|CpWxKO;5 zg8ET`a^5`USsThYb(VSOko$uQGWV3{tkDhYkY+D#y&AgnK;6w8P@7c7(QD_*Hz|$| zU=Q|T9f$`B!Svs>iC} z8^WIRc^dR*j?h^-#P(~Fl_{;f9)Z@XiL;nNaKb8tT_|Hi{m_8$0rDr*-Cdl6f7GHTy5n_d`u|<4rs}^g}s_v|_u~ndD-$ zd$vkMOv5=^luXaZwglq|f-}VEL*ysUSU~VxBc!i zAU09Ws<%AIPY$mabPEdd7azFcMj(jdaFmK--R=w0O&v74-5H73u8QneZZwiezssmG zVsl+}_y?=7-g?txn#NT(xlWW>b(}D=o0-fp0Iz!M8U1qiEi%!`^}0P!Zsb=-@2(LV zO<>{gPnXWTp-_twFTM_6=_L@3A~an7QY~wmxlEd?#~jP8l~5&(G_8{nYV3m6ynRNuJi&FbyIX_;o_(dm$HW{N59M#8m*2N$hIR-SNc24rGxjPIFm3k zxTXknV&#Pdlmq*9C@3Q-aSS{~D{FoUWH^$62V|@d#N;ZE;6ZE*gvUEl5w8=AwPXj? zUq#?D=`JT;AE)_F?g*@Jz*wQ0VGWh0eg;C55vIUVf$hU?G9fgH=Y*OhN2ucs3|+h- z(UFWTzkjrBRpiP?Y3hM4I+~87(JW6>?pwpoWNe+=pmTh8MZARTpi`{ z&pIoM9NhHyRwm0bZC4#QxNf31-CFZ@?6AZ9U^K?KALhjO<5p8~*}ne9rQYN)L@7&t z%5*6+aSriNqAWinw?KvMc;Jgvzp~z2%Vpmqc*yO%UwGVT!_F+fS9@*IVtChG>>R2) zB3@XBa%VqU7?@YpXtN(w^?1^-iQ7{%cGNv{`*0jSea()6D#ulG*+%?BB`bfY$Uu<7 zVKYm)Q=*YKIWF%;-(tq0H`(^MdTnRNyJ^sbZL5d8;3j!m6s~m=>b21iZ~PG-LK7PS z#d-(s>^RG13Kz9;hsf{&6ZNFoaPoRuop?@Wl`&sU*}Z&n%I&G>!FW^Wc|&OooZZSB zL8y+r_%4WgQ`WBxv0_rVk=H@&^`|$}*N5CDbA26~m7hxNnj!d3R{4h)v5yAKNU3(L zjxK|_GxHcTcjjaxSkKWHuB z*bKA&z`5s3SnvdR}+aO80YD2ekE$MjEb2>dZwmG!| zRW=P+em361W=!$1%elzCz2YN@nMy5b`P7;4u+%UZj9J+wb=mKvCwzHmfi;lma5)iX zhXr*JTNY}Cey*}acWj54ucaM6 z5RNEpLtYlDiHfVdt3;!d6_@+wDj6%3H0fb&FB*05E^qZnc&0Qwt8o`m7q{_oeBaH+ zc-nruRl-xzcDo0+H8#8)~>MUomlURA`56cflwuk1QGUr`y zPlB`DZfmv96&<%r65XurF&KU<+FyWoB?X&ZXHcp*3Hl*vO-#7C=rKplIkbflLY+hz zc;(a`xNKdq80D(v&8*h6;?4xQT?${x&>T&rVoxl?c@}4-Xvmt%gE_P3Es}ePGQ>mT zx}|-+8Y|8Z2VIycr9RlG6=%CFOQwP}y>Z@yoWwu~ECRn~PJ2cBXDcpqQPvNreAVy5 zOz1b0aN|akE1h9baM{mZbtGfU+^Hwso;P9}s zoz+B10*;b@*eX68YbuIe>n)siZFt`y=WYlxQA~7_E8Xwd>p+excm2?1-BCv6hwcpu zE>LApBCl7>m4DbRmb!HjFQd^h{H7GsP_LPQrl?0$75{~OtHdsGb3ip^RFDhKD7vTc z5?vB5Isk5DT~BBIo6LG95x{)BY-=*W*m|siS#zjEuPg zcY(T0^DJVBs-`a0U;WnDw*=z3A!DHCgay^UokA`WwYTeX0LX#+Q%rM~vh zuvqRTTQx#;B=ijb0O?v?@dcuyCbjD^24Xr#A?@<6-SEV7>1rOE&=hVe1xKy$FO!I3 zITX%%%}sc0z-G$|nI)s!J8dog4GTRC)$shoBExbf@l}NzjYFWyZh_s+&&-R(`}5_) z`!z?q*{P$drD}7ZzyY?R!A%ni3>40Go-JM z%7$V7kTAU;#(hMjtM;T_6Z3Sc%1R_ouKjMvZ2({9d&m1-H$Qm_@&mBRg5(UphpFyL zA2YGdmr9z3&3jTcr4%n69nsEwwD{^Fn~iY4vNS`w z;dKk&66N}9AtB1#%kv7E^4%b}r4922iNQYfVFBtpzf@#Gg^88=%F>mw+0Vk02G+CV zA|g%*m;LFhQ?4qrAGWj!p*u=dGwR3%g-y8q3QZD{9Ut8*HMQg$Kl}D$%vWI$rY2$! zm>!yU=8f^e^4su+%3Y_cWL)d3o(o9K8Bo_$O*OeFK zUxqfx=1o|8?VVn*iw^sr_e;^T-tMMbx>`kWdT=IhC0wFZY+xD4q@TcgkF24cM%r5JBF`-Uyj|h#=etA9Z}MUya;9#p zv#aMAQ#3UOw~!T^T`|>ob87=oSKPljA7<2cQ>@?Sn=c+Zqt3QAk`*CyBA&n9s*j2$ zS8sLb05s%F7W%LossWI_|aJ%Ww*^1B#~4hlPT4#=EWOLbm(y;Qa z@%2i4)*h6J9+mGXuLU!Nzdo?R5{$gIlNHM@*(2kvPTeKrHv*rG>fTS5UkzU+IGMDI z%+8o*)eI}NGMM+NBvP6(<$Xu)X4uecd$e-YJOQEm+Uy;iKbtNJ#te9@%~ZFY8ZVo1 zCcETRWolsHpz`87BsCHpYa#5)ypkF(fl3p6msdXM{58%j-P6 z?2&HJ7+&{(c^?7+W)_Y`l5R(|?*osP?6 z4|i?K$=!B%LUhc_`&Vu$m7kx1CmP?*)$l) zLQLez6acSb$bn}th3GxG)Uk(t{%^7eI8P8nE(L~1nn}R^ROPS4gRnwD1H$_@{LJ}w zf8wZux9AtM1$Fr_17Y?;knIItx^qQR-J&Y{1D3CTH6)}G;!1ykNtj+2vWb+`TGD3l{7##r`2S|9Q=2HOe3DR%b<^gSGTDv*1 zxwl)qdjoQW#(ZXEf0xRTC1iL1(orJ;Y9Q+gc7<>c=qQlcLa)C706RQbb9`FB|L#i2 zhv&juTvVh_)Da!d{4?Fl!*GEF24W$a>+*fqwMsPlrIPE9k&IxloD2*8Gw_=#U~${=h01(1k8x&U@!kNjltu$Nag>G`(zr^ z&7=bUjjLT)7?kr>xAdv2`W>O_AfC8%(x(H&_C6rvY2D%hfx?38P@Y-Azj{LVE}HZL zF$o8-l>(9@7=rJg>G%m=$p5nR74_G@sOcbg`R9esqLdKvRK}g59p4kAO;-7AUnH$! z0yIpgQrj33$oDkjEf)@_H=rRIeHnMtr4YbaiHGdBSxh5f*nj!d1)9ybsy}}0(;*Jd>sKJ5S8&fw;&jwV11Fyn57e9jWJ;I#^CJop zqDH^`fdG+a8^lt@Ab3cWdWt?UO;TClNw^1ZsXsLAMgq zUUlz6V5&_4jO8Cmo0TdVH1wm|5^+l_-x@TuTK$(H4A8Vz`?&}xB-Jn+(0DKhtYJXt zzVe|@!$SALcK|Y%VAMYFEyjc=TRj(lOn@7n=a2WWZdsnqw>_)UDGhyE3rf3*{=N?W zFVl|Dp7su4DMMZ|0ILf=izDQqH2+(t0qN6Op4#qG)eE&&{AUX*;5JOQ5P)&Bfk4yW ztx$pMLP(tRHtQt^J)KWRGN9OR$^bPwT{sxbFnpEiUiPwqrR&hnNk=cSY5_e&6#&y5 zBAo+W%Y{!AdI`$3ts|9wb<)W@1P&7^stV-wn1oZVhxM$|3W`&z&YKzr8`IAkBT)$W zuhF9SaZpv2v=WF41j|~+uR%{(=4A;Cxi#^^{7NKdM&EsbzuD&@P}Jo5v!>mcj5Zic znSOBBDmcOz*)+IcMoNRUfHazOeI3Pr_KPW5QAh!iJb%BJA9N;0O{WQtyt%!7DjwD? zH*c*Z{;%x%p*yqx%KJv=!>)t@E8}mT`Tv33{lDgY{~HYKAH2pB@B42Y7X9IU5o?l^ zq@tOasYRUDUvdX(#a<3fCabi48C134zmRYiYQvO?^+|G}#8$7oh-NlcCUxAu-v&n^ zrW9ATsookk>i zskOMI;yt-f7mX#~JY~TS{+5R!YZ9H^avrV?_Q8S!Gz?3>pQvKQX)0zRN%rCxp$@s_ z{-pInBuWBvpyrp_!7O7OEyH+Mr+ZxtOxforKG*&QL7wN3+G)r zUsskDJ*qYtu#Cqyk zEs368m0;1}KCbU`OcDa*h{_*giV%r#>XrtXoe$z8e_^~%gQy!=L{YZ2L$7tlpFC-R zCH^Jxo`?efCCAM2MO;DhH%XJ-!J^xtj*0pkSwHdp%??(ah@U-@4u_tP9SdZFDD2ML zHdq1s!>IO1s)Ixgu3FlzBi?~HE-K;6I}2vhgZ$lU9X-+GJZJ6e!(e4Ef>7}L&KD6i zG(`0~6$M(3urbsSOhYZ5h2K`uk1hMZdqIX2?2Q5_uG zemsELMQOPzt6d4&M;vsYgw!4e(IU`T~Rog`MdTdUKj>x4~+8U4&W3R;QfEojm5O);kZm zSeA2bksqWLEBv*XeddWDcW}g1LeHb=e7Ld=w|qs5-d$ptD;{5YB%p~XB)Ys9tcza6 zioN$gjYAe;Lu4K>Tj3@a3%Diw?$x|(Q$y;39)Cwz(J=ZwNqnAL-fYz7iT9Pazx3Sw zStZi)0q-W{TPHc9Nkz>M0;@(<&O8{c(;VlQ(bEVjlE(aUr+YNYa7+#U71sU39XvKT zJ`1KzrMQZB%S3tG-Ol`|zDJ9T$d%BYCi;TRe7DheYa>3Y)v$h9Ehg}R7mEWfU!oqWaVuR-gulp?~#%NvJC zn&eQHjJznZiFV)&eAH!|eTs+nfuXGT} zxXlRf?(jO|&l1_KwT!`4Cg?bvzH0Cw!r|TJ;U^Ffsc3HhVRSCc4v!~?nkFXu5+cEY zyyy$PJ<75(5>*Eg0UP9AELe14kC{u#mU#8|iLA z(Z>&%F@62yd>hxH3pn21)Ok0muK&WAHKyV$i2_r42K@G-|7}|J&CIk7bg0ZN&CGQy ztqpXn{?m-Qd})!Mr9o^sMUGt-`Lq{@7;~H|qY`KE-r&`KIcmRv5<6MxdZ>1@+hRo` zniwB7j`Hor{=qCU?qc*?j@~y{QY_Geoh{l}0=wpx4oo$7<^#GdXP3jrdU*n{ygvHx z#Ok_9YLu$b1(mgae%#PV59R!tsNsQ9Um~V&PQ}Ee_1Sak!>pY2KHi?hdPZlFg8Jbx zU#4KWMC^w~G@QJTEA-(pY;vB9o%#MrwqOe{CD?!A>iJdG^pX#8o?GJ*K{vG$NMjMjumG3i)U<1tN^atP8Wq}P`t$wU#g5;EOq2Tw6FA| zM!t&eG~WgmCqrUMBZMISPbD2?T!hpwLLAj(!QZ~x;y_BV1U{6Zsz;+O_M1g5hH>Un z56g)oXMBO5hTICpi2ugTP#5BFwUoq_L}TO=kDey~qa85tGA#MhUQwM?6*)!r|r zCN_FZ)U1!C`qQ$0`e&I13%CW!r(StjXc5=-k{im8-El>Z`&3} zqhRtrZWrG}>@|6R`2tTlKfOTUl}3AdvPJ#R{Yvv+c0BmMtxhUy9TW54JCz;qF84p$ z_u&36N5jVcf3V>J+s{7|+P{jS(lj%(wz9U=_^V@2FPQ(MjuHF^iBB}HwSl#Pnd#p) z=zsGH`_Jk7*X{oOf0C|iV60=H{eLPN@$aIowDfgMH2yiM|FM(*dXL}yO_H^NiH@;> zsm?!d^V6;V7aybl^*5s6G#S?&*vO~=#c}q3CyI`Sj**szj)9hmk%pd$o|#ttzno!Z zW}>IkGBYtTu>McmU4=rw+3Kfvk_F>_Z7}b>Sn%3gHrdVv6NOLFDE61V0=kPJz(+2S3{6uq+C{Y0#ey#@K7}svX9W<;#=`%cL)hV1;m8}Q&_x;AwS>$LgV|mh<~_uQM@m>!GH5r{W%KRgt0s9Z8SFvS z4qIhP8v(YbpbONXa^I9nxlcQkUwpjzka(3~JW`4|2wi zk2$*G!Sx1IL}gp4(00^p6xn>Noz%3dZrYNDn8@4eF8 z_kO?BzCXH8qy72Z<#9hgza5w5HT(9U;eHchsdfrH&0eh`<*o}3qSHBXkfLeu-3A_W zRTmxmhg3R%Bi*6iFkSun9a^#fEnTK5Tjk~t?&a^A!{61FmYFktOxL8^aJ_g*1{Bf; zscH6GG)3O!%uV2$|9m;^a+WjA%b{MQTZyJUJzZN;vskVUQ!|3Ir{*gx_V$W@jK(`$ z|N6#>Q1U}xm3!C>(-Ij}nDXGUZ{i3fX#F!kd|po2sTOVt=8+Ef2v*D!9}PK(u-Wr{ z6&G6*8)|xg-A0Fv{#~qsTT4=~lvn&_+{xkncog0VX|%$GsjVZj?^ZGt%I9m3Q8QtJ z3hptli^_q~F2Ye(^+&|cG7sqy#Ni?0Z#Q3Tw-p;VXGlF#e8&$BntG$dO-=$$Y$;|u zS#Y}m^DPv~08oXA@02ge;&!PBZwax29OPQ9~+H;F>!ToPB)X_M*wrOxAY$Bv@O z7jDKdQDNGoZpq_o6b*e{fGZ5QV)Y1R%rY$k4ads8)(aF9g@CejXtU>g?x!G1dfNJP z!O%I&TvFa+n3gR^8I+;GA=}R56%iymfnbOuOWUWNJ8>j+Rw~gj2zg4Aji`Oul}V~w z68XwFl@}-}ZfIr?XY)$-%Xp>GNCO`s0h<@?F;F0WkOE>@3qW=&L;4GuP05Y0#enV; z4cl54?JiVMaMGDbpNyAFFWY6eHdihl@ke;rc`>kYaJQ^e{iCMgKfHop*+yK}QOL+X z)Y7j;_wPWD<~NL^r(G@BSyd_Fc%)xys&=URb zsA64x>MTr%9l>}7iZtjHZRIE6!oR8=VVF7UkA*IPUybjtCHk*$_;(3^>5+efZ+39|B&{V{u!Us*I@gQ^jG^EY5zQa|Iix$_=b^=k%@`< zf3(K$(*9Ca|3+Q?CuvbRVMA9m&8rZkxj_vd5_X>*x0u_V6npjToVTAgW4sx1CLeVQ=2o2+*m-sIU?GFgRJS&}ekkS79 zFi8@w(PX~ghHV={PK8}V)mY?ipmCJ;)I{4kc3o!z$mPwutr%s z8yh`#dA3n{!Vw9p(>He!ycn$vzBu{pW{%M1j>O&XLUipsC&jyg@D}KbKG=_R0>i~k zx&V;T=ajDl*?I=5D+@+_m*0qXV?zky>GVY(Zc}zv=mDgYX3Lpme@#V<9A~>wQ85D1 z8l0p_{L5oryX7-bB857 zk-Bm;XuR0_V5fao4d3lE+}7jSs_mlVRnKh`B(iv7LH&gZM&WfRju>3=fyVh22)wc6 zyqAxPb%lwo2*+>c@xn>ahp3Ot+=ov6n6{w_Og@wJyLW2kYau)ZKxRn9#0+;DrF9oS zl~@d7&%N-q7fx^TSK`1WcsmSPht-*PazhKY4Ta<&vLYr34ZeA)5@NB#aF3{{yr44H z?AI!~u-+}o=h$ymHlCZ$^fdeN338g>Y-_`R{8;dRPCOi%k|n*+TlxGYI(UvkGsR&z zw+fL<>^XQjk%M@05<=u9Jzy7~Qdt0k72BmwwJoUVUCwy#*a6xJuO4@6<$)pS=+a#b-bS5NHR&d7I@a32> z@7wcTRd*H-$K$ii?c{b1#cYRF8hM9L z*gOshXM|F61=`N`RsL=~k);QzdMS~WM~Q_cpEOw=%P4uMCvm+vdo*IF3BIz{|0uw+ z%*zvtD)@AH|IC~Cpl#4xN3{D>&xQ0c#%rx&w6_|i>@*%ox(T8hB zGAAB%07(5Bby)C+UTQXZWkp)3HI^5+V+YuLCI#~)NAZ4Lro7R-tkiHa*{xUMO#ETG z{JDLBncZZ7L+F~VI3Rk$lWv1}$yo4I$fo)NfXr77m_ogy(xOJux`ZDE^_)DlFv9_^ z1sveF9;xK@q$X!#?`;{%rHoba7?~7b6y)!-AMKTe^rK2C^dq8j&QGijqT+X!b~N6{T)av0(H%Hz5pRu8~Ac zm+lYQ$?a)MZ$Vn7(pqZi!78$ z$Tv4Qmo!rf4K^z4LHqU~mlyVM&}uoIiSDIf7=u61@tN}xWIglw~G z{;nP)_v8_CpC?bo6@~uMPyB+v%Y{YC$ACoR2HR(y)^yu$qb|q}$2Kd-1cMIisbtRG zo%1r;Ct8vLH|vtsPt1*iztAKF*`=)4AHi2a0FjuDJ-~)4_waHgZ|w# zJ1`swZA#ypJx~z}?cP?)@D46$0t5c``}E83fu8oYDS*GMSsM(`5-l+mt@FOdgl;=0 zCp02FVZGO`)RaKv!t8{#iD;SLI5HKh8gt_gk^m@A>*7>r41$K@1zp9ha!Wa2?=|~* zR`F6*DjCNE8oa<)mv-g1WZ*<%3LqZkrTcBR0zF?ci}V5Afx)j|UleG1@MDl!QQOil z=x%&8z81uWOp!nP;`XDKvF^OiE9R?s7*d7pVjT?Qi<+t`%n9FqFhej;%NM#04V!;3 zTOd7Ht)PDSCUZ91d{v;1`ik4hxX8P5D=$LDgEgg+SyPN-VEOrrO&h2hj9y3wGi7Vz za@L6cFs$~c2?ZcqW~(M~h)}nNK>urE^zB{~MP>}ub_Cpgfz1yJhwcc5Scz`Za~5~@xkBlx z0G9z!vgaU}d%6wJdWz5c`ybgNvtG@)#Jnp!@1LgMkMP$PJy;%}I#|9a_xHnRh4(Y&sRh}glm?Qs^Whw% z`kp=Hmo2j-H_z8wa95}Cr+v?X)^d)W!f>p{>|_2ae(GMtM9@G{fR~g%(-)uWE10%{ z-7L~MXE}V~7?rHqwwqkx#Ft*E-|ZoO>OV?XSs;H@E@diX`+OYr7{BBy?yi5t*2YzE zGAt5x&3c~2qrEg-omn#+*9!|1$ew`kdAct4`0J)n=JUWBQsJWJmYp}acgi9YiHa2&QEEEy4K!us^<{#xq%Gw5I zB2+MHRks^6mcNyI-(dP)P~oLpCF9Z)x|BwoaVX@$Z@_4?;yVn#h=a=0rH5%U zjV>9aXjh(I-^3fn#L9$VSDPQTn1Ym6pSj4z)eX_|NP8JzWd#W@Du*V)RLoP=WC!Gc zVi~X%y2+38+cR~#n$ve8eDUT_O@NU0*ahLlMr7ht2wEi+yq90K&`fM@f8sZE3QJzY zt)6zYkq5%X3#FUn3n5U3?3XgjM)ENHp>Bk{J9e0=1RZ#I`B*lc7gyJ^dg=YIh&Q;c z&hqVJrT6F~d`$W8tnxpWLjJjT;Gb^7-!6tIk6SO%p%^U&I3+tJ7rp|r6R9cH$4y8( zND5Mz^+rjl^=`&+byVn!L?S1A6>T|NUvOG+O!{=!Fe;622n%8ZWg}|gz5Z;$>ph+h zUxlk$g^$#V#fE&dF+N#AUBt8GsGi*&__3)EEYs_~qeA>ZaWJkW+4F|n3)05#R=7H|9nim-Sr?Z}DMH5r!T zM;H-A$@gO{o-!&a73n(>8ai<&V`~DHgWlt$RpK zBZM8wFjsgXM<=}5_3Mi+=qCTt19tFef}%60 zz#{xfz=tE^F7d=cGu4U;b{V7{k5%N`HV0=yrNQA4ep-^&QycI)^~*%a*R3$L!a%U^nLGsr8*-{KMZLg=GqQjygf#&$g%2s?()ganas^^ z>iv&UnLn-u`hV+!{wMwZyHoee{ropN{y#Z&kB+tVgvFBdXsQad9QyANFl|m-e@x9^ zF1&dDoH)q`WhC??6~SLm%|APIzkBe%P0hdX-s_bmtf%=9;Lu9>1kw>mrFr^Gvcy*O zB@h$T>c!REitR9}jFB6D%&S@FCQ__CQsS`V_r$DscyDw?>6~&wh@}Qk6SFU@z2b6m za&D))&CA5kPi&XR#=;3-bT=#aYWf{z4#m)&l^HSzk+pb_C*y4ZhUN05X9M7Yz=%wP zh_VD;I)Uyuf>UNx;wjo~eS5?h3SsJ)keO{Odq&g;(f}dUxnytI@$D!Ch#5)&239s( zf&FY_LFV@b?hgMXHy@K1Y;02+PSnrj>e1s}Kc^|u1Ys-Yy*g0w0^e%*(!+3m!hh&u zGII$><3`-Q72Q zIwJc9?~W06jh~9DnhJ@yV&${9sv^1(3o(9v7x7+G2g}QwEd^vtZ|zUfDL2;w3qccG zK2RYzp;iF76Rhul^<0(Wla<1`mjp-o0H z`|));iS;LZJnwiyhH{~mqO+s~Z#D z4`{{NY=}|nNBthea%912iBPknLj8k6OT|yCrE1JXq^?b=(&Y9>JJMxB8612#Hn64a zJhsfsV1m~g({sg^lUOCb(3zaOGnMMirz)Tsb52kk^~L^Z56yc73#pbz?R$vbv!#)8 z6>?iDk+9I|XXT{|^)TJcFzRJTe9URh^8Lkfqg6zXkS=iaMMMlu&Xh+mvv{a z0?wx=@oQBFEE@n4WdL_v z(8OR>QJ#g-FN^HuWzSvO+2OMI4hE?LOO8$ma9v9inGrgMDWs3jRQ%|RpUwGsd@c-p z0>TJR9v~FhUY#6-uX^rHKH!~eXoRVxDiIM{C}9J1KMZ?>==IA}Bt3{_vaWb$G22NhD3egu2LRJhE!Bo>Rp*%))ltLYM|fN-J2pIZcol~hiQH?Vd$ zy>J5RRtMNCWWZd&4Mv|KBfl#x`RmFZm2FI+dl!F?6toxF9-LB0kHT-fN!XGs8RyYgcT=KOj#HS4 zp>8Kk6srSW`V>V(f`FKd7f&2-#GvlI++Y?Q4JEPQasy{DgxAM@Z!j7{TWZ);$abd37+rCbXZd$@3sXA#D5}X zg$JZtGnMboHokh@dlaM?*;%72D8oA;#*eW4yDd#62IskLa$$R?-)#s6c`xElp~gnS zWVHj}`uUI6v|H|PjGqHDw^OCK(VN5Fn?U3V%m&_&*+#l?@9DT7Zg%eHi`KK=T|BRi zot}?}$CtvE`0vWNaUIs2n~gJ`Dc^wpbq-N<<1_tpX8%p(;a^_mf4UU^?cDjR$V2^3 zm-fF(^D&WsKmNnK`l|{3ck}9xoBaai{}Mj_7L|V=g1{9|~MysY(Z&Er9g`amhBZWvZbB^l!$ZiXy)oCNJr9q;dHr zn^{S%@gO}L&=#1654DKSa4cpEWX_{=Dlzmjpr8mxow(H)%%d0dqL2-4 ze8?bqc?W}h>jJ|jlSO49`fqSy9=G>wI>qs#Z@ zU`-epm_qa|@m1D|(-+Rl<6bvW~pHn)q zP)ixJgbf4n!f+1yFntTulvA5kCzBwoEWE8(OW??O4+xa^nK0lP8jye*RRO8>E17Jh zgGim9``#BWgR*cYNB}dX)Q^ym>enmyY!I>H)Pu6?MUh3}`l0TA*@?zI8Ra_NeLt0A z{P-K^`q({sRa_z}ym-y&7`PEF2{3@N5};e&6f zu>eC5J`=w~!tB{Lw9JlszCCS!f84r1Zi(Xk>52GrP0c$~4VzZ8=zH64e1rHbEM-z% zX3Csv$=SulRMx=-G}6=C@Imdu}NCAxO{ zCZq@X69%9{2$*)5GiL3q4&_K%_I3fAqKae}Ajo%$t5>_mv5x^{{ zHbMt(fyf|gr4pF5u9%57_#Ph%0Z27~<4*-X-1|%rNs-ULZOGG_m!b)!^DGMa5_&_;APRv9-S}&^1Dz)*?fR+Rd*55c9H}jv%8yD{6@2*U0Lm@}F z4Mo}-SMfk~chxt-3xKu-(G3sHz#{T>Th^*5Pie<`J_lL*+0SXlG)nTp$YM;rg6sr^ zsYb_O8hC)`CgU_=8yrg(|qhM)bZCEzPDj$dI&+0;?j(fdD3`~kEBBOx(!s3fTS=LgYZxze^Mea@1_-RGmIjR->`ZP zpt%o3I@zlN4Trxu5T$$p6C3~{Y z0&YdisCv-~PR%}Qd8}4Ydfldf^EL{Anv5IvL(~k14O@oo$T+YjRF`BK z>Iy5~+-&0NNa8e~i!8N_g`YpRVxDfqRmt7qlC1QPBhP9%hiV?KNiZ&VYt{<#ZoFtT z{q(~`B$CuFv-p@eN<^oU{V&5msYMIJ19;|<1~ZTNcACH-X2uQ+*2qAVuGmP!omvvW7E}dpE~?05##sOO7?t+2M`%@63`m z{OV`#3c=OOV&;3?zRySPdEm=?U(tP|zdo?8yW%l;4j9V{&=p=kD@s*_*Z?ZUmd+4r zeu-qM1uwE=VfDKiC?Uoc?z>;?A)JP2pUaI2q(?cY|wwS^{Fy+d@wWliXZ2I*MsL?PDgf=S<5u7Wp1D%MtQUUsAyQDEe#B zb;M_{06MP#hEB1!Ca7sl(FiS3Jm#&K&_0p~z_|9|PsybPX~BnP0Iz{1VJ{{tmJYO| za#i^yvx14?xU~qvK-TMmt?2L$e9UVN>-a58i*(73*m-LP21i{VW z?jz3BB7OF5Tn_vF4RSWFS?+`dQa5lPh1qn zTG@`P~PeBh*3z*{uR+TNrI)hvl+tTAK1Ov>aO};igF+SqF z!f@gMhq$zS{giP7&*TF7TuOyjcQ}jXI$;y+#jR^Qz$?q_2{O6u9LhBOey>LHZO$7C_(| z4a$wFv?=&aSaLH*o+@Wq8c7F*7z!7oV_l`Be8(jOxbGR}0eapD5hPT5E@hClHRET> zS>DHqE<5dWSbaLBvHLJ+PjmD}n2u8XuMDWuR&}Js+E{&zabdess-_e~Xn30^RZo6f zHKZPr$9rq*J;``EpYEiNS*!|CvWA|stMI3jw0-roHi!0;rAtmG&GyHgp_TZ6EWTH!9JC<(g% zOfb+ZICX}$BR^wBS>ZB65^UFFV(yU;TL?81*5CM9<4a!=3L-Gv1G%t#nvr6SNO~sn z_Kes2c3vE%?06_a;?@JVWIpbo4OmxWt*n_m+RRsD+o@WF83v8S)A=ITa#+c*uhmM2 zv!HJ|0;etmu+#ijiiBW31yYFU>a~`zp>D>~*p(+85!l-pM9}M3x`>k~U~5_f1BqWy z3?HMGMz|c8wX_R|rWuYl^RdPvx{zGnkEvwdNKh16xA1SmTbhM<@IJPV7ASoe$N>Eb znxqR}e=$c(9_ipwUp(x(gC*3&-BuHsu>;kZICsR}8*W5CS797*<(FP* zJx%T2K?4HBPI%H$n?w!Idb^UVz;*cTvwPXtdoV>MER#4bOME^~WQ=6ZnLJhGHNy=b@T)3F(6^}6_x)S~9zd+Vu_lkjPo=WrLyhMA-;sA4eN#K%sAGjNEW ztr_5B9uZxQV+t|#L)Yc&Vc9WUu;>SV0_IFMnylVdkZQJ1RTB=@(>Sk@e&*!f`Cq>- ziK5}@Q>d%+6d92TXHRx;;l+*z(Rv(yk&|YqgNgDqSA)q2fs}-9ggy}+ao{EB94hB> zBM|YQXV!PqCb%{yF~7KX-CPTPn*_%5h)PUmWcP`bBV+=l$KXZ*ohfSa0N;1-{JPV- zcQ=pxfWRwy0Mh4G{L>0KNApNTJy^2;{Lyh?V=9qbCU9*~-=?6yy%hhufiwlmF~Co9 zSW4h+Bcr)f_P|Vv*Hx_aYLIL-Zo8hZpYoa5RtYsw7`J7=W%}x z1Rr82ZubNUy^JDA85X8h2vlI3GPkkYC7#J+NF>LjMC+7D;XvaSv4M8tVmyTdJKg0i z#`Sr0{aSnw5LXhv5KY|14Vi-4&~n>D;2i>cD?ely?y)W7+>Y&N^&hu; zcES?;v!Hmy`?EKG-kNx3aD}?@QB}@x_RG!kaunE~!y~baHLUKh*GE;zAof!jG52@~S0q7f5I*Cw8 zwfbOozAN1`j@x6DmV7!6uGQ#^HeO;gL+HJa1xE^8(nxc5C1f&00!jCBlCOugP+q4u9bXUD=djtzDCmya>`67wY%r<)BiXrdV8rA@PZBND5 zL|CEp>4ES)&U3fies-~w_|cL+YRlvMI!88PL#(PX+K@8Wzl062*{rS z6Z!h!k%?E?)tiN0U=43#PzGN8}W50C>IZ8DYVLHgW+GK%Z+lV z0KNINJvLW*0uyYpnUHIuLM6XCvGs`=L`mo+Y*Sasek9B+fT98CjTq z%83Tk76{?Yte%HdCzQ8D%ds?6Xu_+0F)`~7iCWcmZ1(H)COl@sue0>HMs&DW@YWk^ zTHOXu6ytMCTkD9s<0S$I8jWDDsH5B6n_3-_9i1Mn%XkH-g9CR z(!HjSMCI1m7XA$HP3XWk-=scUn8u|A2qg*x%G_m)|D_c%c68Z7_c6$Esy>Ky!>&ro zFEtaY5vmA@-~#%PrsDTZyhL!T zk4C;3UeI8cFs;CApUM<%g>0xt^>30#U4;=CZr3(qNy$pOdDo1e1rU*PjoZM3{3yLK z*XA^uo)hr60(USOd_Y|FbHc$9q2gi^jEw&xnFcvQhnq7NSqiRyEfdA zsSDKhLNF`bhfU(RY0b1 z@d+ij^~M%K%8X^Hb^M@i5s@2%-nsB%@C3=d{31s#;lj{2z`XK$H*mhwxUhEc>1CmE zv)6N;(4y|Zk71ZK3-UV>Dok+tm%7g4eBiM4nw)M9QtJtLl1Kux$YhXgurM@oh{f3N z_Z^PoZVeJYFt<4DgKnn!cpLR~b(eGNggAt9Cv^n#$Zf6;4N#AFg8Vh0le>Nx zv?D$YAF#)!9bCLOJ`^^|MW)CY?BIZ{mYSLtABTzMousLqoD@Ei2hvw4`qSD(O$UY9 zfWarjPA(8gRbGBBw@LbzDoAh&iaU5lN2_PSq0yyg#9$%R&qHIeI5(~peEZ;qfw)xH zZJ+kOK{H`@x(C>mjt@LUXg*Ug-ZhMD$OzUlOv^HB-E2 zIuzu3ju~dK^ivI{f$$d-j0DKSozrD`n^kk_Lf#+`9@~B?uqwe#<8o1$?NZ$l=n|0) z*E41&=IE>h!uV}3hL3lwq|P%~la1CC>o**^{-H_*!Ctb`Kz4;IX>1vS9zek|mfn2$HOEp;`B(ONu@p5)YPnN&&>8nACN=2a9 z!dM^csO)nSalWyb-t~`5cBD4vw@tR($~wSrxQGd*a!k$_8 zHmDF+GAPE9{FBF5KD6Ji^B+w)2g008ym_8Q1fAgpc?!#V8VG^zJh8Q-$E}OpQom$c z#!?`qjT3NY1qBJh;gMR`eKZE+tn92}|md&?ixdt2oq)n|Glt4M_ZHD=x8o{H(R@Kunb;;*DxohP4 zJY(6Tby5wx1Hz^-PV!6Hn&+6yqrKCEuifkgpkk`r#ZGx=)r2)4pD1Kqcc8PbY=~T5 z+6+WS)f+YxryjlH)6h9O9G&`&0F&qP#<`GMAqSD5^(Cypou3t9Eb-iEC*ASI2s|rc z1R0=SN)WD`41+o&wiGGvt?P-R#$kP#ezW1Z9^*wxI_lQ4;ss#gTw4{d#T%bDl&ODHvunxH8x>of$%XgQWEY{VWnh+rL1Dlzr-;Q8l}JOS-Z;5VIQ#u zBh%&(oql`1d51K5E4p~&RC#&_1Ac=P7WSGom9T4Wa(o-yq^iJy#XpQL?|8mPv?W)i zCwIN{p4wmr1fa-$7x#C5m06OZ_I!?zyAk7L-JI8a@am~0$m zPtPGQhR@6@-j#p!JUlW$dIM4O(D20>Xnotdh~a6HmMW_5usH!;+vY`)qJV&~n?b;eos(wz3?e7KLkqYL-7>1U3jQ`U;GLfP0EP`BpIrBHtMbJUCs&rXl~ETL-Z z(5Sj8&qdln>|!h#EsKoZxr^pdj8`kN7$q513kr}JIUSPK0VTFHvi{X{e6L+%92|K# zSDmj!2;o3Xxl=as?Ic32ezgW>u1Ae}7~e?4#?0$V;j6h))gBeY!-G#fTJZdEStt8! z(Mo(G#`%!}D-W+6l-b$0VDrBIz<@8nfBHm#WGJvH5KO+m^Mw}2?Cq=T0DJ5gIkbi- z6lrz~aCbB7(1N>8CDG9_VAFN56a$%k)SMsVei0fmHL@{!l^oQgbI{*I8vR1Ta_Ege zKTEm!R(J}gV+e2r6v7VJW@!~MRrKOfEOu&rbD|@3%no==<6N{E!CGIZ%Ja}%;)G#3 zCaA<<8vZ&}oTs6U;_oxn?tkKPNJc&6E6Qq3xjvM3h|3^v9~fvDz7aQ+No!D@y47P> ztk40*Mo_8WG=}wA5Mf%jq#ffNwzNaoDP_tRCLLT=YbHKg*wJ6!;4~PCg?k!jgR*_e z@MW>p7)>-ohzP;ognnk-_<#{azgzg%m(eI;y6*vn-H2?rvkx>z)2!L#dGa9!Kg+BJ zWQ*6*(l$k_2z$C*r>OQQUuB9XVNaIg%YERggzY#m?2pAX{gex*G6@g{TLI-`o>0ZI za4F%IL)?_FaN&GdaruhnoSf4k9m@$Jpfk4f1k?Fyiw)Z`-M{I>(S3$R^?ssrP%)O~ z(Y6yNp;FJ(!B`b#>|ZBvs-Lon)v>w&cbtuHdv29p1zy9WfX5 zh~3CH?uVbZ9^r)QNfrbi$B{DobMh{`#Hc+ob0B~KTuPXGZ6-pUZk6FEOO-UgL=+U}ZT#B^QBGh7bNUBB0% zMtFD8h|96-71sGl;>X(KPC<9}EA4z=oamyM)Ns25R*fKwwP6gK$Y-jGD*dbAj*_Bb z8;;AHl3QA;BDOsS@?-Lnlb%lOxIB52%Y>*Dsi>TA!U~E70x;UekXKY}r%!@HJ3pw% zJYbl}^cFo{;O|c=sJr&(nh=bIxu(F~Prh{16eC_Jc+a?^4@E-~8}(kRRVUbvU+#k( z_pY4fVM(NP%XON+@}~Flc@eK=e|NnmNFv$$w0Hw6WO+B|!jkw&8g*!6E4cFpkFGAP zAui*xnpEVI%Qsb|qnX3Gr;|B+pQU>WrTwd(aIe*n#e1EA?5gsE$2lbh#!kQ@-++0%GPi`yLwa0#V<5?^CQb^0mf?d({x`?w`_zy~z_FxOH zk*!P8^}PAo6Z7l^xk0tx@|k z95o(YmUUd+BHN&a`=*T2QoKF6Tw?TPSJ*E)MHOhE%PL;<0j>IzDkRvb+UE($>BG`S zR8*ecC<7}gPRPuq{BpzMbADp*DTGR+hbevg$V{n3LW%Cto(ny95+e&nuHF-8NXfv< z#3ZFiqE=b-h7}Jd7ao(`@E3~&x?)O6!enU5MQ1W#8rSYR`cqDsglQyxCYV)>E@ev4na*)o zFtwdH9}A{ezhFXAEJD2qt%1?q20fAsbrMOHpvo&J3SvyVn?eZdn-i4OoFZZWaJLz8 zQOlrXL-O=`9%Y-;_*w%<>KH^3S@^5ntsS9(O;8eTl7#ve#?7v(Zn5uKxA95gR?LMh zT`oQ%R?yg$lA4blr@DdZ?7YLL2j%q+K56h`QnS-o?PBMl)Vl*{`eTkLy2?n(-}Kd- z_p6asKSi#>5v#nE2n)8Oc#I}Vsb{dTS6)@?G#(}#s$~&Az82PB(}ZQ#!)FbzmM-~b z7Ai-Ct?)t>H{oTriri(Wg=H31sb_-ar096-O(+K@UGoelb>GWdTc1yvMwni!PEho- zOYSvw9OGqHWxgIm6Sll~^w;`Jh(_XS>;qML{nTnR)4?NAaA7Zqnhogto^qAlxumjj zZPKEh1~cwI-r2zUb)4E$jrJ=x7dKk5R&!b*p3ZrkMohV-6ae*E6D9@IL^=%$(fw_? z(8j6lGcc5rU2@j9Bkd~4uev%$D)Y*AQJ${el(^3hE4ECo|OMZC-^gm;NzU0#=_c|hF<%FYCui* zcVq&EI|#>1MI#>oQhcK?UHJGWK4368Na=hd;rR$KT>mSXk|KI2#joO|CLc}X|4Lly zzhyuC&K&ziGX6J2<9{YHba7z&$O5i?u+PO@QaksZyTl0!v4Ay^%mgO}?ru=wK>I7F z^;Q-mAo^Q+W9 z9o6}0f8K@umrRfUvkEK()MAoiRAM{wPLxB67cw+*bPRHIOmz&* z)eJ1kHja=VpXnHO#mB}+N2wqON2KbR`GA8oY^F8jp~V?2V4MEbgI{Ev>6pbC7(``sNl4h`kM=KEJpW-_ew%oIF(xZ&kX-SP??ouNc4z`J3TY?; zD0=pcNOna%`g@&uDXxXCnV12m*%f5s;LTq-a184X)?%09Y0NCuIV{L494v#!-tRyD zq)+v9?)~2Fv=yt$ICKB{)q}VH`MD`mxEY*{QDjIKub<>X1kGc&qXGWaP(xG);nE{O zw;KVUO0vTUgWiQM>E$$1*8(La2!RfpL`VCuhz-vuhBz622n1|?@|)QA=!nz~LO`Rg z-USG7gj#wud|Mxrb_BRE1QwJqq)9S(!dyhrF3hW7A~pyWgb+d;ma*!T6nsPis?=D5 z@Ic!pGDP8Qjo8RUXA*zXo#^L{0DxT~dV$ZJvFb!I)zN}<7NW7ld6^HJ0~)w@UooaQ zf=2nbenN5bP~po0^}1|?etV;!B9enc5?*~~UCBqa0FHENL-e=vhN1G4>S?gR@VXI- zMS(X<2!I6MMYL481aH8niWLA?_JgCRD!_;?-MD1)wfT&2l&g?tp^*%a8>$nD&{^>1 z?9*|j4loHV(lJ^b8;6Xz2b{2}^zb=2uGkJ~U_2UDZw4Zq0RG8cJt9UQzm8VR6A%GV z&W;^Q(VXBU9h?w81iW6@JV_wQt=S$69)~U$kgzLT01!8bMluL8h=`5=kQp#KTqhSx zJtB=KjTrO>1Oas`5qNiF2L%Y>!}me=h64OW65O(>Vv@DAY%{7XIPiJ%G zLXZWkFlL4RIoE%d_#uQ8A$_uf2ayaOtw)d;6a*T?<^^4#2ADBG1VC#8CgD<8OrJ-eu5JHR$TBNGo0`QfKf zHcFWl!a#1OK!+SiC^A`@`$J5=)+*>Ut$=9S<$w=xOgTAYuW(+u2^OAazON2Xmj9I1 zd8gt{Y8BZh%jzl;JoH`A*vp6O*L~4EMp1xdiNJ^-Q-p~g00u!lbcIojzP<~PF^x#t zK-o0}@jiO}z&q>!y|6Z9KHxyQ4K!$u(KV+5(TI&A{3%>p(*n5$R1Z%;u_51;H@Wyr zpT0X72t1>0$%WJ^KWeaJykPEy5f8Qp2B2)O2|4ry=+wy!4-_7>2SzjPuN62DUw0Yf zpv9h~;h<8`-ck^L(&wQj`V`;p8?CL>dKU^AUB+7hAqG08gfisE{vJ1d9$qa=Gx7){ zX4+*^_Rd5Dkn?%S} z9jt|hAO&$`@1tjFK66BeSTRdAqY6qQi#q3VC-ORUTJ`Yp2Zu@A9%>F{!|e+9->i|R zI7w?{$bZ?lS(y~m6cBMHEf=wHx@=aQ-ss3{L9Qi2!bbq|Z$@Im(3r60%Wm#*u{>p> z*}0|%Ty&if&R1s{8vArCd6)?D@Qre3iYCSZ8z>n)Fqtos4ibL?Sz#4?HeQw*BA_l8 zk;Vd~*G~yRF}7o#@C)C>ER7HDg1XNoXfa4i+ia7K#=Ya2cW;dFr*W~C)y>8#BWIhQ zpCd;zh0@8L3-T zk^mzA4{PrjWy!W}3s+_(D=Tf=wr$(CZKKk*ZQHhO+qUs)zjx0*XW#Ff`|f$)jn<;I zwbqY0N6a;1&KS{W?;W5{7M~caU+9K4YZt3qUsoeq-get>YD<));Z(foBV6=Xb4&jA z%ve5eY7sXYq9hC8-D58k2^v5#{B)i`JTn%C&<$2Mes#Vz;S|HjPliYzbRai%EFqCI zcyN20eEQb+0%PecRu8avigVja5vnJoYFRABR|k4b#~ojKK|}sPlR($6>8M83O?~JyHqhCp*$q^@(%bz!!agv%Z!^4%GbTwR726Tl(FQ zbC%|(4b>JLV{fR5l^CE%KEE9F4%Q}fg_r9k?Ynlnhmb=Egq?7%V^CMQCC)dHomfE8zD+fOfQEfkq5aA~H3Or&ZCLT;D!}Z;WGojfdAsL; zvyfs0?p6WpN9l+3vBnKM)&n$#ANEa`tu3zQ=Lr3}Qi=Om(DPYdZWd6sC7c8>#6g5S zPVEpUq4D)2U=rjx=`Iw?WHBI8+EN`gB1w7DC*;SfDR3tzK4vEU+~@?kbg|QTMq$kZ ziQlKvi2we|fI~{-nv^fv^|R2_cg2)J6isehDO$4_u+X`#E!gpb%h|c1u%C%28ai-~ zBt)=P2nZ7vFh%Z|hTgY*?O>pdZzY&0uJA-p<^(xbf(g9N>O%%-lP7_=2qS@1&X!`R_Obvt4Q`g$ zB3O7Hk5@AAB#spztuM9j)>ux(G=9UuCU1_(@peLDZgvlFCOL9dVyUMty{SPSqGHxZf_1ehCw> z6j;82^9K^me9Z|8Ev(P4&yJ5f#J02qv=ksd3vXe62G#xQ`jvMyUqeUtu{Ab)%=t|6 z=@az0OChBgoOFzC0%EHqt5uzrAz^xZoup-sCF4!4(Z+Wi z?n%6oyiXtNGZwzR2FH10I~PV$R#894`nIIlh7&V!6D+6)5f>uV67y($Wx~R2rGv}u z5$_Bz;e&gWgGT1#V=5kx2|9~`So|pF+FgC2o?I*O_>%hEWA$y7pn_Lq* z!l+I8f!wSrsN$)E!8^Uc)Sw)=cD;3GO9XL?QQFbsb#g&n*xQ+2{j-l`p;s5SykWh! zdpd=&tKev!JzmZ1r-ZH~xARVwKBPPIo2w)zb2*bGy<2sUp0P*a`oqEyrmcySS>QSK z3=_DCi(=wZ%j`O8T;3yAeQ*#Uv+?kvBrC&;tnJ|-F34h3B(6MP09#7?XOkpoelrDj zl5Xvl?C0DmUU@IeTS-*f#~xBvuH%%0pN3c!Q#b_s_1(ewYqo5U7(L((ZVWM*>fmFK z?T48>McPzA%NX!zZ)zvFSERcO_t2*3#AAHjVpi$RrRzd$9(`9+-8cky%jpk;C$0w% zk4F8zJM8FKZHld{C&^phT?aDU)63J&qJ_)T%IZMq z{y-J2T;;IgITA^-8y!g;@)Xv z_ByJIlef0aTr*I2d5{1edyrX`H0-+qrQEvB9bwDz)RbyB(mwGhra z0>Y@~e7J@AKm||dw(~Q(PS>jp2%ET)r^z?RIVNMd- zQr9lp>77-!2UXPC+JW}y;cC~C6;1E756sa9$ozS+9YS#&QYqKcP@!uMvbMMK0Q#Q( zlqCNm2#tN?Vq8i_@48AWDaO1IXO$x@?g*cyp{og@ndWsVP$1?y_Y%Gz>naa~`(f5P@cBF0ek?@@ne{$5iOyiL1p7krh46i1+q0}uGV6+i(uHq z`YhzH>t}!d3!Jjzf=z zhU&@cljlLCpJaoX>&kReOv}xc$3fcWv%!ujBc9j4x+pj{r&)&dJ2PV-nZ>?%TrFoC z`faBc#ug8=?#h;jB5f8 zr7|fd4>LX>a!lk2EGfGJ0i{($CuX z_@)$h>pZO=6eWOUbnNA&sM&q6Lo-&Se*l*RNYf8(pY`Q|m-W<@u5%zOShZuNCc$6tBiNzn`3 zHC$N`Ql||@7(Uof%sNb|yJusa$yE3rWmT4x+g__oK&4dP=;t-=@=L`l4mXwr<_)Fq zs7Hz+pCg%OEh<*>T&l21%&C)^Nl=AUJ}Ob~SGXr1f2(W5OJ@^?Ub>51g*6%XGaymy@Vh!-We+9CI~{`AIA?lKs`P@J~D$Q84NNU z$731-uGZm`d-#?QuU2>AHSLGKh<>s4oZ_FScv4?-({0yOyq+3!(HqJkWM$AtpY*DP zy>Fz{7_#qm(#>Kk7ZyRxjUiGf$i&B>rYxD}^d3J3JkS^F^%0c>BUzy1KQ&EN5^}Fq zw8R^ydk>mFJnck#C}yiwiZHX(Ii2yeB$uatFn5t>wJP78#_dSDVmo)eR`2wV_5tTO z`F0A$ox{`DNp03}7JBlsj^p*b^+DrS8~{{sExw9Ds}&+3?psTdrXSVex$h6(M*9J< zHF0aL^QuS(E@9tRQMx8^>qy^{*k_alUTJ|W6$MOgCNz}Fbb`ES9mqsj7=2~IC0UM@ zenvJdO&XktQx-m$!FMc;!Y5Q@1%WJgc+o1+Kh-oora2sNxo=e_IY+fVbvjt|Ux&?E zqT#*AF6T0z;rb6v%WD3;w!1lPoQ11U75(%RO*a{ z-5sDm9}OCgp=m&nPrE+Mm~)sdi9K_?db(sZco|HuU>YN9lT@mnUvQXYPvAjv3gT<5 z2+!HQzIp|cM)a#T%GL9!EhdDg*Nqt!bkS_DO(}F99kV|2$1Ce4Prl?jyY*l_tqXBD z|K9f&!fsErB(0?ie{ki>h~K*q>ilI<=j-+D&4sCd?dx;`Vka{lyzhb24l=0lmO281 z#%QKi93n|w33Ez*dfoiO>q?q&uVMu`-s&Tx$iw5@Ey>Tc&UMKVf+H*Pz0JUF-Oq37 z^bA8L3!Ya@*wcWjt<@dskjY4)y-Gu+rBSNLNiZS+kO+`Oe#$UYO52LLmBzd`H5|S{ zD?z_}cL@4N#EMB#$TLDvzLhFciubT_*ttyD0~tF`*gU935iNi1gdH+eg+hpD0f^zD zTiNYqw)%9PuRH~YQI=}qjx59-+1r_r6z-x}-CcGjvs6m3^`MpOoMTtdghODGl= zT?vTACI`Mpm=rUwJs*<04tH(=u&mOOSH;yW3rN(^@2vPWsV4{(4~K&}`G)SP%;@d{ zkQ8+m7y5QR7SEkSUFQxh-cq=4UJ>Rx!?!K9=QE*WElTzX(3OFYMpiL3!@JV#8P(e& zU$oeZbms<1H}-Rb+b2pAIt>s#=)LZ~b{55)J+DwEB2ZyAYdW%N5O%Mz~g`X!e=R67M0D;8+S zlTIs#9{M;hshhF6rnI-bd)^-0+Fb~K1d@6FII@bFZT;Tge>K}%Eg(PFh*!LUL~2+l zOCk5+Sy6gT5}B>b+!&Ml1e_C?YaE??Qw{ZUdW`FuetJ7Lej`i4OXm7@-)rZ-R8dpQ z^1$?HuZQs>H5tT)JRj133N)I`N_wGiXHyY_mj5m@wh!oDa|Fr@|FgJpy8iGk#{!!shJR!C*}ZM;oga{{ z@wzjpE}!b4n7{|uvrL0=)8SH?+J=Z3Zr)mp!5#7~CdisU5UWhStB4f{vZgowPJO`$ z<5?Z)Z(YG@W$Zx#jJu6csTf@ON+R5H!k*m=P(dyZo4S|?3CBd8s|yPBmCL3dSq2%* z(<`_DbU-3N1~wX*?e&#BUgIm$MOC&zKiIFuW$P?by-@tN!MO*DUR5G|o7yI@+vBQm z{DT`!@4Ji=V!#s>!Z?j)v}e?&=-jR=iDE_gZe%vFN(r_zFTzp57s>p-W!?TSoJ@EzlQ_44(533z)&uCK$TkMd3^ZBWK2 zBZYx+spN2z0k7xlt1EGCJDfh~WEvA|5=V<8;F@TuJZaD~a64&Q12w78M24oF>%D#7 z&ZpiRq2aEff8fhJY?Ts}p?u(WRMq?pTVpOV#gx1|faT<^Zx}vI545IELFUIB z6174`p*S?Y`1@wJT@z9Z+BlS3q(NUi(njNJ%TIPw!cjs<1yTGD1^S3XjD|a9`&NPr zc#r5FbHSWfjT1cRCi59vqfl4iF32g@_N8GX?S94SF;!n&k00DkHJ1sm@a1 znfwh^tOg`+tpx63rU&;gy*z#5Hnx{y ziA#CAY&&qiYI}mh*W}DV#YY^D;-`I{`|jcHYb|}6=@%G~5|Gf65>U|6kg&pXX3KOq z&%gQrhK5DOW75cg+uQ4JkxCl@eHXk{i83e@lL7`+r;Fi~F@`Gr@a| zL5p6{m%|r6*|!Xa{2%ZNNoO<|l;E-qN~paIm%lxX$RMg-FMtLR1(*OVmAXLPLr8Q; zbuj;Eb#TCq3rQ!KuqzNIVHI=)^ _unq1gbS`E5g@{E2P_G`9R2`uR!Jy_1JZ{*_ zGPAD8Nizu$_!rSRP&JIOEyCU}^7K1N6_8vF16bFqEt@#yYG9Vxhe3wfj~*KmiKNZi z0#_MiI99BEGE{i9(INHfUZCIbO0)o|bRZNd#Q_RTX>hL~OK%)+@h>UL+8Z4J7nC3N z1HmFv`L8VkzUU0^aqMcNF17`(@k6n8RH27Rlaiw zJ)dmGotXYF>>uK70gbP_6?7gTW)iBvFN~)#5S?9-+R@t^G}RAD0T7~Mz!P71Dj31B zE#joA3f6+d;QSggPcREqq`&bMGscsQP~3u>+}0VYG3-(u=E9upKE#oS6U@dR`k%Wt88aLam`>47x(-MAF2d3($mZmi;U)v`-x7RQoG)Ww0`H2P>n z;1^3XK8-cAMF%SwPewEvVXlWfO!zFPkFbvFmRLVS&)@JBuE4P~crbfG5qAgL17m4z z7LV&#%5%F)5sEBIgFKJg`x65;wghOsP!34i(4XnT{@Ok>!)T}`VuBbgN>V1t6GTUz zG|H_nO(;Lckz^=xlrA&I(bm8hBx@hda?JXKF3SbMJ`79Yr06~m z`a^pGM!vG7%GcFZ9l+BPO)r@-o}2M~`-h@#$><^KW#Vy)baA6g#~nR#7*4w4*~rFf ziiLOmjjsq!vxxZMQH&;CTKyc(uJ9O~6|_=8xv4YclVa1QsQLb4$J0pI!|t~%s|Pp3 zM|7f+P|q9`|B?MeUqN&@|0K7|LAsmWwe8C@oGBfF==!7QprU zw)%K4dvbRQ6e|q97?P@!i15u9rg?3bDpWL@I>DTxT8KR_s)*6|QlqQrAIH&607B!%0E9IV&1lTJ8YeOcLdcb8?dWM0s;(det7aXo=lSUF8#%28C`3!&6mpmOlv< z(G_6!o?Zq1;@rU%K_Y7IC7X6=!tVMki4a+v8Vd>QuF&KOWwR0e;cClU?uya5vGuMn zFuw54-rd;B1S7t^@as$_!8u725?aM<{YxMTL(%JMC%@8$ec=|ok-4oYj8AC%5{0lx zl@Z!Y)oMk9ekV2Gz%D=1EuluN@UYl*+&L^(%N(ilq9FyhhPkBEIDV1bw0oyac;X0p zYUD`x@QrtDK0H%m1L=$F?O;EoS|uBEIjKg`QcI}g_B~vB)Filqf1dFpjPa$V<`Zzt zyi6{u)$l-^0lR?s(t{JJCVzlJv!Yd?`C_}Ift~pap`oBx|9qwNaFs4CsPW0{p|HD; zPC7#TFn(=IXM|&gx6OR-3YgSn7fGlgI<0xV{IE+iFExKq+w^_ot`uEf%^}jGGn2 z$7|y`U(iGSS{X?1Z()|M7g5LJT)V^9K*dEy?yupcF82G`NiL-xuRi4feF$&vZqc8f z(b3i0JwI2oO~M91iK+y>r##p1rYH84OhHls7C{#eE?s0=&0y?(b`F0nM+v7?Do4^R z5qE4vuXxbSsaBAl>$Tt&cU?-4UnH?Q3-9FknumVJZ%;&?*xoFK4TN zkk`tT(4FUH>`ob@B920udnp!6m6KI+rurDS`WhJbW)MOWa8dJj1^xJyO>(ls{PXMrvtZetTq`MV^uNf@ySTjradg+)Vw3u{E+Kw z-#Y!|@$|b7L~~J^*6AGsvykY;m|K3dSR36w9^>+neK8(Pcksz=#itVp;;_1Ef^F(GloW< z3S3kt)P|`>XL}iOTGa9uNYQ}x5>%=eTjX&T9#%t@Uu0Q=Hzo>Bes^b6Pl8HvOUE}K4xLvebxeV2wU-JX`-C% z!D-g~Y3?qlV!e}C!=!~mn8l+)2&=qpQSPOEEoT`#_^Cm*NrkEvm9vnMvaZsuv#DH+ zU1#OZ9k_F*`sHnaRC=s~LF`!s=W-cEf`C3%R1b zSxjnn`$yU-u8})b;Y9w!*670;+Mc)hFqgpxeE&iABU7TU_8!WwmhTNkR}b?rRMN%O z?JWC?n`1QqF>gatTilCNzg!up2Juu6=I`%f6xmK@ypKCluV?gq3QsT02+JhG@=Myv zf;{Ram_&q^jDyA-l4%EZ3XzBA?56>}nV3quJ#s}?ZX3e2R?QuwH+p8+zh7QBL>ydh zmF-8pN?gmhpD@n8rmw9vz$F2DzID;kRA#21Xuvs?s1|X})WdaT*t58L6u9)X1zn|{ zSJYCNjVY0cD7{FzmI@x&1>SKyt<^4k{nda+Zug!6$8w)S5=hF7Sel9LU4YoJ)kqtavo`i>H*_T&jyB zaY)3ByP&q-e6CP@z-0)RN!2qc$6UqIK=dw8yRNj9)@eJ7ggiZlyemhRx=z(nBo5ZmB`R=YT?;dD8f6wJi$ zPfOQ)T&r{|cK+==CT2O;KT<rSZ`-v$oiYuxiBCWNOUxW;RGzpzel01d?#6yeVf%JU&97+e1s>JkLFUN8xJkVtQ<_uvHIP7b6cS=)qIm()G0};* zhQ@i?F1(De>2Y(0p!EJ4S9|uN7cUvyaCT_fd zgw2NTQN(&?mZuuo)=7*RwNly?PPD&1&vL!K!0n6ZdVifNr@sW77{qb!d7g17$x;7K zX1^%Eb3-~4+6Vx`25=bx9T2sZWcNIE3b6rMbT2wB%E5H4$hR%tzUlRv*>adEzX;_P z-!MjEIR$5=0j;EihRu~$0nSfW`K<&jhM|!cK?pU#QT?@^B&4xLV~vBb$; z(js)~rWdXi)7)70x>%4d=X&k~=!$ClitX&xYx+os@tWy*m0PPhhnK+(`XN$~-*(N| z1AJ7wP5k|1$B6AZg5%uwkqiFN{Ck$dyj@OmdAdtLb7yRWUw-C2`9dF7#_giw5AAt- z=3S3wFEB2*q3}Roq2`Wj&^=W6tt2ajcEw2!C2{&Tg!R0m>iUxxd-#aT3oGc+u{QUcA78jPg^E!P+Kip%P^v!C zTF9tUM>TO_4vy#FnUB8E;u8ZW|n6p zW*m54hYPo1;Y;%>O9oG~g@L%iIOH22=q07)x^~(KU#3JQ^N=&O4;jdW1qClcWcr5i z4%?R!j($xMlqCH)S(C1KDYLQRMOYJ>t6rV5>8?1%JV1Ja+76BMA{Cbib>k%AX6o*t zVG_hYza~z~nnHwJNUP(E(7hF@koIqp{wy}FV$}C^QxGJ{QfnW*sA_L3BOYJ9RV(2Dxm1q|gSPh7xRiu4vqfpW>_b)xF~ z0$?sxa3?wzXTjZPpyWT3=rXZGeIFTA=?yh-S8k>lrome0!el(lolhp1AAin%mBOiN zms_=yh%49-YZj(yTn8xKzd7J}Be#WoSHFW=0&k}EmbjrXRZqUG0|UePR6WxPc+er8 z%?eR8WEhY7XpaW8)P2UvwOF3&qX~|J@15E=?Yaj2ez>cFpgtK~BzTFb|FLG~(3)1N zpK)cI6vq6v9HlgphJlczD%Eu5?fG53NtPV|c8(BLb%~IR{|H@^qVuD+*ujpA;+M-aPj>V>n@%C5YgBuFn3#-GvFY$%G zNYF4>VyQ<;rlx0|ltT*pxtkdCOQRiT3Sx!OJXRONc^3r-COY8?cZ(&XXiWq3uZQtC zR%+_e4%udJ<%}1h(_3Lf@=HXt!|JYjO0w2?(&yK@g(;CZ=JPg7g{Ujt1nnFL?_Qw2 zy)Lc~mF;%v-{z1_Dd*sQ-gTbR_dl-K@jMP|_!CBpJVE^*wuTDwCB87+>>>)Lo0evR zinuqu6npy^vi2k02JW7w07=FCiUv5_=+-}RIb*=4N-Ub$wr{zf|-kM#) zOu$2P(HK5%wop5LxG-D0t1ViTiB*(Dv1#NbWUNP}s~ov_R3#jg%a>(QhOeZ{v5wWg zq_l@NZ04i~JQ%KksyQQCNW71bJEl*qEtVfe+_A=Rrq21<_d^eQxFkDM8R!O2Oseik zKfaDr?`Sd3oE~?|FnZMuU=1N%YT%=i-5IcM{g%Tey8D26rJqW2?}cT|e3nn6bl4no zs$=qGr`cI2zAK|j3!h-3(s%hFt*PJmF6#y;O`#`hK}9%1=vG*Oo{+@Dg+dTh8-!L_b6uX?|@Tf4Bnq-$(JV@tbLW=pR*2$O8OFi@7QT7{}$rkI5IL5%U;i(rh1@DCBQ>kXpK%%EqSd^h{Bh@ z@GY>dD>^*}CpN^3AbE;$%h}-H!PMVMS5z_l#V?1*$zwlQze=}l~a1MG9ujW61MKx z!**O?r4MYeU#*~Ec(BCswJ0@Z+(L77k4WXhC0_EVzUSxgRz_UQ$milN zx++(nn|i3tNjFvx-h$#ct?Wq--fExIiZAWGL#W*MJV+SZh0zueG@~=e~=pgC$!Ihok@8A)n)wY?ebsI@&39hNWys>Dgpi<4-?V< zR@?t+d7`DEGBC7o(EZOwFMm#C|K#!-l)J3|a5h;{K}m@@B7x(hK$Q;#n+76;#4P|Hbm+Q4~n_1*-=NUgK~+p4AJ|+bPa}d{9(XMB&`nb>->h^~71kp2}6q8K6)m zgdri*-GscmA2t;3G;^XH#V1dEAok+rI<>Q&DqaQ1? zI2;_1JABa_jax)(s->|-E0hgMFEK*zSeeq(2>`#A(BcDu2XEyo&I^dgo&qR6JxdY9 zYo5FUE{>x#v~;EoqE<#?fU)LK=f7-1G~~p!wt^sMLJ^pnpT#{Me%*)qt$oiWNcEMV|Ljs>?fs@KG?jVr8uj#GOO*x3_7*G+F@t`xg1U|zt9lGyN> zGm19s(5bwqrS#T`-T+i}5ws04H5E$5upC<-sk7BF*+Hp4Z!V#G6;a>(gUa(}NbHP& z&Y2;Tf^*;n%uigKbTa<+9KNx@_`#0xGnlP$K)IjJ)QP7}7e#+Y-bo0PH@q^OjNGZS zM8df-&=YVgs!dRY=hrVa<;+&g8ECLei!MdSQs6jZQZf5M0ovh7F&G;~zzTR%3LD|T zCFi%zuVuZyDXwX+Z?n?smvCqmTM0NQ0+ahtxG0x%K($eM{vV&_L_8VIb8t-O?`7c# z2(kqc_O1iV?ooX&>&ztlH8;MVKouIab_0#qmbtOl9%C=L= zJY;;3E8c)XuFHE^lTlvl9+{h7FAuk8ds|y`P26AenG2D%+7~0)ca?LTSL@n0$bU_2 z2tK&1W$@qLM|9u5$^N$z`+rkh|M9U#`!Bz+{&rpe$5j6_!T-rhy@4iajz6-1GzP;r zImX9F;xGbapo%u-q5}rtO90l~K)l_-7dnH83&O=gJOpb8$ZE5pKgAjTD!L-*yHQ4Y5X}-9TKwbOiy0#Y7u` z0vHyhmr%b8P3pV48|itg3i@^**}xC#*U`kw{tmOCL#jI&9Z(Wo`n$REyXn{OPGRn;!z>%LY|K3WUJB^&D&kb zPtb%fBAsuR(^ODG-qB*ZOv7x|i1FiL+(C#3cGw5JIP}+O*;0jU zm3mL1`Eom{{1qXM+sgA1bK1fwE>DY_{q;m~Q(9@$O<(;&Br22INnE?~S2HVc>jqU& zI_WSu{;*XF02NPW*r2vo&@H0C=*l-;J@jBg;9-?rV|Z~7WOm`{FZw&{+=p(DH7jH6 z$^0(;<1R}1Yv5|vndc?%=|0+`bhT$9r88s*+A)&5xe&U*>8JC=-L9^Ua?*EEFOIR! z40FjHF;Jez-~w?5OxmPySC)N0e|{mj(S5&1$st7xvdd-u)E1;sm6gyWNpIWT`dnQ- z0y`c58+ctkatTSuS6==DjaDUvG^Dk#Oq5|cc$4BM_?tbirO~Jg9z5=biI8Z#Gch+!wMZEWtM5U31nWuUQ9opZYaI{m$Dg(wUisq@AZ=} zja6~E5TKcl=+3Kf6<>PH!gZ#hOTa|MBEy3uRH%z&xj>}Z3)h^Q@Y@WBLuaQ{DMPYc zjHqUcrhpZpfpCK{5s@JVFldvleUQ>|FRg}rML@6yY|?-Ms>YSFRI4MPR|160Gs|U`|3E5@amvVi^J)SZhQK+*)H(u87 z#T8jiHs#o#oE{YBMdSO`y|;Kl?{3rYDs=L@B9Y`uZmy!ip1SJtM9*0I5o|mwq&716 z3}`@Pi?X(dAvT-=v`6*{=6eW-6??-Zu!%+Ti7!bNaIqIgg>_&cAq{4ph8VB zq;dTYsE1pDaET(O*x{fuy@;Dh8lQ0Mc`3>k^a29b2n7zgV+IRt2sD0b=m5FS=3gIQ zR>06U&k6TSXxV%D3ar5~pC*V_V@7*2!#jVndTkB*cVR z_ngZYXfwP->}HMP1HrnL=L~FD8&a*jl=ZwVPWalJMWH7SJE>%_b0;iU`Fy7&?7H;& zb=wc))78Z14u0@4dj7J<z zU)ggb2{+yz8R$8?eOWjK;ddr4K+b(f4JmckY_sZtz|aB8MlsCWSD5S+xoI~9!w~#ynxb$@r4MSwCJ*zAm2`iKw5ctMJA)^nCJ5jBi}01LIlx zq&9FWu?Bn{21_i23WBwrVmWpObPb6)pCPBsE%6I@3WaDz^HDd)?F8PWp~a zxU}>ZvzC|iR34Md-;|xAU1Znde{FnAF|b0<;p+4E^+9y1=H_?@j9lTWdT*ISIK$P6 z-dM_qR&gS?N;>64jEvL)j7f5R zQKOx$!Pr}J;I~S@*5B8m0)P(*p7)%Y&kUvNl0V@dvt1Ru5|9)N?IO48jcNu%K)y~@pYa~fv`+kyFcGTtd4LD;4by@;TrSxF>pcxvS6 zM3Iq(QgEtU(1|N;XmnP6oKSx;Id++!#F!=pG@%MbgV4aT-Z^!!ANNqiZ2*or8d1JXQ}DC(nCoY0iV_a>HCKMgVz2{D=Y?B)C>EKCA$#Yltz}cS(Ib$n?W}EV`NKw@>?`lTNNbRGGe$%(b0piX^w+FJ+1_mQWEFX z?lLgfa8$ju=Znf(;Q)MabbuaOqgNL)w5)`rIsN<{(|3L+)TXR(z1rpZC%w zOQEkFnB1 z_nt3|ctV2&M|~KeF5_k%Uj7}{Y?NP+C7mDSP!4VtHiqbgCJzjfGf%{@?;E8Lr#=V= zLl@q+G83XPNiB=D@Z$Sz%KJK28_=!YtZOtVNa1Va_)iNM%2hP9f;UygVQGzK;uU%Q zyBr3{*D_K+Qjs~CI0X#UM-`KfbfFg*j*C>K{{lBCnVkv$vEQd*N;pEI@+j^pNiB z7c0-hIk@#(Pc4pb;~>DUSg;w4%XvG$Z1*V-_g+4ltn=>p{H`TdMaWlRw7(g_1=Vy+ zs_|SBpeWyD!I)s}r2b^ay?xx?)+V>ZS|uXB+U-FZybsg?$po**Jhh9)u5BncpG=EIvX1{^tf8kJ`up$waPc(JS*XB5&dHO}%)HR@iwT;$}P(rAIzx*}= zLZsw~6x0m-ZR*V@B8}9+xlFc#Bpbl*P^yN1yzy6?+_)`hbR!bVuJ>N|O+Ag57H6*v zoSCsGXv^@;&%W2GWTIzT;bDF(0RUPD(K=#Nmw{6t=S(c$n0^NDLShC?oEm z${9bq7$~~EnJ57(Qya|L62i^r}Hwid;8v_$cwL5W=O~+h^ zC?KaK=m4xsam6aMuCyBcU#y*FR9#u}_R-+(?(XjHPH=Y(?iO4_@DSV*EVw(t-GdVx zf@^T+y`-nRlb-HOX5K$5XR&fW+`IQVcNb4p{i-S_GqfHvUe+IGa|YG49$l5E7Zbzn zkeoLGZ~cVsAv`-xPD%WsGJBz4vTKM>u5r`7OKLmF#UUYBCNN))ZT3^2j%cbM4{Um5 z5V&jkW9f0Y{`rN+YGMvbp;wzwhFxbw*;`$VIU8jh$2#J4Qi!z7I_<7QpWQE^_M`me zWfEU7osRveE5&Zo8m1E|By3KGIa^)~Ig^H5G3nxAQM`POJv2=VXhwDPd|l*#pUmq+ zVhRbmKr7ozrn^>b(1Mc?$(GYdHyL*svQAWx%lRRRdYP5Y2E0k32C97n+Wr?Yw?r)@ z>RYm7g>U$9mvohx*YKtudsCv1qlFx{yL;67HP<7{7<8J`*ncPS86(o5QK#9XTfg8uP< zIU80oI`aCro*7j9-DUz2fFWo_L$v_$%I!fCzlQm`wgm-R%_eEvV!Hn;WZ6Kd<5Zlf zrQ@?<+ueEmkU5n{)0;_b$9w@w4sXY4NEN61^1Z@%j3+%| zZ)ognX=nRiPqF_|ej(l>5&)DA(9`u_tJ=SlU;kICwqW8a=vd)n>W|A^n?yu<>eBgi z{cU5>-}5eiBuD7~lEGkP{8659Fai2Jey<+^IrX3EwBJ+i|DNu~NR8`j0#zbj%85Gg z_sDk3_lNnSKkhrVzw7z`=f0W#vTqJX&i~7OFZr?FFfb+rdh-jai3g%ahrA?Ojjo|j z>O*LPRJG-c7pY>=#3py0;=XwU4A<{LSWN(R=L6U8gt@IG}8x0nhsB`d_}De?2V!wR6A&`8ulfD|Msm4RacSsCi`p8sZqFw1?)8%l!UK|Kc+L z`n6>G;bHu@r|8G!0B+VFKa#)X!~EK6es(T?T*3D@d1{QdL!*&V4-AAB0k4u?)1jPRjR5Yoe6Bj@=-qGCd>kjncP z@j2xm-tYFbNG*(XVn>%GP%n1~ln1gAv<2814iD8kh`b>m$#G~Z=W`n2bKAUG=$E=& zdsPKou<~}6fG{|dhoiJPYOJII#G4+ZZl5{p=9*+nDC~$U_Z_b|6tHAHUL?u!IV$}P zn&KsBF>D-RF{aS~DCr9$22o@Hnze7%h3Oyj4wlR6lz}*YHjdDkjRnPivoeU%fIdt( z=b17g6m`lS8zk_|3qho@ua^_&TmE^}yTM@50ik4FN3#>9G&uW zaODtCv9@xZp&J!J8wM_4Kqjo+)X)RSCX1K9kT(R;>|1XWAy;>2h!-x5vDbe=4g)68 zVUSY92!MeJX`aR@DhAFzMLFV-JD4XggyjE#(2kyQplV+v^ft#4?A*HMNPZH-_^220 zCWD#d1%}b2X?3L$1;X2%g=t07qGO1$xY zLMIykjwxh|SjGZBCpCA^I=hGHbnB~4P@C(v{c2NsBE{=PWU%~Lc0l5XTP(%znlqt% zxycN~AKk4v2+rzJOB{OKA~%vnzzB7uf(7^Uvr_`SF#0Vp|6whd<~WcZ_?6e`39mJm zvpJQyKRG#9QJD+(IxJy^Nl>=kk&h+yE*_4QpB-F2q#liC*ySuOOTRo_PdqhW)SN%# zlA`2^u8}L|%8jck-Qvmpq<SCiR6Q%vpH+jdDM%?b%`n*v@ zuKG0*tITrhK&z4DMm0rg z*S%(5o9f-+3y*9~JJ#LM%hy{3|9r;(N2B*6P4K@Py(a=k3Za?gNyR)}zqp)lXPk)< zkm_Y+X5$3>`?(GNhmObpNfCSkb@)U}XPx$=MN-_o*NFyLq&m3&-XgJbvM{o;vHb07 zf5{@f4o47_Z0p%0m7dZ~unY_nouWi&{6Oq%yi7Xr%^ZbL6O7?4w^NjZ67l;f_z@WP z$jfSNt^v+93Znp@lSoo`*;a3{1d}f&oRauT zP)GWGd70p~rwQ%Kz$WlinE#Kt25hmf-Z!!vve88A7-1gCTN1gqy+4<+i<1lP zISk1l>TaHWmp=h_`b=FIII^w8Lnxp&UU<_$(($9V8N4XBK$s=Nzlh6D zqqj|yS;|U00k&J8FX1nr+cE|Zp;$g&UHH zx&S1@7Jy`2D6w?R6FIUG(&Cxeku%oQ`6U3WW7nGpbEw}giC8zTNHAC|17|2`?;|7E zETa)(diFda8KzVqp9Y_|`8;Bv*hki%g0NfP6@{Ui=oJLBa2c6otXov{;S9~SAQ}Vw z(d8O+_af3_yfy?LI`gU)3Wq#KRVUesf*#=+hh8hf)ptSLsci-su1ylJ2Syn#1zs#O zDERmu+?YHahCee=iySqdEC@K>xv%03?I_ydbJv3G_Q3!S44jXc&qR z$#|*b>8XK*#6XZXgfJ+DIWFXkl0w<=B^2Xbr{n~(m3-f)J_uvzo;h)|kdu=UIgodi zf2SI5*Iu3C-UMI}mxZGG*e=zdaEsRML!Z?jFE$VQ?QS8Y-2xvvNP~JrAis>U=fMtM z*`d;LhRHhg(%A0yQDp3jzm=TqlK1P1UO1T>j$r~L0>!eQULg=8te_VXe_=+vGGxm1 zyiEl$SP|~Rzw3j=_zWSKP7$z}pv1Dmbmp^^4&gaf z`5LgQPfDZl!d^KKV)OS!lhS$(l3>g#xSCrMX~+)WMB2s4)OW~-9+C)YTeGNagw(c^ zB?`j2UwR4!gd^%`Fy_?g>6*b|rb{$yv((t7ejw&lBnY76ouheF3#69XfZUyXv5cS@ zWE4PUB5#ste>9HX+8Z1u!j)5DQs5x`c}4e0HK-u)uM$0}hNgT%ZpFYYVCIr6UHnte@;j#%E&Qn#G2-4)j1^5KVOoXv-@B*OLM!fjGXFADk;bdY)Dw zFbz-*{S?9y4!KB(2+#el<+7Nfn-)ui4tqae-qmF<>o2u>ouAK)EOU%xs+yY)uaoBO z^NzDutbK7DDa<(f65`wc*mrs?9hyE9ZoG(Cj4@TCH^gQ8^}#eRQ;R7fk8!$^X;UvR zbc}Q=-?d;G%JHK{Ech zff)+_Pura&7c466_%&C5Xbj|;AduL5qr>nogDK~XxRU3++p>Q~1!XS-u z;IDYJb0-W{4b}$TT&jx)wr)-H(eC1nZum2Y{6|i^%>!G347E7R*~V0#2e6njADkX5 z3J8R!c`c+jA$Qodhy8A@?v;sjMxug`obItECUaAI>{KQ3z>rRAHb6oP5Ug zHjcjyhH%9^GjTAJnfk3oAK$do*6Xb5FYR>A94X#Hz14UE+!lJD5fsX_+)h zmHN;qb1Hl)`Y{pM&mjO za;l?#sEA>Pb`drcqxN3)0Nq_Ep@t^Um3gq5hEa>qQ&?DHEFioJujd_O`>@sd&Zs+0 zY0yOwt&{PyM!DB%M|CYLZHIe;T`*zP9<6F)F<|BsLp$!o!>2sCgVPn~ArSBGd*U~r zC69ORv1gf~SqY~#>YG1bp4N8<*uih-iUdxXRA~w`_qkai*26RP(}}E15sS6qMzM__ zRVOK81m9XmvPN86;XO|xu&uVwqwDG5Zt?WgdYjC$j0j4^D;`L#UG7C^nFp+2s;x(R z&L?Arjqw5=Jv9N1_4Uj!;j0*_7iJ(`9(r!Ao}X{#M}6;(4pJ{4ZzeceZ~IgA?=PDo z1@29oz8+lSu8R2_YZ+!-sRg#p;GKHpgZjF1M=qNHyUte~=^wdV=s7o~M_*j=he+~4=EK6S|dL<_`fi>-aUocTEB*9sukw6!C51ile#fw|YVs{mr{(nbdH zF>$6$J6pdlKw;*1Ct+s>Lj1m`)ki;@03?2|4rtl&zP2TkeV(aToW?uua*v?ST+nTP zBgTnrjwNpYt>%hM%+_F!dJhPAtfTnVix)u#V!)e8Ydo{O0l-Tvz7#JIhU*qxFIIJ) z-G;RLB!?UUrMh81dIZ86%X57^mMflNNH9wHv`K(v1Dor1khkXBx>!o}AQiQ5zD!?T zXdetp7}_;X79N!>L#`9;E_fCe8^ql39(2M54d*q_MT}WlZlzl7uBoy?8gGe367DNZ zY{NTIK=zB}O`xZuBAH9puPb3JLGTdt@LOF}*&<^t zH05*CE=URIQdoYh=VH$yz@JF9xfyf_kv$EtloLTDcu&dYNqk`l;&D>GSK(uAY!NE# zuwx*c*rq|DSPEALXu~FdkZJ({QVrqw4^nN%9zd$?gPA1Zn*vC+$#0}u1(D|Us%M13 z>?h^3c;`gmW1&t^?6= ziYzQfg2uw~g|D*1)w6n>fg~=jQ}x{-uW3pO^NotC<2pGsi$UWd=BIh+&Qws#mMYVtxU+B40o@FDLB4mRKSoXMp?Ni@}X*R)#7 z5>%Lerdh2gsG+?;8SLGP^`c{_=W^eUfFb&>LQthq;c3Szhi8?Ef5yTk7~cNy_FM zsYWM8>G21tw)`8Z_TqO^Ee$}b86$zKx;42j1|)K}K?*Vr^z;{``6jP<6d{TydG1~e z&^Kup=9E^){6?w)J2W&FW&7Z*phcQM>y&fmQwG-1&V#eVeYLmj$TpgY$@S>AlemE# z(iXbd2osu=YLfoaPLJD@1mPY;?nIMeN%}01M3IDX{+Y1(IhK+_#oTsbODB~+{{m^4 z(`9&=54|pa(saHFk0*9pGV_$82IB{bZ4Abgome4duQga(af{tXvt@~oN!L!a;3Fb3 zak2%8``G3|g_6@8R4P%jdDG030?+X{65neD4U8+!s-baUS=km_<6_Z|_wf1zsN>VJo-N!5aWL)Dzy!6brT)Zr*u`f7D#u>V=S758-9Pv{s)hzY)iO<6!P+)AOjx-XOBMG^K75+c6gl59*%jX+I$@X`W(K0$a}7S8O&l;j5E1UvBNgNR z!`VPKaE-*lo#)97n&|{GiR9u1FExtn`jnIBxoN$~Y?@9hd3IEX^N@v{5hQ;Km3w1|TNATGHWezh9n zuL}D=B#VBga~HLE>`3~N5-@dFX5W1>Sb-2!LY_8n8#vB8$2O{hwYT7|B7J0Kj{zJ`v5+2&w<-e=w-(F1=jrP47dm!9XY>och6HDuIwiB z0!JXnn2hH$PDRYa`l8iDvuLr$g38dF0lV6ifRL+8FDmbm5feoWsx`e-0$(@ zB-!>Aue`gsvuC&P207#TVf^gdG*|XEzjBmO^@dS*!X>V$`KTWGkg_-?f)D6lJWF-R*% zHTz3wPlXWWXJYc8JA5Tx535!J0XuDF5M|hIDM;p%GhaT z?g=o;PG>I|$BL7bU*ujCm~D&mI9(XTrcNh>n1YQCmH@3?FG=SP7L-cgKzA@5WFBfw z5n*5$lMm4$ycT}JN0c*)lcrX|XkRYgxiQtk;Z>_~Y~#1&u2LL2Yzoy)xNOt5Lf=7J z2OSt{8tw-d1@}!@6102I62cijdAnaYcs16R`b1QuO=RFcZ9fB*9~vC6A#?<`SC~Y zab3hUMsf5B-}fA0dmSG&`5tyYf+Koe4L#RL`S>Lt*tbQ2UmROz=E@&u<>B`BJ|!)w zQJ&$^xd3V;5kDpku{Y6z+l)Wio@Ff{35kTt+)?JX78C9C0L806qA`x?w4d+@>y-kR z#Wss5JZ%yq_!|qu(s$zw*l`1&4OL3ChMIMmVsfd+VxaSn->9{%_R=P=NyK=J=`>wi zrm$9;#OmsIN7(a@)bJ^YtGwYi4!G*q4DyB zqb|UC2~H7omT9w1WRFHvGFDyl&+wLJ40f`qS3d+WOKfzOfFZl~ zYP{sQ0mUUL{^w#(h1NQM{^^K zO8bCIoI-J!I09w3fnYqoB6sVk;S@uOHfJ}sn#vH#ooEeB5SVyZ|6e*$hQ+Ms9cu%8suv1wXr5ym}{__l7V`k2!kQB8?bo`nSf%QjPd?#!8(|e3M%WU z-F8M8Le_S2KfXiK;&73*asM-qHa*(stYlZ?+U8NTiR%tFb>hv5uf?&@1=+M6=r?BV zyAAJFMeBFKD2Tfuv&Yms{E#}jgtD6AiE;8>aK)5XzpTb@i?bJPRZbP+NVSO@;&Q*q z0aOKr#AM!)mPlV!0??VbNVCRXEGeQeeK-J>2nCjHQltNU#jX+i4t12AL9o_Ig z+%oE$woa2hm(Q%rbA$XkRc#xb(XXNM@x*ZteZeM>kWh>PC;xeadoM279;0}Q=i#B8 zxKEu*zb;O}HsYk2x_rQuj=+>dD@uKXnrt9cLL%yxd)Hn9j!}1alsdlrIXdZ zv;a?H zAkZ(+tbL`~a5B`+j62v4_DiaqQsid-AYp-_m~s%inn%C3KZk3vd+WL^5h!}ao&cgA zYb=$wAmbY8ltmqP?k&#NNv>{%wIuM7+m_yx?GMFN&f3iB*rujKE(i9b82#5Mm;PS7WB|oRpeVN#8^$s^SLrrKCXn7eBMq-R0QHb; z(^rDz69Sk~?Zyl3t<1m8Czt3yA9dnh*SMtEiKF=BtjeVm#^=TX`44Jm6=V(PYBm4G2&@2)Z8bO{!VC;3C^R)oDNt!I5YBaNeT(Qzm*iOqML z3mEcS`1Xca*m;M1=cHcPO1Jj1XI@s`=13f47d&5R8u$`Ctk}<4*`4xy7+Bz(@R>`f zb9;iprM0#nDZL?>5`Xve^L@^F!XzZhaqT;KHld3yy)tN^)6W;RCF??aRSsb$FT`Ivt~tSGun+-xiE*l&0PB-2i?>cQkREi)Xr21h^0 z4U25kAEp%0zOD^8dKk?{D`{J9qqnlUZ0gIommU z{Ml&XZ&3apKO_IKCinMZj9<)4|3!`erRe^@O_TrAZN!dp^7?V9{|D60KLo~q^^yO$ zp4XBz^1pu-f32$f-%-gQskiTnF@AMP|8)DH`h{+YifU^Y1Z|gW-3w=$K09gS5=|zc zAo0lzjU_gQj?PG=@Y}j~XF^6eEN)vX#8AGqU2+KJ$AyS~+UB2d65DMkT{ocQgcX1; z{Mt6Z9r|y@>;FOlzS||>6;@6*uI~@r&)qcvI}$O}(FQ@t=b+%-eQ<4P z#OPk~I9;+^@;ZH(Q{XAXS6O-oO0_$%2XNOa-PEV*yww)Vq&J~<)UBq%LarVv5c$m| zh+!LiI#|yO@@!Sl3qrruo-`j;crsjgEHc6ekAe{HJEC6|Ha@P=ug>NxISq?B(`U$( z&L#g=d-5K;8`xSFIJg?&H7Ir$?CiJN6LfSxp(Jm9JYQ-H>i6^r8iSc|LYDp6G#VTf zUSbf{hxiBTqx-LK2eqcmb+I8D8>8ostdc?ZmNt`Q*vqHqlENtABOgx03!i~ppxfD0 z`q4FAzDGFhNkA(N&hwvp>E1*fXiOEX6Ckp?m!EnD9C#|KQxkavs68<_h6L1}%$JnB z31>I)`4+W5wdc4!iHREKo!GuCRX8^S)S_HgGGpoH5zbejmrYJXw>lPZ)?k#V1IFhCr?cQPg$%>oqH-inLt^|I8V7f@!4x~ zd%XF&npuX>^0<1wtpD+W1A1J1fYQzW;^ zyPIAHoO!QInQ^9rerjOu{fvg*1$dmU!i1ZMc9=sgS*>~4P$eZXbx30>Z(0FR?48bg+y0M zj$Ak~>q#WxTNlttpy>dtW5vmE#On9SwCWxewk*ZtfgGPkHrw!Yx)aM`LqkZwSd=`c zU!l`oM~rnAQ!Xg*eyc8d4yZ1jNg=YvTM90+EI6w zrmC|)WL{%hK{T0ycJJ)J)q^W1WR z8e%ad88#8o0UGLR_*Pu9Pr9vsNXD6>P*2{8ciarhc?A)!BU1-MQee4bM9aH* zFv-CN%v>zOVF%3^YZCSDFS){D>^Fy0*z#e?txD)x1y^%8-PUoJS3 zdqr^V5{YYqaKOAcG}}0zZ;f=*0l6MHF^Ve3|Kw`=ApWRWp- zXBI%oflFGW=#Yqf=pu~uBF1Jcu&DqhT-8xJBkP_2qU9Ih@#Tz}L|;B4_uwllS8O~v zN}K41bXcqP3L-b~e2-mNoakz4yXibt#&aWXHsNSdTr_Vb120Wnr~pF3ZoM|P6|w9F zLX8ln@pz{?&=u=-y*^z_q8+pk!)K>s-cByMJ@>>N{+TAwvW%@coK;rOE1aqwN*y^4 zE39!68NRApZ&=4L-zC~?d?rC>*<#&EGLB4FOm(S9{8n2+HaiWdEx`fQmP7(-OFYZ< zMcjwVJVTq0J4c~D@y!bH--CI+EZMK`lTXz`Y(m>1Wm;K#KSk<#p-iDK0de84iyxak z+74RShri>rsi|G7p!KEG;U!z$jwtL`8*Xl9@heEOqykN1WRK-{UAG5K@%t@$h48&T zJ~m)RSH7yxls3>}UncCCeXA`&1k{$CNpL>Zme@=R)l2PX(e~F8T?7I=myv=jO$vzF zW-oHndaS8OiBzkrj%X*oeCxMqQ*tjsSLc{$KFG^MfxDdWexeF2KT6f~BW^mp8jcv@ z?AbRQzvq<0$OYjsqP@2N)FyBUgS!MoCJp6*Xb&zav|+3lM9Fu`MYt&u`aEE~J%)lJ1FDO>fJRgKI^4wT&Fa)Z3^pCV z_@M;eQDw%cQXlE3W#H9~>SxqOD=0*#z9C4y|IUecQt-Wle3JG6Y5R2jqJsZZQ2eW= z<2ykCc;&C%aDLEEf721?KPmW60<)In<>Q{t+o}gvI_|$nob}?6IUBV(6lx~1aIEsk-0n{sx1`Id!$U~soJs&EWF6RV}4SB8->w$rD+u(zv} ziyak${Up=PjZNmT{>9}T+b~X29Y)ImkqZMlZDqr=$ho-k4h`Xz)Oz{+yE4W146cZcjv~t|(;6{Y^=vd!_TnaefrgrUM@7Z{vRgbX zJEk>9P0uBcKYK&A9N!gR2?x?hZ9yI`?k&u73u_red)eiX$=R>J7GM)~usE{h)$|k} zKmy}vQBqRgeXTG%oiRl-`~_6$OL$or+UG+**WPu=*BO;8yO6}PgC8o55>p49&8=oB zjyvnMgTQz>Ke%iZ%%k7?trQ&870TBj-_Q!#Fa%LQo704Xww1}Enw|!BGjaE>JGzgi zW53%6J%Gr%uil^++P*6VYbTVIJ46gv<(DJ8<&>U1YBdAlMRU{Z2vL5dCIMH~uw2*C zW4c&cx97ivL${Xi>cf1MRks0RN7p_8epfz!;xm0oITDZRDgW7ew_!|m;S?!K>v+P`=1UptBkqqk8AzjwU+p$8k9c|;ZLgEe&)*jXa#(#!y+F5 z{3a-%=i#qmWlgUY{CJi2DZ%JkODxO^x1^5M; z7>*aRou*gGqRZzG;a#z0m5L=(Ir`L zQEHGun-tEq{Gbg>!m{x;fp*`R-*O`i{sRlV>b}{H`6NCXMb~U$o_0hh{?r59XDv~? zN&5Hl(LFlzXq~Au`O4SYu58*lp&)EWhqA9fOww{GU23`jg*j~PO9_(5!c#^IVbO>! zY}gw&lp6C$RxeN4MZ0#2U7|QGD1%0l93fWH|6?8wtN8 z;Q$8HC8RA@5Pouk4j(Yb$c((j4Cd-?e3fKOAa+ZTdbU~+OY#{JYn6he;}9oBJWVVW zIbol>lo`#ZfG%iUuZvpKF+OU`@?)~bd6c)Bh2_tcpIPwAbwRuD?->=A+ANW5S1FncZw9mWuK z`jW^mKcZrB$l-Vs>t&?Zrwq4N;#UuSz&RKz;YfaguS9NkS|k@c`H2LWla;m&Q3SU2 ztn+5wuebol1$9Z~RCaLr{Ck~{` zKtf#3XdW1ps_<)o-J(`lF-Ti6z^Luve46v%i$d!ot<_(2eAbgXxt< zaEH=~)2GpI#_G?g218}lZ7CqC@l;;=Yf+7VvsK@TkiQ8d{_G(BtD6QqIz$j>87F%S zP~1AFmMcQThFuN%906DVX1S=0SDwwKnVIM0qA`W22ZVex7>GkK#0ru-@}c6?A`8CP zKGtJEwVtK%rHZ<@uBxu@MbY7H^o!_i#qhk7XS6xKye(yWO-WM-yKx@EH|OGf?}>bT z=z~V*Dj-El^%>>73k$pojdqD%#oUtzUxH5KlJeVO_q;pS1PbCC0nwjASX#=*HKt~p z(GkU1lXe^)0dS3!bbfW`lu!o7&nR=4cyY%G4co}I^OVMUzGa_ptnZSulEXZC4Qzb* z!E(u>Z;ltz_WAI}^R&4?WX^LS%tSVhda^<}-viYNusT9a2OcmbT3eJy0IA(w8pK?PETPtckpr!|-nNx{mahce01`Shqi;6>0XxF-=?HUK|2(U0 z1xqJLwN6|68XmNsR21y0Ic{-2*2qy@QR9kG!vEM-ZPAxDN7g94QZrUY0!%2;lcVS< zoH62Sa9|;J1Sk3k_XawxtUghq(P*zoPocs{wjlmmpba4q4`3-S^WN zY43uh-%SK_GwHhvI!e;&zu%6zHt9O|lMqF~l;U~=^O;R57n1|fA;T;!HzKZe(gTKP zxqNKwLi7w4kZNA9W>m#2-~n$p+}Kkr)r!bE_&{M+C@3U})9FUFxYlct_*<%Zj^E8(2nxn-22w@b7S*Vkyn9#lYErj6)2w|xC za2+dld<$U|t9dloIu%z2W@e=i`!`*oE_WDnvf>gYy=gm@ueSC#55nINTlHCM=j+eJ zmVwN*#4MwL^u&@Qkz;l>VKs=YaopS>DX%t%hW2Hsh*ZIsb`!QLc&x8A=<#UeeSZ=~ z2wpLD>{!a#4lB*M0`3-e>otNT#Q?}P8;1@CNbsD@0UBg9)3s*?ld?iaxN&7VJtR=F z`oVCW#tl?J#xXL3FE#>l&1{{Ze5dh#?^xASys8!vr1ZzJg6GGR2f}*&L}F^5Pgt~l z^goXV!4_WU1cWf024NSfga9E7oK6V2Zk&kT*Yihxo?CChHXF)gO)HW3e+*&JeG6e^ z^Bp&fa9$;b>xkE?P^GwY`ZF~FLKs^hHWH24PazBsKnO#oJzc11d$5`V4LKu!=6vV)WQ<&Z&b#O5WPF&I~^ja&GgmY@8;r%kXh&B-gJr;rE=z{d> z%?UR_Sc>X+UPb^V{|CzP^st(s5;0CpGH52_w;G6wSmgcyniIUp_}OneGq+_RVg`{+D{Fr)w> z3@tzi<6QI2%9nA9vACqQ(aE4Z5Yk#G+ZZGjqnJKEB)H_s+F76&dwMFP{;%u}=vgK= zoJ~WN2`DI*Ud(dxWVt(FjA$?NJ3pIPQ|h#9W^niR7?+ICM_QuEeoT{(95ap;HC=|8 z)kBByBZQQ!z_5pNpChD)R5OfVomx1k=UA>!bX3=p{0vc(?1SG3Xpd1?uP@HI^hVYn z3OzCYiz^+Ct3UgQ>R}I`n$3DR*+WW68k|zdV_}Jo`H<0J8aa-WbvepzIsQeZz2$Lr zGT1zJ@@orF`^0w@vE`x&7ZMPo9=`TnGOfF5$b5cyUv@ag)~!r?#aus6hRl{oazp|r zCLW*|nG49B=sH_q?5{I%-QdzHis8RWVj2Bh=`>BQFSlk@FK2EPL{Rl9mZ@WgvR!sq zDYowxpNxNh$=n2CSlWg2;!>nTQ%p}o zf=>BDJ@tD8e_!k$BN!;(A{a%)zeO+-{uRN9 zFS?&hcd1lQE*nsC9Ya7~_1T~04z>%`OZ=)s&I^wjX82q{4hk2L zgKG4?+He|RdOu*_nOmQ5gvzl$pqTAH4t!Nfw+6&+<|mBZz83V136yt={y`tlwG}kmrWd_X{F^FVf^jS@3^)h`Z2DGYsS2i7VydAR==~FMzQj=Bb2C6T6*+nAEixF;-4~5QJ^WyZZY$u zBzYeWpzKHYQf>hulx5$WQ-WIp-@BsL$9)3dt5uH2dz(G|R^N#i_ib;4aKpVK9oBg0 zK=j=Xwidmw<`?a}(@XXqdEG3tq&&JU6})&30t|^xROMW9)Iffc zy8O{HY_Q<=;L<@rU*UTR{_>E7n(I_9qTR!*WrpuDId@W z4WD(UG&B{wl-9tsSDlFjx6aAt@(t6JNm~-3PbJ2o;rhp#X8GTU!ZZL;NQ=lF11RC( z$%R5|LCZO{^}}u}WP`vTu)_*agH;};E}OB5QguC6uCC%c22%e@LfiRn6|y~GjpRtQ z2t8kI+d&WYFcfGNLpEdsL1%;Uvn=A7VF5 z_A$?)VgRv7EzIhglMuNgpA~%}dD3`969A1z!g&?V)__zxM4jrc)Rwe}7|{mfe-UK) zte*+JnO{&gr)Fh@NR=(SGpH8Mo&EXr+AV90Gt@a;EI{MYt$T*$Ge0vN#`Kl}XuK#V zK53p^t+n4Y-mpv!qHcs`&vpPnQX!bg+~COuoi*;Wz({y6#ys<)StxZ8qu!L|AOoWWa%a%bN9$;KTd}_YGsCr zt#iO#La5G_u?P@(^hhhCWMO)TjT%aBmMxKkjzMIn!toh|C$2U^>H{WU3z z(wI=gC41tDcpuF{OO>!xC^^D+yy=riC`ubpnFyj@i&FalW|^L z3i<8vTPquBBd+x@;xPxS0Z~hpp9fK`7wX#c#gv%jxbG&rxX^nSmpUzDh`5Gdn&i3N zT)prNrD;i-E*V3?dfnJ-Yr7C|>VgQCvu*n3OpQLn38I}lA~r9%$txu;lWxj!Mx2@( zq-HMm4I%#wg4}`~m^^@tGzi!r>u%633fDCnk6}%m=6gX7=aDhL%fG8BS0}zFo=91rtLZ4t2Rle)M8|nms3{3(Yyx%9P(-z6KZ`Jt_<_K9+OoXs47aa z9ZMlds!pk?`@`8bXG8os;tORFOD)w56Kl>2M<0XhGMRO^)6aL`PCoV7l9 z5QYr)H!I|V>`6_+7Hy0Mgocs$3tonq?QA@oHFnuUukbtJPgY0!fm|C3>TwD$UpChU zvPLf~oXNkc@<_ZB$88{3IElbrN52zf^N=AP;WZz8@QA+42`hVJ&7G)xmm>E>6ow=3 zcYY%Z#a}TrjbhEoG*Bn2OLX{n*nm0Sv`~W<41`PSmr99myUW0=j=%$m!o^_#QE2l- z6dnjK06j+WDJ!*GCgQF}j8~aTFo*m4R$oKg8IYGIQflX$%Ta#?8W6;R(;3!-2G>2I z2gus-_Bs*;IrzBs%uVMWrXz(3B8B%mjgB3BQkL%Xon!!_Fyc3&aQhok7!M!{*YdN$ zYk2IWag-QoOfa8)6%{7a_ePF)~^9>=jDX89Lsrq;bR5 z%*e|by94ddh_+R;%)E!Uf5%VCPTRWLRne>T8t$!E42#p!WTNsiJ-1VP$n7;oY0hTe ziEiU{ePTMrVnDBCt~Ybe*zJaEZ15OKl{5!|WE zsBVz<2%*@eHS}=G25^C)0uLrYBCIa$aUinFVHa-a8XmoscxYVSsN4QR3?iv!` zq&6*j+bxZZeAK+`NsN9mO8BV|37iG>_N;1d&B2tn!I$x|Nx|L7S+n zDfvo3Uy}BUbd2AaJb9aOHEWUK81|-X2XfdQARAc($VSRx_OdV15;3Rr7o;n=xk5p) zCWy!}_u->wbHC&CPkI`}Pb4zs9QC0O`2CkO0gk;vO%*k#L z@*G=v){BI$G=N6`u40@Uei$#3LPp<#KV)S40LQg|?HvctrC*<4Eor4Lm z@jr*|$V1_{KB$UAhnJ7{G&CCRf9(Baw2DJJn-*4~gezn_2(E_%*8p>6ShsOsd?b`c zMRkTdW4)U^#(U@jpCddO^FU#r6v}t_X>EISl%NbS%jNO>Y;SO+jv~L+is6OCPT7U~ z?EUOPy4z9N70f?7uRonsW$0*ZVd-jW@{82$Q-6f-?H1|J(uedGriPBrMy7_&zrFh} z4OKJQQXWSHe8khm@bh>7>&U)8Y|T@a^e5qUeQ@}F9wp!3ZRm&hrT69}jtknj21L;o+f5724D;%PkF zAI9!)`u7~nZYba`)p8tTIorFr(Hz1a9v;$ne^gv}-PD9m8HkRRgBmXKjuUlet@~O2 z<>i7?oRiZ-z?PUmzqD(m+q7rh$#BBk@wahms)pb|h^;_Cq-(wpPsaK|Im}80im)HMF>6dOve&UN^CpZ+8i(#oUl@R^Jmc)kXS~N9_u4KnIX}tY}?iQU&b8J>c zON<2C9;{~jBXbq=oXyAQ?SyWZx?#aaujz`=@O5(7gsCNmk+q?hCMcsK86*9>pjTd_ zxM{h_zP5viP>;x-c&#{;XR?GxkoEE7R?_B}4tB&vBSBR%B%A{)OY2rj3u&B(Yo6t@ z5AG5o!C5CRJTE6zwBv`HENMAdmjPl_?MHDB^0O$P2S;|gnlEgj_s|BfPmz@AT3g{bGO&%D>9AN%l}`tm0U#w?Anr#X{_w1khHHCX z(z`!+%SoWBe8tfB;JT11)RbYxl+aE-kv0CP!O3md8CpmEvj1zur;xf^Jq2c3_guhxu*zZQQIf`*wnPQrQzVT^= z4h;qMd}0peP5TX_tCCww@((=j&EUV=8Ih6`k{v{BnJGJ5UK-<#h{hN{Y3(Qb@$>Jp z7`i&f&Ie=?(VIv2dE@SDq+aLpc!BX`8r;(h!*%3l+LoTkmV2TrJd`e?VR6+v}9pA;C_9?bX~#eBcMAjz-pRgZOHwGE%B}!)&^brST=V;wo)y zj}XD-+U1~nZE}Xj`)02RB2)2E9L0~8$*hJ=sEWnOaVlt0c)Ll4A9}SRm!D<;gCAk= z(oZglG6$UJ#6MUEIi@}>Y2FrX2oHs$%`a(J!DV5pt-$#!%J$dBN2<-u_7=Io4VMvH zMyw}SNKYJ%Nv*{whqR2?3DmX=(_u~-2rm;QHw}hN-mFvQ8F6O9Kzb765rtjE3kPvtOR$crlW7y!m2&qZIN70>OVlBgm;Sj^~u%X)v#=%iZ27iB?>Y}NP5&;>q@>5RrxZ3zlIM|@FsC@ zOjeqIwIah9;bK2v5R8C9H1Rzj1cyrf`F*B1OzCLaRJiX@x8DwWV2J*L#EZw>MkZ|P zH4oh4>$lYIz_u_nDH(Zk-CRYh6E{YH%3b)Uw~9&soL6FPq31x$@7we%2f3xwUsw5W z7i>~Zub3F6ul5A|d=$yfIthOG!5M>KaspLdQ2NnHh%eD|v9(sTJXXcPKsW%z)Njy0 zS@fpy;s|r+B5<=%Lus^bm5P_7=AuYsE+n2RrcIPel(G+qSS_(qsZ1R+kQN=ve7b@d zH1;M-s3kn|0lAQk`M;M2KWi3(2UU*}K;MFj>QEo@xMDaIjf8L2Q=Zc?6fg**8Q5Sw zf0?bJ#W8>qm}KoKgT@SyGM`Io9yj>YEY16aR!}g#=XrosEiz)HvwQlAmCJ}r!E#3p zbOWWQT~7^4el;~N9e%m?Z2XmjyapnG?)SZNO&a{Zd|__5enbI&OdZd)ZnP>R3ar6vU(Z!e9|B$lq6tC1AYG;6i_7Anak{&+!v901R^+ql#^K6BJBNYn z!H=XR{Hz6Nuezt z5G##`F%X;5F@3bwT>p&pMYm>^@YD2DdMFyBRIFLkKrPFl12{9VND?!F01mM0*&g8~ z6aYeK`J;dEz+o(!rkTcM1Mq?=YtoNkjFEv#zBj6xO5-U0nh-BYx@N}8G=`e!W9XiB zgy^Tsa$LTvlohbwk-Qt8QT=#@s3>=Po>>KNL-&Q zgWD`O4GRo#i5N$dM5gge^ez%|YC{$tpTI}EKm=A3I>CcSxZWoMEW+XmGp`+L$zhJ9 zL{ekKp{Art(uuOe=aLRZhcz@mR$d4@)oy!J4#$>iVRgNb_DyK1l9xC37p9}|G82XR zLanFJA^>f+;Ffn{N&Gp!zsRx3&5{9M$ZA$>?VK~$lu`HFH>pcYvZahCQF>LtT548$ z7{>(5<_2dkdWVi<{4P4SvoOBe;SDEMj&aP^Zg8Ybq9*p%zt-RFnXE&D%KYHz!701v ze2C9$3diI)p+2kk&`-0b13qV05zri6TFGXve??k-ZeB-X&unM&(Frh7*B@VL6zsK8 z6QVCW)KV%qrKFN1g~P9Upg}b7Eg(uIZ$drfNw4}NE7?V+Ba=XOMsW4@l%>XH#y>2Z(2n;p4gyBy9tUM>)%7%qK3%vnLW6RvPql;VfiUJ!`NxEk%d2@FWKBcc zr9I%zj5$OGwN!S2rgLdVt8e*Etd*8+x+4EDbOkK>Eo@Z(Ta)tQmIZ50nRj2v7i5`7 zp1&EP_yVNd0#%{TIsfG34)|l0<&?^B0YAX>D!b5v{Nu9tiz6qcQN!ypKRG0VjdDvs zQbRUNcHSPWt%5rQN+CRj=>jPSg_Qpkfp8^|xs6ht{|^sNmr=2AK6@;!?4EskC|_M% z$@#%Wv@jt#xGB-F8MlKFOJ~R5X&zUXGr86jER8vRa(PmD^1XeuQ z#NBJ+#C|aRq3(F`z(fF7!}dZSo+lETqL+B}gOwv9Uc!Pr4B+_Mi_RQ#REzqncHB?V z;ZkKB6kLDvCu2J5#`PzH5tdswdp1hD);RO$8;$%!k^0SCf!X~w7Wios7Bt0I1sP48 z$LwOo4kCv$lO@8`1l`?%0U&0pDY_Mo7P; zznL#jZxa>8w2U~__71MSyRGxRcZ98FV?eVGK^<8pc-45$iLvtD)Z);&_(2xK!)hZE zWOEdUS%vI$oVhS87SX};7F}uXI1*J;%J>oJPESjp8l$2mBS?yUZ!3FavJSHP>ZKR8 zI#O?@sfLZ9r{(TdVYS;RDU}nOor@jJHJnC31%j8#{RPC8f2>F@{zz7F{@_gVNZYVYr_k?m2iUSF3eTG-q?}c z28B4+R!8w4&{1`gn~Hk-?kUz)`EGc#${U6is$3sZ5i{<@xm{(PD-(`45qJl$gnCG1 zi7Zz<2N9yDLV8Cz)u(*Og-$NJ!NMsKTtJF98ZXJo9BlbS4BUz+(vcdAX}mA_+at~< zG>0Z>Mf((Y}Fixm%0<6V-FtFw>Vz>vIK|^ zN3;;sXlOPp<29#KOQg?kBY)op?eNGO1x*>g+x);h+IWDH(us6X^VyZNaL9odgx(qT zObkYbY8w)@onqZosw}<^Ef4_!3i_9d!%I_18KjGp*WU=cM~8tv0k?|dI-eL+pV=6# zMTX4lr95g1tv+QSl$*BLrqB~jmGxYm_SQJbB>kpY-Y%aTjEfGz6V~p zWXRXEA~dtow3RzNaUNy!>n1qq%ssE9)yGE5eT`VB@Q(BuH^_cS{=0u%RoQ zAe%C`HQ|98k|0OTIKzyU&u14uH_MNl!Jdvkx&D{msCT{yDqlUIz~8VU;`DUI$c8b-zAN$DkHCdOvQ#Y0`FxIU~%qLV_C})H#l*rrl1VLh;_kvNRg!57KZz?iYT54iA#1mBD{s=Ld&E(7 z6KTkhN`_8wIXTsQNc?TCW-y!=_CR;sV)|XY-9%Q^ZyK=Z=4JcR`d4!^GZ1n9kHwLt zSYJ+0GbV4eKHxMu_Pl?Cf&7S56oq;)SyJ zIE}-8%Wlj))MF`%fDfs(jnb`xCg~wlHYmNKnMI{eOUSiV6B(Dg! zhJ!Uh@B`Ye>`$9t(cL27!&Y(Io{RFCBPgdoyLrm5%-XJ6FToNVdwToP^k7REqiYDI zuXdqxv=2-!hr?-d-m87Tf)=X)_Dzhed|af&C5qU$x^b~0w3FMO z$!EmIb3_)9EpOuJCOcO;8~$AC{Fr%txZ2Y zu~?oE^WBfU(O8$%y)#0j?Wtd5gHZWo*u~dHt!^3bQM+zIVf&`|6;|kWE(#5<%#`7% zI0TVj8^-OM6+SIRt4thQ=b16y&q6CxUk#(#^bxiL+|I*s*m8ZKzWLdSe$J@^RycZ} ze}3DM(k4ibdR4%2cIz~2I!CK#d_Mdg*fvzOf<+mBqiUV}oPI9?I96Bx;C*im4<^iE z4)tf*)uOHVwNThzx&iV?9yB4zs4q%Jsw+9XNeC%jiE)vodto_Wg3px^wb2CNOY{6O zNa?g5unzwBEya3qtmoiv>>5}JfgNFObN0?gfmt?QTq`u(I)Ej6WUTYhB}fSU zCzB~#5qFm1pwLH4v+To=P!-7Nn}Wo4cFLd>Vvda4unaEmc6y6YRnR76@U$d89RrySf@N^z(T#px+LJx=~2jh zl@^3qvDo-TyTHyPZ4BO=29veagXo=)9xw0sq*9SxH(L0(9qK|{Z*kY_ANJfb*5{5w zSM2>~VlaL^r9hDI#B$*rwxrXJ8Cwl2Yv?`r*xyg_?9;r&(Q?b`i3T2+z? zWFEcq14h=0#OJSp0hPVR*s=GyT~(z#lTp^!DKKNC)|<-?P89?*y>7 zc5ni;nwXnsg?$Ynf2*WR9vX+Fuj+a?k>jE4Pwdr%g(7fXlg~Y6M>8hrSv!La(~H~w z2HPFHepzRco>urVh0Xi>PWsIoSC3?ssZd24FLy>0lZls1F2@SejvBc{$j`8wM<$TAS z=uh2A(C~`}q2%ZE!y&Mk$$Zy@G!l!eGL4ZU1^S_b?**gQeAmX~{l>$YA$Kqo4WUO( z51qr>i(%`YCGWwAfeku(=6C7iUTDYm#U8%c)7OJPEj?spWF&?rMkPjqlNt&xF&%H0 zJOV_ideXFH5&`6wHq};iKq2826N&=S(Ih0e=KVk&9n zd$X$&aLIwXwPl-qiCSFMd8P7bx~a=s*;)D3Ru2a9^?p&)^UvFNZtwTM2&wc3K{Rh}Znoi2hz`(#rBadb+9nj-C(&LQ0roW=g* zoZCS{8%?&A+pcuiCr-|sxt-$!*}MZcL|%I7sP9yA;vUprqFRjBdMl~OiT5v)ApW;s zjkc&s|G$1Q{6$F&|ALa({%+j$1KLcdS&rr^HDrSB_eKIuW|Rs-j+s? zE9KxLdf3e_a&3j}FKFV|fy6fWKdZXEhNrfqOO^6V3I3j1%|E<@lP6+pHp|JoBiGj5 zx0-5~A3tOxtz&6up?Be+Xp%zxD^*qc|B1!FPZ-0$&DehxiGS@{sh4_F{%6!$wH?zc^EW>R ze;3DpZI{KrFzo*T_WrM$`42#^GMxks)fmm#uk_+{?%B?(|A_J2m z14{!FYb_I-y1g^>-+u+VM;?3q5r6` zu)pj7xx)T|ivMpF#z^~*3i~^3``=7({Ljvx5HA^5}b{uYhf4V-^Jo%wS8L)3kZme z^8Xo{|2Iy$|CQhWAPWD0=2<(-?Dm^Y0uHWcyVJXNffu`f$@z9cUkweGI8{d*prEq~(+Z}JuXUBe&1Z)s8HubIxZ1f^D@Kq-kR8UQwx!w)#s#odk1LLcs7Bbzu#a zdO1pgtG^2;cxo*D&!3xYjvVpHVaO9KI|UAo%#R=x<41SXjw)m~6ZXT0j=n!qfAR;p ztA}t>`lj~<(SK>&Z~}Du4A}HE;HB1oe4)sp;UkXr_x6j6YbDEp2~R4q`0knj%!cs4E1zvp-tj^2+`c4*z1H3`D}e z?wFRivyU#%yfXkG;GjJ;-dYQ#i*btII~i2+EEh%a8KtzMI-@F!=dsO4IKN9#6Nx2@ zTG(i$Jvf+bfbo?JAJ#p>4@aSmOOuJw?cXznIxzW5JY@rFHGZU75Aok5Vqx(cCt7D~xpuN(4C!eq3o1qKM|5;ZT7^u!M?!vkkeLyy)92pyxjB^Xk3hfl zI0AOYFgx*ETIj=tiz5yq^!X(`oXT^)UJV6L_KDIAk9dxAIHo##zyL~>onA9)ol)Ay zAKD?q=A)NGLpq)uY2nY3*(On^DX2=1)L}icD5pwh$6WsGs#m~Way-K)M#S~J!#188pn-ydscxAvI#55K!kz;&&;$6} ze!uvATA_p$qWJ(R){%uu_X}^83M-Ill>{?pCI_S3F`!p|($UiGRG*x-V~w(Q)l0uW zHy(kv5xNc1r6mI~ls6{UAQGXpS0!BOm?s=a{NYGihOXPTH6yMvtqF`@WWFwXUKqTo zI+_!XBER$nFaJ27O5%9-eU;GQ(^&*>{-XWOz%!lJct=*Od5Slt`gWp-L6)Qvbj1I+-XMtD8Y(V8)*ofY^d z4y<|BE+hdWXYxGXAf18|1%w2d056c?lx5R)B6%L(wqL}?N6-*PZ=;3bN6N^rR2Qom z6|{A}f-P#qZwJ-x3|E&|>i6oTI-(NxEX>o&D@-Mn{_FJl#vh(HRS-0XwLOFmULUKdL6Qi&lYuM~(vqH^S& z_e`>`T3!*fFcK}u_^wikd>h zLN0g;9B*&UO(T|1X4KB*w(nti&WeQs+L?9xEU>z-s;Mo*yOP{X;_(LdIJ9IXnyl3} zXn_CRILD$m+8;%u z1P%`SMQvJ(XiR(=+TFB?9m}~yQoLd*ZY$}&xZ%P*z;3e@Xdz&uY{P+3s-6(ufCBgGvc5};uHO=_l!AG| zBy-{Yy(omq{S1YKXT|wtNLS*I(2|XK3Qz?P*E;oPN+HM<(?W(rxSC_zeG9eSp43Hh zm^D7?B$`vQ_?nkRix}Kz<+X#6qm5n$z@l4_js{R+B9zurs@%@KFqq*X&;1tkkm339 ze=fY$5w`JU(`|twj8B&>V4@V(d0wQlS{$9ax z(z1oUlO-;hy@ovG0i>@D?Cs~#{cXX4lnSCU{@e!xO1J~sV|cmoLijNRJ)-sJ zgL|z-CX`?46vp}Pj?eVGeP1IWATaz#C5ih{Q1noPl>#-_W!Kcu6$+^+E8FBOS-e@6&O;s`GH_Sdhv_W8Ni7_av`D`8YP zl>k@^MRZgcFXa5=^ml{>C{WdFC}5-weo-qj z*9y6P{+2+%E#p9^6chRKfb#A9K$E_*krZ!T{*);3H{`wXnoh`fRAf)SvIY!M+5x-- zzG``On*tS~)B$Vn@1{@JkG1r9_CKU+K!NlrlZKcE<(*3#7N#l2NxvH(3-*%dU=B=J zPtytRTsM^eBq390Oc+Cd-kkmZ0S+^6kuHfuJHDE(i9|kPTdLPyBzgIy_k)X+ z+PlcwEb(}#ag|>+NoV;dU_ij;G8ElKrk34e3EzdwQjwtLYU}ISI&0fV0hqie@{ui)XBJHdv%!%wSS2b9i-v>tMN1 zkZGs5-JHHP<8boD7Wt?mv>DXSD$)4b^rgjJ9PDS6{-v$6AHR^sqy1h=8i^WEB+(U& zw1sGH-+O30FDX_fX=_K1q`ll;O#&O_{Fw91FWSHa;2XmtEO@@3(y4YT3qjIN<}CIj zO_Bn2k?q9FLvO-1w%2X?@qJlIEDxF5O0vXV>Knx6*FZW%uD_yrfg6AJ*x|tnEl;ZFkh&z0p zLVwCB(5$DV>;j~LnB;6y1cu@4d&dq`bDadd)505DoH`plP0zjpwHIjt)_QjnWtHKi zq5>?qemz&Cc_u>GX(Wo?CZ)Yyc$-l%m2CJ7bL{NQQi^gV1>UUe3rjGpSY&cv+esIO zYK?_Yt*$qe9s2JJ@k1St#GVr>L#MY6-MMYs5Tz$=O~S>f-LN=lO+&=9dc9%|ZNr{;nOZpq1-XJHjz~Gbi1CQ{#sP*#BTrRZGd=hA6e0{FdjoPd zId$_{g%YENLFAJXWg@964=6qw*kRriO^G;-ExeK7D zB^da+j~eT(A_8#%qd>HEr9Xr_(vgV5>gsWkcpoug4sR;hby_kwPRkDL3g zcC^YV%mpcR9ujjQkC)i>=+kr2x@5(J%5~rnxI^Iu_F245&=(tniqm%@?|CdRLW5ou zJtZp8E5UxF9EsscuX|K@62n4RBaUYsT^q*+)=1i^RoKYFi5N?BFty9pauWb$jJ1z= za$2DR$s?5_OD*)l5hF4^w005hM}rT4`g@e&3*FLn&ENeFYN^4#@92h;9eyK|u*{y$ z5WVOS_cvFUX#!z_8G?r0u_N);;Ru@Rf{{#BY72hK@gZ4e?-u4l8p2UA4J60dX&VaLHc+-o?nxnEXI7QXr-PJkP*?-qTx$VHD|hw}5~3Z-J@P(B{=)fGJc zt<>N$?RiCxzJN_VIt*5jQ>Duw%&S2rJQ|9E{L>NEogreGrH08PnZQieZ01qxT)E7w zGLzSMS*^o#C33&O>5sIO_1Y3gMq%IVMN|8A$#s@yS2ckTFwZ@N%0fihoB^30J9c3D z$~8OHj~e>RO*3mEi8-7C@(BPx5y-UZoJuX)h#9PPW7kjUboppK5gyiuSWS1H8hdYV zOd4x%qg8b#&SP9TJ-s2k>26~J#DTA$=*n@+3qbyjtzVoKpUOtxXd4DuUnm^`d$~mKH~tadt^||(pz(1f(<`W8zTemO)96LtG@ymwIYXhRJL)EA zNv5bOrr&2F$k3p|=5P923K8E6wz|Ia?yg`jz}2x39&Y*|#t*I-gw)Y6`3nUAzA zGI;Z%nNTzq&}I^8s4cQ@3JoS>xotUm(x`xKQbEBN$|1zx#`1l-dYSr!N9Arcln4%C zj2+EiP49<~MIPp=&-XHwf_GD#!YZCr7cD2+hPvd9Ky_@0QOUB@OAuya{A=FU6c4DE zEK_e4tolx75_uss#%IHqSkCr_&_7x zL#`+_opjqT`2MW(c@HW+#BBfAeY?@F*s45%J$pm8bu3L=Ctqi&v zS2kuh_rDqCYO+jXkl9pLF?HANhXRX_@^u?JSL!UlHEgJwNv|C}YqgCxf*6Od{@i1T z-E^+Wy*;T28zC6}$Z2ex+nBgM6J*L8n{1#Dk37Pqq#6jOPRi-;-SpSroRNNis}#o0 zEH7}6Y$VWKR4-X;a_1qWQ0EOaIp_eg6q&4fA^d~~kc_Lt7|VFxyQ;v#ws|QG*nZ>Q zWitjA8EbZ+9LoPz#UDH{J8e9oqQL|#uo4oSb1or~1?@1sUco>CYp7|qKzs(TEidUh zfiDK}xVN6E@L-OZ8)~6Uq9-5$0hc!s=yWED#JkPt@bn(~FR(2tfH{aFRL@FvpU16w zGSKh6c$&f92^?yX#*eoJ^{#1FWf;=MD0zyw-D`)rA;sFo*iSjggh#@E@-UHSV}OHd zx;ZP^1rLetLo$xS!b5|IW~PtPNtZfLDqi`yK0X?bmF{%U^ti1Bu@^>&%)cHVPadop zwjNvp_*sP$f)#&i=qVQSDx}in?$-Z`haCJhfmv>U!B~A}#NofFX((+kce*wBYhZWS zPhltNWv`!+^?6fBml@n199QOa<;|yjdV{z1PC#UA-i63Xc5N8#V(>i)sL=!$VVbcB z?w4@=%PVks@iE3xLY>z@jBvB1lUu&O#|i{-*%j-bqCM>DswZ5ftzb{rKWm_Zwt*7o_KwIv-ESg2i1$ienHD%I+GcKiBEHc>iQf2W=sECkX?wCotY zZ>}e7{3_+djf8#WS60lxIMv^}<1Dq2UEp@9>2OHlnf%iiOZhF~YaRD>*9jjQpJN1E z<|^li%C(-+kk)6Iv*kqB0fuDFYAb>9Xk(sUV=@zWYF`R>W(KKFf=Ks-!eZ*(narR- zQlqHLW)Cq6bmER)Fs+JXITto{E01c+2@(sY(8$C(KHW1I8YxcyamM%+B=d#LQdh_3 zg^?-rmKiIT52rq0Ti-52BhN@n$5wAw?;B+iv^>*9J$8gSQk&WS$h!7S!N37 zVZ3d7P)+`D+|~hzK1xn-R!f6l;?0=LX@N<~?;`u{l1}j`M~7Nd;Tvfz&A%B|xtEosE$DTG-7}Fz&f5{SRt%V~5WU$!wWZk9Sw8{*u|82MZA|V)1 zV*Gi)FJS$5I+rrNZUOQ3ea=T-j0xFlv=t{WSF z2q*ffbe-@gCYp;t2wd}b4)K+8vm`6cCW5}sv#}H2Y z7AnuO!|Oadr5LU*0tjRWiz))X|8ICFt$MU@T&z&{d*&b{E(Bpz86yZ-uV!ku@Aros z`$&B4FI@6>;(!+Cf#^Mx!}7%@1zC1O+Eh9Hku=5KO-t$PmwdV*^^C;OS^|PSdki8M zG`Nob78@8C{u(%l#4hVA+;riW*OnhVuALHfw3UIn4iuk~GeB>qduDBK2d&Q*;k9p{ zR=oo2e@aD-N>Z4U!h2_u%qJ6%`Zt?CAaUOuKOxa38Q2uf13c5T`8>rRgLqg?@!QZs z7shJR^;~PHxiZJ>gW%4zLz&j&t0KhWwW?%Mk(z!2txO_0d3QDvV9`8m=jcYuKIRfpzlSz1zeX594ZncBW{^=5emFHb>VP zx_95Yj}^@or{Lc%SuWc%_u}l8N7f<4*8H$=G_iH<+ru6;w@+;IVe^h^O7r@yxHi68 z1D;Y)R#Po&*2B0nVMsZ5o1Z;$2>reXjJZ-W4+aeh8`3z?DB|Jx`qGZ^gGSLw2@xXD z8hI$Ck5R`0MJ-X4J3m;cT4Ehbzl@M0HU}b~#mlT48)qKbE>-iA!&Ss&r0FLbNn^Iz zW~qfb(4{cv@2fi$Uq6hHtZV_lWsiI0#-)Xqu(@f5p1ttB;#Ex)XR@1F6$n!t&q#vg zk6&mH#s*wkUiT!N-h`q6f{1-+X&}t)oG#_{cOAo&~Jl8NLqh11D>TUUS%P zU&|zzUf8V^G#VG6r<@8q+LQ8yM_}#7-sKK4cGGv7=#Tu@^BMBenH?%w)v`~%6UnD= z+xER#OuyWrEN!hlZ8E73X#q+xl)`g4Kdwgk_xNs64#kYAO zpwX0W_a#N3jsAWyj|!24CXK=_%}t=l358ApRr8YqsD0(mn)V*;+krATu{P8;fvkFH zt^;4{0O94(U7N9~0l8yfNOM!WOEM!+lj2fhkgGGhHu&BonUE;+id+l5np6 zUM>I9UbH-3%$l6Uaxxp&d5_dqOc0`w))ZQ4O3o~h*4a8FlY|ltzNFM{#_ze|G%qPZ zd%6GSow;%^Iho|VdHL*~IPvbe`N)~eJi8j4O(gy<(TDtg_lpI29aVlggwRaWlwo0S zsAbA!nVWP>I5KfMC#>>l0cPex{rb zO$UP}gDM=@&4`{|BSM!7B?iOHj;uq3ConOT5imkf^dbbr2xusfP?;3+NQ{aRP@0dz z!0!ilBK2@}2ygaN0Zk3y%|ieJTs z4i*IvPK6MJGhK({$I$E>?gSAQ8_x=f{E=O~GxSG9E5Jn1fnM7QGNl+73>fwN4$7+Ge4f+3yjOD+%wk;Z}uqH4v& zAkv`>YCl~h!@)QX3`JrP587oSdWmtys09whh;WXS1bj-0dBKZW$&KAW<4GQo2PL6l z4`n056NzYCSqn2n1@?BjJb@E|356d~$W@1~{+)+H0EJ*Axhw`Q_GER;hRF6fo^bN$yCE8rr#FZ^ zXnj>M-X=)gy$$XtzIQ71%W?cv8u!5T@}ib886?M3g5$MhQ)=DuGIy$0%%@6q_`vKgqO=jJK znwBUWQou|C#F#287;`im(Vu#Z6bKteC|IG0QXq#{K?gsy)13gx@APYtxkp^td6&mec=Ui2UYQ#%VjEd45lCY=R?FtQLxALo# z2n;Q(iXjQw>5F{sCWt|iZt@L8lCuAX;xb_xJj#Tc9(s@xxYwI64sHm#ge}D^Pac@X zuh+}y2L2Ajc^QyI+1pJ30i}gPC;8_)zqUyGw5+C9H)KX&N?J?MC)(pM;HKsPgTzf8 z*5=c6Pbfi2@Sz6^-Lt^tT2YE@NgK(r9}#kqSENZg0^X!ROV*>sNW#La%&u|dK9p)T z+LGeb-XWc@OB!o2fAiF>)0Inx8($|oCT#kwwJ@>0DCU}|QryPnu3cuqV5YDgrGOZj z01+gx9hn7FXVyU=uYJ(X_KJmW|A7&B&Rs#QNQ-TL8uvnGEd}f~jCx;*F5U$PBn=~E zmJl)x2=O$0>^jg!Y7`x;Kiwt>21htP4-F`pZK#!pR191I`hjI|Wj0Q>fdY>)A^t;<- zgFcN#B10M(uS*8o#N8v`&?X<0;U~S#NduA}7ci@oh4W-ff?Fd+kUYQQf9)uaLXR*}~MX#dJuc2p%}b2-qy~ zAbr2NQEV&=DN#V1nwUk}5YdS@gLwU03+9j32g(!$?iV|==MP~zIr)8p` zFQujMG1BD@s4hG>-4az@a;$HABhfv-!*5q|HM}btk099%mKPj&FP=CBBNIqu`h43B z_(iZVnPU|~)Br^2K!VF5_@E=q%5;KBfP&#lMXB<% z)~^8II?y*qfp*e9Pv|=1q_G6H`m$!`rRppgLg{kS#}Z*r{6{J}PSO5A%1nk{Wk!7j_EUMflSNzoOnlU3l4ZQwC9(Nx6K<^*E50fQ4Wh0}u9&29cHPnd|E1 zr1cQ^sUdp+X|2RkSRSzu0W&i0@vqS?UA<1~q-aVe%?vt-WN*ST10;rvWqz|^KbT1n zNE}5U&}dN^k@GqQ;3uL|+)gDL>AEeXNn9@rGGi*fkivw*H)i)5S^P6(A-HnoGS}n; z3^qd$nL%;{Yp`KO0{%%b#;^rKh+EQI&y7Yuf_!8M$N}_-*RGcsTo^Dy9L$8Z2X57H z6Br~$(;o)9DzkW0+1r)RQKa(7g|z~IV$VZjbUbKqyPuO;RI^=Qs*VquDYuA2_JlY7#8LawZZI68X!CMGV_evrI@B3$*B{yaKe8c)a_ z)>-9Xb!csn`!#lX&aC}VVsaIhTf0!vNJm@Ym2xvn&|zE^$~tjFM{D>88ymfO@pP@Z zY=te}E|BUHa)|n6E~}Jl*?6*e%Z0i!=?D#tmE2;QXWUgSo7)i0(;Y_F_bk^Pg(t^C zp7*GM|LSVQ981d4dD&5P;7G|k4Qa@uoZW4NgKuwJELM-T4BIwhc3YyS-S@&ec_oXO z%cr@`2O?AMMn6PHMKZ^XBK4N1-{%y;6&IB%%NVeXVNn&yw(-&`R`_GvswAbJ4W;?} zqRFDAXL{hyoviY_HJ7=O+Oq~`$?`n?`loW8T&?evdvNyYp%R_HQiLtvm)%>ZEvMi0 zV7IYiFwOLVqRj;ma{@;tnObOc+5RLwynB>W{XFn;7@2~>NRw+YJaFM({PDTHA+_1Idk2- z$WM7t(UU#C%>HazwK>D0DeLH{JQc3~UT6{xJ|BCK&vev2QBlLU`ZMsPlW{g(rg&r3 z)=rud^79>5<5szKA!sC^;0bzfgYM$M^=hUeA%%QhQ+{W9vRjL>Rpe^!R@Uva`rIv4 zJ5`AOrEtg#FzYBOTKoI$*;mWitc+e3b+PXpu=_q;-_fC7+xLw@?g*p+ZmIf&admtazEhWgVH)<`^jcWgIjT zW%Zt>%LsX1DubFcPLuVEXZ0w7AgwAhXPx0FRy1T2=`7d|DHaDO0^zXA>qz+fDZpd5 z@a*m8qI$|*^@Yo6bu4*fUY+D6o6Ai>HPTUGr{a^0j1j|}X7kvPQ)R>U5pgHEjqjbl z1r-ljJ8h70(pZ-q-$&^;{MGE2rf0T4I5}Yng|6@TSsLYM<@1_mCF6FSc1>|Rm)+80 z^rjod)74Yo38OcLs{az=#lUi62nl7;V#bsTclKBqzF!VX-9s0de7K6jiOy-wa6YNM z+Ee!3oO`Ym!3+&8a6k;biYAuJ_c7okK$o~4yOwN5X;d!ohiXV-O=w3)DtK#_^q=`E zWt9nsjlv8pz*eb4exz29>+^E)D8*ILLfw%2L84slW@1XRX|_tni`-`8uJPyC<*Uk7 zt787COCYc2fjrsK(U|Vo(L%|D^O{2F>$dwq_!Z7`a^k0xVJjE&rcSSrDJUmr*A6$W zii@Sb#>WBp_U#xp(i}HAMQ2vWirw}^8+_({fhGHi3e}@_#w6~^0L>VlIZNI=9dkdy z7g`sAf`JigYJ9jVL!gUMYD%kkaa6s74HxDel`C$8K%JrY9@NvvOUq&^D+W8S4!`%u zaBPoF#<42ir`l_71f-N!@8pL{esQo3WBJ|mnpTTGdJWowUM^@@o~ax`9{CXb*xMZG7V_8na z|EtCH;$7tysuD1DtTY+DCYq>TKg<`mj3A(qyU6R6JDrw^x?X|rMJ1XXUU+d&(l zd=I=BUMF#+kigEEfR5}<`ct3Ng{+}Xp9$E^xQs(h1_qlUrKGKTs85RGJhIKvASf|h zTpD$3&a)0E$m{#E6_iZ>mOiJOtl!>{@vh&b-u${u?#G#`bb*etfuBZh*Ahd=XBnej zEw1-#kS6Xf;(cuM5~xPwcKYc%Uip!5JRH*Y^YDE53MSQl$k%4@*WoQ91ugY3wQat~ zA%}3Ol_^0KVlC6Fy@G&M?ljpx$kcsoMqZaD%(SaX1=n{>g-izxo2BWp^?8D!Xbl&} zr;E+-HH!N?mLk&Q<7cLrX=y*V*8tBJN5;u;2fn}jT8Fp5o%`rXaVYJQwoNUvZb=U7 z%KQEyJ&vW|OF5PfjYa^&ASeqOd?FO*8ZJR~aEeV8}$=~cXHeQ~1i#Wa61{d>TO#qGf+6}uyM_w^ORP5pXxU%kB1J2mF-P~ha&on#WI zxw7WQiOW2*k}lF9xjNm}g!)t(T3LCvoyYU!k&D9*1*fi)59sOhCNZ2b7p{@}W>E1- z@eF>RA1!U(^O*G!?aR1)W%Go7Q*5Hge)g4_JL?w$nfo&35sb(!va8%SbPw&6z8 zg|o%Fw3`?CxLWEphDsA)`?X}CfBeqog;{1T=Qy9G9ZYn>x`z&~%4sGH)?A^KI^(M++eb!Yuv;xm z@1-USAs#>)FF;Ro9=!W~=;({o@LzmJJj6QU&2y(=Fbo_me!V6&#MY~AWLS;KCOgP6 zF!m7KbOKt;Fh6Nqj_+=ZBfZsq<^ylVh|o!{?dqi(VO**ZcUr=17_?Ofd0luain-yJ zJXs@bnjU)5OsZw^+jH<`H4HG+>7EwbYj1@cT@*A*x=o$NPMXz+nW7QyTF{QNSiE9& z2Y2XX{WPN13pmLGu2_cRDpOnKh$u>DyZM|$rtg2E zdno$>WXtiKo)l7{{c8AXCouTtS+}@oNbTG*rysNC!Kzoq23T;dz??CxTF;K&cl0r>OqT+`X`M!S*#}527W+M}#4-D}9UTGD z+FUnO=W?>3Q5?E_u~)A%@fQ0WD*g2ib0JZdvYrfY=Q<_aDQTUiQdh=&G>W;0^}2A$ zT}+V)44To5pA|4P^NTS|nI4a)kp1zu&ukH8r}tuKc4N0uT=fKqfDCi(bUnW(ZxO}& zI;&fCDGm%MmFq4Y7EzzljO!jOl+lReP4801#+aWLQB5NHnNoU5pYr(}^b_iH(0nL6 z{M)WIE-P~>jwqwkVr26<2k!LgXTgW(2a8*HCN|?f%+I%Q{mqeO6jBtvn0iVjoSp=g zNs5U>k*IKGdAVds@vW+Tb3mtGxtMEYVz2oLNKBp&v_h417zW0yL zZ;yW;ulpBWi{+J|`85CXTcCf|=-3$aYHxj@`SO=(9q=3l#kbhqOPuFS(R2Wwo{con z@4e>?E9D7%7@$5&x-2I_CwqA6Eq$9x^dBFmU8c1RDCNqx^uKE8`Airlzed^t{Y|vg z*1DSM6*&}{W(ZIo&zh#ResOzT_SP?5hv`>z5MOv+m5ER(Ywx<`b_pD`CY!i6`D5j+ zBCPVf&%Z}5IvuMkeY-Q7>F6w))c)=*h~rhwj>%n#%h5XV;=bc8m#0(|^(e zid$4aGc>9_CnVddA5(o?vxfG*F)e#GJo7`;Q$PfHGB4XNy)c+PN}$>*`(aZN-*sM7 z0MB>+Fvp*%dQmIJ)=d^z{pa!!%vyb~@1eoI0u4m%yK%LUY5_QUzR7)1>UawnxF7< zURs&Curk!Q{VY==QWJx{&8n$iZua(P`%eM+)vFNVfUgd<*sk}8^Kq9_q17k5X5(gt zFHid0>uJ4<)z#Iv;?>h-?7>E8J*_hwr_k;RAGgiFxM}Osl__UlDFCz8l*dj-0k6w~ zXpsSmjId(8l$a0&`v2`kAPk`_&B`{^e@pxS)BZQ#ZYO{YO#4fU{LgQ}|HS?O zB|iN3TGap2Ntik4IRC8x{ZkkC&(p;n4g?hZ5flUj9Ozkq{BsMIJ^ivvU=N~Y8*~uK zw}lgqME{w#Ak#j#&Ux_H$C!fe`5<%Jf9X%R7KhkV`V=5qyVS{)CUs(F@fb6qNxK;d-D^89?C4$Q!Mo7%~P@Q zjj0hYzLfiS%9z4}!$;$rz}5qPP9o74UwDSw=!2zD2HXiU%py|B!e4gp_N1XPK0iVl zqPP4dp4AN~eg}g~3IzfV*cvhO2LqGL>|Gf6; zgA0!W55ARu-VJaKiJ%GgLYKrme?D0pxe`qEK7&WzIs9|*^!50weaRXj( zElG6lbmD;rUtuM8Y5I`c->T4}&lU*)8*Lc1%W&;D-y$^AIQdcO8%#m$iXsW&hsS$Jq?F*g}$@?&)-*`a8yf z`k+unnVUqgC>kOr$9RqcR=hHeiP-&ZSG-#+K0S z%ClBo`OXc!UFqJuiPus?`_q6NvM4zYB7J%k_BH7Ug*(t#EY#jq{wZ)sLkiJ7DQB49 zL;eb2uxubB^&sKrJF##C;C=YIxO0+2Pv3>bc78!L9)8b3M7 z{!8+GtrM(w?DnpG!NB0DpyC5i}zViZNO@moD#-Aa;NJD+R7w6bc{mUi5I$;p&X9RHfKA zwc~&XUyKDl<3{N);K5g0)-=X2)LiG(wqg115B}VdIQtgwI)1b%jN!wp3=Ge5)&Thi4qQc<9`U>g+kbzL#h|nr2nBAROLxB)r zG7_%%4JfC{QFTto+{$%ip!|Cf_vR^~?Fd8@0Fl&KG8r)FhXECU5F42a9N4b4Ul7!} zP~LwWe4&X+Pb@J6kuK;4z=c&JC%VpbI4h310o{8Nom0n!{a5xY>h(OFdwQlWmh)U1 zjBILWhHliNjS;7jD1<{bl1iv1M0?7sIA(Xp0tPa3-swC-z=iuvx<_DWhmq(4YCo%F74KEadZrOq!Z35&0B9Mdn4V~ldCYtRe1k89T=+( z{8?RNk#pob=pRF}CmnquxI@-xv+bV^qS%{BAEW_|#VFCoe<7$fm2C7aI*m-qx!SXB zATzr1qZf9>7jl!MTTTrHB0DbGXM@PX9vQeIZ4JIXl{4Cl!FV(q4T;iu3 zI|O?4S3NdJ$;C}%;{&VaZ$45y!91245lVnl57L!POW5DFL81JiI3U}-*7+GQ6{)ch zTm{C;CV$M)zP4-0kIM%^;%XggtMa8X;l>21v1i!4niYD2HU`*#3ZC}liDh#dlPYaX z^_fzp0YnBx@Z8!!@O_t(-BIZ z{O%pB>CqS3?ld|E^S^-#F$R3fsgi0EqRM4}X<+-Nt76Cq1__Fk32a zq+Z+^y6UQX`n&JKrRw$7i`5O$3O0Zj_3MtHu*9UE?DREvq~a4crVoX6IsMG$ZhNPK52i8!Y6xWgH;Qy>BhA zqaYbi%`zRtY*(F##jZb*Eh$x{eAK|k8QqF)wfQ_u^_DLH26@)jG-x*A>s0e9_7JS> zr$H-o>#N8e=2DjDeaw0uB<`|NV&e|M->B|x=0)JT=rCxUAEc9EJH*sW-oNyJInSbx_Uk}udVy@qt?d_Zo1RG zWs1aax&9AKTAwP{@m{#5V_oYJ)6r;nGPYDq6Zl0G&4&h#*HQwL$;CrB{AW;D6PH{Q zA}p8H15$Azz8r7+itp0p@a|+d)>|Dutw-(PN449l2c7i34E+|b4`6iE0ZS{>oJX!f z5B?sI#9l%BJvhGM1>-te`=#ZfdHrmDnarYYJ^0imDHdb17VHb~Vvm%$PiM~ssxw)jisEr=pjqh_F>OTq_G^Ir(37H&U2cG zfWR;0wnCBGX#beMkw!+#TlF~kiS01w9L;Q%?G>U6F{4{R81d*w9z?`1U+Ya9)7RbU zh?NA4f|P?2g4PA*9y*_oCY!K7l^++V$CKfu@b|9ui_)`Mr%wc^h-}EBCDM!%1W%{Ov!Fjx=dkGM zo3fj0iBdWfJY(vfhJ)@tL%LO?JqHSP%S}i3t&KE3{rT*RHg^(g%Hiha=??EMPq%Px zLHCcAE?RKJ420#P!svFRRJC(`V|lX_grt5$WU9Nb*@7tuGs$zFz%&n>L7I;1-+T z@fpgTDmlEyyf{-ZmhvcroD)ui>uJAM#X~{=8pGB`q_KXK2{yB#wWuU+YA_N7%Aq6 z@h;xda-FR@OjT@YRp%QgxhbA2f_-bGa7Djq(3LqtsO4~&(_cFbwy^Wbj_1}8zd3jc z{MfF1;na0{9WI|;M-TZ^u|g{nuIZq=^|~6KT!i$N!pOFrn0%3GCU6F$J~W?MLY2g` zJWcwV=K4Mz{$ba3Z@H$TykIHtFqKhWwnfDc3eQl?N1sr>=bRpP<|ek^$*^>UZuMH# z!ZbB0r_OhJwpXAuol?b;#-Qk?zA+CRxPL;dG)|Shtwz zX<}{j$P!!nE57Vbzn$y#XBV_NpSMi8itC%p5ABVT7cCp*!=8ih`*+KvDD2$>Ll>ge zsofn7$FBMQQ~T(0*4^-ftJgU?{2$DEJKvk0Ka0*Av$aL04cP-Qt&%TX(wzJGxaak* zb1Z%_^DgaQotYR#OjNz_SKCl$R_8}>8h3H8v7>&qA0=DN@H8jb_!jhIL#DDHbj2j$ zFg;x7m=oVt;~0vOm5oeg<2=W;R62TN zrGb;&+P))Juu6)r(cdgTDc=MpF^|GU#CsDnaCKFqri&X*E=@JQtKSozz4H5)9;0<@ zM}v=61Rg6JviCY=PK+;VI7`d+FgM&sIApC;6PQ?0o@?lnkly0ZM^4=2~+zB-$P1*6Lh(aF@N(OO@r zLdBxmG;ZQhP-aH9nlw~Pny(t%*xuol2iN!pDUcA7$4Q6k7rN`uFvYq zs3%e7I-~lr8%DPUa`Z_nfsM%J9<>gDqo%XCI3fpWrD zkJi>S*Vm>Y;C?jxEo_i?(Jay5s;x0F;QDH}C?k??mXEdibO3~0-ipZK*5a!Ho@NP? zyk`1dBAv=ReW9`9@rTyiS{n|@ z$i2cGpDi?XN^y-3VO!}e+IU@V)%2YoZz9Dzy{Qw`tIDfL7LO48if{Q4TIGE5zfOax z7GIl^!2A0sHYjQ`B+ZZ#0`0A=^mAM)X~g4hjxFx6ZuNyB+%eu#`o^U^CdRq7QTkYp zY&qoea;ffVYpZ~UMWgr!9Mo`YvUjS5Qn9_`<;H`cX?IguhSMe#`Y+#QjqaW24+9fv z8fiu3$1#2CxuVaf!nUXId28q&X~7%TIiDHhjXSQ*OF2>3ZD__ zWVYc4j~R9EZtvBo?Qd!m@opl;W|qS+!;u6}w_aJqfvQne1gV}pdKVfv*{YcOiT>zow+TV{6lsRy7YFq zz>)sZ=pKw;1ENP&u??07jnkZ6gfG`oHdzMqyxDG68-ijw43w(}Ysuh*y{ei|%j{26 zmqp@-(q9`ViNpXGS16<9_WfKmPCiL4UYsmh;h4>vbH2{iN9a=-tW3}_c49MH54-p3q2N$Y-N9nY^<@qf2}p&yHO%DBJM=jDGi|zyJZ+93PoBm55Z%% z#w69tMCS{cn>zlm3Wx8{6{^y8{Qeh@0|smOARNm~1Q-3N;sp^H%6w(0Aa?`g|*y@OErX-|hv^+W7r^ zr+S+3)`JV6=X;Dgu{N+V-83jypom(5bf{`_hd7m9(AIX!jSTPS?zylJ?K)x z@!+W*?Z55(+2-H@JAwJ_^nJ?3##Qq9FQ|2#Nq2`AU8nHS%>M4$iQ~52jo4fAs#WeR zOkdRB0orqXmLFP7vob-%$=_W)kZCDHE&02$?Z;l(3F@*7y3J*kMVI)>G)ZvW;Tt?I<}mub0= z^nH`Vou7F*4)D`jvTACaB*=Ec^qqH6cDx8&+f%p=F+gCY5*BR2m_LbI#At$-z8-tT zFX4>NX^obodZCL+GUEd)UG(nTe!Nq<#&<+Po!aVC2nO9th1sgXMXbijxfIf?Q%BGk zt)`z6$ao2Rr+o2c38NH1Z2a|}o9XR7A4lf0aEx-DN=~iV5QlPZ@|aFza^hj{mfX4> z%O8U;Z{<>EFsXl5EUIN-V(PEhD3sXs7oYNa;{6wj%dH9IgSjNoA5FGH?(0s032q*S z*Cr$IVol4%!-8}x40Ovar@rk~SG5aojq`cU=P47O7@zabAAQ=343=$v%aNklqT`H( zaYfkyJs%_cLDi{)p^hc_gnm$J@Wg`QkNZ$?6ur_*GIy`?yF^S-E;|KQhRX}q_am5G z3iMIXl~To35B55}YpATHsKYv`?)dS$m7M4q1T^ulm=!Wq1>^&C3+IK99VNl*MM_aD z7Ic3|=`$$CTYJ*?#tAJyPu$m;)q zq@`S>CB$TFc7Ej7CN%eHMC2buGc+x2A%xvKofhvBQa#IAiC3wI!IP9ZPoNud%DlX$ zLICsQ#K`fcU+NvrY*XukLJ^KAMwRHq+fdRNqVu_FVC8UKpwZrg4jdEr--~Uw!5Q-P zd{UDbejd0*J=5u88BL)R~x1*q(}#$+%-GJmWC`++Fz1VVPq{ zc^cc{Tt4gFHE*Ybi-NnJMF@;WTNJQCTzMHnk0BHRBViZ(U-?AXAD+!Feb44Y5*1Ls zV4p5NyhoI#SC+?34Ty@#xc%uyk!Xll7@F#n<&d+!0pk5zyalo%H(EK&v8I&;&JmA1VZ*)xR)O2~&@Ry1IW z-v@;MBHqJbfu3h$F+tXu6q=G7eP6mAn zA;&Ynl~Hxo&1_bYa$xKEwQ{7lG~LtVAtVansMAOQoF)aX^WLG;j580fJU|OE%uro4 zCdb;B#tN-5Y<9%1U3X~N$l7~2@R2srcS>N{yZ{8gChZlqS$stzPl58Hj?xRgElh84 zxULnHpq0cz=?9?QtD6M7}q zzjSRsB+I18XJwhjq(h;3i2#X34sp96K)IOiw3(>=j;sph+4@LCMnvHM_SpYA&dsYgBnU*ZJx$CHG264xwAxsN;K|yWRg@I$pP*1*BkG&j4Xxg z%h#{!td!HoiF@y^oaY}>4k#03l8g?aSnyRMDD?_3NNj;5gQ;mC`$XP? zBmFJE2SD>*ggJ4;ZgMFTfwBNHO1L2NR4fMWfOt=UKHBcRebHqHW*RhTULriGjtrq- z7zo&pSm%b0`&40=oDC?7ffB}`Q-<}p4Q!YYGVcHj>G*+k=RqWK;-D3E1Br7?U5;+W>YgqckgExTl`q$qzpnIZ@dYw(3kg=P+s=c}rB8 z$z@a|8uSO@HUet&Ve!DQxC?6%jReaA#P`98V}(ir<>=~Q^ikX~VeZMG=_wom8g8IC zEii$#a!g?baEAlOMu|ra@7X(XGQLU@9(jrzFuNq+eO9_SK?dI7=uKq|$%M#K^x{C3 z3=MW)0XYDgDBIbl!q2+d)b=)ygq&1gB~*lRMuhgaB%_c$GC5H$y8SLP?SUfB?%i>G z14uOTU0?k(COL>25%O)Z>{h4Ep++O+mFR1l5@)El)USVM@d>UM;A{jReo7?TjKyz< z>mWG@?Zhvq!*g(g!ij#_k^o@_F=;NktAYw8abf31z#Zj|MHQyMf5NboC~vBxYdNKm zO$B+sJgfiCgxn*HaBvMLHZc{U25m?|H*&a5Dv>bQ4VW49VN?4p~QGV0`J z5U(+)2ke0l%uQq%QR;N@4B22NnjLDU^KVut=O6C4OkJXc52X5FU(L{oMvE|DtqaP~ zoZ*5FV27@V0`bmbX1xy)$a99g%sYalR%0@Xd6XQ9L*h^tgc_uK1r6)%j6XV$Xr#n( zF3LwK1cUjFc|X0sYR#ReP`({G8Wj9(2qh)}0nT_wp6<)&6S@kMRu!gznDauD<(AZf zcpwViDsmpsf&14eV~97!)gA`LqO?d0Z3Yb#2aAqs+&n^%ez$1u^Z2duXmL6c^h7sM zfNIiw*LuVB%rb!?6Ym2DLp8Js%;HM|v-rtz;5z<(xb%peP+1m@fR7mKd6Bo zJTaaTKOG5PVi&NTM6JU$0w%Uxpb}aVVoD8qEcoRsVg5SEUJ-QS3trNq&yLbs<_%1& zEhzDX3=g(N6Ti?EE`)xVoNt}w@cXQGr8jo)LP1I>?CO!-z;xF+U zu}ne58xd_zH_Zws`g@z#K1)AN(5BB}ke}N>c=-B&XrhqeSYYF6mm;%HQRPaN@55<( zUWk}iQ2%|2jGf0_x5~mEd2SPC6(tM>+FwW)1{)sqgp*J~*RU_y6+0zVG&S{t*PLXI z4#(uU$)(6!8qAwC7TP>zqz!?I7G7wA5dD8gGaCw`AQ^&+|AA&miOK&1%?ybes)S5j zo$0s>9lKol4aSk%OiPw;?n3BIUhRbXr7f1q;xWOnsacx+gJwccgMnzq{vR|GSEs!C zids@!Oxqr}%=KqFfgd<~@DKL-rCA^$HMyLOUW=OPG z4uNQfUwJ|#Smktn)OkS@dI~KGfhisSfMkxL$6O@}p7pOhZlju;&2@ZJydy)&%xC%%W_~lSQ6ozZm^8ap5QP;wUhS-wC~06oNQk7M&33$5or}Ab9&# z{GG;?CW78zo_X>k{2Ku~s?2|&nbrS9GXcoV`msTqEB+0cIMY9HS~Ckdlx{JVq%Y{@A0(OW?GV}FneqtIVU zH~qn3ZL9PqpZkfgf%Z~wA}q(%8@l1|$wjyxe}E@)g=vL7FO0rkOKo9?NcA?|3`r32 zT8$l8cxDdGeAl?*&;Qp3SSwWVv-ItO7u8yh4WdWGZ~$!p;kO-3%85Q01465lY6uqY>-y-*f)rQM?W#!ew0vnUf04 z*W#lgyh_q;>T)x7N*&xj2pdlaoDqzUVhEI@y5N)q4;F$-`ge$u3BFh|^I-M!=J9NW zJ(Yy*3J@{?Q@7-^HIJ(+bJ;YTiCj(0=mJvoQ*0DxQD|eiy#7A;^i^txEtD~G)-+yO zsKBxASm*sX#`6V%BXVFC|65oTZMXjrDn0SNMop@3+&t_rtYniFTeu^3e%2J0BNVU{3gx&21(H0 zN0xu(HQWRN{D6fYy?o6oQ>n8jO=pLwp{KUxqrTj#aE&69hb^e#eAQb-uxWMH~F^Tx{a z2loU5*}8xCHyRv=PyD&RLoRISIBanf`-&-E?av=Zs8|JHK(10>+jc1KW?Y(mD(5d3 z4W+2>Z)fQ&xnB)uXg@{>(9b(O@w0El8`Kj^4(f1--zJ9dHI^TxcYQ;!K`nNITn8we zRhIBt%$P)(tUu*;sgcWaCOdn)?~=4H(^57KpcgtVq-?CGe%0zv{5h_7z1n0lVbs>u zvA-|c*A6K?ODC{8woShkXIWQEsCAk>;@YOW1{iG*C;UOwdamMMy|RHm7ut%2S{12J zX-W3mp@0>?^mF1)EmvVS6I6OsB`Clu@V=^?hgOsCz%O)hdFM48|GlH#o5azVqLoIc zeEjDrVx5Jr%C*Y%B-g48UF?te`Y%ss>!n}p+B&g27EK0Pwc{CF@>>Ep(XzIlZO5m7 z;)j2ZJ3-YBCs+C87vCg2-WQ!Q?W&)_6I3#+G=9$2>|QmtH`F<~4DI%T@gT*N{-uv3 za2BV8=u>k`&4%qosWE#btd?|&TB0WEst z%4W7HxBrZD@btCIu~}W^xG&M|d@bmDWf+hDgDmcA?``s)1sE_|=Bl!isb7fBz?u*^ zi@@G+s=KS5 ze|(JWcHhVpIIqLd5~5u%W`8?|uhMdS7@RDP$YDQnSNpJech`2EWa!^}Wuo=OUg`Bhfwx0U5~#&*o69q-Fpt%twVjBj8* z(Jz<5Q4h{I*tCVD_nE{}m3u}CGB?cvGh4!L!`AItl4JZ|9nVtZ*-T+LJ(F5TNsaO! zkv)@+vM;Y5%ksb(3m${6HK#Jy1#oZVKKyyFm)Cx`Ph2nCR?}4Z92l?f@fAn(n&0R~ zYG__QIi|+@(nE8Ui_3SRe6b9V@i5V!elJecmU5_a<$ZYyd#`wA=?wcc?nSugV6|cl z(^vTlSW6ZhG|sRsy#}wJKRk3?H{8uvAND;P|uVtmf}=p4SE0o z3beT|?AKSxV=;}vW?&%^rauZXP^3{P*yi`Ryt=ntzxeG)-`?-b-b37u@5Z+w_89{7 zPHZ-_xc=(xAB7M9ZQ77>O^1qit66aRs9uPDXSe>>4}=nI1gV-iS7!jd<58Hhl#z5dRjO5x&^wi~kDRFbN) zN<*7o&Hj~N8*>|32yqMWQ`NSbJvIq?>R+r~-=g5SCNRASp%0%OZ;fT-*I_e17zi*p zr>4FWxM&)`(?hjdbvJamD5q82$QyZwk%C%6$#?6<^L~6douR;|SB5p@XE(F)W$s*k zn0z`o%8x*2!IyQ%sClj4FzW7}ZZR`Q_In=vgAun#4_YqW5R?FdQ;ccPZQ?uTL-(+}1@6MeU7R(8u5W@_(_*lC>!*?9!w0I6{|c9Q zdp_20$9b8Y+<7;R^drznRJ157FVVE*kodztREt7aA5BWtC|ZDCw@N$S73&SLYlh}>%K4$7-Mc%Ps1`0M8m-+67^ z{m0>z-Y;P9_Mm7@_oE`2&wB;a@BZ%m(BrzwTC=og1QBb0Ho=2dXw)ayRbQbNY(jA= z!?qIw5Hy68nr-IA+bXflKVNbe^APwhGu`lO*|EaT65CK)^^eytuKS&QGw}8{13v~A zabK7Q*|!H_@vc6^KmPE^{ObKe9yTXrOX0sL_c*c9Sr~G7;pi?O&D|as zMlZ~ylaBH<-{U%`+)H!n+P_5KccgAe2W5w_{5`1it*H1C>d`(pMZRtBVd>Ns`K5~N zKvxPLX1HMce-`X1sZ-k2O3fpr+w3WF>609#se05rry7oi!et=)aI3a*z0{daJ=B4x z)b}h`>8dFsH<+^7>Syld*<&CyhVSr6^kq+I^`F#$=>$Xf_s+zR2^G~2o}HtdZC^dHp_QG}_*)d*Mww-tXJS2FLe(*suw2jM`Qbis zF@8F1g?3F$eN`sG&r4ri;asXVCH-S;)Hvj2A8n5FsZP?3nrBTT1Z7L(;g$G_?hD*$ zNvu3_E2cZd?925_0!B!_)c?M zIUB7?BJ$@459;@DcVg>1E)E>_gpo z@u&~&r{3|!OaTxXX0)D1+16h`4l@YU%^@yHE6!^sI%(>v&UdRDempr{n-Z*Y6TL|6 z>)38x$XcqZzw+vXGgppRRm;+9J}G6+tT+k%%&$5t8513j3KX8=IYoH35pfyWYUngo z7nV{@a4p4Ticf747dJ1@k?m9{#BU%LC=+O~y`1nBE+;+^^9Ma%{GB!P9gu&q7@1Nl zU|hDt2OMVvIM{^V%XOG-Zh& zoTsQ2<}adqxv4dcP4w%yiZ_ECN5^gTLeiTCA6 z^%O5%DCoYDon8FgQ$);+a}CckRn6eVM#F}(@p5uMgj+-zW|=6$Oex}K3AnD%!)~?y zTtOkr{8k%2om#|=ke=qH_p2;C23ISq=?tm5?=bi@P`TEm!|HY2AX+o_jcWEW*z+@5 zQAE>e+-h|{d$Jqxgt0`Y^sqR7 z-LpS~IfTM&Ff;at^Zv~CJloVerum6ULwk14NYB#x`Q-KWmhE5}8G_V>FxpT5Cfk|h zu0Bb8wew}eQ-7c18O24mQPrsDvnKQH7sC&qc+Wu%Pu+KiUwHy>+UOZh5jxpRWHa?KbgP`?m@Rvsq)63?h2@k%c{=+w ztd`>~r_OPK_WEs?$s2*Iva0q%237lv8SAT-`=XAHu=e0>XUbN|hAY(gGlQBvqqa5` z(mBsdl0d2e6USxlroY%Jr@9$0PT0~>BXL7Y)7q`{M0{~hvaD&GW%2aOQyQ+u5q)t^ z;`K_YOUywso;Pb6v@EZ5^C?hvyldbe6k;KK-3=cP=R6y{W8Anel=NmDAZiEd}pH8E6z?o(nV2($*7-o!p zo7bsNS^w4YewnAMnVRJq8htbF#^bJl=Ui_5lD|ciNjYYDNII(7F}rfJ8FiOs_D|O( z@00?!qFQ4~Z0G1q{OZu@9sjIxXeztvyF`?@dJ)%i|sBRS-RCHQdXDSZv+j* z4oF9@%Q9j#=^xS`*yib24eqY2N7e7uA7!7S7o0pjTmpO7O1P+)f+b6XVFaGK1@5R1 zHR3I&iYD9)FI>;K=*E`tz0?kp*(Oa=)8dCa6>jaNjUp`5`cLP0h9Ty>m&2;Fo1LO- zA1h5)DaKcoIbFM9TKnA=wW=>Gdw&6d^)GECM|uJ;zKZ!DQN{7e)6RaJp;r%X+!Xh( z4+s6;0?QTy=UcS13SV({Mdnukk*V$6oR`Ca@7*TvfQynW4kk4Una6vU7@QZ6j$2<> zgHc#@3@U7IIl6UZ%tOU&bca{#Ru{ig#M zzi5iLWHW7QOxt_v$3^NyfkVbLj^m3i7@g{D6;p_MI-4oijAfT$fYj%FrV+vQq~0Fy z^~m3aV!Fm`1{bThrJj*8|XCG>#x*uF@UvqEP^`$zB zIL2GfxXz#o6P0S}Jb)gUue=XKw08#VOluz6yGPkBL2?M!MaE zHkGs8J5~Jzu)DwCG!t<3GZ*kU08IBSF9YzR@4L6(KH6Qq-G3EoU7G-o#)nY>>skjZ z*B>5t8?8?F#Z}AU0whm0WE9HYuYZo(gPSjYB|mm{E_jW3TQ+!AyWj8G`LB;#(U1D9 zHtxUGJD(@qRua869o)Qao+W*!Jk99&n+6zeI0~3V`UHo&C-KU|(G(64)O9EOVHXYQ z>u=S8DxbxQiGWGViigSfdoqWT7dOZv)YUC*F0G#RtE{w=7YO?BCR8s>9ydgL{Uihq1W?F zkx|6mgv6-`5&CYc#VsJFD5R;3p@@N2h{DIAtCV<%KMbAmhs0Wb$41qHgdvFD2HSI@ z8HgB^ga+Obp+NZ;W)jJ#j{fWk5JcOpb4dSd7o-vdf=DF-0=Ivwp!gg55CvH5)WzDB zDt!?c5kx5v$}ZkUQp<)49nnghP9}tEvGPLz7iQOJI~Z8(B*=*Rsk;KBp9lRT59s-X zac&2j5RUx>0~R}l0E?aI{wa1UjJwK#`qSq;N++ET<$^-hAE-lSA}sQfk6z&(ppA?! zksnC$^AC73Gd#5FjlE9LO(F7Gu5yGmuLcNyNS_W4-qjw14ga&)iIWkL2_*=rmduLeSk%?( z2%TWfz#HYiJRe}Olc}L0$^zIwo{zO7i4HqG4BSqrgqpkdzdRq&(3^4c>#Zi{3N5r| zigy|63DRuTlpQ7o?2?{Ao=0IEWS{S-e>2v`Y=2h#(I9jfwUjNy(E+~)2f`#s0><5l ze_#?O)UNU`TOkk38&>Jio}SSmLr@&@tW9dz`l!XiXBZ43&PrvUR7wK$5D{<)aJXRm zXAa?71ePEMuns}^JZoSrOP(OTOi$$xd%D%kUNe&WFv{4+|MYxBQ^1{2JHj0LK;v0* z%Lk19I1KW3{eec`1$sWr72HCE5#B)0Crm*nsE+#&fza8UCi4e*Py=R+1SfKDI;%1! zl!*hSw{z^jJ)hQ2IAukvt^()8-mZ*)JRjME{c1oCU9cPh(DO-P*OH20+yi<(@ja=q zW+;%36rwv)jzG^RL;(fO6h7AM8`|qxJ%q{4Pd%XLbX77&&Q zcLFG_p&s|CgTo0ui3kVtrnLQt8)MQ-j+2rS5FD|i`PDuM&RO1F0WAb!NCR!syYVIY zuuejo2<6l*tQ39&1&cG=^*he`kjkHv45Wntq%M!Mcfqj&=0q6;fshezxl@E!DC-e! z4AoE%PN@y;FP`ESv{57(lP|S*zoB0nxdSkatYT4Zvz^589% zZ5#Nu*71rik*Zw9Mcv2YIc=3VP;Wa{u3tQ{~IQ~17Q*d9C&K% z2`?py5*?1kKQPG}2$KfB-&3YY{Nwo`37^E$$$$VoA7g+JJDD-i^PxrT3q;6coPqr< zh5F2mh7w-#zdWDtf59Z*f5W5@oBhJYtDE<0Q#a(jZBrLVWgZ!s=%lsPEEz>QLr+8E zSR)N^D*rkK3l-PFwC7iO%nNSq@9QZqZ-99i5LZekrln*b3AlM2PcD7P`e3yhI= zP&C$>QJG{JzeSPkK>BYb$AlR_!UVN)&f)83*VPwSu)IA|Ci@Oi5vSb&!;o#XG)Dl+L7vLd>H^b~#C8@b$Q~V`%WDYJ{wUhahC! zh0dd(xJJGi*RPe>AwNmddt|H(5=kZ+DYymlj9xE_C!;U_$2t^?EP|GYwBY$Gnr(Ia z|D)|4fMnaYb=|VfS;j2ewrzWsZQHhO8?)@HS+;Fkv#eKZoxArtYoERMJLkT8Gowf3 zh|Cz-`sh6)+Q`4P@23VAi;R+VN>bX@k^_rh@hxBBECk}%|K3OrBmvICe*23+G3@1m zfkiOPfme!SKTfT!-QM3lVRz-kV!J7B55FfAgE^PTc_&Y6x&%<%Z-f&&eSSP8E40ERKEuf8shJBjFS~4mT?6bbL>Q{e$ZS_a9s* zg-TGAg}C-i>*0slX>B>zxgn32;jElH?ek0_Jj4=u?OHp$sWrF}3SJnVn9 z`7~*@eQ}-c|KvJ7{MqL7#dR9|Tbs|HTqkrGcGCEl#)#;w&Zv^8s=G*fd;4%lcJ}*> ziFn@KNBUo~2F5U&MMTeLQfBp(qHB^3*PHtez|}cq3Kben^=&j8r%n?kVyGWcKRa1P zH!E@F($EM}c^{l67&~LYWvd%EvD;rygY)i~BD0tt7O|ab)s+(=iykL+TML>QsJ%|6 zzzhz%zdbyeVeYveXvQ{d0!u z0`0qT46y7~eUAG`Uf59LRJ3Dc6n%F)4X5OWZO(g?ASbMQP)-Yz%|OBWCLSpstvROo z_CQ;_&uu>^xRWj_IVP7v!%ue}CAKGx{Y@@A6o#gy%y8(s6yAjw zDI#Iffmulg4*S-anQa?S13UO*@h|M)&6QJkIfm1Uy%cpU79M*nI>rJ@O2>`6nmiNgfN^4& zKPg^Mlu~){KcC~CnME?DIHliSjVYY3duqE$&X~HT7kf8P_N`A#nr)xb>crP#jHyhb zC>2sv>1QW>k}S9_&jc*84n@3=$EuP>k|cM*;nYmgaPqZ14vqJ46*E+wTQj@N1Q~H> zb>G???B2~2phb#VCv;m?Wwurtek9LAfQokQ*+$?To_LFMdKU*hmUgD6uhy#iM53{{ z)xOF(B;D8D_l*LdWx|0(EgdaUo0;E^W}Y3~RNQ1jysnZ!8~GH)cyVDv+nnP$+48bn z6d%1-g;$S>l9FZCTUS@NJ^h9ScgkY$<++|$6pm@@Udragu=H~GdgCup3kYss+_lOB z>o-V}SzmSARs7W<3)wb^aX%n4l;h)%6}}h4b%v3I*nrCXZe&R76~Jswur_tWB{SZ_ z2+PCr{G9OiBI|}Mcw|D$GR!FEsOvnff0p6kx%NZmgFhK_e7Q7gLRdLF`~77z%KO@L zFF|!J6$@z)dTFx5NTc5THE@`*SwJTXnfmSRgK6XV_sy(}Nv6syN-BE+kD7GHz50DZ zk<6Vz3VIazQnsvn6TP5y#$uAi@Wr^qMa%JWRfdzz#n$?iS}L8{4Bj+!0`AJ%BRG7R zLi)X$JhfWU!}A>{Gx=lE-MDH^ddbqGjN+A*bn6-Xl*n6%F0=8)IWKzH?m2wgt6uP} z-?0bf4pR~2arn>d^yuDIgs^y)>J}&WCrMCd3dJM`3oL>H$&;H zmtGH(t}Z)kg+WB+0f%L&xTj6`PRk;!$MDDpJ3}VUH=1a@yjfFr_Ki1Z=p<}S^vx{o zQ9S)TG|Wba#NCYBpJj|&Eh{8PLNnIgqNwg_+T$3x@!GmcKO51ZTC=p8cc{5tqHRA* zQ+;tD4-nTB>naed<1B(TR=GJ5ULM4dD4R_-Ruxy@ak*DLyY~m4>$jFPZ0aQt&?n*wjuP)BSERcijQdWc)z^R?Gd6qtquyvThs;sirVqBP#p{$W3Z)NF$&vCEe z{0meR_~|J8#MR7QsfSwkU`*x8cuUQ-e(Oe2vJpf)C&nn4N!P96;6+1q%Yr>eP~miE z+VM!Ld&mpc@uS{kYb}rS{WM*mL7%-Qt?i@%*@|5TW0GB-{_F1y&&?K>v4LRq2vm2x zMtCwxi3$&z%2&1C9ag=g^&z!K@n^TWE^_#x}W${Xx2vd(A)=o>#`a_5c5YYn5{+~PiUIRI1k z>zyZgG~ROwN|;_}b3L3F1(QWHLmPt|7w>1)*s2wZN>;HCwz3Ol+`A?9aDalCu-oxh zjjG+7#7oTi^0-OKHJ51kTg&on0yZKFfTdqso=BI8;o&4~UGw7_^JvJZi5n^@3R+Au z3wBM5*wT$~tzsoc25$vW4@VQEN6jNWFeHYgE#Mt2z^1>eOh|dM<~uhT{0;%a$|-6M zS8KAe>8$Nj6c&Zyk(Jwfx2KA!_B8<&I4FKaX`FMQyeur$I1|>uD14%Xmvu9<*NvK> zr1x>i_kIvC3C0n<>(A;Ke(IEMHe&-VJ523V&23MGWAygcFi}>oZyQPa-BMYQ?gWLN z1#Y^DriF(;-+Y7hgL*1&Yis^7*t4a4t%LxMCfwab@mOULot1+Obn%d8BJ6>6p7Me7 zC*KG0gxFr%y5IO6>SIa>T&1MC({ck}H@?;2U;w0L#71)Vj#=!s=7KWGO z*uig36T;aYk)h2{;_Y&rk3A*nAXim!izTUQBM&pu_?!C0 zlDC)2Jbno&33jny>|@wsJbQ4vn+$Zb8PQM+;93RF1s%wTJ`w`!$Wn%l%He z#Sq$Q`sa>Cn@3c#o9B;prE&bK$})xm-_{4ay#V|rjQ8MHrNNyTeOXUj;&Ep2?)n$V zA%k*>Lk!!P6}dEYkgIi`=TOzv#bORQg)zWVT*QHt8quwCi^xVDYkXTAz0S`Kfho zfAsAb30GKyo~lxkGTb8h8ZQWJIf1j< zQX^mRs9VZl=Xcu%eGKjr+Z6MLYUN+)CivVCavSI(0WW!cfkXyf2Wq9}ez|=%YXUKP zW2$dWPiyO2bsU~FY&dMdb88ik#Gb#1TtQy>-Lld$1#M~F8_ML&mY8vqU(0&b+qjh% z`R}mQDCkxRJS)N_xH(L&N*}o26ERb!TA4XCt#wpmH>zIc9Adw^%6g~Wb$m}z$Z48) z@8VMIFFran;Tq7V&q#?n1#YKJgGy&K*U?^`^|4J~PV0zJ98czgSVKJP|CaekZ8dYo z+FoHZp^^4_DsCZ`Qm2=M&Fzl8~j+DDq+airpWsmvzAUJKdRwZJpDkXl^u|Q1Kmkf$>5oJcumCd7=6t zZtO?K@}3s9S;y#ZV7Fz6L2271_yS&2!vG407x9C%IBjH$3H9K_ZIis+<<)&d9*diSs+!fr;xDumo8w3PxSGV~T4Yd7!_97K!OMjPe z67l;ci*7B_Te75r1h%t^2Mje&0>}2 zS;?uz)D^1Fa(u!F*Sp}|MxQ2&+uG}qR%? zXO4xD^yY+%{o!mq6h^&7KEI21=SB7Bb7})+)heYNKo4Duyj>){DW?w_$*P6`*l6!zJ~Pwh9$(m zU8aAh8Yn(MINc~2`vQ>S8w>T} z$bt6ntt|vc|AjgB*WLNNRALYFb;>^m%fCJ$4b8u_tN;5t|6y1EPb}-deBZy$4Ftde zqlWYU_V@gu)Br@d;!%IS$M-cv{>^v)IVb8ruJ0ee25LrXroWoaGto1%(U|`6slUYH z_+NQzf0L?~?Ms{ou}kpKuqDzxC?>rPG9Ra>vmdk9vi{s2KcYdyp#lFI?3nMxd@v*c zKnf-R0RO)Z_8($pp=SQa-!DDgf5iItE%8sT*+sI6T*3lLHPq(k=6C-2dC2jkFg?khhdxCa{mKv$0cD+WIR-_2oJH8UR3Y2PiXK+$K z;xD46hQ~bf+@PS+0=e9)QNajVom2KTdu(c-W_SRAUR!`f(uM&Yy0Zvwi~5N&O-+)W z)_4C5B$RY-ZT&YO&|rqk!n*MxvqYHm^RGULh^%1`wY0G9;wYn-{-F|cDN20$()DaA0nD)u*O6`Khw)X3O%5n+1LU; zW$vKWvLy0+$oS8EygqpC*e|X}cJnt?-vevk@jw82e&+e72pUN3e~KszXMCSL6N7yB z#4&B0==7;Nj$I2wetNC)`dR;ZOBL%9>nN_Y`CbUD252cW=Z zI=|RAHnFneg=g~)K#QRH3Hey_2?=x(Nl+TupQqLKi$Gzt;kDtFchjRcOhWBQ!U+PX z$fKE(z%>XeH}<*4k=YgST~I*mP*!`b#tCp11L0WXUQx5Zu`wB0IJnkal8^Pr^F>KD z7ZmgS=wk2e;z&3tm^m^6Lq!E@(>G(v0+C7X;|Whx$G zq%}n7e2PW5i8B6WdQxAcPEP{Onw%;Ka7%xN%PQe_%3-`8$YxeR!0ec}Hj~2Msc}=G zxU6iS4}!C&H5Ig2BDGitM}*lMiyRy$?fhO#Rxn@zk{!j-BA7KF83#(7KbLv7hPhZF z^c~?Zhgr0lbhJ)mm7`C?i;``?6f$>DIIFugGBur6zmZ_6Rq>3im)@BkL3qf3Ty@sF zxiMe-2DHX+P~h*)sF#W0ug65!KO;>+f&7Je!s;$Ufmh|4N7On)AEH(AT}MT%dvOXn zNEyO1J78!ghNzsjMZHPEW|w;32tP9SXLWG4obI7!Ym;F_CrF&rfE#`70FV)G@mh4+zH8(o8FPP03cTn;O*lZv^0TxUqMVsP9d+j8=prGL26!^My&Q1&j;1Hr(^| z@mOwyNNcJY2CSiu2(0=`Eo+0MeI~z>rIL`t58!8K_ilrNjuL2BggATbdc zxk4DjndH(PL@ousb0WPjeAT34`I4`Y{wQ>|4O&+Ju*^j%;8c3d?y{=K1bRmbR)%utwPgE@N$dNO`iYR$tV6IXTuW zMYymn;TISIZ%q>Z+IfW0Q-oh1S@vwb`B024?$-#<7YwE-Mv=TfmGWENj7^VzHcaIp zAS)Qh`f?_55>^U!%QPpN++s-grwZw{I1-bNvR}DEEdtvS(5zFV?wpg;^|0R;A=!xxffRQgT;-dTDwJne1-6KkimS_gG>CSE=GOB8;CNRk#mIRsxA) zQTK=;GThKqtAmVqXA1jUle|E|sVVyC3m27Ql>7Yv@SFR}TRXlv8zDh$Lx$<)?OAkQ zUD|KGNos*UL6NI8`APPbCrHzU`bQKMcFb6O^gXm=in{XLM%z z;69j%X=DNWz>}h1=~~1c6NOUMX!8P;1U?Vt^;1Rj?H0dj)F(b;Nn}#C@4e<#w>i7Y zdI=ocoLfT`3Z{uUaJb3`_)R z_mQPU<@AWFL?VH6c4XFn#L%$YY4Q&3_5HuQAIWcSQ1EO_wVS2j8@${*P zO{Qa~*|ZjdhMq}p@?{$fy0(~gn3?@*Xm&S1=)us)o1=hs!{rDBGRk(qP)t2Lypv3& z7MrY#7k+_vmWB0({ciB5B=S(GpXNrB@^27e_oc=FHtib&_ZVk-!(z}j;F?@37mK|Z zA2}P7v1h|X-KY}>E=^gX>ImEjKiJGkb9P+w$x--mY=RCC^f<+34Z`N8=F9x^3c88k zl0@o{=fC(Q_bLB4DzEDvm>jcR?$%>=7PY3-n32-UYX>pabtZ~~g{*u|sOEL~I< zP!ytJxOJDp%8T=htr``?_EZTrX8o}?0~X~$aVkulD#S`ih7vpQtKz#UZ1E|DO>Ek%Gp}w($AdVv(fFn~A$_1+wI61x)FsFm@=@x}p1W8{qO)Z) zHUCIkivf5j(d_g>6!U%Eq2g8k)H$j4GC#9etF4+tfw3@&1m#{2@J3$w_r$jW=ws1Y zAy~o&Daq0F;7Mo%toX=CS6T3(2};3B>zwSM(Q+9=YP~Wo1eCnxj9N))KFuCJ0AfAh z768%5FW;VYZ_;-lI{x(3IX{AQ0=9`-TkK6*HnBOd5NH_4dipzP==w*xAspV5kjFjh zyw4rw-Vk^^9?z%io&Nvz;<=mP*XsxHUlTxBYEVqOuS6z;2>?LoUnhVxf6V~@dROq3 z`&rtU&@lXw#j;W}vj5v0@QV$Myo3hlA8Lx)?C(zpD>g&e?!P@qfhgpR!kgVS7l|=nD+%;j>c4N z@VtTC-$KR!`c=mv)$>tqV@E$uAWSD72nxejxG;@*ch=iHnHGl zQZKv^XckUJOimtzQR_z>Q`EIynXev4JR!r~!|$D8EyM?Plv$q1a3Ou(2o8EG0y&@o zp>5>ePbvzxQh+pJ?pMxLB^)IrQJ$?#@gg5MY*5W%Jaz`)V|WHsoO!AsBoYe3L@h{= z82m`lI>A!Ckj!KXC^6M*wM@Q1C@8P(1f&cEw15=(TE6gYidfFWHgLKgq?=k6_OOCG zrEO@sQ3p&P0N0`(S$`$as9oz&00IQ-J1v%ANCdU{^Dd!qJ~DQkfwE@=yg>pM|4{sD z&uS{I#UH6j?2ZFMAd%r;hH&0ufpp?ubwdUUKnn5DA$o`@41pEZk_

0;T>ghiK z#z^3+<>RZ9oMdO;Dr8Foc*dR9*|8e{QOG0n^@V)9VvrM7#)8?GvmuBj0~&x#oMU+x zlh0`iKOv@Rt~0mOtrivT6ddT5F0vUGzi}r~SMK)NAvXd-4|IUV6&-Jc8ZQ(pY8(TGA}qs$O!=m;3f3r zQaG2yFGO*HnSo6JGRQL6SFqOwBC2XNNTk{Cx<-r$LU|>W3|dX$2ozgNour=?K7saq z``K?1k_q79bQB${pl2nVi>-F8tNv~c0gn15F!4!1d-K=YRl(EQEBe`o&jIu}(DwxI z7AlBpQhv3cW8mv?m+u9F_&Y43B!2JZ(T9z#{~nZh?bG+et$BJ@Hy=ZgR0&!s6Q6g0 z+q^I3QC@vJLpHD!0#zaifw+Y2TwG5cL?5uVr;vO)UM__9R7xxhfFQG9G_aN(1y)91 z@B5KnKADifbO07ki4FfRjxd-~a{ym`xtM&ty6>~-NPQafOja6eB zW2@J_NR|<07yB*j<+kG{5fozgB?E~Jr{Usd*`3NtWp9)}!1O_KF+xy?JpkQ+H@l2) z&vfObMjflO@JT^SSuSzD7+F$4qQ15A?_9@S@?7hVf z5l+4u%!dsV{BeHyMPf&sg^65i%ax;E#L`X_KkI=pt-}3&0+G>0pJ|q+l54jMtttiN z=ci%~?Ru^A?Gu{(4gBKNz&65u4KPTcDHchH&5`o4Ni|$a%;WG(WJ8tyR^GvpA5WM0 zlW!xPTq?e8Pb{K7&Q5|pO}Jzyw&L^26r%R{b4f2&0T0Zd>x6uduSNkytL!7R zLz1uT$guz}QW1+2A{hh&(wuj@IZ@CP0f-fPIL&gCeLk6>yWr(BjO4Ll0l=-+E~w@f zOBZl3uST8!n8+#-2%1a?Q4u83i^d7vS8&n(aYIQzIG&kPW9EJM`^Jn4pDsOE)AOs? zSkfNN;P4n0zPJWrJ`i9ya3TIawVF_Vp$~DM5&yCo3wd-ZeidDcq!kZN$cFZ%#h6af zyD5df0pPPfTXeiu;iC%#n0#aY@((LOX{*5qA2A~b4JqZr0N?V>`GyvviPiC#B=%__ zhu%RE+$`TA1-Sg-Z?Xbgwk!$Ff;7!ydCK5xG`-;0Y!|vaV+jahzh=<@jFo1kdQ{M# zs~Aye3Vh%Ve>c?CV;ET(8JKXdPWCR2S|?J!YuTh{IOH}&G%rD1A5aJ6scaJAZG4Li z+m8u#7YWm_Cl*iANnoyYVzN1ag7Qpnh!In9l z(PJZG@*AS1Pv0w<`L;iVP(fcl(1B;cdceN0<^CR$;BEe~T|^G-%`vNcYJPk$SShvf z66M?8Yk!#RF(#7hoGz}&4W^nU6^+@Fp#BYMgUlLgT?MD}cX4kSD1VSR=i+m3yA~IB zyaKW+PwJtgqDSyaf5$yDtj88NH4}BV4wj=^jDd1e)DV8ug*EX=HM|#e@%Yxq9dHEB zTtRwu*W?^^SQeGt@SB6@d(-2+30s0J%(2(+ABl}|e9fYz8##|Q8o29w1=`hvdcTU` z+9&NZ4tbr-b78MHpx!W=!(k0194y%y(97IlnihZ3skp$#mN^(!H%V?WonT)MUO$@% z%ifx(N2_Srpa!|a9LO=skzp@X%4t8h<&USF^xis~JiGdarB`6U1ca-IF}X0ycJRD& z6kk=(E%6e#rRJGX?PbI#Bu8AbTQ9+JHT*EW5hh7j%SqemE~9w4X}lZQajmm%Tw8*U zN`Lu?zShC@B&D8TnyU0-+uA&dfjzUyu0k$c)5K)y{4AnxGQK#Z48c~6zeR?=8I#pv zQz^vS{I<7!O;JE!dMJLbz47{L#EL8Hs?}n~AEa?GqwKXP1>v#I`fy!Ps3;=4w5uEx zhwME5NV!@|ceNj+j4Z%q0kp}p;h_9`-0jjU07#p&AonKd{d>GICNXs=Dz9zr-0Cvz zNHvF4`(WNsA229%fvBEH?;fy=a?7}fvYn%#%uBf06k~{5i0e2N1g@dW`QiMy37ia~ z*&PsVX!hl|-IB9OV*e)Q`Ge0fGV5;H>YLwP>ZENaTw~`0Z{hD9-5!@O;zd7f`W;WL zmd5Fm*v(JU&cwT-N79y-P19}9k`CkBNvkfk%dpQ_NN$Lc9HlP8bM~@0*=Za#Xcdisu+ z=2ete*JcCB<9?@lEcuRNi>M@%Z2X(wm24=^Mu|9(jwmsI<0M1_m$ubj9<77d3Qq7&lOZ#b+vNrRnN6=r8g z_p42_oMZ=47gYnez-W&#kH^j8JsdpWZVq$KaGR;D6{froR4}qVSi?(VUbHP1{Nio5 z=K3*?1`OI`Oj9E>a2!3xIZOf$PFC`kd|tXdVpBE)eriR1s0$y}1ieaq zBYRz%RGPUlyh}CHugaXcj%{Abj>){fXU7>L*r+iYTlXoG#A+kfy*S+t7zxwil(lTh z`GN&coU0_nHoG4W&#IcMSnNp_WDX<|7JGxm1 zqE3nLwFi%22>Mj(*xP8-ly-5v9AA4iz2y!i)t)Wj8qlV?apNov?Q<|~Ic+cC(r7+n z)OJdkKo7bpB`#E$e4sTkT;Vaej-(Bow~T$Sr(SA3#~8pmqTd@rhY`P-8Q$444%`>9 zFq|0dJ1RD1$#VBsFL~2*39Iv(X5N#;(y8!haA0}WIcqvDy4jiK|2{Yn?ydlrG z>0V#-??%_r_SidwuNF*n8J$Hiw7lRI7)tnpw+ttjhU0gaw|v(lTuqQvd(zGG)^ znI>=Ts-^~(&J)q;$ZlhjYlund`dCMP+ulHXr_To5I*qi2=`mK;U3%PR+$pY?bK=%% zawCn9b0?kUR%&3q=Hcl9C*P*!9h0If2t z2CY_U`&=8Kl9TvLPLZ1D;E*W3N%-RVFk=r5lhtpADL^KZ1Nzu)Ws6J6?J@tJl06%YJ+ z>f6}xCUffg@M@?$!IvDeJQ$f}sB26mn0juAB8(#7yu=WwA6pfO6lR$~Wwcd9B`-j- z|FY>Z)%y4w-P7lq?Z|8LDD&}^%g5DQC+F_NG;dtq)N7p_0N_iy672c-vb&#VQ_FSe zt3%kf5q@W*+EceaJTXtbm`hA1EDA{*R!EXrxH%^N#KOLbSr1XM65J^~akiyHuX+&U z!oHPGp#3zm;b6^$fm>4MiFh}rSGeS6Y#^0Y4pSr3Bb7Q#$EOaAz>JuD^aACca~%5%GyPyY`5-Z6p6sgNquV(lg(<) z&Hibw6*^3_#WqWZd}5icw*+7Brc2|66~{$JPTRz~AfLjM^u%th!&v63L*2PGvdlA+ z=<$8%SUVhD*1d5?`AWX7?)~VFz?myJ5GAJtfH~Ta@<)t8H&8ZkH}7sudm}q#jGR`~ zJJ77UejW(j+eWb8gYEz#oCGo$AKjpCxB=gX=K9f%NdF>ZM~~+BH~qUYvl%f_6iy5P zRh1@h>=r~A@e4fW-}j>lCAK1{Tf|Z)&W_Gad$pHSRd`RjHCSE`=jnd1pXqJ3IO1;k zCltljc3CvflUv?*h*0j3q?LIX;8rRX{gA*(VKh&DGWQeTtO9`0a@DmKld|i{6I^7? z3yMI$tpi3MSkz*!K}aYLF@%S#-ct|7Au*sJQaSfs$r$T%x-_hk$F@SZQTV|Dam7vN zNSk)d-_eZR=|HVA)8@jGGUJ=jZc*3|A@(-UPPX+HeA51<#76l&XdyLz6X5mXB3S|zM66#X1Fya3 z$B$i696x@B%lTWNwFZu-_C%(j1o=EuJdWk=fk#sG;af{JnV_Of-xJIfyQCiicDOKk zvR(xGoSb!>&d1yXya+N1=CTcm&nS#wdLNO+}qpAys22Vha-&&v zhtGYzdPBhO=jUJ}Fjl9pEmr@Qicm}r?|I;#*88#YDT}hJ2l<8>^lpP2#BZpEugTjLRbQBt$-PM$xH_4CbfCqpb@)Bcxwg8 zFaxv!6P={@E0M?{Mqkhz;$#u5pae)-Qo1S`3%CQFl;V_BAM%()4Cpx!b7*ERm618` zwcUaeD*W4O5BOCNDDg8o`*)SonA=$1zT#uNyKd&^;G6ro`JfAGme7e3X9a`PgGJdD zKKb$VqTNL*Db7s)j6MmIfn^rBc!BgvW|+!>*PV>J{4kOB#+pSsk%6l_uRhBs zP6d6zWB8*qxR$MjSi>}MZv$M0%Du4`iDNnStt~k?BwZFSRg*+8t3I zPH3ch49p5o z1gs&;R_*Tg-N43@5Wv7#RU?d{zr#ESp6DN9i^@h#&6$SOaGA4$7tf%ms+O=-5pZo5 zEb0)LKeK3}+|RW#^%A-ql%zeI`GRn$f3ls2eAGr}MAEgn5)Ohi;D)~4%F)I)h=pC* zg|~g1ST9o3R!%O(hZ03}dwrQqB10dlvu9$ne~zUI4W+5SrYU|Wk<{iJSE`F*Y+Pa3 zZ#6}11qC9mw_Ed^eZhR+1~sMm_CS&2+WI>SaD4V@eIgW!;~`KynaN7uP_W^Oeis+vC4l{$N!rd9-7r`oT? zaKvaPSomUs;p8ih*vvYjNlkF6E5IZk`Q@uE;ID=!y33s-xYn@D6=s(lC}#;vKd(~Xz(1~t<9F?2(-W8`(HosGvKF7mFIhD4vfd^{x!>LFb|U{9un*EntZhG0)3WF4XhqXob||`=2W*MPusD zPi&bxr_==s7$5EAx?Z1%{m6U!6qCTK)CfnP6m2X;SFXMuW&J z)kJK{8A{;}Q1c~TQ^N(|u4li~SSHXz)bj&}scxmwG+}WP!ehw&X5U-JCbHa>=0^kN zPifW-D9vd(RnGUA=7f(xoQ-&CS3qboo{jM8rA_<}pH-JEO7%zL3Y%F^r*>eVRO}ji z0WIvinUhEB&(O3AkTJZPDN}oe2ev~Kri|Et=bV=9Vo~Eet|M^*W>-z1OxghnR^>f} zO|MrSr$-ean+42vsul;w7CE}wGAcBHSyZr7u?~viM*I~qVKvp|n0R3)9qqnq3Qmph zHH^RK;ykUn^p?Wp)W@lO;<@KLU&U1NL0Usw=sTd}X3E0A0d9Sl7%xcABh5!bF2^S+ zfO9uuh(lgbD0ltoG%_>}%9uyJv~qC4z+a z=(M7i*n5y?V*=aP#@t}a-D&d_`YZqyE|wI!q`dKNm!Y3ml{Vr;z5!BZNB%e{pS-=G zeWGorGHV;cJRNYZulyjcFg1?X_B;4fdU8yOT}D3HyALne0S`<>I+;8V)?r`xmbC7k zxa2l0*4y4o3?a{$s+`bfk2!bb`qUc?rg&Ncj*<+U$$|$ajXQr0zS-0iN6mP#Tj}?X%*T zrPs8p{SY`pZh!k2O6Ef{8tDs;@R@i0x8VBu#LlgXOxFtkWC^!6_J^03kEJ~HVU{@R zrpq^f+2r<@hRKm1N7H) z9Y+m@L*}cFOz&&Z{~LAgUzV}`Jw5*?>;!@&N;`<8Dl%q=5 zvNZB^4DxhL4GeX)3@i$^PLN;6bPNX)lT#BDRFEU%(v8e~puu2TwsTqv&=L%mu&w`4 zcBlqK%4YZ#wBy$h|2KjEk95Vq)WQ9e3H_}O&Q^arlJTg0?MZaf>&9j?h(CxV--kCN zGyW!{4AW#$$0n5^MKhb&pRp?Y-YGuLy+?aXVt>( zc}(Pu+kaG7{ur35!T5)1{=%0grTLm7-`Dsb)cXCK`nmt^9Jg&M=Xk$ zH?s@g5G_?IspG6T7Sf<4*ai3|EL8O_&1C4-_UyLzqH=3=fB-v z|D^yF3I_HcN8i}>>nQMR{P#D^zcQ(1{F8;w`1gzX|H1fRt2kD& z`Tx)O*s{`MH=>%$$$27v;xboP8Yt_WyP{r2ASmiKQ#J#o;gZfhLHLI?c`0yiCqe=M z_{sG@(BJ=@_CJAneBGzhQ?s)EeN~+PYpv*o2BZ?Q>LdKHisZ%;AsZ~2Kf;Qy=!);9U<{&GVtgD`e=x~?*e?oL$1ERAoZUFK(HgO2{0r?^7KO^1@J*lhVh~i z2=)8{gabYOFA(BO5eT@#lL<0I!i2~$Wbz5aqvKcusgt3~oFI@y2jb4@2EylOLjjVZ z#G8PH;|&)Pd4a+Z4e$a93JjjKh&}>p6MERf;>k$Ba)wLc`xJbk;%r}5cz24+bCK9? zpN%b7-)C&8#^ZSH^isYUOmfN(8%*NQmV4 z^CF~0mV|0RI)i-z+FjgJwUknfQvLFkWBo-o&+KmlfzfwI^_E$fl8>D#9D|1 z0)%get%%=9g!JqhURqH6xd~)omRG?Fyrm|^%Rveqgq2Cp^1O22;l1X2uTS@ILA^! zAu{TyI*S5VU0kV=`)9V`;`FEYpt3Cp#-ycgCGpp%LX^pO^TvP zC-{qK8}tET6z@AYArV3!%6d9rN#r4}J`a-U1(32X5SIbl7x#myZ9g?(Pg+6_E;a`c z2NV}wiVQXET`ZO8gB(a%tucTP_Z8XH^Wi7by(EXnoVKER`Kkj$=>{Pl^YT0sJlK@KfHU_}d6M~pv<(n_p7Yp;NJ2jLa-*Rc7j@nS^5`hB9g{(9r)w4O4 z3>BKE!*`y4wahkqyVSkeao+EN{#OC~sLX)_vc4_)IBD)MPjDd9-lVYOK%rip%h+(q z2~Th=%+X|#WAS7VjU`(6d4VPBi`J=<5XoSNrU(g{7YM-?t=k0(@^+b2nw5pm1$1bUZ^Kp8WE!MaW1`av%0z7=S$vkg>4DWW{icXt^?gOujv{O%5SX z0bFCy2Rmx$4*&;me6T3o1<2Vu{2IiC!GT)H=J3>aC&1U?hq{>FjRAJbq@fD+-Q$Oe++cB^}QEBoN zN}9z3Gzzh(V;)sP%+3NT77%ARu$vtfGNxxX;m!~;tU9U{nApP7P6NwBZND6QV^#fu z1VJd^APiGd)zoE>6~aLJ%SHp}2^H&SoUPML%9DwDLOS2;CMYAY6ut_fr5cjPF^hgOFiZq4C94!x|!DE9WydaihUr^3Hy>u`5&` z^P07EVY>`i!Lht~Y~biDg(fLllc+kE2_Y*={rCo{4--RJB!^b9oP=Xn-1UJoh%(XM znlMkqyRfNHvJQ#Vzayr{(sn=;{6t3osJ#ySi%SY*QVFOf_l#0p*|_tnUi+A7I6@K= zH_H)I$?c^ePQH+l&W6-7ARob(Bh!pHf0tg5adZg)*ALZ+vay8RIM)_r9baxOoL)g0 z50Hu|cJcUD^@7`}?irypb@_*GT>53eojiTk%g?zlqJM98zN}y*0q_yx!IxhzhbUM& z17~!01tHV;(cC+a8kz2!AQPx35sYMMng?g+po)MKQYCPb67_OvW+OyVw}4Z*Lzokd zK#8behPCpV%pmW%8a-R?^XEo%k_}@7+>e6uL+_s87+S(y4QKpog;fewJq|ucIuoQk zh(M@sC_hM@bE%GcUBK7IF^JcUxvTVL?q=onzzj`*g?-hfR5Q)WJ+8~Fu=3bMfq=NQ z2nLU^$&}82ap~!>CMeoLbNNCJW5(w~#M~6slz=mp7kL`rif5-rzLcNG&dJc_spQgv zqev=rAeW&gj3Xx*id!|rFCNA=T(vO=rk-z^!-SVt5tI&>`~+^0)C}ZoQ;(RPO7kXC z%cS|T>$Uxb{Rb1CJjT)hNEL2h=$_2Yah1Sk;M1dWy<$`LDc$Wau)KCWT1P)Vmp~LgtE63mpQjk z1u;)R?oW)z8g}})5uG6P_++lQd@NA2|6Gua9tSI@YPH3yiHoKMg=N2Au>pvmf|{Yd zkQ^dGENB#*B;5WS73~b#s4+|$g)~+`xsI6oHF;{|86Gra59nb)K$OH65?%o}|2h@G zgvLI-x1y&WXpOpey)OifqBvd;gn~DH3-K}Wg{)Z4oGi0%U3rw&lF&|yBJYdhK_Z9G zI}Yp-yUmB&lhSiFY7Hpd2LeM7Jdq^9QG=sN0%3?kM^v-NsoZMl5a&9G($hw^*`chl zQRy5>%!^q-hx-|G8=S1gA%(N~D3Lxgxz?WW(Qf=5#lNW*JJk>PQL={;Dzm)FD}&D`FlaheHM$0 zlM${@nJO@+uxZKRZ`I{>9M%^ZwIs8xQX})1x!w64a7;Mf2m|%ncxkbTtmhRAqMVMl zX2(?ijqznpdD0-a9G%iL-sAnE?bMFXWp$OUZ`^c25hYJ(nRk=fm#ub_&X31w(stcz zThPe%k<9c%$4hg~RWXxc(~oQJERXWFgPjteaab?o-!QV#iawkZ_S$0GSxGTBF^&NPbcWdwn^mp#REeg^sUE-m79qzb{x0jsUa)S$6 z*Oa>HT?^_*F4b?I!gtSiqv>XBx*j|BFQB6`4Be#B2+PV|YPLIH6xMe%y20bJW_CDv zbxve}ty%D^ADr@b!nr@ayTs{^Th%hY?frtB*?a@ln96xh3sn9+x;EbWbhKPwHR+O9 z!$?sLlbWLTw*3v9Gv~6aS+sM(HmlY>c_@E7pgJk%BhEs>h;EsgzQA^3K^`#gzSv6XdS_0C6mCK-A!3#GWBOPN&2$RQ~uD-aeWIJ|f4(~Rmn}+lC1l48; z+<8@Y47eUmnYLNr2Ix$GBjerSg?8M(MR8sh_H1X0yFqGuz5#!UQ?~R0`$qg?_!mw?Kh0r`ho{4auV)9JsK@`)+I7cM`Tg-*X(>?@MO1{$d%0F- z2^ncn2-&jto+TkWiX^F!O|~eCRI<0qi0nNwGQZD5-232i_3i$C&+A+LaX#uwOJ643x+X$}MlcFBP2=J}00iJINf^>cwiQ6m6S4QCzCDLm`w! zlsRqY(@Fmy&x=xbI2^td8k3VrHP#h+v%pZeW^%77zE`t>wfTd(JmK5pgcm$=e$PzP zFbz3Kl5~!!zH{E0aClF^7oD<0M-!Wb)1GL=8&rlA**Qgp1V7Gxa}36&$Ao$;7bl>fcE+!BnU+X$uBV_gt=nm^%)*K$Vn$|>7Byf8hvUG_PKyp=QEz=Jum$p zrIU1BrF-ajnxUe>As5fA?Edtt#v=zk`pdm9^v0Y_Boi|+_{DzY+Ue;}$8&7hAE~?& zd{y(Pz0)FK-%P>J-loc?+tZDo`}}Q3DJ0VR0tYALsXy7}3=O4e&3IEi`*5?^BvRT= zN}>Bvc}e%3SO@Y<)2r9AGPEz(UJLO(dDA3z7jxjLKvLvKLxqOA_-9l?MQ|?N7WL@e z6MhrvNm=K=d=pA16{%-@`MB+QLpFW?r{<)tr!Owy4BX^*F%9Wyb7TVvOm2w~bdPHV-y6tmS zwMhU{#1SVayq)u>*(RtE%Rqgy^u$x~k%H&@ z&WKP*6mmV(@l(&Tt~$piey-2y>6?}=g#d-_py*&njU@9EEqyW?)Ahp%!zUP2AE>{J zJ2>NB=_by9cJ4+q_tUSHQf@`XiZ`YeJ>Op#FwS`3z&AEiOC2V!Ju4D9!8uY}#QEw8 z2@>x{k5IjjJ{e6Hf84j|0Qzq=^En@PQ=T|SW7>6IF|0L&SMwo{nM7KqtLsCmy?OL= z?mb6?XNP_ezbX2hk(MN%P{?q%eT>j}rosHDc!to0>;Z*=qoji&2j`=InkAf(OgH~s z^sAOtL=X0t4$=au|GC-CstB46X>@NaClVHVjU}bPkuIL z5YLsIp49PPs!~N5g<-JcRh8VE+q2kfT$vh~NBFr-JO_1RFHb$My)b7!^pRtf4kw>H zNBC$DPjQ~p8|FlX9rFf^NgoJtz4xVs+|Ye79``QN^;1jM3+>LMX=gR%`Z}i+3vUpZ zu-0CwN=ZFE-0SDTB4<8l`GVejdM1!$-vN!Dt_ilj_+I5#6Y3gWeYcP^(gTBC(($bR z4WTX(#SD)PUvdAI>Mn7kykNsFH${KIc{YYP{xEUp&GCMXQYU*3Z!Mou@iBX8VqWgf zpsST!ql^ZYJKx|I2OC-A)y32uZYvJBuH{Emi7PwX+47L%>F%2+Nx!}?v#)!YFN%Ms zWU8-Ll2Yb@JqL?re*%wu$q(TBxr?EPIanEDyJGXhShkPLj|Dg<$X3MCow6J(nDmI9 zR|pxrnyTy@G;Woyo>Phpao_gh+3^d++YBNoLRBwiBf9%8HU74|F({eWLH&)vh#*(a z(B^ng-e~Ki7=`GGsl9{4-#$h>l*Eq>rFL<4R>-g(@~ z?#6A5H~0B}%aS7ARkc+Sejn$f%8-utkyu+ZpMeVAERl&W^2RW-Lc;9?fv*p}bGj>0W*ie6m>Z5zlU7jl9dM zo~Pbr?xG+tLF_x0bXO~fV4uiw1==Ggt)d7{SnZ#ZIeNL5cH}`%9YTyGAXW#>oX)1>NOMC|;fDc3(hz>s3CUUx2Akf4Ship4* zgioBAeHZ>PE>Dyk3w2hMudmu&Jn}QLBzO1S-h1?lEFJvV)%R*fTlaqkt5L7@w#wq% z5PomB+ld~mMuoiF%Ai65RHKZ|o;yw2c~(9NXx{n8uka#nG%eDd`cjgnaF;V5@b!-)@on6=Z)c9V+U(gPvY%G+*^}`W5kH$$VhMiS<0e1vZ`pZ5PD<0g z<2f0*mk)cTz^n5gM6@$3mFa9sX&!j9*VFi4yoEda*{Z>Q{%b|o6UH6I2lHHb(%h(( z7P}PoFLf!T*ixQJASO^D46{xb7~ajLb2^|)VOGHXePW?!KKqE@;07<2&A#RwzSWkVto_~0eBpql z0B01e!&30d;~`dD((k8l-)AZaIQ!%fjE8v3*Go^-bu={h_7Dkd+a*D5IY7YvIi=yU zH%aeD@?+X~+lhu91y!`u%C9KBkaJ;r_Ax-imm-Xut!1>g-U~M9ayueBLy`81P1OuD zy9Z+sizV$z*M#`A<-t8K}Q@c-uQJ#HcETCexh{O*hQ*Urn(s0&dEAH&&UeUcWA z^Q`rjfW^Jt?d?dW{rBJqIqP!aaB=o8rHrEz1%5b_=a20#XS#otR8^M58-ASo5e`Ar zz384hbYf8laPqi#&cR3pDxML+f~bbfiGU`Bdc*?wag2*i3QI?TCWX`11N;T|cQ8(f zrT|R}LyHHLJsh9ZL<{vJcGq0f(Eyqh78ad!Q#{@gZntN`}NAsc1^` zS(MNHZG`0OpE%Sm`M1AD1^5%RzOYi4X|TUbCIKszCd+&_O)pBMY2TOJ*o8drA&v`{ zqlm1ND~!lhb{2o%{rxKs&NE4gj02aS@2~f)WCME?Hi`{s`yHkw+rqJ{I{mEYrD~eJ zK#xLQv!sKz7aR5XCJU)8>Sd@RZQWl-`HI

-Ij43zVZI1wE;Oa#S}^j-ms~QI*`@ zk7JJ^i`%cv@;zc!oRLCq=Mh4pm!oFefN~TqP>zbul`v0+mZO^cWGgtIP0rVNKePnP zQIvxn4tXv<90!Ec`viWROnaCpV0kM_S(4`L^IeDG=kp}{1g?7*m~Z)Byzk_yrA#>iv~Mw0@HU{N~au(cv>}k{uSp{BAD8}{elD>u$+@*FT8e>J@RGz8rF76 z+covzd|49pgnt1nI>|UFp;F=rex(52k#a8E;7jkEHH&~2h1Ivk(|{I*=VEg1IF(!) zY48bPvkng!UdDCm+dAmdn1E zKCxFMLd2BV_+o;O9&WTt@u^=xhr&E+1y9j))hde}3b~Z=TP57K+wGoV8a(7nW%lle znr!&FYU}=bmNUn!-uXW99+Zyo6FPxNJ$1|cvhnCOZS$0GhyXiU&%D-0xeqx~+bUIh zhLU6|%`BRlLVhVNG$_2={%$YWpzzI!_t9G4nf4wgx;S5X7xB5e-Yogvy`;O*(XYGJ z#fN+x63olq2k7f|cEpq^(|W1EAEX6`Z+nw^+>u<3m-@_wFY5WimiPz#=jiDZ6T92L zl-g8tQ|i9zoYeYm{=IRp9sXN(PoC@E9@RW_IpvbYy>*rkn+FnGdI|GdTJL7_IVXhb zTqb$c9*Nxu^cllVdc?HFMj?p4Hyq>T?W-&Z@WfnEDNtuHEr|ya?-7_ro0dJ`V{@Ox(bT3dH)f7@=6D7?r7a@&f>~A7 zF%Ov?4VCyR+bPpZ1EzjIV10b3n=I4hQ~1co+o>VuDfWuOUX50bh}i=HW^@zdYVXgM zv;r+GB{j;zFPBSs(KkydSaFsA8g2{?9QI>3KvczirrT4fq+-#W@#~IIO@@w*hR{dV z1Q#uvyJ^53^~BwWbz9>@LK8&KN5#%ny$5O+GV^f;A74K9msgsq#;@kGkDie!GHoDi zJ}M(J5gI>Q%Bn*UXZNP{XsB&VQ_Sm8rtu7hxeTTDpqE{Dy=w}BD0)Wb>|ahs7<*4x zSyCs_3DHQ(kX`4~nkNkOl`i!Y+$lxBSHZ0Qpprg&L%bS%us29$its~~VSiQ>kzq1I zgLK;d>v$b;aaBi7x2c`DsGnlvmn?4hI@%!`Uxt5o^ThX|ugo+g42kt``yM$Y5dX~b_xS$KB`aK<0yp4LxU^SP z!zGmj7a`m2tUCr(4pL`bpm1y>O<{aWOwWr;ef=W|-Y)lgK6{b_Ry@j$3DvIzO>-E( zzvO!%T_UgMOga3vlwd6O`jMWu-{Uk3ozCJwe z)ZeFZ^J)@9HlJG9X!0JTt~(Jf?$gnxE?*}1+k4S`w|huPzqO*XZ{~KKa<1`1QY5=J z9vRQO@oUe6FIJ|81ct~__Kg%dzOMPvYl}k?ylSQ3A*6`Hb z!0UOvh3Vc+!KPFEW_uq=v(lDN4BfUH+o5^;E4(c`Gnw<+!(scH)=mqH-@v_-^}6l( zuKQa(Y|rR)RnygR2dcH;9Z&ySHKo%k%uzQy6ubZW)Y53=mK0 zeN833P(D^5L@8p+IOn7>Srz4w^m@)(EhLxZjb=jS9oe+L-}H?~EajMkD?BpV-WP?- zY)kF4IG-?J#C7_7L3>D&@tLX~@wloR9GZd0=fom^z~oz+LM+dDT@&HpJAZ_m#j+`= zrRiITn%Qjp%muG2bjc=c;~;Y&BjlqCG}wZSc!^ALmA%XDpo;a8#=ZCVBI-<^$rnTfc|L{n*k+ZdSJ z>6pzHnP)cm%J-CJEAt-mdAeFXnpd3hlo$~K4=jV zuAQ)%HSV>tyL0aXL7la}^4s{~$)S^_Z?Yo=8-EpBQ3mQT$B&tQy?oGs*wMOctWc)N zqUi{in(fPRjyhy#t{U%w{1l6L+b^T3#^UkgxNu&7X#*nFfv^l7wJ-UxY2S+s&x@7O zyk(q=wz10Xr%(45FYwV1E|wl`vab!7gZU>m7A3cW2Oq^AVK5CzRJ6^``q6JaA#_LNZT4pd zdiM8B<6P3nNpa&4$Dz`SM0e7o%{W9iyDq%t89mpWGRlmn@QOh*T-;AkzsAWm;&)kX z-A$aAB;nof`4t|Kag5!WcZtyZz!yFsX8$?IzpyNky!r9m2_3WMn(o7%V~^!x{TWFn zQpR60ZLKli-X&W zC_^j~QpPh8qdg&G{GFv{O8Rfd-=+BF46+B2-|H{v097iTv}W!Y$1sA>Po+{aa# z`v|^=EiYomH*l`>U8QJlS56`3V`?am+8$1>K-c*e3%W-f7AQs zXRBduHF?otl7sDWmP4IcFAe+IIrTYTHfnKp22L10ee&s^{I{?hCAK-V(;x25?0@gi ztjBRhvX#GJ8#A8g6ryAMWj@xTG~*BUqn&Cmh_$_RB>MbwUnd9L3L-qIaI_|Wrk zKn%MW(_UubTiU(N^kF1Y-zqdKgHrZb^SAWe^x1vx70Hgovkypk!*2fad|JV3BV5*X z80{~Ujz@4aaB6escYth#iUd5JWz%nd zOJ=})cUJhZCyPaB8{y0q$EKiE^QxFiTm@Mn<>UAZ$Kp^*xOh&D4m=zPKICXh_;X<+ zlaYy`k%ozxwuUKWGV5@AzR+^Yfjh(?zqa7qCGjm=$+w}T0Ya94+hHVs{!9StKLh@e z1&&|??w;HZ+qTB=>kYLFG8FKU+Tu`fR;)%-0q>GT`|W{@2DV0900xYPU|$^2V)b#q zmGGiH-`Xu1H{J4pFj)BQkI(@(yc%;oIu?(0f)lvOA&jdB{M8|I2TTegk9`Yd0&FU3 zY${v`O-43#H7-UZQU$@N%Ef|Y)I_psa;R~raH*(rE!>mIf?#C(b5kayzHFpoVr;Hq zwxB1BL@WXWZ|PJ`I^ZG;Oj#LtPW}M~AOeVB2D+@FF^l<%(Dh9M#er3Hcd|JS3}%N1 zgULcT=vX1@Uy-or6M%-WAYz+*`Uy^;yniBnrH1-}Z$#0Ez!DsSk%a{~eT#ONDUy>F zj$~tHLe~&RPeN(PEWbC&5TMO!6>Z7Dk!v(yO#cemGHn>qEryKsJu`(B4IRfn@XSf* zfd9ZVK?iVBmo@YQF8&%xU8#31BxC}tnrs@ZD(Y}XRSgv+Ba#KK&Zw$};9_KFL$Gsb zssoquJqH2Cge5XK;|4KfS?^f5i!=efgdKp+ zY?W2|i~JS34S8EehtVogK7%=`Jr{$bTjz@r>>@iNxc(tN5I0yeR@#rOI7AdXD+0lV zKp+{};VPPpNKG!leh?g*jOv^knp|oss%&a3|1`2!2g9Uv*R-EsiV$E-SR-R)V`O7T zv+Si7ntvi1wi*Jhn{n);Aq=36dlhZNz*4jv)i{{{m2qRUAcDY5*uoVp)}+nB6aox; z+Ww*2u@#YcQv$ScuA2&?Y9h=mV{oVqwKe-u3=Usa!XzFObG}J#y9W&5I zTgq%PMm`ig%&FrHbijYKK)nm~t?LCs+oe^hqn`U8X2IN-jhRJLLtR5tQX&b0(SSbPF1?)btUSfoblaDDFhzWqP z!H05rWN>F$jA7(ODNH)1h6WoGf-+g%B#8s_fDjR2)h35*VY$Ad2tl91!fY1gtUno) zjGX7>Kg5CYe*tE37$U>>8%oBaOQ02H)*pV6tCfvE_APS-YJsT{H9V7qH${)a+ zG;Au!==e=&B!Ga?MhCakX4+=jh6dQ1=F%L%{~m`((qB&!_&u<;`rnBmfQb2wF2+<% zOG94;DY4_GSk-A(9<^1*uY#+79KwuN^uJ?BqrdWxR-|o-Bod|VOM>6 z3*8oQaj*kl9UJQFYcE!!Hc&s9PFOV)(b5760UtO7z?HT&=Eb!7n`dnUO+D?KT4u|^ z1LJy!ic-RRYZ;M%MDP)SNNvL`6}_a4ao)=)Hkdo8)3RDM1x^3~_&`zkwPQ8+MHGvL zKvp;_63O<5{bdvwBgUc$fGA1|TL{Yl6EFwLaZnRD*RckQRSLieVj&D}003N!s7;jm z?EwH3t8CD%7ZZR5UHn<-(R2a)3z7>C$I1p582|wk{*-5T1Hya{P}QJ^ScAxqTeicxQa!Q6p57tRWC>+Dj#eC89T;=Ay4F0@DOIUjDH{w97KNhSb&5 zTnuPTCK^D6-PBA&eL>46n_G&PvIj6R>OT{@mS8PyQ!_&oYphlT(1^w@Lq(~8nSl={ zWL_g{4Ufj=Qi8m4V%V|@N*8+BIkO9p3_eilZok)H6;1k*E;lcI(R4xSjSUf|ih##b z0v-$1>DH+=q%Y_c-G7$CBvhvueFn8~(B&UPE|Cshu8L>Y5R5kMCDJ!*!r*iPj+cMX z>0X~*Ln>;zn7GC!P=UUKS<9c+61+HF$crhATE^%=C@nkxx)YiYXqjN&$^dm{4y$P| zagEJ%SMm%^vb48sS>9x+n5b!KTL7KU|NpS5Xv6Ajn5n3%nEhQ!2+e2b*BW*?V^%UX zQ!)FK&thDjQ6_(viI#8ar73i=y1o@f$};+uV*XJ*Of9YX(pw4`>>e`=CIg9fKd?cu zjEZq#@~Fmxz^feARqg!Xh^2OZ>uq{bhAMxA&KtnCaJA4i2skE6Y$(En4%%o2Km|6M zS-yM-J8aU#?sPH$HZ9;$fM9F?M8vv-yDAc-ErxJ_$1L{>AjGQdX9AR|AMUFs-TNt+_ zrjfv46K7YxMUSAy1|kLv+W_H0fZG5a7&i|%z)#;NhQaF2tVGwx|9{b;@4EijKg+^J zPwd!+-*dCBx0eE#JVd?tve=-*q}~<_#si+y8=s6_xkw)}(UvW0SMaEM(!62(AuBt_gaz~=>l?a0!>oW&48c72`U|4HSI zV%a3X)clyjI`aO$6t`5NT+IKi*A?=&wYgJ}iBxPT1Ff8ftt&0t0PqJkh&ZI;dc6JP z>KzM2bl9du|DSRmG|R%jvEG#bM}6-EWO_R5iP?0)@4^AQj%z?p4ep#om*p%Rfx}A9 zil7HHzf;%HGgDca=C5y1D97J(_CRR;ZlJo(0)t(KzzJ1Zhu!6AuL^N!#UNnw!^OeE zf(~$fSg3Hv^tDBJX`6Qe0&A-tHn7&%_o36Yv@h9A1omKe`LqN{5&|jyUqD*k32z1y z`t}ftNlt&0G%f8^(P`QQCX_{8vErEiz7qz!3RIdTAUt(rg$ZI&U>9u z*y(J6A&mM$1;*tN1pEQ2FL76F7+M*H);4Nv^tH0s4c!Tp`~x)0>RKNTN>zNnM)fO! zoxssmJAq4X*pRbIRf}sZ>g70s1AUEMUuUSe*t9K{gBgIu1=O9P85s8;|mfe$o$>%j^R(+Zjp=-C?( zoLJc4NC?7BWN+wHE!CD5%P_Se*ig0E_p*`mX%R~sHR#%P`YAT@R&xQF`KpBiV}TK+ z9?YS+{jVSaCoI+v%~)XaI8QIRWHM04G7rUyCe-Q$>>bzLsBOG8Lisz}v@2r)uzf@V zYdsW}0#Kk_s%tL4?OOVWH3|}c)>O1qANUs}CmRR#Gjou8m=`8C7YqFy!uJ-+9jp9Xl$^d6uDL!g5^SjGIseb%xz2l2hFbU#$f|z z1OHpbvoS>|Oz1I41hW4?#AAaAMbRHy2M;i;tN^TUODGd}NOkj?1Ck=}6JTf>lKlU{ zEYpK68s_Ab=>)*xz=8zS;aHNep=OgS$NI2P8q1Fqi(d*)P>0ifjSVa`hr=om{H--X z7k`;7Y_P4M40v(1W;9=)DwI9Zzu6lo1*AxCSEWeb{t6EI+7Fof%F509uuvLA_c%l(O>jiw3A8FgX<{7MeNhyV{~ zD6KGPAm7yj-y5!P!6<(cs_M1^J_BCZ4ngmIc+j>HCSYaX9H@?lr(0x}K`}PYPf`aM~3#yileFM5c zH@v7ms<;#Qlt@`%C0#_48zNg}?U=j}zTkz(^$iP~h#9WFAt)$a3m?<344HouF+tlp zbmGa7)0%4J(fA8o3v6LFuFa?s^MWj)xgj!$8aE^i8zrtxXo;6Y7I3(;QuIvg?U&}FalNpcnr`-7YH?fe&M%s){(sc0dEg z#29!L9gGAU!=h87XMzz1Hg^)T09h+52O}#Nwnze6%bk&XfUm7dR}v!cgb~RKLe|j@ zwlJW*uonqg%&l;}F~ay$omN^t{jIcAxW-m2w39Wu9`e@_yD3F%#yX)Lr_tfW058Iq zit$U6Wd7r&E3~nWd2R*^w4XFO7Fl0xu|U@mpl-61p#Y0AHYx(cBD8ljI*woK;DD+K z)4~DiB)xIE1g`(kmd@yK`tD;%3#LVXxxw?sU%r6EKwI~qBk>O343ae#)a6D#>v4ig zKcJVYq9bw0z>EZ{BB-grp|-LayM(lWV4-n2C000R0bh7_-_>(_}GqU^-IjsLp0F-~?z{LKWi8Ne< zg#ViuFM#E5QQ^Yff2jYJ2|c}uwY7n%p|*{#or9@^skN21rJ;kafv$rty}q6PU#i}# z`!`kL-Ee*n0j<>fQTJ+N!D~-`SuIMRx>W^U(}91;|8~9qdb$78rGWiq$e&3u9xs7& zIJXc_58phW&#&sh#lWm<4XU}IRKssxCpb9H&DEeh%Qj?C>o4UKR9Ac$h4MV0zp_7G zXIZMz4R>Hv{d$jf8u2ret2xZl3Pfzm%J2`dr{K1Z+UhUzh0mhPGXOy4Z(_4@GO=)S zurkpZ=(2Lsv9fS7(&@738qhH_u`)8U=rZcFva-@KYBSL>vCuJbX)`l2voSF;vodiq zGqW;turboZ?Q833=6;pKM!6g`WcpsfneN5bo=+Aq_vXCSQYMeC{utfS{G1T?;>y5m1|$ld`@X$y zDtN4|qe8A|!^KQO;EBskrm+?Cl_Q|hEYRRhKvS%8nz$&*6aIRL>~eS~(&L$L%0%tC zNXrUY64bXKD-PNtXS7D1sJFB3#|_w|kldPh`{4`7?_-cduWn$tQt#?{Ecs!!7Ya%)_UdKJt-J1?; z*yy%13hEr3-WyF)GYuZ2)4UCfxSBEpQTdp3Naay?uap_`8+)5n8%8xPpD1^GJK|ri z5hO6sCTTc+CsMTtJ4)llfI?2PE=cRH{PY7U{12ruZd4|t<1hQ=jc@E93nP8-pq69@ zF8fFguvVey%*B1vw~7O*?|KA`lxJ-42r zkrY>xwhPkw_@%o<#m`R`*Pd7IJL$puY9JR&={-Lj2V1b%ZqCo?tmpOlVi14YAq`uL z6~3O^tdW8`EZ}r}3R!Tf3(n=kL^pq5-Tmoya8ide#wuD_cE?ja+H|k*4!&fZXPr4+ zqh1#jFV!>cr1%41J@j;lm^V;OUG@I+ZZcJ4%hBO8`qbkoRAQ1eT?lF2+2)i7C~4ZF zEUT8dixzufCT#!Rp4efkHNr@$As*lDjI0@ISZc6x$&cNe(rZ{Co+=szm8VC|SsE1Y z?ekLNb9Lk4GmXTKX1q=-%|xPy`tIr^El+OPoSA8K_Bg}1U0fBWcSye$JADlNF`bVa z%-E}_2YKwcf$C6}3km_(jC)OkaLD#CeWp@f(1%C}(3!BFe{5dL`}pkb#@q~T%MlWJ zP3JLFO=zu#WA;-`G%joJqnj=g8iC6GWKB)dgl6iqi&5}Y%bu2(CYwPO4v6AKbZnMp zFrPjqdegbhYo_bOT}7oYTg~8tq4YkcH;qu?Z+M4?&1z5dcWdmw%%L}5$QWfCar%^w zE-3mtYc}2xcWS#C4I&y`xsQ3!j+|wt46F*{nwKP$A$J;KMv_w}b$85m0V#Aay*yUz z4DYUeZgg%NiTw;~`U5aU+eOtoa97dp4nqSTl~u%g%FRaCS8v?aNr~9T+uk*QpsTt) zrO#drQ~>JHSY4LUb=y#84w{UhY8$}vN~5{jN_-#oWi3Dy2YmknNFY0Wx3n`l$p z(#x55L~orIx%RFcSs&mVJVfy23ZP5MKVQ2mSNAWCA8_MxLT@eIJ9;|T$OyxtzdEF# z4~{9Bqe9m!`R^TEAm;Q^)SO`*J}pa2h&D7RXp;{^8>E zox|Iq%^=tMJ_LciA_C$jgb_5EP`vcSwV4g}w8uU|WXh zg@@_Z4D<-WL;cKyZR_kGkJ!P!<~n;dR9znJJ2y6~kkn`f<&%g1w29qsL3}>zr7z(t z`Ta-vZ_6V6|8QOW_w~@i+L)e6oAK9r$@$l+c)y+P{QIi-=T@WJ8{m{+OBz;MH-4-6 z)Ft6x*KbD#o1b!xP!6rp2M2kp*nu`@X!l9SAUOf=UPA>Ic)*!;xgWGC#b(tB>`nwzn zuW-t$P|8D}&>yc?m^nQk<9-;&dpz$EirOzVT4@^f(6@TVa}k{+Vcy+t@Xl&8seULw zt_U}J#l`kql1V}V`Dgds@z>H<2MPGe0>)G<+o;!P+&%)sbY3;AcKP=b zc`BrLyvqGFAJLJsCw5UO8as3tdrTc|ALo6w?TVHT2yI_4%*#`iSB|1Fxz20qN5O@~ z(n3DGFLimP(4hK~vUBZYQ3-p&D%yd{yT0lmfuFe#X3Ry#t*>KA5;+>LcN3mJYO34( ziG2q-Yv25CZ2>D}c3$WWLAS0om_)%v$LGKCMI2LN;JWepX=|Yn#9{0lp#QB;bAw%+ zBDm9Rl7Kb4+%BqfC7;c-fBi$$d*(@Qa*Jg@ z6MByL5UbcSq3`4+mge0f=-msdW#%_0SLPD*sk$CFLMO$D3|2QY`SLNuYDdFei(a?u z8JLH4rZv~ET6sM#0&WXBt9UJWIlTspaQU4&lS^nA7&kPTY<}{0zC72wI_u)c?(^ek zk#}7`Pq^B73o3*;^n!-=rkxj{xT)@Av2@h~gXv%b>8tAj>}K@}WzgAvlH(YT7r${^ z>f`NxP+_7Xv+>>;4XqFbYZHj~4o1}1X|0HE@8xW8%2R+oVCuS(upXsdzC*#gmFzf> zvx)WAE1ygA)euFN${XoG)wXkPA@jKHB0KQhdbUNHUZtt7K!YO9;*YV|m z80BpGPD1lDxu8Dmm5iXc06TVT;u;kWQuipQT=%^M-Him8Pyrf4v8uz;Hb{BiLwWtW z&&y#X?oRD0m(92r$+^zi{qY`{(RLMMKVwJ>mkJ6&Z%)2Q@>od$eHi-2@pMcQr*Y(? z)u{PAn(F`~9vyc`+m?+vs#}z_vBQ=IMU)s>NdkVOTn0#~tFUsG!r#j(7;CFGX>BI< zs~nO0NDm9UQ7h0^ae^VH`Dm0v;3X`RVOZ@wRmbSji*6M9FAck~(Yeaq=K433u4wIe zvrG?b_wu57oQ-`eGzzfL4Shl94GRmi+D#roeAd0Yh+(kmwd9URv9%QYr4028%Joz> zJyUlzR-<2(rLD&;WqGfD7`7J~UPym(-U$_os$49N2fFQV_IYa7oR0P?9)?^+?+3Oi z7s6y6+gi7s`6ADbXX-&b+H{LYH_dJ`+90~k+IlZfndZ5TL8w{Fm#$u`>t>Yd`P-5l zkCWvVnO(kQ9w()d6v#y?RI{(XDQBMM0=9qNMLQMZacg7@oec}U{j_D>oa?>|t0w{- zz$WqHjt*{8_wi@$pkFo%qU?D{$fOhB-fd%qN#y65Uh7#cCC(DIts#}vq^VpUoUGhD z_9yGbuG*&wY4~JMAJ6M}Zn%@w!$L!C>_9u@p&fJdZkcAayzTunOv*0ly7_?6IX?S3 zBjtE$EBjt8OAQXjh2EK=Lt6>Y5b2!YYbqJ%`DM+LADTmVrpA^G$s_n3UtqYX)B(8f zo8}H{>#AyUhgz>ly!uDjqbVS--rvNe57z(q)CW?m;43dUIcVtn$X36t>uz%rOf>sS z#`patG$L$>BdlS&yzxBig0^uZ^byZ~#w4t2DHhv&FBWzvRFHCO0s=<*AG}(O@ex@l z6|uW+OWHS9SwT91hFhv9K0;d81F3Z?P?u|=AY$^cvd4ADAqMYMdRJ( z-WqV=LIu}#HQna*J)6@4Tc^%hU!GYvLIORGD-wt6Z9`?Ux9=UOFj)X>BpY`ghe^c zQSKKnsA!*L1CQrSeIBDb;os>jdJRSyPk2HSfRRy(j{DpaF8f5}DFhW&o?h+QBe3IQ zatXd6)fk#2&_%>VPUpSHnze}2SAVMMd01X^TIQnA>owG;OEe zKA42P_HAok+t-z5EQa0JpNF&>RBNdzRmNM7a%2HaLSfH8(@}h;+-qaFSzC`~-|u%9 zE@O00W%qvDC^;GHvz24NzR0az(>KfIu?H;z^lNIcuc_yutck48(LQ#Zg z`&93r9`l7|2??dpHJHlw-8trur`TG($WEEc#44^Caf^95W0dW1X}ZvI=2e)&YiB@i5& zb+!i{hzQ9uW;GlYK$~1gxNzJ1m@au-)AJvlBxZJpC&?DzUd0etxp`}!`1B-yOwK8r z-F@P4dti?0#sA*S0TlMk3ndiAe_8{)1aSinkPBt_*W5N$zv4f-)QItZi;=daAC2zae6w0 zL186ZV}hgBPLRMosjIOI&sf@vvqn?_Cx+?-Ib)}M7!?*Gcy@>E+E7Hm^W7Hta&>qq zo%Z?K!&&MsS=i!QZZ*~VNMr3apX_uKF}6ePEW998u;00yX8LTXUlfsqdP~W7$I_Q~ z0jDAZn=-Lpq|y-63R_UsLN?cGAhK(wt`;3T=QGlWYV6_J;jiM;`LbY|p^%7b%xV50 z`QBNUp!VCNNNek6LBxIKTqT2^RZH-+YLHgpoJiN&p82<4*BcXcj`L6H_Aq-f^}z^s zHtI$G*y*}5%&{~+lDW5wXb@sz^W9FEs6$gN*1BnQpRrPlFnb^31gDT$Du1t$=3nJvejlIFW*GOtJy($|C}jp~7h$nQjn*J8|w=oa2VzUoB!W@)bD?tsrk{q{9$h|Zfxwa!`^sXTldN|5?`;3rVEJf zCc;2LZf=}Kf`;67^|6;4Kucg;%6HWV@jexY0JrWnVWi)k0E3#mKs3P9&LV=hKJO9#XcV>2bD-kw zur3hGQcVuxL_~QbxMC%{f@4UEXe*;{q{Hz{MaaKA8I=u*($A(9>k`!727&+ruq)#l~>sy7QD#$GXpJKc5@_ zq7LWfw0~p}u)wM6%_jhaV|swhC+GDaF%DbA-Dy`0+Q{>NkpbTIt|mvki(G!u)+DeV z+v`$!FXg|c7O|Cc{NGKP7z;#jg+KTI#Jm6)yd>^Ixo@ZZ!qEh`T~kLqZ4#-{{REMI zMKJ`tA;f_G<`BY*6F9Q9!?|&3U z(*j80=bTTG0o&gA_iAsk%@@MW5msRZ7=0mQc8`g_AOl~o@S^`^2mpYY5dB`?EveIL zcm^PSRL1h;zVJJNTYQ)v7?2h-{Qj|tR`v=bea8G4z4a#npeN)}&tH{ag{2zX6uM!2^N7aZ%R^fR7**u@GQc>wp4m2XVf2$UROP&+!&0iybZtbh2#XM+~Y*Ki)9 zx(>#_FZem7EeaEmncD;RA9w-=f#rW)13>DUyanMI+AC zn<;OKzajxhD{M~C0HgsP0`O`2CNPB#rx6W@YXTr0vz7d7z@LTBYv1|M+3(%ra})>L z$b(D_V#5ajJfAMVUlbn!3>|P&zDw;(>MPBmggXnkO%b?dA3sL%uq)79?p|=xuNi%R zvip^XXI~#bWI9OrbuMw=(h|4|#K{Zr*SMcK_|&qcD8o-W)NGahFuv$R4TpO<4cMg@ zC=2rcSW33C?-vp{f5Tz>hkzfkM;prbgOAg2uPXpHgdaf(GM4?$@Fswq3mDGtK7CZq zPXcVFZQqwaYqiN=MIqMTy{lV!!2|zgb2jYX&%Oil1Hg7zCmJADH(1p4h-TFWcCJ;vME&R*->G55YAzW$k#4I}^f;t-n=zqJ$|z&b z@Y{9OY64obQ*Gb;haQ6wl>s-OP6Z{aQ5B6UijdlQlXB5cwVXHG2xnmlgs@t=kMG8o zNF1xytj{m;MYgNll9l+)k}5h@9mDKY3J=sIiYClhIR`!#%~-J3IT|I;P)BcTC>hsQ zX>5K^vVyAm`b*Qi_!@Ob$>|wI(+`J!?kN8Of8FDBcxHmYB>2>-!MA{q{|bNoFTr%b z(69dj(G^8YnP3XM^otUudh%La4!x1HQEYM$Rynir;<+!5dp^$BkA6MWSWN$pzLt@TP6ye?RoJMR&~q@Cti_p?XLaEzQc6F1#AlarU{M)GP= z#xnA@@DoNrl~(=^eeHBNc@M_LL%uwG@-pH9BG;0B30V0-$#xvtcP>m;1mCO9^OyPJ zVq8MPZp6x>wnPJ#PX``(i9G>7tMb-Fg151LElC-5yU5X_VtvgYsbatWkrh}ggPEK zcq2KF->Z6LKjkzO;Q#!%VIh^O4EZ9W#SAIkF7J&By682ObYqNvTUgW!8~b$_N^2Y1 z_;l5CE@-ZecSkhm^bBL$>YMOaCI{HT>5nilKl)H@GuXbXYtgi)e5YjGB#1>5bK2z* zU5Xn4j-evEp4T?U@+A&Mn4!T6Ic?TZsJ`q^6~ zU)vCK;V|{aXUK#KZnP(}ki_XU%`oRFsKD2#>ROhXwF$ox<9q!Q#D}^yI9#s%;AtxYQ|b50&_-VYX=ty zjh#h+NP>#|cT+hw)fuL&!LC&imW?$YhcoMwVW}5WO~Bq_pmffHx=4zba4b%&Or}$D z+RWm;pZs2#o<;rZneM+FsdD^2QoTRb{eOR?%E9ozex!OY^n8%W$P|1T&Zztww#~ty z$Dzl{#BRt$XUNROM8|5VZ$zi3%g9A%#AU>8z|LW$$DykWKbmCz_oGQxW)>D^u0M|^ ze|h)4^8W#V-v>Xl(Ns(Z93^51s?7>Y9I`}Hzci&C{<^lFLHB{N;{)>d2L^b;Z=@F7 zO;a<)#6G@@0M}9byv8=n39hFv=5c=Xz`_}mx?qalw?|$rviB#5ohLL z;U==dd9Gm+nW(MnkYjy){seND>b~fHku+!8r<%OU>Mt-&tc3wvtEN26iz86_;6I@& zg+b^jb^0-UTeD2>{o-CegOj2&_>-ceGE_0I__`R0WN#LuvgmsBnaBstbOB1=#^ufC zt3%v9n)X&qA>O$&3&*3Iv>YpbGuD3J<+`9F6m;I3>9m_C4ptqw)>@MjRTk#3=^Lxv zYso2(Gxb=@8f(?X$;89HiwXYeiL<52>+wm2(zCKU#gGi!O@HIBVb%@n*ZCe_J@T0o z78N;~6`)HJ%9iM89@jLx#$jl%oS>tHpd&03NLM zNo}^<`4B(zvGU^eLRFV1-CcVz5C7!^k>XrT(ix;JA;xp1PJOYt&e*l4Y`t)Y!D`NK zi)X>EwYhkvT4ℜr2p;hX%)Eo1cW=n00%3IsM34*;<1qq0H#w+o9)i_G&`O)lW)l zZzAQr+vbFk=f=LV{^++S!HZm(d*SG(g(@SJyvcm~l`JxM%g5DYY9yBGP8-=IbTX+6 zu+?=sx`F-e)A)UD{<9QPl`ANo^B_MU)>6qu?{X}WxIQg??-O$^HjUXNn$(>I1}~r2 z!HN8bt_C)Z7gicx`(UJ4qLbadAnX$oZhh6Z3`;4giPp{LhU3fajMWXlLllymu1!8* z<86&edJgf;u=iNk;}rMSo01f{y8gs!jpnP;@<(CXw?)PaP#BSY@e^sz8((8{aUt0H zBniyu!^|sX-n$GsXG>30sJpi{(6h6+Jv?M;@lIH3=9Sl_+u{4UlKAqYW@=Bj&5~kq z)Y{!^TrU;a4v^Por#Myfc76kdw5Z~E)f)K9x%h6)?--Zi1j3LBJHJt9M-mvQuLggA zc|1qyWR?2Sv%Amy;L^Yf0zpDN{N%DZ^+M?2vZusJA)3g}wku+jA8M1d-iPa1M8>q{ z1!_9lN?YgezgRe~*S1oz)p9J|K2-KFvk;Y<#)7Q`I-*O`@vp@ke!vU8KCNsn#^2W> zV)tWMI7jDCQG)Rv>=z;P*`GZMBUVTl0&#d<5UiK%mw2%|+zvI@LVlVGWS^ARcW%hz z5#z#!P}C2Bxu3jAYtEgc!<2+OxL9a5#|I@jvgQ&JM~eo6M~k*&Voq1=9Cg=|#-~5q z2H4=qY$+tEb1!wg??>m~9ST_#zBpvrR^W1(Qn*ae3Vur|b60@+A_O znV6Z0#YDAM2wh2ZxxYb+GzK0YDQoBU0PXD3<17|niKk-b3(&DB%npN-V9-LG*L7WL zQW6zKB7;`!Ra;e2GbfjvB)c*%*m-1W3!8-DVDh%#M_CzcXA=%5;)jqJYnOj=ICWu~ zx!u2+x;s8V$NCV4i#eb;7aA;r21zgANj^E{{4f|KlJS=mZVzD<|QS@sSikJ7x2sLP4V8I-MK;X?AM za!6MqelZ?RF+VFIK232V&UpwnC^jfek^zTffC|T&vruJTk($rZrR;LJU(a^6@F*Q( zA@UVB6x*3lZtuuvX8FpNsScH4D#fk4Th?u(>6ib}!LkAH}PNJQA{q@8qQQqIM;(2#9x@`gW>gfKvy ztRPa;DAako?_#qK$xR$HHvxBCI?$MvceKJyYMv*zXqyYQu3)g+ zqXt#uPgWcAwZ0v1M!w_W6Q*$RSgFaHaTj_qj*Trg&P}WKl=YykJ~s*x$^FJHIiQ(9 z@N#?b{M*`w6i)_!amsVLw2^KLdQ8pbKKq%2@_sp zt1)h_&4F+1U)-Rzp#f!EQ<+1{e)I7D#Fuh z5U^g{8t!Za<(@enPt@pZIgAi3`G?3)W7fq}jF!6boESIMe z1$wW>+~o_>Cqv6Z;1NAkFsrVfT$xTej%GSZk&BYNj@*m_vsWv5d2DhLB6=5zR39Sx zCMj8XF^ydgESDpHctX=UQ1uHOUUit}rRC<4)p|(BtoF$_k;iZ3UsDZ$t-~0G>b9QR z?@X`1Exz@Yq|Ca8Uu%@FK_Q604bwaduZhsuWqTiD>~hnN>hq`rb#4`PRsHJY+d;-M z_IO?Sq~(Q-6;!yF8NFOg##-fPZp=8Pm8I^0P)<^;S_iK3YX>x&b`nD{PZn7-GEN!g z{^+`EI3gw_>S|stZTB$vgsaeR)22(5wHfgT3m>pq_NWbBx@W$P zM#iHUr(pSXt|{jiau~qE>*&JNmU2_8my&~aNChN_7Pn0atm@&_4m`Q74&O|RH$2&M z;zJ+UqXHWI*>}o@*Bqq3a5Q*S^TShQCeByW0%Erp zpoc?(C6RS$Cs`=9;mS04(!#Ua2l1JaJ2OdP=2;fZ&mO;$1BGtQuv}v!GTch~sq^zA zgAU?K=|wG*BJvAM#5~B8h|ac3fO-~bmV31hCZ)EV71W@*ob$zOO*!1ff?erC0`SX8 zy4+xArV+^Ws)oWy?9iF9|E~1yF!rjYA^VbC^T<1rd;*oZMG_)3k>`(zO{3YUAmi*nPRvT*B&W=&DWhL3!NcjEX)ZOM0fYsu5btoctK+e8SX% z=_W(-oG8nMjNYkE55%D-aH?;}R~Mbaqo(?)4ZqV*v(~uu(&`4;8{3cEXTMB#&$#<% zuAj%CU0C!oHc|@X>gY#CCL>QB1Ho>=A%o9At6R(rEPM%2-P-a|LW<n8`Ew8mswLGoSQzhqvNpn`@y( zl%oe9BWoGop0NH&^tZerf;RcLynpNXN0pG2hVqx$?wgJ_{Qw&KLKd)dwMo_j4ul- z8wZ`PAqN{BE3=*f9hWW>8yyETr!Ko5lOd}v2lM?+_P@pb{*H6|Be3lM44}!FjK9O` zbXFD@IQntAHn1(l&K_lv0B*GS6F~Ey={MkUh@7Dqn^e1ay^=?TEQs!nB$IxYdA`+5 z%zKvZf&tlj@Ae!^KqX14ZqBkov&c#(vOjV0(}~mugqf4u)yRSoTNApv6Qi^rZWdp2 z=Arf2E>B$GIl}n!2M<)?AlgXmgM$1x|P&nGP39KzKCJ7-BRfBe3pjuw+q(8mh! zbXw3kU6Q6>gs_h&AmEBQhlq69*(X;9mWlZatY{Fx+sTEZK73s4)15Ek%N3^`;8XNy zjCTX!H9}RRo?Io8o(2^LIh6cG=Ck=Kr-zv+-vk4#UTx6PvWOQo&1eJJ)l6`%RFj59 zXKNULbl{bVaVH(-uF+sQo1~>gAD&@4He)60TsJ&uFq6xIAOUtVxN2@a|EP~7h1bJr zWu+{))TGV|wH-SX!hO8jnNu+MVf(sdMgm>SlPYU6)Y>Yv=sSD$q!rNXrcR-`8MeNw z0PVQ$-B&)m-Wjr5@UUxw?ceM)7WsP=gpKB1Tu0=PjMUb4d>J2{;D-?W!(Z+(Da5dX zqboGRBqS>NOEqpD`F*X)C*YX*yzYwK2cH$R`sT= zRVYzl*G*Ti6G@|s6X;rPB z;gOOu6ZX-|^*0Dv+gSrltTYI^SheX&`Y*(?(`)HX@GqYYD#Fimn~{y|K0i?=&%;O~ zwz7FDZYjnBB4Eh0Z<@x{F_zoH*PO@IAo}erZ;USxx0!x9%RS;8XnsbFXp?5QiuqVi zcaEg2C(P7OWzp)xxVlez;e#=RZ^Js_pAjrPOVb4l?wy71`STFu91r22ByZCBt_Ub2 zQM0=-CSGpR?ghqFTMA2q^ZPq)#VM!f7>g1o=bzd?D#KOzj^l5e_SHniOD=|e*btTX z>(m?0<+wq4?bqJqQq1uTnBAgyNEkc$hdbQlRE5i19bkdL#~^Xd27<{l0va#MsKRit z>_e7i?>fo1-+v@`37%npi7Z+ZPwxzYJ?28o_lT*b^hqoXZT-oQH;w}I?=VU|ZWk1$ z%3)4uGB19OkTZ=m5O1_;AV$RYLw9$$^we7yFWVH02U0l+v>Gpq;!An(RCPo6^;gnS z7R*u;Sv}o!fdpsPV3$z8v?6mWzC%}vQ=JDq@r4f@-3b#`>!NB+5??076giBSvGwI* z6a_e{+kbmFsO(^-+cmtjk3_?<=qj`tCYi$LZn~qQP{aCUz#>PmiX;bTVJO8NlGXsZ z++0?-Xyo-twPO9y98+;oTwLsNrC|kBg`RuVRaLuf!sf!ria4A}ni}S>&)ly@!i>rK zhQ{(_Y);R(>li`dQkI(jTF?jo@Rxb`0nG);e#kfY0o%d9W4O%vjC%Tptc-Lloa~Hr ztgK8-bb3t8Z1BfRI1ITAbs5>%jJV+2%kOY$_J7%C*jSiZ{}sc12X~hL7@v&dpkWBd zaO1=MhAsOt`dNL|BupX%F)0oa+62|M(mn#|_;5)~Bjd%4#aJvO1!rclM|=wF>=fRG zp&ym1)oD7TAC%J;)+{AH^6nMlBK?ueJ19pSwDmmG-IyXw%s8R?WgIIMDlNle-+btK zcQ=1nf8puOTw9U7I$tO1x|Z2pTR>(QaV)!aMpA&f-XBt;j$05m!NJzKvuhPFF~z|) z?5rA3t~{Bg@%YYUokDtvC+)E(br zmhEjVkdREKChF@L0sHV zceP`DHdl()*0ts4^6Rmw+O-*$I@@U#i>_TU<_DhF%R@!lob~1hsRwtLe$`=~*96A= z*Zj)+E+^S+&cKK`Ip`AJ+}-{c0#MWDBWwPglRB?{;kRM50ZFQi4?Sm+i5`SpfZ9Dr zM_o*|a_k9=2H#GevFz$QO69cUOXr#Qco90B6N{fuHm*ku`H}IoA>s{OZiE?YH&9Iz zjwH)iHl`g`BjR6(A?s{NgV5f?dkhy(fc3D|*adv%iQ(ksc zG5JW$RuCH-`J9<|nY{dzto+oxDr2!3hto5Q*`${V4Kd!&I1wvm-8T^ls(bQIHvNt|bSm4zlV+(M?by>vI;U6S$hF7NERdwIeR z7a_Mk+igOL45`kpiX)=Nw+D&lH>qwEV;e2#R-bNBE{w^r>4|Vnk7R_(rdgU7>dr>Z z35e^iixcmfDz5tu?`G^73a9bi+Q1+Ub>7iK(EL@PJc-+D6BbyoJHzelTvKX@LXCq) zrak?v-N}u+)#B;O?O^+!46mKJ+tfhVGfn$D>r|Iltj=d*$~a-C(>wZRciWXBEHhrm zLB2dCw}R{+mq%?Qs$n~qCAX|qXCnpm#5~sNn2~f6(IjUU4iqA+m2|FwbA^5@ePtkZ88*#v4h0>PQVEF z?_vcu+n2U4Wc-BcalCF)geXLpvO=_>vg!M4)`!=t&%3^+xOv{}0N)(#UC`4sTYaDy zXQ>sLK|qc*R($UPYM@J0vaV`R%ZtT%OTaGd$FMnpa zKfMDXE-$)_IS$>fV+iZV@gaxqT zbT%Kw5&md%VVDz9ab&A|hetE3xS4Nw?ioC0;{ao0@pf z{N2zREk=$!6>+%US~ny6dwU7MW>Qz}pZLPA;>+EcCCCgDIM3v;5Pjx|mc{-i=Bu5d$e|K@-1K-V@mpTAOhyx0AjTmtzwL;q zoI@s6Tmx#B-^Xp((#Y_2#0r^6P*h%%u(6jiz0Osn?1XOomXDAo0lSgeW1>Z_@FvFQ zb*>Jz{zBY_+`+V9c8U)%{oN^sA5s+8(9Ys>SXd_NF&pz>gH6S0kfAYX?WRA)^{ReW zt#N$0MvzZ3!AB2|yjF6NO@?he(J*?h1_toh#1xlWg^g*p)9|a-(fB}=7f_*ONO4+w zq-t7sl4On&5wXlEg1WK9>jkcww1N`ro_?K%!T#^WOih&~=07-|e<#LPCHxljqjp3~ z?3q(1rbl3!FfUe7T(b^L;zT-E10$ZTIj_7y&vQg8?7Ij}x^nb;5(b{Ma*xOo+D(SO zz4(ao`C4v!BjECOxdnGxFKKf6Bf7+)!Sb!tp90`>Xw!63GifAPj=7AOfh1VbVBUz# zV>f*3T*y0NKZ@6VU}$yKRkPGT`AuS9*v}D$XvC^H?bfg5-Py36{<2bE9&Pp&OY9MT zWd!Uw&B*-|a@7dBMrb!;#PO8`b>HSDQC~TMiwGx6IEZAXi9}iB{X-2P$5^z4VDw2B zTIkHVHSe4286oc#W&6v8mqY|2_UD`y*59N@OQB+l=T#f*8{H512)5*xxAd2PGRUhV80n);d4h2>RCxkWxhB>6ZiT-Q1_uFbq z9QuI%{t%zC3e|%EywC+)i$vf38{4AD#U379?8>BrA#>Nw^mAwpdyn%lW2mnuP=;LV*Y33Ij>pw(ds}H-#-<3_-+Kn(?m29jmO(e}Xk+4AZ zbj@%O?58c_8dT-ke~Pq?q<-~5-p7>O3a-4g+xPBHn(v+|p3 z-D0lkj5AT|!q|3{b%AEzQkFWYJH`_0Z(h8o&bpz@XZJ^35R8-0ef;ywxUAj} zCn4@!VqT-Q@ z;za^OtB9aIAT}p>h_8;i+|E5T_*vP+G7l5*LpUau`%maF4yF{%{Xjem0^In`go@nD;#V|GgV`r? zoL?Ybi{~z#<3FLjn07+HkkDV9e|@2$6^JJTT4*ZM$DabcLe{gS|LTHYobNW|Pa)wn zzXiMR*V1olwqw51L_wnU!TmizhvY2gUL&0Z2-7`}&w1vq_j>#z#jRvdEo#>+lK!I! z06;2I{O3!esf_LVf#C!#DVGwwe-LtFT#fVpIjBXei0YSGNI*~{!qUYRCWJco zlLF`JvHtKLwtZ>+{@1hwG0Xpkcy&fE-H72Ga828ba^pTT*wGp!!mIW1E2;i!>c3}u zDuv@0bzH(HQWALQG|wR^Jo_t2k~J}7FHJ0e7zebluBX6}X{7wOFsZ|P0>bZu)NoD5 zqGZbM#r@pueV+@dYXx}lDtVtpRYDQ9xX``DFtNf@!Npkr3p#s zOPSNo$VQoLn}%|nvQ~EPCLT{7%raZ;9~^XM#}s!?CS1EHmpc8Y+c`l+>)C0xQd4`q zT%)#@bd(p5eDWOb((7jHR4l>inDrbv*DJwwYI7?+lLqSQLLSccuV?1xhcUv~x)SDI zNR_2jAH-D0s5qzdXn`Ey+dIxzK`&^tteePt6RECR$A1Q%@82@=U*DGj2h#q3`o4^N z{BLAZUi1EU&fu{5}5Yq`!YnI7A`uODMrY7HZ)EY@MeX{dzBk0 zG^u~b-#_2JxZ%JIwmxyIeaqDBP~PqV7r;-ub^}gN*H*Y}Bm9 zXqu~pNffibTPI@W-LJ2ec`8Q~%=h=pJxjr7LUT8-J$X(pj|76F2b_TK} zB%FKg!#I1Z@$NqA!f$|4T<6e@)CjhvwmG1HClMjQIFZMgf>9XZGIcyx-i0ib@gf>R%#5blQlckRX{Lf> z?7paC-N9l};ZL*WMFj;Z80jj9UBr{qDjNie*B(gKIlCoiMuogsX1Tl|2&)1uf)?+Z z>lZz5_C?i_r|dmns}TXY++bWI`6iASNtc^dt-1Oi)vMbRR|hPMARF!bkc|ytdl7XE z;Tzmfukdk1e6}J@HfL>CcTA)zom8Qtf?(l>TDB4RB85L06053+Tj(lx<~=HTeN$~Q z6WL>t_Y~jiwt3D>#K$_!iVguisCCYVZ3NHql>LHYr>~fP>J#`Ux^>}<%Za-=d$FfA zxHF`HgU`u+pn@Q^=xVmNBf|dR?%X#yLzf>naDh1fg^OW2*}~-HVrmfii~V>NbBeL& zyXG=)^yrE-lec4pMeIegi#!b*3b>NZq_XuO4(=j_Y}k|HopUeBVCUzO+2kpK&J;Y^ znb?gNp=8bs+6_k`U0OBUSJ7R~rFw^v7k9_)*r~AVy9FnhYUiQRREJ>$Ld0fL)VY;Gn%^Uk_yjXPWATX-ze zCkst0rdH6Y&`TepB7a15qOtO=T)RzYIeXp_C-H{$@4_P6b}k=`-_9&@6^1Un3|zpE zj~Iz?UUXcmUkxs3o@OoI*e&1(*3?@KsHgB3USFBZEZU!o#Wq|`7Q~=CGIBkmNFX*a}mVx%`rb8xnLMo z-9_ONPvhkrRK!}r`UnnTg9oIXFp1iTGMXyxI;T_Aah$<-`x8?|`FAdFo>0X8B$GAT zA4mN;O)z3LQoa#+Jfy@U^T)r57!`OH-Fx_qCOBlI~*Q}+2N z&*5bk63FwYWn+(BzCyQ#$HefE`?odjdt6N{pK7XUdT5bQ&4ad|^ zT~#5iC8?GT%UdI0H$?|-*z&iz)cd|wMrhm z%onZ6?;aZmolbEbZ8ZOXn0xEEDwii<90Wu_LMdrfx=T7Gr6r}iyE_Ewl#~YP2I=mS zF6r*>uHQLwRlMr`-rxJ)Ki>U(9ysSY&oetavokw8J3HHp-8sMZe7g8B!(5I`aH%LV z$b(dl0&`*?|6)D1Gg(Ijd-#0(ImOLs@5%K@a{9=~m)cJ<`sL zdp!4E-U{)=F(j@@&Pr-NjB9e#JZ|U9D8R=JL4HQOkaz%f*+b+Q_;Bg40msk*z5FW7?DI{PPHp zhD8bLWB(G1fUB{=AeW7W8PbQH9L^VL@DV{y-A*x-EM?_h3ql^Fv?Dz)j-T61?{_6L z7($&myt;}MBgMJsF;Tu*-7=xroI90xdfZiV;j4rZ(CdA5URTF5xe^p#Wh!D-Q9eAw z!8BPkf1_3%B6d+(iOR1a#Jue`t1{nm(|DgCBHj49q;_gz2hOcTMe%$hdaw4zbD-8` zaSQ$=-f0ZEO_T!p*m-$86RbICKYzla4RywB%1np+0e1OL#Pim??5x!ed?S9hU6oP! zy^Wz_w^^`WDK_yp7`ghkN!MLXeEVZtXO=F-#FK5((noEF#@$O2W0oc^#uKHR0y_fM z3ii@e^46QhpXLs=xB-E<$!#!At>$M^JE?2gt$ypRevPkM1+4L6FrXv(N&`_Tu_${q zAqQ)?JNjLRv)#^{VEVoILnca5^*z}>G9*-eEM7|fT=Hg=Vls33eRgSj+Xxxiv~VlO zuz%pHN01-%@WlYwSkiREV*5~UMN)sSSE`ppXlil-v{o`P+=+1~9Gte7ES5rL8b?CR zn*)CLit9zy9iHwe1DKK~#K(C`VYRGmbOSMhfhn@&a3A?SLR|~=Nv3eH<46ZHr{|11 z8XBbi8ZBxv#49!R*HmdZg5iXM0`nd^@QzM4%XPq=qz$;Qy|*OE$ToiSWw+5R+aQt` zm7!RAEf1RgR|GbBU_Yux;XyNC>3H6i&nc=H6lAJNN-lDDnR%}+oeWWh)_ed6J zYAM^2(rR1VGaB@7mc==2PBllPP*tesx1%d(hTb7pu`$=I@~6}ApA!m2@MdGkNRQBm z`D?X+T}BB^x!Rbii2M$e;C`F>MUeC`n-VfgUCI;#^H@+|bj<|c zO7;XcLt%a>m?L!`%SK+%yOt^1crC%An!KcPyE#SwX;|MF=jRe9I?`%#Czv>|J}=r_ z?ID7;8dJ4AxWUZ4(3O5OdgZ`s9Okocldp`TL}!geu&vO-i4*j$uyKEziPbR-JF!jl zL%+FkM#@r~=xTfuH@{wI8AXDw-K8?~^fm`?;{f0>{jk!DMPI!w(Pa$a2f!V*uu_K! z13`+3y(;StEx^ahmEC6XWxVNXeRpKVD87vF@FN zN)-NpAxeUwJw82YDsh5IaIut^GeQodlifmve~_&H&W68HjUcD5CP>dI3gsk63589+ zcttot5jmzNWxM8mfy4aFW*p#~fyYr)tGlG({Ea z-n8rYoAjp!m#^>xhNj9Z^O$v?{H21VNz8SaGUs%P< zFf-CnBS3J*c3^%WtNCfIVaRkL6Lkxdl9X2G;YI+xSKF);}0}RQE zYtv*?X_B~C+fOj{SWCpUM#jg+Nkz|>QyV&>rmE=}jzk$KMMQ;buM@uzXg@*BrSp6~ z qTU&z#`8BlmEwBDt5qb;O)xm}%4)#9*LOX5Ebc_OVbf6X9A96r&{@PR^-n4vgd zP{StmN@mQ46j5W7CpAt|w0V5Dlan5kfbxkFlUbvBvWE6T-dX_iNDsHh;5KNHO|V|_ z=_WE^+g#L{mKUFPifoQt^gywijW%gZV@FcP;ll2*rc1}}5M5F3+-6lHG&$`jffJo3 zbTyAg>#@0lNQU@v6dCyJcT3$%h-7ff2Vt|e3dvB`wHe@uafpBrQxaN7;b}O6URf0i zh7Y1}FSaXfWyR|beI2dw;!MUPU^ZSLm*(YDOoF9A)Sp|eWIh!Gt|`;< z>Lcy+aaz0uL+(xu74{HLLMJ>OeiPe7wDEZ5^Kc^)W4cO_D}EEFVPXi`!^6WI1ErB7 z_F=1OPBF2I(s;L{mK$0R=s1s5%iiVqq%MeDw!3q45U-rKf`1QotGC1;i32_b`A+sN z(Cc6-J|?vW^e@0cZIH&*vmPt}*1-QqsqdU#zJ*-h+0B{y%{K&R1cZID5{W#^DX#(m z`oZMljGFEFcaP6^uvGB$(#)VlVk2K28$@k09`|gS4p4O%PgldgKLk?GrJXnc^lH>6 zErMLu7DkHP#{2C;FHflLO6mzsP=fC4T=|*{pi-9ta#<%q`z;@9zqhQ~(;V`|Kg0HQyhFOR)+U1({7?=E+Rh&O#eEk8$6|9@( z%a1Bj0${1Gacc5R92=#L%t_mO*{7blhyubVg*9xc_X8%tIRjg z`T?=q+Kssvb+2`Skb8mE*gtdo|Vom!F47N=)oqW7w0Vqr^Zt<=^DHuBh7+X8t zX5nv870;rgXRG565brvQfDt6epFEKV&FnEXWFJp=UZV zF+jO!up%H$by%=Lr-&4wOb=eCSV{z21BpW-b$5#Olvce10iA*4uxk}c^r z68W`Am2cd9-;-xvhsb^6iGEWOWLJB9ws%EZPjGU>y(9~t9^P?!xT9WYVFTOum2{A;a>7`BqEE`$AG^PKDW{VV;KI!jDD zuDn$^-ackvWou9I7pX2i<^iu91vPHz9UZ#9l=d%r=w8CKy?%exu=5Q{AeoKs$-wkm z=!={pdOa{Ln~898)V+>Qg)ru((H(|2MA)`cGB9^MRX|_kG6Q|bA3at7vg_Xs&b^JA zx*MGPBMIdnT#es6RWOok8H`4v*@q(e*UMn?M?ZV2G+ydO3;??m*!a#ej7QJ}j@ zc<*-oU|SUwoF*6=E|(M(pOC0JWBo@`pASH9vK17ymVbnpzFJL1?8${5lI` znJ`w;j`|AMrIsysLg#4%##W2MvNJBAQ&e8#c;z^MRSlj#_Qt{mhk_}*K2}q`%a(~; z9wu@UZo)P2?f!&n3wE8e7(vQ%(O_Qq;s^3A`9r%#(D*uB>jPSMz7MB1Nh1M^ANWZx zHz$uVaL-F)&NbABY)m4?AvE~oqMG+|n?rJTpWdfbKzQRN<{asbz>%q)muo-=(I1MG zcYj{;B$1o}ng`On_sxqwGjN;1EnMCwRf;#kkp5TKey!;0-L|2!FR)sBKDV>*WyYZR zoomg8C@?q-Vw-nW&u8xWV>->~s0g|jVt_5{s+XtFwcE?L_7*NJmvPbUL`Qk`EnWAB z1t;pr%Q321L%-uxf&bS9Q3Fezz%Q0u^9E7d0=s^+0?}ik+ z>{>j#WCxL^c>TvlcZ=L*q`|PP59_0?mFP^8QPSJwUpK{SjiBqwM+Uo>Dez%e#Gf~# zf0c*Z6x%>~&|tiGlMWaOeiwWD--yHa*xTQ12mcX!OO66gnrgdsn4-D0y=hzv?vCRn z2%tG#m|78S44kO-z|>kKek7*fgJ*7D*wwzgJVG0neuDTk!Sunu$-mZ5y+rxqSlC199 zU0sn@&HAKwYB)%O*$x`IffmA0pX^EmAs$Qd4U&sPmBaXV_t1 z1sVqCq_)W8t49u(-f2NcSlT?bFNoULsRQ___w1nx_)#55Kc~D@Y9{M{q*g1go#N+R zRm0QV;<5l9GL1BsOajOgV${E08z9iB;4zL;g?xVkBdvH~9a62dE$AN1z(=aqRD`1q z%619+N`KdjU3ADc%uwEW=y&8?^TO-CPiVyXKtGJ`SIHARIg=)}HG>y-YY@%!v%ZSf z;{QC;!q}O9lEN8OflD6_O{}~8TE&?+QB|-atleyM3b&|QFj5P>lnU7k=OG(M3q@_eqE3D-vTM0cZhW*i3fvKFr z5tRrqPhlH|QuLVM%!C!p4*ip#*nbf>&q^l6Gten8-e7vzJ@i28g~iKZ!eT>x^l|+$ z0?(|~3jeGpo@vBdu+h*%pYX)UhUwy}KjyKG>uxeSEqLXLs#MTh(Gw~9X01P0^U!Cn zMPt92yUG-6VWCy1V(C=z<<93f8Bunoj6bv2+nOBJF5I)cm5oImzg5k@-QV#&Uj4gb z1`Z*=iCO>M5aN-w^)-F|?-Nu2PYw8M4e&>H3Lx~qB_9S3Lbnhm0Cpm8+R7y`sSog? zy90sZ*C9|-%g|Cy!^Fr)&yvDi3y`AT!ct4~dppIOVt;2VMNR77(-z~8Cp zD|nzyEA%V)Tp1;kaVjLInS4$=0=fk9!$KuO>G71T;&JZGXBsLQ9vV7yil?}eUPcN*B%$k_D+(5hLmL?|xj z!3Cl)G$n9xBEv`TwYfb4W04@WBNgh?7V9G7e)n9lMm7ZxJJoF9ZF#@^8k-WBotf-# zK~2$?+BBBTsDYo0^npX7V3E@n$r(gYSu+XpJ9O2l57ajhE^dY-O<7j#2vv`LDJr`Z zNpmAT#$GZboCtaoC;3r8If&%;D%jZaJT=?%i)Yc?^_{^{}SG(-J(J2mPE)$p& zbgaQ4wz#6__%4V?tGR;JZX7c?tj-d;u@ zYvG)GGG&Mgwu7fNugmC;*&oTJpKQa7;cl|56S!i5M`7u;O7w|(bp1w*^^CO*^>lPCzncxeQBqqGA|?}54?TCzkUe{m zNjfCrHa}2+-OJ&8pE9bp`Zgk}JHrs_lSDD;`HT*+Trj;T}kG8(^j;!dMS;u z$Tdx=Jz=KxL}~i8$yKKBXtVlC?70IgQ}F@Y70dQzjJ2{hC+$;KsL9LYJH*$RoOzc5NBih7P}JV<)x3&d;l-xUbeM~ z#^=LzbM`3mfrbT@ADsi3WQ-Fw|1=fQIotq&W`F1$v{Vd?)Jzn2R4>I(odXyM@Vo6P zG7X{_)Ce^zq%sbH9$QDBCa{w-kut-@9}jW2sC~j7N1iTO5=KDU+c~Oy%1O250VZ*B zKHU|wH#pdC6H3;JWvicAa`oByfP5~*mEP`{x(dc{79Vx#5oKWW9*FZy!`pI0*q$Sh z<+DRjOHK~Rj2ZMINQ|jX>4Zl;3rCq-l@TAi>e&R3CMYP7CQcpH16SO7WFsEljw|<-zf{k7JQ5hW6*^jogLPU@LsLgE}@{z;9~5>aj`y& zqnacnLw>09gk*h!7K!=EpeCy{xo@R}o1XX!N#1TWq{JvB--CO$$zkf30WZla$RHvG zha^bk!Y|1Y$y<4DbbNX!(jaZh(RX$~jN8PIA7W87<&sOuQP*c_3g*4I0QsmHo-=1l zF3yGD#==T)&PKpo_2FR!wsTkaAgYSnnCiPdPgX}>w^_Jm_`|$u=0gO)kslO+UA?^# zMJ*%=lQ;K&9`OH-neiJ{@vE8fhf)7mGXtofd=Zl}fSaO$7w|NrZ=?TRn-&1A(8nyV1~;FR5k8$-M<}p?N8%NwAx06Mv`~c0TX#{J&>K&A zRVG(iSpzwp6=hP37R2E~P9`QQmdo@gMDVkp!*yih<88e6saEG}mYJVOg0L17Shks< zEqB$hWPH@$tDxe0LvO{tm{(aLe%}}1a{qo@`C>@H%-JT2fW4T-=_iH+)A!r2uKCcy zB3Tncd0vhORV3dup5P7-tyT7deu8NYU8z;0s6gU%-pqt-bhYq9NwlI1mp2mMqb4_HR|;g0*jHQ*s@ zY1JL+8y|goZ;)&{f%hsjfYTR3V`BTB^3Z2WI#34pntYkigI6-RNvmSXwjz{g8R2y~h((d56c-?$7WIp*-WFR=RSJGw zY|rwlkSK*@+%9NhVH!UrLykdbN?)$73y=}W6xEcNbIV?`t|yNk5RZNX^--#--)_?X z=Dikv`?2prEAf32hDLeCaxuqsrCS1Ta z>}DTC!da$L7BrU9m8Y6*4=(8WK;Lm<&h$3A-)HhqvAU}Dt0_oL=`g#m_kiQ!J}P=j zko&^J$5+8za7b|)%`dhe&2YJ2zWT(zs?WB6XSm^!d95_E z!uU;<+)-sWDMODozQ~!*yKvRUT@r}1dm$ML&jUE%Iv1hL5ep(Dg=)Li^qr#R8>>?@ zt>{ajLu9FmGnie}-OwN+B1kR^44(}-rsADVs?uxvqZVbBFdMvl-f-g+KbIRoCLKg! zZL#y-nWWp-E3-cvYPck5-2xg-jw%y@c}jb*%{%M7DHlS7>mg{GTkS*cR;py)X26aM z$y#r;P0$1N2Ty!0iSpmVX@qNqNw(^afm5=KIHYlO>AoCwiW_Zj$b*t$8}gZLdo2sj zGW7W)l;1Q*Rc0A{d}9By(#mS-8XK%Gzi;u}_68@+*hFI1Sd!3FADVM$J*j2QD@S`q z*pw~ME%D6Qv?bVN_GO8v!h01?gIq6>d+IIOhU;3TT1T{?GhW;7?D~6;O?zOzxQwu4 zyIh3yE??6ce2KMpTLxZFVH4 zDtp3VL-PpsheBHhXwz0uDitDtN&{Yh8e4v2-+UWefXgKjf7aY0lhXoz?rur1xzd;w zz?a0EV84(aHQ5@{u7idPX>!v4(0RiETr8BWtQP^MRtsQ>4egF*zl{i{Ff}(Z)dHN^ zr)BX~&(Q#?1zvv}G^`{=f?}wUs#j!jx6llpE@nUW5{TJO!g)0)6>eXMeGE2PQm7S^ zb9(aPDHr~@z=O9lw(&Dvr)#PuTpJLT?G0#lDKSJ&qNry3nCWB0A{-usLLDS=H&JX( zrO(D`aZ=^Na=ZkK9hj7?yYzD_IOmgs1*QwK3%FpE6&mpvBMlTDQPu0?S#`D`zjTJy z!gFreKJrYULGu&Nen}@F_%>uknaC15ENZ?-H?B{lwwl~xch=GXEyRY@ah0S;AZ`aw z0!&RCXZB;}{^OLUiM+y>NcKd$s>WC$_ZqW@WZovsdoBCZx{2I$q-z!An2=f_TMGr! z*X8XJbAdryn0T1csX~Cga?$9>&5fD7ote=-Pu z(}lm8=-&;3KbYy?41xn5*Lm8&K~%yqMU?i<_(E;*fDgyL!#O{Y0a_i-ilel;9^ zwT^(3+22fPzag@n(z1PuR(^vVc+a>zia{YQ>U8yoFn@JffuC0Z6PgnNg!vEirr*%v z8&keY_#Z4Zk5D0ld(=oCC)d7DH{9fjJt#^f5#$kcQv3#Zqbo`qs>?0-i|1WT9$O8e zPYl>I`51s6ync+-oT)v=UAi}PFag*aJAq+&al+*|(TUxa>Sl(_ z(`pR&ap4ddZ;)u%_v2%u5!zXVi09cQDWe1h8l;1(<>F);ai4<{aZ`I0TcGlg0z_cT zC)E1ZHGQ(*P${Y)VW|b`?lC}Y5oC^(z9rBzJU8S>$0d`=^ucC7(>1{HUvNtlbbvmC zb&<1Hle%EKp~D~{Mak6SLn$<~EWpE*%pQV7^^@{5eWuo1_^LTOwIfnBBMhBFeZbKQ zOI@RVLx%x*jNaj7#)yi>?0lX{_5}}`%)3hn?+Z&?)`OZEW=`xI_Y%M7V}=rogCSO> z96$QUBA|Z)y5;xs2Ltvuvhv+z|H>C(K;Hmff784__zxUdsPUISg#&p12@w182Ltxk zioSyX!UYF3GQORkb9K2m=BUHAX zPgtK-LTqhzGb>MFUggQeNqec^;gPu#5-iTJ`@9#%CzAj5qJ58+M zHBC=92#EKk$7<W4m50Ba%PUORv@I@!h0 zX{9a49$VxhW6N_RMN6enW;$(rNlaf~>(wNz|GObF% zAKYw=)RpPLcR4EJ78cv%nai%8(NX_KlDa*^UxDs;glA^`3xU0|BC^re#)~f?+ zOVEa$CuwmtgYq}Uaab?NE%vW>$sgo5yqlAIwU0{V<}PTTU}1F);sM3D5tXb=@_`9Q zJHR{2g4IG$!caqy<}hLs9Q7e%F12IX_VeAx2E4wXRjoA@!#bmin6|lTiLbk{=d`?Y z_WVair#|V7%UvMbcsFW)5La*rsv{oZ>gH+BkCSUzWR`7@TY+)mu@EX!qGE_ZsQ;_*8N;`$ug_=xrWk%`@aK;Vh#KEc> zNz>ZXG3c-%yIomR?7=l|PQ1tW`kbTmW^g2X1+|dta>o#bpZ20`qUp+@rl9G;q`!T5 zV*Vt{>;m$FxrBP;^dqOZt|TNAckJ6W`?OqC+CmEML_Ub*d8YTB{hpU$r-2oZ60Tis zIkqx6CL|Ns9&n*)=JaHo?Itx%B)~Mh#qC~p9;}Ai#m##_!HDY<`_X4PD7+pf;gc3I zp02@!vv^y&Nks7*4yL1~2Pz0Ys02Md{k#w%A8y)T2ZK7-R#uePS_RV2{B6PMyJi7= zfV*SBQCMmjnf_SK*}nP*-2vSEa?)v3%B6P=16m|e7tRC+u-t2Eh3mt=92Aye92ZozUQV1K5~8uu+ynq0S6akAaL zDTnR7Ib|G{&&y4^i{U(ac@<>aiz>bb<-h!BgntbVldaL}M5T}f<5M8(;3KNGW=|$f2tNt^4aaiC4UCH0CUeue)%4Q~B0Q&cM2n~4@{yHq z4{?oC5MW*=oD*%*moVr1CMmB}mpgjCoY}*iwxlDZYG9qV1|Wp5t2C_2Oc?r`};XuI`{w5U~K(-}wSI73%Y^ohYXu26A`0oF`h5kH7?wx(1FND zsIF`eXdF^tGgmyek?XRV9(TEF3titM%QlxVS;H2ETs!}Sk>!8E;>@<8k=KQ$%lSHi zIdO6pv+K)&??iw(_5=-BF%cb+GC~QuVY{pk2i3^@;0-Rju)?|y|9TNWF`AdZESg0b z^M4br=R%u>F&DNcZ&^@w=YGpR7aR z0qwszq;2*~Z`G8OR%t&Qr5sf$kxS#1g=QXj&26fRR=g4|O-9a==1YwQz1RxM*W zB*A-+RU)<=LKKo+pg>2Qw%g>Q|N3 z!s8N2O=8T*pmHPtibD!bH!h!;M-jB4`bc8DVn>`{Vy=HF4YQHV3VBcxeUyDt5yl>wf zCP3Nwjr@G8lhl7-zClb#_(wg!2H)*u5P*7UsHo|0=>=@UZ(I(4ejOMp^!0Tr+J6ZF zD0O!c`0F$rdP)EU>KC^_fMWH3avIJzApiMkI6$iXCSk(&`u-nHm;j`8idK7a9AFj8 z_D8~m?`YF7Flo@!(^HevYO8CJ(=jkI08a7I1|%4ML8r-}sivhtuSWMh10>^586ab}#sFvO3rGLLx?6%Yz2rXo z9&zt7;~8Tr-PGKSXL7#rxDpHEe%OeZk?3he#-d(~<4ce>vxDjE}wiQ6K_oW^NdLHgTfo?oD}F0<3@(RzQW zIvzyfaY=9@hE=+n^N=iv4)6z)Wc*Y=mwiFR&bUFzYLp|orFOPvssrlnWHYGVG>O}K zf_wZv4nt#h&&J;5o_Ahk(B#$Cn9{?O_?)=H$4Z-$H0U?K*TdWW6+wqZ+nT~QFE!YR_8F-@7JB)H|(@p z^RY|gIU0<*3T9(4stL*p`y2ua~_@q z`3QFJBBX!GEhP_-UXvd|dD(a$EpoEM*po-S%k`qJ&j^R+Cf(e2ax+J|cM!!V@pR3$ z#QK5vw8dgI!Bg5ZL~M?yEwJUgfm>b;c2)FzuP~HE$E~n^IpnTlZ@_Jh;aju=`-U7m z%Qt8HycNX!74jhC<+tfwKB0S0<*nu4E0uH%q$U$_TiprNdRczR!BTd0-`@zOaKXZ6 zE}f;II>4XMw}#D=!rG9g*)f>?<+kI<93r2wZNgHX9Qg6bp5N^G{In&GkhKPk+Y(EK zEwuwe&;Dc2tzH42M87FHtsTgZHZzo{9zHLWLN}Uq0$VHj6(aZOlZPHX77Bsaof$vD zY0j$^fBV`~tw}3wlZErJw&nms;E1w~!yS>z+I!^uYUn|e&THOByOt8YrXLn|rH%F5 zBqyzsywq}e+1?E~IULSCE!lL`>GvJ1h2J2>M$qs~v_$MMdk4d!BFg9`#-2#o0>g7c zf~UfdlAufX$iVS5M?nQlgJFj0NYTwXVcs_*UtUB@JohgqXKq+?ND`S5GVk8?sWlD~xhx_B6T~Q9FS=^U zxvpiOTIsZU-;z2e%2A$uoPpUKAt66FONcrWS3>hxXjLMj21DVN}pT7E3w9 zg6MHGyTfZvaDr*8IUbVAT3J8B-uwNZv-+!5%NXe2P87qb&`)y9lpTKYQ=lq++xO^g zopB7K+GR`R6yfu1r{IY`$d61A5s9N^VOpQ_*|-#5Hua`DR8{&W~;oUmqg zM)^*RJ5XJ^oVOuaJq7{6F8zMy**A6hUunzlUXtHvNKrDi69*+5)M@=h*iCUJKNr862&N%iPUsi@oWf*09lJBht_*DT>5DG*gXR#>4!BU4s zOZBs@U2*y322FZ-5rU%FFTtw8d{0KQEngbq<(*enRvy7xRk%3^Yr80??Dl?YUdj^R zds%*Jt`_?w(ywWjXoLVWHpBHjQ`wj0FVkhLW!9VKB1T)I<8dcT&M)j*hESo6Rp_fV zgAIDT3&nz>=uf&{kTkqm{cs96gL7qQ-gi^f%zy8L5L+r=HkS!`*_%V{pdw)vHLIw@ z2QsuLHZ3aKL%!^>6o#0tkm+^HSN1xY6Y7Owa)Kfe=c2FUPsh_@ zd;{#(8L+QU4s5PByqVog+8K<7@?AdM3I5< z&zJHF(d0GdX~D63WVEvVjM9qYO|8xKaypr0uW_v77xQE=q-?9J!!XUzf4#I*KEszV1omfK_3(&!Dm!`aBVVY*cfu-o!6EY0=9OuVlHCm7CAtFa(o;dAR3S z=U}GYna|EVd?3}(8XgKW5~spaB}nVs39~^t<8a1)bN_;&$!$bbhz?cf0~vDO{=Ru~ z?T{qpS~p5IugcD0*c!Q(c3I0-nBtOXk@7VCQLBVqvT3tX89sSW{iK})l^s)u5EX1P z^Wk_Xs^Mg%!|W%tI{Ab7P7H{-cl=ikrSdPyMZI(0R`;Q|SDDKd9>dwWZ=T{&z6IxI z;m3_slDumDk|Piil+jYB!)tw7om|@l-%G!&Wx{%0>c!Sr62qQraYeNk1_48P(GoMW zb#v_#BSs=$sy&0Ta(VJu@`ipRsSTate6gych<>KBU>}t!6lH{i#q2Yx0!gQE75hx} z${}%+vGa+iP+Cq+;p(6hOExM;-L)sRn2v%oW6jyhOkg+mz`4r`tFry>7z}0Z=lHYb zjozTCi*c*xDHff*ET>aB6Qxtshh{bl=u?|d;puzij}r2?U9?AzCn%g79JTr@hEcny zXQUO`w~ll+gD7RmhtpO9+4)#CqA$}WSt&B}@|HF#8cf)j(z2?S6|)g*Zv@NidGJ0e z7N%%~z8^8}$;pb2Le;g%N|Tlm+ft0bh&7yRrIzjP4n%q(nB$vZRe_9k^0}Dy$Rq-7 zkuQC7c)86Jf1C~yqS}}^k>SZhYMiArQO4?v?xKq&msY)e;Sb{|EwxWq9`=eMwW!+E zv()95f9OqGO0X3cV-fT#ml-Zh&^-_xS7KX>*{`uSnx^F-?Nd4)j5+S>LFFixj3>$7 z9T02KkTt9AxI z;W(UE@};@Kf5!Cl#90-4q#yD!l5wZ~X1poY9`}N~T#?3p)ym9`4_oy7NLj7M)m$~( zF*1Cdw8ZtA_669)hRx*JLOLKe^ARENiGl&o4fpZKmuqP{2{U*zi*bdyi11&Nv|9Y-)7cs>#}t%s4$w@*=uEET1pqxVfO@kXK@GI4c?Chl07G zTG(!0GO^!Y4vJ@_q)YBCbAk@X;V5p~hvBhMJmaWaX$c@dBbCCyQL3Y(eBHl3(%MOz zF-^nH+DScQPb3%&Z=Vwvy=OR=Pmp`55-GT~b{-%^ID(P^v9d{SPhHt7yhf)tT|ijM zl-8q^*w4p-HA5VVwFBXNd|edOB}YGy2gp`{ZUZ~*B0RAji60m=b|K|n2qFWyX;YSsu!fTEBG!3#x4FUW%K{K`0g(ex}Px|)l>Zvqb)=Dk^) z4cZ7vx6v^_f1r0@80`V%WMF5CNB`^SRiX+Vl*9Wq*V<>21=P&;wwA?h8=;upJpymn z)wx)Rr0fm)#)cjX66ZVe*Vp0FuUzsld8UQxe14=j`i$3W@ts1=YT2Z~AmTYQV_3Da z{w_U>V~^^}&h>KpiAIXHP1|UDSzoz;cWxj-e(#DAIXkt3gXv!CmSjIZyB-kYF*>tjDp_TZN)s_^IlYw6d(tBlD-%M*mdbOfhVsBX7 zN5`|8ZE!{p=l8UD0%NKVhd zW*2&lj?liD-Q2srX<}{ObGW1k=grz2Q^uD>vF|YC8RJbpy|{_MPcJz-6R0T=c(_B9 z^a(DD<+EVHl&4T7f$TxFyr`a#`3Nh0Ah|oU3>O2-E5-;0spuK)j!Yx3{t}Ubj1#yu zB6{Z)n83sEEVuN4y^^3*N-ZK9P*{K;2~yU)~|HtImb@3Wb6` z-97EgB12icH%{7SdOfqOOP{gS&u?*{q%c~DwPaH@mPal;pj|chZnv5bMZ4Gy9<^8^ z)s7!OsH_D`7r$M5*k@~uN8ZB}r8k3)v}n<+lpUKJpGbA9V+*4A1>M}^>x_Y%kjU1K z(k4cBLGjNT=}12A&kbsmub@$jWm7Aj3t^jVG&M-1%dZWU$6j?)Gnp_AfAM;rFdA!v zQnrF=B&o4KC1<$N#3gK4E7WGyQ6^a})HZf*%KA8F2WvsYdPa8xYf3S%BcyHO+_XMe z6@Qv3NeDD4kW(1cMzr8)&K0-6^v!@W=ZLaE@dtMjrtn;NtQ(#KHGwz6o0DYWE3K*< z+&Ap6snecSjG$8xq$MCpLQLv?I_e7$3tkvW|7h~fYzDFh@wm{v|1l&Gk#jt z?UM#?)6_UvmpG6Uk!@F$8?)_{uKX(M5;b`VDqY@ zAQ>hyY?GNm?^VaAnn)|2iV_?Aaf|*AX=Qhc()!XKPJZ9gOO1g;wd+b4yzz%8Qo`kh zo5WlxNgKqx4qzki+qV*AcS0ZI?K7N&W#o=AX@&sa2-zf(2C2W`?hN zU1@1MTVLRMCLVjf%nq`-+dw%HUZkw(Q^E%#D@NgE=2Iy&Ock*_*d~IQs^#;d60ZH# z&p_oJkQ4jIvDI2az=!$yE&2*;*RPxt77f3USJ?U_V%6f#!U96k2Bz8XV}Nf#U2}5#NX8fOWu}l8wwBSr7f!yrpC9hC`4 zXb1wbDkGf*#(3|r_-&U&+&TydC)9oOmD>y{mO&cV&+8ijOTT)v8P>1hHuNw3HTgk7 zI9a!!0B(XL+QxbT{&V8&c|s&AYyz@_&`E{@K!AV*LoqJAMH1N4Zqt|neg{Ml21#;) zAO%kN4j`c}1HQY4=?hqafY99AF~0|r@?c61M6gl%1>gr>@g5~(G#BywSZOF+(Yi0D zL%=k`Af{4#rtH8Ehj5X(fRo>tqh&$rIe9K1Km?mR%?{K%y$8!On=%}9<2P|fR z5WzEiZHIuYVg=wkXM$-X28b!$t}sEOttBn!(mXIZt@y(p7vM`EAh=DYpJAR5+2*rW zwmB()xOz%Kx;_sbh@q+gLwcgh$SABZlwLnKmvbK^aRzYUI+dyBDlAr%Gg9L2!4Y(V z6QI%8t9d{;B{YY{lzH0hjbu^xHS~% zsj_tSAZ>7|ia8FPXrm)Q%vBTT@x?$ApG+2n5bIXIU&{cr7$l60+k~^DZ|oiwi0LyZ z=oS`K04LCsWPx1NT3~Ah3Gw7Mlo|k_;CkKUlmNI35{Ph83>KA6KwPoQhYa|hLP`+~ zgbyHwnyn1L3kX0Oy`zF0h~d{lgbe@g9taJh>>zdhuE4#%KpbQ+kQA`-=OCY(H+pVg z4MbXry!~|hjw{k^4Hl<;rYL}&ARz6Whctj+E5l30%F>Kf^}HkN0bg;!%(!p+6o251 zHZVsqpxnpXPU{8F;RSJenF@;6cq9tqMe`hxyFdUA9Y9S$Gxm1{VL=42r`Z#DV)G#E zgRTJlQsqp#0K6ak==3cG=hO1*+cY(I8fu!j4P;!4m@r0{~q~>2&7XClQ7xnk=2;v;B(8z3b&c zL1w9jQkVfkQjYoH5KJDp2Lw=X$xTHuGZu4yq`u+AF|~sQ>EoySE7i{Ja{@kSK%sL5 z)l)4?)Z07ozrFqaFQ+`q5T(3@EXD1ea|IC~Hh0S*M!d$?WII~Sw7@hEUyHcgSFm@U zb9LLJ!vFdmEl@=RGWeMk2cU)b>XyE~Qqx9_b4qBok!cb8KY5j@H{E z9#UK#0$^8g;3^}iI35Y$U66+lul6$(%`4kjso+3fHQB)&VqjMT`0>4PQkse(x(YWK zS$w|*5)7v=@s$9snzOCKfc}UrnuOn&|CC6+?$Q|YK4umKi0d=6;zS}?AX(0_V6p6j z%G9-(Dfn-jeF)#v1b~+Eh*{@VgVHicaBW8N`_gi|M;-h-*dW38EcF4u4nXTbf~3If z45 z7mzGF*mi|D_+IP=m@7~DCNvSE7v9$f;J#0Gt_@a*1DPe7vuA{R3j+q=Tk_MdM8|C@ z54&w;Uw--C>gqz+Uzx>$H)jhjixQDBuAIwc&FSo7us}PO;LEWhU%`}U2r6L*b*k>J z2FMh>GNF&cuNhtoIqZ=rYf&dI3X3K?qLyT#+@ULJQ(Py0ArEVBP|@;{!68=b96Gx* zrA(@vOY*cX-}(_l5PifGSo~Ii^L)k5X}e!EoAvFm z z-RB$SfmLn&ttvW3YFb7HI%;xFH9AIeI$B05ay42tO~5%|bW~J;BPccK=zyz0e;p_J zzgB?&gaEDr{Suh^ql*DK$?m>X0K^G+$^JxmaDZ~ZO9+a8iQe6P__vEww~2Ock1zWx z(at~hiXWu2qBLzZ2l#-ozWRXgk}mgWi-b0?@3Pn4ZVO{9)3RV@Gf2=bU)$sFY^`si^C5A zIt938Lb7t+I;enSzI_kEOF;=aR>f-WYBmCf}if5bQGIP3(HFVgp zgR7A0cd&+&9$Hs0bMT!JR`~c~hgTu;^||tZT5x28aQ8Df&mChWXRA~XQ58sotP*J2 zi9Y(EmTgR;YF|iDlH5Gi2ciDl9p$GNℜWFlB$?VKIm~+j23TyQzfPIsZx=`3fw? z@#1~O#0+W$GG|aVm|6)umjsbNj3bBJq9biVjxlwnYY%*;&du4JvYDwfc?4@k(w1kA zAk(ikWHXMfxU(A)%9%cbz~$SJ8}!h1>7i1FB5g5_ey+0B%Hi1~n8ojWP@7OAA+_TN z(6Psll^-UrQ!!HtIkTISKNB^%dCnsSklOCH;yD8cLb#l(&KNGKf0m*nouemezC&kf z069TBO+XTqW#V+7Sv{QsuGmT__g)b13a{<%I(cE?-sH>jVozm*3iRjKCXoRX3g$5H(H0N@Y zkyox8&lPGCarSpYIry@K6PJGoHo<4MamD3#PO;J>SxbgcTQ79^O{koT5wr7Zxm2*M zkjpN-Qq1)vjq3M~(&Q?coR=m^CUth}SIGKN!#Yc*&W?1MkY=w7*8q=ol~46yB*9ob zg+=)YVcKDzc8+S6r6RH7D8%~y6%8NJMj^W@8W`{2tgo9esHil>H?#^| zS`9648`Y+koLKUi5udAXln=RPX%5I;D3A&V<8kK%AL1NZBZBfbBa|RpAJh|8#>K3)Q X`qsW2u39WD-3osz;p4~4&+^|t_kq9A literal 0 HcmV?d00001 diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6_parquet_log.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6_parquet_log.zip new file mode 100644 index 0000000000000000000000000000000000000000..22c849cc5ed37f0ff79b27c040afd57cfd43e047 GIT binary patch literal 55640 zcmd43byQqS(=R-P;2|WC;2zxFo!}bW-Q67qf&~u{+=2%P?(XjH?(RCs9g=g-U50%B?ruye-r@W?>Nx2{AOYg7NO$*ON?=z zLHbu<;hrywe`G>KV`6P>U}~ssqig42>R@VZrEO{GplhJ(pi84~r~j9#&+7h@s^DTU zzZZb^kV58Xwcmkru)lh(+J3Ue7u+2#hyZ}fKefkfK*wmPt4mK!r)y+D&BV%TNX=$w z#7eDCuglKPL~me3uggMBr%g{y&qz(ruFXKlz)Vlaz(mi+$jZRRz`#rc7Sz_!(BU7X zbNJUDkf0Ga($Ue;6HSb)Vs;$d_I4hop~!x)TXwiu1c`Ypr0FO8NSC!izWMLv~bb4PJeBRrP%X2JCWeO4F8IJ9~^EL@PTS?rg^ z=kf5bq+mb@ceOBRPxQD`f5CY(TZ2Y|JAZZglIA>Ok)1y6E7n23VMCCud0e6Z zF#`~Eo^~C1dt{*+Z9MGmF1Wh)lF{Il@3cXa?rPF>JBP4eaqH-))d($6c)?BM$OPej z(`O(}#Y`D8!}HK{AhHu>m@Ex)w48;rd*yr@IV>0Hw3ipulRMW|)EBhIlRXUNb7+Xx zMg7tQ@;2QIov+6Aop?Z5PTJrk;UJf>bZP1}}Zbls%6blf=OWZrxDj)26f zYS5(%DWg5n4O=thbqMxs?c3rs^3FKs0=J+~#+x62NSczzb294a?rrDATr(FBXm5Hx zqOC6tmsS#fmqXU%qCa`f6~=0mukdv(B9%O_WXm!dVMxC4C+cPC>6cUiF$B+I8a;X5 zOF}9Y1~I1`>bq1m0$zh-ewSb5lwHu-PJLj+L@JwWZ>at*+K00P?%J9a&;D1I`7 zQHL<2FkX*Ym%^loCUhH#YvtZErak)2D4KU()2C9Eyz z_EME8JXm)0HYKWwT4#U}BzN-jDLpDh`qcz1r@M#d*cR7a?c>+IQynrj12Zx8SRame zXC1w2i?(0U;8D?R98@1XXkWRfkqoh<&MG+bnBDdooWISO^Xs`$KP9g^b|*B0%L5&b z9ex==n>!ws3cZb8LgLjyDgJTu*rDArj}g~2C~{20k5=z6ON!i8WlgpeO4EFHXo?^k z?>obrkyL|(OWr+)>k`|R!r!yF#;9^E6fhXmO^nxhVx_t=?Z@(_es9V^#(+If6J{x+ zAmSj!^<|e5g~<$J#^`*0kBjOQuFaw+=>s0Gq{VUwGYmfyXyC_ME+OfQ)~%v=r0_VX z&1J=rLD;8?cf&hd8*KoIO32F7 zRqw#Ce2)~hn4}n$*oK@FWuL-{49zDxhEH@AB@8Up3@oxXj&R^S9mA&h(D2|O6>Frp5&s0Kl(;2FU-T)c;#E`CmFd z|CasJZ_ME$f=BdbW6LQkL18u@)J9mIf{(!a67pJRDY`%vE11_i-t4?}L8dg;^8-xe zrum-*Ax;BCwmm%@7`ialHd`PZhSs@)U!{B}WN>5~9_ba~@77t0h;DU_X0n{i8WEUZ z@LEQ~OKdof-o9Y_l3+U#1X0HkDdmC*ke(tsZ}nktkcCmNDHfbr*$ir_K*a22twJLpcrt z9^~qRoE#%TO{u2uhsBQ>_-Z$~;-^MJH7*2l4XWsrBpYiZoCAqjzK)0RCQDk}AJJ96 z*V)wB$~0%N12ON%a(VA{#+;ACDIWCN{dAY!WAQeBs&##MV6{YsoA{)mlrs<0 zvDU@eRnk;S)j>~&BB*)(sq+Kcf@iEECJQmT|Ej zj_y93C#BUmZEh(4tP4T{ZE1jd{g5XkD!0itX0wnbI`ZJmXLyti5E&aav)yz1CS-o# zTB>P$vzVk{+PR29ap;0`v%fcXHQX~82vWj}sgWai%q1kqXh2)Wt}xhWX+W3^!M!ae zW<|VPHq3DyZuVT+St{=?Oi^*_hssRlb?)3O&}O*5?{9>>DrWakt~~1MjF>7ebZ)xq z5o*4A8h9K`2$TVqug+r`ktPLW#NeiUvg5c^ODUhqWL8uZQwvd0&vxA3gt8uUH@lQB zSK1yizZ5Q9+7nf=ro>EXg&LImv27Q%t%E7-xRy;iJmY70-JxiItbIF1ss2*;CWr4w zcC2jB<|wi38?6^m)3mhXTx7*xEz>sGzZ#f;U^mKgrDiKdFb3^1#kn#38XeI{?Ty!`8wy2 ziy?0<3GrOm`rhKUI5B^0t?@wY zQ$5`H0nJnTf4Qa{==OaoN{9`XXU*Oquevcr!D?orJ+auVi9;VM-I&e z5!brt*T)`EhaQi^=+&7XN)qn^p*6U&$W-6?r+&ZQi(Ome3ET#r(Ra@)V0J17S5XVe z+t^M{S#J&9_7suV>|6COxP}!Rt`5@Ln$&Z>M_s+{zS1_pwe2?q9rPb*Ivu72I?ak$ z-J6fn=CjVwJ{&9Vvv^i?i#KTq&X_^}6U8m<&u&OgE;(XO~(^+pn;=DGmiS`Ka z=F<9U)!Xwd^sPK6T-W{?-Ye9AkVVIC!)8z35@uw{wUG`TJ^c$AY-GW4o~k(Lh;JTG z$ubAfpqgBoh9gbTK{xRFmY3pcPTv(rp_$DtYgG!gxU9}v9IazSy-5EMAOwsK(JZ^I zx@^|K)RYVoMDe}tHh=Qk@MS)pcRnh*S@IThKZw3!OD!mN?Z*Wjh7t${)`bEq@RCbR zH4e9}!P=@$PFq%eoL-7t^M_O4ckZ^B3)Ws}bl5d^?LYV51x*#gx4QZ2768yYY86~CK${5*tr1^uH`gW&?<4R&Et8ZDzKZI-OmZPa!k9NuNH?! zvuAT5Pvq^FP72!+`=$luYBdhWEUyX^@i$;A zu%RNmbVsQ(RthWj9TBBkrF{dJPyTe52nqYqylpV@#v-mneLzFnNi6?|r|EP#_pVBz zc$UUb#2U8cj>+irMmbd=)7I~~j~nroE0~oeSEefUl2;GxE{{vgRt0c8%?S` zSZC1pXSuST78hS~2yh3}j93p$1XUVu!amd9CG?x?wFF!iD{Tlddh zlllTWk!)cg3k9XGp%$dmGMNq6Mr_PneR>2DvI~fTJ*&0>S!F(-8${2A*z-i}`t7&X zde=?NZ0$N8 z65kd=RWLC{B3F@5T=&}R`yCQpMMQl&yZC`zN>{frKUXZu5A%KxCg#jVoz*FJ4Q;<@ z6pkI{-j@9s({jHnRmNi|Ew*wC+gyJA?4e>{Dl8@b#v?A`@T+ybOI$>yqdt&`XIXK5 zon={NA4gp?mVEQH%IbiQNcBrWKj*tQN3b35OX?P;!f-wt?CS}cL*Td*l;9i3rw~LsA&i;#WVam?2P8`ONSBE|W913SC#rQb+yF(~diWNbpt`F&c zCxqndf_t*XBwNTBvr6wfftF?tb?f=L!5s~W4IvvZyZp$As+FY)NMWCb=81|ovr#BF zH|&sI8wgyzBZKP$UxaSHM;sZ;23*EY!4`%@E8tlU^3R{yLA~JV-KEv!0+xQqu#+0W ziu(K>5rSP)=w=#Iu?Ff5lX11d>KW7v#8o#@Y1CPVO|zK~rtrC1UbdG)jJa$ylW>Wv zK02EI@~LHRc^OT!>xEzL$?mWQJtNYX2@VqX_V#*ndMD+%rq~@PC$|FjN-!dy%-i9p zR%PTa7tDZ%#5rckiM4sJ!Fw`?*>4Ik;3qhJrQDOl0oX7Y1 zzY@EZT0SISux4y{xT8H(mMhK&aZ(%$hOJxWE z_>N@D1)G51j${t$>1%BF0x;sK;r)vvw-2S}Z{8yxYF<4H15T#l=Gowf9blUp0t3ljhk4ZuYQ7|>`T&S1qC000JDMZM28M#zUkpXF)kT@MF)-=g}W+Qa3+ ziU6|U>p1}411*R$fIDnW!)ICmM#(OBE~)wt>^wal^3`=afw@iR5xD-WAJq$)8F1=2o%^z|S5b$Gq40@+T@z z-xD#xbt*;{p%X}daoI8Vbh~B}0rw=}tMAxKkWfS9AF}KyF7(Gc zpILf`&!|HKE*OsTcHwb70<>h~6w%B900z86WVHrXQJ);#5`yT=zQ^=MJC@my{m@UtKoyS=PpYr1yZ83EL?|KBYCW zFz_2bGXr()_ zv63|)<@hIs_eGbF!JnDm&k;in7g&S*y5HXo1AML5kYEE_#i(k58=!0Ky*dmrC; z^FZMiBi8w|#q(inRsZJXJrF}q+Tv=$@!Zh&=@W%Vht?zBuZ1-o;8`QQR*+Pu=Og7?(Ede1Hq5Kwt4|Vm1>1+6oReHA#l9!C$_(z!x6lnZ?!}SjoY}{aN zzT(SkMSQJdv-IVN@4PGL$b}*lDyA{U-z zN+DCGm6#RKapNXM3<+Vl)$~WxL+0}jbBB2s=)}=N=FIzuYVw9LV&=Bfi4|KEnZ1M< z=9*DiC`?aNJFjR=Vg=>`R(cAub)s2U@bZdB_9-qtbk`*|@(hdScJAd>PN)?M`}R{! zCJqW`iWpBN6v|}}Svo4J7C6xKZjMG!^4J*#;V0kmw%(dF(dD|OK0ZPGEsXOVgkdO5 zCXfYd^?WJ+qZ5pV=D&1<{c9k@!rGXIUYqV$po8tNaK>{e z1jSs?3FF9Lx+(^pDB{aae;Q9RUx~HmQP#as?oy3hNkbkg4FX`qh?i&6v0xXpC8Erz zBeoFAB`xYeriCf~EnxGkAi+;q))eq}b)OLYUHT^l|9ABJ<>dPhx*aAUS)fc~w4^_r7q-axB$2F@RF(ObCJVSLtjqyA;3Tb1^=x(Wrs)?ZFNd{^{E~m6 zO`-S$QK~a1uYknh&73SMb+)0JT5E@RR*_~`MLv|2fO&UCWBqxWbCX0cK~mSZp3-Gd z)3MjF`*D-|G04H3g{O?523iz&b$%iN^F+DQtQG5HXw`Q30j@gE;v9LY%6piUk}?N$ zxJ&A;>R!*?E9_6CZ(T~dQ!^t&TB}9o`x(9aOtNq3Q3a@lw)ggBfWF`-0^IGfbS9W= zYW5NRtB-iY?N;!}$ctCq>Ib23{L%Q|ws-CD@K{=-NkOX_t!K|1!s%*xpJfTil*kMr z6vavB)H0D0=};~*()OTtmlKFNaWkF1&Kh%QsOBp@f6;)7Lq~p~K=m;-V&)rt^DMT{ zCzPn7ME!W8?vmcxSMfyE`c@&Z^Mk|!Tq!h|^a9#`#>}C99U8(GpKm$B4H#I7Q{X$< zQ@7=yGQB#(Oni73HNI_C_q4Sk+IQ8O$cfIvi0ei<;OaC05CzSi6d*pMQ*d8$1@enCIqoPsjn_MdKD z$(&Xhkv44D1@TT^_IX6mHr~1 z%ANX{Vq!^?L9@?0=JL2xxfj=}-D&>~+VmYH{nUDxaEe6-mZKY7*`%1eURJd3!V&B! z=4pjcZ$bHIlz5TEu8XRj0KN8w89#kH22NB5Nh6$`0ZAvD&;15TNt~R%ISDH-c@iK- z9iL{_qKxPLEQLwbcS%H-3;nj0#v-cH*UL?ql5Z4{rr8+pVocWxBwsb$f!ejF`UL$; z8+Y0`JYKEpnkG6RJA~T0sjI6RV_}&)3X5^PIf;@C&3cetGduhqaV|Nbrqe!RQf{he zVM7bL^RlRR1{(q{V963_`>@89j7~TyiN`m zfpsEST$LA)5>QX7BBjKiC*`1SW^N*I7uP}O_SL?gIL1omzG>R;7IZ<2vwtu+p(N!6 zPDIvyYs9%f%{1(;XstsiJ%%3(|7sniQ>lOvz{~cf`DA|i9vN_6Hy&sNp`AQq%%uSl zObz!lilSB7ErM$h+KS$rIrs4@faNCd8gF5V_%`-eWeBg&`O9vr-*!%Ws#wV^Pm06w zZf;Tng_h5#-~NoHXsO2GzVGaAU$stXnJ>OL81xxL_wf$Pofd4)vWETWCO)#NEqJW- zP-7PHDIlH>?FGFz(&8A}$9Hpmm7JLLpIbdib6+KvtDg?r>Nz7C=HWZ zC-7XATPotaO%J7Q8Rd3QebB-2#`_ZG74sV6WC87t{8R95p)wT~+9tB{08~l7aHCO8O5<yDnINQ1D^*sf9$5&P-Xmv}|kvcoj! zA*rT*)f>~FARi+k)a0JpRna+eZ8wUB&oso(-P8lqM)}S6Fr#Q38saEJ*7NWjOn+Rz zx)Fk8%Choq1EkD7nqe7Lumm$wS}BZ$UhS{k@pCH*4@E>p(7s2LB1Eg=g1e7}NMqo; z3`4eriA6j1y8QXcN$2)bcs94vABytmVGP#P<0H)E#5Biga@{achK`7m`APFtLjJ8EP>)$rm@wjuc8>X$_))t8T>G5tX+#Z6Qi+%+ z3Ri3g)=!X|SwtWrDx7JqWG^|Fm_5dCKKOWw|ASfVgqDw;NqW}HR|ywyVHAZEBhGTC zTf+yYMc1+jSXQ{0GR4O{Uw%&02;29m>@34RYMcknRD>)=zSGDvLBh(V9$!{*!Kyi7 zv01fTX`65Hul9M0vxcPZcOt1}N^46ucIH%?G&CBb$u)M2CB|LgEe8Io=Qm5f*^jbdkP1uyGawY&E|Z{MN3Uy8)8;YaV71eM+%rK48U ze_p;KfEApEvDPyJ?|dAY~|V4w@u@AopCSY4@Rv+1X;G4J&c&hkEBzZq z$`6OnyE4c9WGdEdmL4KAIDy0wb+chD6}~Ky--zAWocboy>aL?#S(F)MCOd2-zV;H! zYCs|HdsV`>wGmcM6`ccSI14bC$!U8GvZBO1>8Oo)f}wxM#FCwSFMHUT$@`(krc=)D zprGd^}1tX$)~=unE^xvAYl~r*DccL8dqn6CKVA*?7J? zvipr#J)?N8Zi;IkPHAFfl;cj;oucrqzbT?N|jLR z=bZEOQc%l-bH&Mz+reUdME?I*|Cx}mP;_xlcw?GN2!A$mXGTZjug-)}X z4)YFHn~ab~ItDZFV3{o_Bp)iMBxSOjn9ArBI3blSY?*9+_JIdEqj|`G2hMo5d-;kbXn!OC;Gp84z!|Kx|66ANYv1;4^8aQ}3I=C5 zXc(pdPQ(%eMzXVIjS{D*V)=YxNhFuV5GBT6NLVh8uc?nXL?vXE6D6>SrCS_+?#cT} z8y=AXqj^%IGLyeuLQ_!2)4~e#^1TpyiFbx~YNmJa4{wL#fem`o9McdVvGL{O3ns>E z(9!kN@B?_m=1dyRxj#2l02k*M7ci`jMe{n&PoUUKTb0aQ%cDKO#Im9)okvaGUZR4; zt$%RzPIjlqAFc2;Oy{bsi&c%QY7LW0Ba?>GCojo5E2)aArkq+HwebZFl)-W`WR6Sk zJ%2mK-GiADT|wpRfU%M4%;9@uRhFAeq19A}3t193{hf)E8sWLj!^%xpv~6?X;{Z~c zyxUL_PWEv;2HZvz;%~%*2UF>KU7)%pG-g$eM^LikgmL{DpNncllc; z%9@H9dFAw4)2h?h#g^#NDmO#4!;eN6KN2XVHO^w}@z~ZL(TeU3&r2mzx$HNsahD%F zI-P;eUD?W-4_!GarGtj7T=qj!)DIHj@ef;^)mqNO(MX!*r#G{uuE3__heqn-haTl& z&y5h}<%_z({Ua|z`tmcM_w_vXPiR$|HoI8U70WlpyJWlfcvF;F&83^d(-g4^P1(5BoZY14kJi5zTz@KS9-@W=Hjp*cn`j6#Hb1t{#tHQs zEopgn5{K}xvaN?;$$0*JfRy3698uSH-yn@$x`;&ZvN^i#e{wn8D?Xw*EUGCXcAA}% zzf+=~b8SnhXPNNa&S-w6aTka7bjrv%tdVpqlsrGX@JSoXa_2|zq%c;&9u`XJcLbE6 z{MdlIyP?F7#`0Xa1@OYsic*@2Qu?5pPl}D?Q_ZdtdNyZj@GL9u2F#vP&@5(bHLrsL zi?!9xn~)j{$SNN5bAiki2w-b7bpKAZ2x*8{uV;J9ER;U7lc|HSp~}JHi+?p+;AI1} zt?JDUX}f-Xm}E$M-Q+q=pU~>EK>O%&oj2`|%V~OKpIgm+5$NoI>o|RMz2Isu(d#-) zeP`QQ$hq9OU(o>-alg}aXr1JI4NM!6_n0STq1|7)zrAfDxT|CKbDRZ8;iy?oAcS+vjqzG2a!FqcJVpg|IWgT=jYeu}Io(rF zC!gL&;Bu5VP%#;IC!%P%&nN6I8s3l5OL=Xy&ef+rjO%NAIVk~0-J5`Cm8{NB2W`9! zy9qFIB7s#J%uo4e4UtOskNf)|uLp&?!N+6I)l-?JxF9C4-FQ(*Aa1B|nw!OZ70&cD zx69#T^`pJ}CgSmZ^D=k6nflH*_a-~fM^WI6+cZoZ@6E>%qx)^%x|`;ulSRp_Q;F`6 zYWBTT>dUjP-CNse6{JSMlht`ScPmealRA(WsG~gH)jaNK0B?7+5~1MfFjQo07Pyny zeY|wr`@8zwRX+;v!}vy(NKqKK+d`qW#{!}HE9?YQOoP`d~^0~Ia!0A z<%#}**Vzz~5YN>ErxnmmrvH6*dZyO>wf&aSI0d5eHZEj{*W_im!j?((cF;klgW7sr zI1cG)QN~DWbd&3GC>*kj?zr)CiV6E+lKUf*v+mCH@PS%=#LLvjp%`M;A;~~D3J_Uz zOHDRN?7O4J3$&`HJ?iGV{qXr|oQK^E#Wc<)PT-C6Vx!mjK11=vSh~pW$$H!o={;{K z!c!vc;?2Zx@zO~S4p6h2?%Tq*JHy8t?WaB%pjZF?IPFChnEYlvP8uC(RpaGxkJao6 z${C{!!F;+r3p2bGh+8OUa`7QvR!?;8WgjMieGebrd#OL<#3;a9M2++~Z~hsS9d7Nf z6dRBjT;+Axv%09;8UMcDX!(o}lsJec(iez_!Yu9eN{d_&Vj6n|Xh-+Ar82ppPMm=c%W^R$v03>xn+Xy8d` zeS2FC@@ZX|YHVw6;#b#-3}iAIX=4#p%GeaV3^?vubbdO^*BOnR9zNuS*sG^T7k6H` zwT77CH5kb2x%3VUs2bggv@zs-Iu zMa|2BYZSvGr7*|^#Ap38*ZB0OPQ#FR+z^iX9LoN&^vmq-02z2;p^*jAVj`Hi^`O)a z9YWmTZ!|wtPTKCH`zhK)s7=Tysa3yLPq!0Cj`d7-Ms|nD(3q!e?Ds+SkGUJwu(=mW zr>V+#mAR0mjC{U_q>dgwTI*pH;$NKdZJN-KeLHkOwyP2m8`{;!XvG-WME{}8Tv)c; zG0M2O)>R{6911ie#!!3zo$M@c?Yq;yM7-?Q>;&r|CL*WTcK^qQMy4|jQ`yo<;#mmo zsLk22X*~J2G^!;vR5K%)-DLyNO?hNz+Zo>*F7bhY%?s~)wms%;#Q=`YD`6&3fj{B^{AELH)2-(!zpxQXD1jexr^JG*=>uRx;Na1g9>467bEA7E<@~{wpQ(LPi$%%*vPE8zeogfrMR?{yF-b}AlSWF zuo2(q*TCF>D>3gC^={lIYS-0z<>nv;e)(d<9Vd4aF&bn>gP=8NS_nfwK8%)=PZ-~B z)V48@$nbVa{oSqDC)sqz&v2fpsA&a7u*1FK$&I-h2}aXGcfK5j`bIakA2wKXx7B$W z$@_l-T|e3UfH$i=($7zup9&dW5vwZXims8gHx_dHY*y$lb#0Fsq)|nLJYehAw~d&R znXiv_nVS#8!3tj8Y$81^R`p#FRM@#1IvvT^{$${WfK5fxeD@NeE-EPXhHvxiyc#iX zGl2s7vJ5drd$u||m(+#hO~uHx=IxbX%&CHLiChU;>Q1obF2+xd)S@!a*`~SvflHF? z)={{{EGE{gm#pMc;RCggIa9(Us2>UypaUC@`WCze<$Xsn1rN6;*uP9y7k>wr%Byf+ z>mC~H-IDwR{!%;8QkY?^w1B=l&c-!2-xY`CasAqkkCTU2Z&R!7uA$jhPQ|pXqX%U| z+o<5Dqj4Xp(xK#b`OBLs=%^=Tg&$aPLK%!06R;Fn(-N;=W;WQl3d(Z-K>jwpB|~Sw zlA850aK4XZ#)A)TiYdZiri^=dRsX_)ZcZB~PeF!oO1bvoa9(h;TVo*&Tk=BrI_I<7 z%{Y*LguD(FO zm2y7*$5IV(>d&E{-!yaYGwab(Q=M$p)#v-}Vk`^RsQ2sU=P=#p$Aq%a2!b9-`VDOa8vX=uc-aY5Hz@gmTJ>B7zqyOvW;+kW&i-d_u?fz2OI&kSlU70 z=s7sqjMO3gYVubO(Biis`-|dt_K&-Fd;4?SV5uk^UbHOh8K$HBU3m@?Q2(#&iCnVg z&aabI0B_~^z&!|v9-#I7DU2pdp0Cd;9Hcq_L#Q_At#$(Z$(hXuGk{K_-7o~eTX6E-Q;zeq ze6J;p(KsNycYAYHo?C3u=6`;?`=*}qi{%PW{4*6mk|gaD^lPc3wn0C5Mn0wYL=?|& zTVSo_|8=+lfx-04DHTn!zhMASW4}j~P2uMfFs7%>SuJLyuONR_O(CIQWXP*U<9T|oXp5`8jeUmP8Y>o7SA z>h>_}^Y``>_X-j-H60a$a)!EGG>tXw8>$G54`-(6&7B=4SeZKak_yb$OLY#1TqG1F zB$VY9E#(D88j8qA7Z#8aW15}he*({AJ9HGSpE5#_A0T@bmNOFf5=|TjGvqm|h}_pg z^aiOuYLq&7oUbh%tTSROd)kOU%Ft3U?Wdf}$NLhV_5E0;W$$ zWv4q@ZcpCx{w{|SfhfKhpu>aQ%nf!K8K6_OKcB|H(krvO_Kg8GnuN{;Oe;y?K z%R%|7dD<@%2tvX{DRqBJ0Rci&p?OyFQl8X|4s3t_ z&PmC}!pN@A$ihI)PRGDV&BS2HM6GAY$WG0y$Hrm2&-m=5WcZgukxUFs>|poi zzc?xXgO-0Diu^K|bagb5PxygGc-6Rmt`MF@#%MN z=V{ef`8v|=Bn;cdg3>wsntr2tL1BZFg+Nn*GwmJ!G~3-{-%?1br{=7Hgta`g*o%vH zoi1qeqSTy9obC5XNjsZ(GLP+UkT^LL9Wn*ON$XMSPKCPMG__-Z7~o8Yb_3!i-xlRC zn_VdGV|=-e{WM?Trf942B2bD@c=7lcexC0Fi*(4uZ0@NNwC><`BY)IY3NgGdp_+4# zX~NoKbCkX3UQxBqB|nH`7H?mAV~R|`{M0lw+DVGXt37M;y*(NE-k;a{ny=nR>t=k4 z``xBuL{m!=kaJfd}J;ogF?9B9;=fOB{^8IsH(x&7uYg*594xeI7yb zIn|T@Xr}+A!|iW=^gsQ}e-8s;{cp`wpp`Q0Pc!|dMg3%@+0RZ<2#x4Nj!wk0lOX@Z zI);O{-_A2Wv3e+BY05CnF?dE6X4Tvb^$@g`0;K^T@$Dyax^&bB^si@sGt=i_K}k!l z!82^@U!Wl1@8bVXtl+98oswv@7qa~5-pffs#GV#5>@xFqtN%3MU9Cq4O4Q+u*}LQ)0bn_8GBP< za!}9tDu$!ecAoE-g!BD7rg;*+pYz*e1U zbwv82R?eV=`}}8~w2Ys)%L0b1*QW}cFl+(2pP^hj;{BFs>!r|`DgoseaudU883YCzqMNT&^%#j2kO>wh&Hro&ZV5DI5>gsVemAoE0On>m|I^4J)^&aoH31wWnhE55co)>l}JmL%8 zPPw?8^NARjoKRoMPmdjLSiJE1X2W%_p7cq%ZY- zuTSwprq-tBHYH|pNS-H=g>e4yq;9qk)=7_@MIbEdV=viHYU;2iVQzRTDdm37Ldvst zY|vDhd1W1a!-3pUW?|`QxST|=ZlQwzum%rz$+YceZT3!t!jm<1-O8c1h09rGW-g~V z7>N;+--1e+*RNQnSEc8 zC#RK8)@7clPiA5|g$huQd%Vz5(@Q^rSB&j@SH`K6FxnuAd$CVEZ{dG8p{=2xQj{jT z-GMZHft&i$!|Q;3e926~tXC(j;?;d|D#mdR9%E)3cT`5~ zj~1&V2pz6YH*+srMP47?ZLG9MS^LMnoD!cD*@|r3=AFfXj@VXNR!M|WhsA_UlBR-a z-mnJfabFrWUU8lr!4%{tNx^qW*3gN*cT0{|+Q2I?BBB{s_u2r3#kzhv>uSy6>q4lOM#V-|vlm&QjyiMBpyBdnLYz+1sF^l@HPr-yQBj3v-bfBd0+XFU*0~Ae>Zy z>P;~eN>HLuZ4dCS$WgFj9u)XU?S~y78^bNd>=*PSvci5st#%7wiBpQ6@xsP&t`8zo zIBC(zF|}Qf)M(bj$e?uhh(nh!Y{L+;wtJ_fWL$%L=D-iT@=ZW^8m5HTj&S@z52di( z1Z>p-u*R;1o70%WH3wy>Oxs#95>L^G7{r_GziN-!>Esy&l1rV&f(JF0BxWmf!NV5egpC!;I7H(64ujXz-_lY%k-SuJtk&L^RSOId!5W z4X!K0&9>r39);{O#b{MQQw69pT6nhTE-9?69I+FMGUee6R@E*7V&Wi<=z`4xOVy7& z0uiI4n=c%buUjd8punohRmkq~Em3A5;)Ynw^j0bB*Ut4Df}M#C<@6iT_7`?NT}oeD z3b50P2|~z2xj51a@qN2{<51uA`gc=xHCA{R;`?;h7SmF#=ihsJ5~%Y(hT9J>_%B}g z!*B@m*rTcQzlex-W{Hi=&H{WmwlJXVS+PA^6%w*@rTBb`M5{+Bx?%eWB_5}}^#m1M zh@aMdTTD|WSVnPPdLb$-0`c0-DcBE}^w7EtrP5WV-;M%fTF{Eo`z@!KS+M~7g{<9+ z04cX--$MMs?%Q>3pYXP6v9%3?*l0L9{r)k}R|2@|A2A~fC)JRQkGs5h_Jdz#2`Q6Q zrOF8ZTz6jZ!g2vwt{hhTR4+p>d12iEZ&^k-oaNPe&P=f`gCGpq3c%US z#2(kH=NJwj83c|X+cN2SqlH^f6zv1XT>JlP==Hx{B|R_Z8U7uz z;*Wng`RiP~!oO};z@b+g>`hO=38yMc8~QY~bZm=v;mpe~G7G!Zgz#mhFjzf00w%~& zJ*0~%9{f`Y(F6UPqj1>?6$G@2HDb{W(J+!)#C~GqBhaMsqWS_b@u*}>l0W1H1nG#0 z{NAzdE~Vg-?iyr%v|)LC=+Zwtygs~sIzFmi_Tr3ELUYhYk5CYi4%2Ya%A%0Nfls;F zW=_w@j80AKRZrQE5>BK2$k?P+e_<0MK{KQtP) ztNd`i8QQ=P&H?g1+!!)M*h+OA=-k)VvRzo( ztVet3ESA=ItZjbURi<4yiP84lU|d#-ZhFK@p~MX?E^fA-oL;^g313cjz4br#8Xxj$ zZ+>cXznk*Rcrt!E`#4Rzz2JISc&(jmUCdF&WOwy#yrI6@CQ<#dko(HUys zxn$usW%*r5btw*uQZLO!tQKOO99>%Aa{iq6Qjt4M2PSc3f zcuzZNL$_ALr^8qGd;4(~JzOJlj^}GY58$%uvB&O2ArQ2L^f(<}{d&3XJYe}q?c{d2 ziLvy|e}Pr2>5i1vZ~5)*PVNw8HQF{&Qlh@4+@q4R9j#K}SBJM30Sx`Kza#+4Rnc4zH6hjs0vn(Z{_ z0D*2#YECY@JS7pO_rrWrkFO>)+}r!=WC8@!PqSBGzo~N4gF~!y%t%~niWX4m8;@^Un-62NyO$g~M2ZV)FOKRTk5&iq?i7xGtb*d2*K?Tb z&6*xNFU`|x&+3po?=o1iFMUIJJudABch4Um5~>?+HnxZFFF(vKqG5kQ(;+J(v>WpC zWxKX4nAWKK$yVzGRECR&+ikf@c;6wQjlp;OmU7TEvN++D{~3YeI!esru?F@8E8?zT=|%Dp}sCki(fM zgj=M+@U+cNbwFl|Auhs?QgsM#{)`y3m)uW?)fHi;IrugXc!pO0bl%IjZ)SFtgZZ>` z05d#reVu1c%2W%=j_|bYOWEY*?F3Euwxt39Q1Mze?iq0!GSLtGg5CQ?vQh;CyfGy1MQx zImm^%RQd=txC#1k$)cMm%{c`k-v4l}-gQ4Q2kKRJaT5H>j^@H?n@vwZ23+Hn0?*=wxDSZ=F_j>`G zI?r2&K8rC84jMZlG0P{N3}q<|*N>AmLoh+q$X_It*~e!z+0JWgxLq~fl%7u6huvyj z4)%|h9cFL6IkD7#^d#5YWtZW$+)r&MKip(%wV+NeS_<6G*gtoOOD^3`J>UdWZR$v> zHJgM+QK9w-%@jw*a5JPE2@*z z%%q*^iITS!yfzuqi-?nx)U@l59FM{SbuTKdzTnNAYZ zlwP^%y{R%_j(Q+Bd?K$Zn_;|#Vn&>ivV|;o@z8U7yVsQWG@v#7>5IeK_JP*vxprRL zhJ)d$Y;$4kn$&RgKvPJetE%6h=FjEiOXTmyZ*Di< zt!*mFH?o|^%m_Q@`|nP8?e*JN@^DcDjIX^s>(%Mqt_n0U$ka%0m4K8`I0a%YlzexN z@`uj~Zy6i(9HW)vdpdlAQalW)!g;9Ryo}BBAdT!FVJq6`eD1nBdO9SUZXn;=4;3u3 zQ|HcXj_x9JvqiCvacfpG$f&o_xu^E@l()ZXrc}F^VIe(^Z;OzQ1L5W*cPCfeVn3T% zQ8IYW%b(+v|9o07>xvr`c%9)rZZ|JTP%&YusX|v>vJUEkH{xt;4hi0+E}Q&Sq@t;; zxzv$sc5N}WbhoCx3O5(>N4?&SkPLAP94<48)&`iY?}_B8bw6f<-pLy&;(=h z_-tJ_IbRD`7BoE%NcH>JsTjjk!GdT36cp#7jXwFb+3SfVG+1(5ar$xU!;g~u{fzRl zenEEK<^WF>b^6$|nCj=aaSynztNwv4NIc-8tatgg;_=rp3^EPtnR9zmkDnGt378AI zJ2&f0)}@xWCZhQk5d}@mdqc8da+r}wKEo7XoY8+@Iqx0+T#BD03`6LHAY_IeAsz2$YIBw*##|gZh#v_*NR-e9^vT*r ztNkJD8q?pbW9y*XX~^o`ptWEJrj#t1*nNmXGpXHLoc7v7(G2T{&anzoE&trq{V-)a zdCUGO_vj{!*a^IRmh#q;G*_Y4##5UmEqMwGAR$~*>EJP4kX=Cy~?Xy0yK~WN^ z^QuBG;&k1{f6IVxpq|HSzvsn+2CNtAAc%$3SuzdCH<_n1*56X{^REvBH8ePRf5|n_ zfc(0V;V)$XqT&AV^x+TBa-l7E|6L$|yaVQ+g|h7KFQ5P08;8YH2`CU2n!;bpvUPv5 zTZ9B?;vCWZC4>WIQ8BPsN^rQ*p@4zG2}^MPl>YnOiquBY3ec!{Ud#i+B2`rMyMnxJ z&UrBn)Aau=&Xa~S*`?nDG)^_@VE=yqPyfC>Dg#UX*EggkEk9N4w`V1wzvp8Gx*^4dm~aWvz9V6hNr8o0 zv+$4U+K(TF>4}D)IkH>^e2&ENpI$#=3D%ML#9wR&kKUQ^gNzp{r}YDAMlkNouB||G z<15(r$F9tMNAr5~6eF}Q4{owd!(6z}&+f~Mt|`}tm79S_74e>YBRDP$H-qlar}48; zo z2!^}!j45{8+eE+jr**Xo)OSuUk|=bkrgH97O0@QbL2g4mcD*X|ynerP;7`5Sn4Y8G zRlRV3^kg)=?Q~SUT>WKykpirG(Z${@^{F{I`M!FRq=pD_9;ZNcoCU>R*h1yxW=plW zXijj3greiFxb`+{?_Q(+i{RQy60=z%clFIOxd%IiN0H>?s`F++{NtmKzPS>E@tt3Y z#@Vza$5Y;@;j)I!8a-~qU7ox9sz+FSv%ABSr6YHJ<@Ihrh~{ZSgS+#T(h#p_@X9U)a|y1fQiXH-HMixs z^kB;K8VLnL1J(Uwicz;h;(Il$h5Jj?7I HSa@s(iz*cG1T#yA!l-F53XXs&Z_zY#TOnI;i(Azs#&CP?#*l zO?H*XKfMX}9v((hJs;9iE_Ga=v=^gp*kX>8FNikbHs$XiFclY{^lNEyyBw|PnWBW| zLgToi9(J{}yS3F$BWW?eACv7tO@17OQ)kVwiLF_1Kb+$ILB&yj-Z0$w%-evX7R|CX zfSO!HMtOTx1ovzwF-sJ~`MmR3WA19W4vw(w`cSF(MU+X?d05iu2i0u{^ZoGr=`DKC zQ}LSq^35f38BvYq{aVLn$Py?oNz2*!dCL6VQD@DR8V6_f1|}wVQ{F?H=<^_Ig!^;G zeGM@J*#Ay{fp9WK+9lFvrt`5|&7)eqzG!-3R!5b++GPr4mV^!?Ozo8a02 z0xtG_S$UoD0?+Q1u9Bzo5dA23Z2L<^A};&QrVufT$@8Wn?i+<$o%@6oINZm8V-qfW z1^h+V(ZlsiIWWPu&-d-s4cSqWp+ahpHk$;<+SY9gKi*bZ%Zle!Z)M9FTMHWJS5+`h z$`w=Rs)b@{!i!-Xh+i@t7I9af^j1H~<#Tg}8un)2$-ortJ`Dk$u5460J~O0qGmEaZDq}64m)m79C;5jzo#p`TNJgm?M7(dgxfPWce?zD`5is_F|u^s9h zo@otSRU>9!7@3$rFz@ly2Uw#@4ES>Twd{?8KV5pqA9#{<#-FZYw}@l696fTg<4py^ zwK(j<)Fn+nTzDzbbGbYLE^v#@9ycM%^jvDcY_~ezmJx7`E|}iL-4_?ue|5qKhHyV? z8EVwpoZ0i!bGdQ8HhGxD#-%DuHs}TvxH-sGmY)zzaZ;Wd7#+~!7+qmYUL-rU@%Jgw zGpU97K4Bg!#TG&o7YY$W4jPkt7!%JiD}W`C%)yC6QDK{#l|b4_*qGIj`sSA5;_>B!Q!3 zsu`waVjhMXO6avyvN&?Ep4RSyfE$Dx@{3>9cBs4B@D%W|-Un3drqaG2P*GEMRyvM% z;fmGMk04<)7#Mw}uec3x0>2lqcTdK}aCVcjBbCL#c18&b zisBq=U}9#8#Z=}vuB0ANuPrT&>jaE-+=g|XXQX9irSXED?>4tBElv#ulmtn}`qG%= z^L%B|*1Kvy7%?!G2f-#vM!nK*b6TzH6yrN%*@3#muN$?Sxc zV7qfv0d@~i)bCI}_aNv{Ue(dAftE#DK3xzpCuWbu9@InV`|%bEwecbN?u6W6%A8Hal_skT9&maN;E%iR>iq}?jwN3ca&m)y%{qy~OsV4~B;f)N5^T^AF>xZQ?O@W<= z$AL??C6i#Bes_oRS&b3HG~SUR@`UyZUWg4wl5S0?96_rT^;)iW7p?q|nkSg0`8RED zL{2d&c`#Z=-J9Y~E?A zpt%wqB}#L6WvQ2i?sjg(TT~i9@Yp>FqPf>>pXN&PN!K(}2odd>$0`T1RTrS(Cnu_* z2HSKMyXg$lSj|2;0QA$LhnpyrqH~uWENDA!sMLilMXXq`_WU)udge!0$+!_Dt@&MJ zw}R~jee`)9d|0g?PLS-Mc(T_cKvxYEci9>z{JAW0ct#h+96xdKp8S$TzTz9W8<94|>!@Kt< za<294Ez{)`*w9P={^~jX1m~mwnx?(b0SBEWvfO~hBA$BIeV7cz51KmWMXnJ7i z?QYUxS#OT{?dZXZ`Tbe0ngW{$c_V&D;_L471@z6jBB~!1j!W$)OKG7bcDBkwqsbi9 zSVtHQ%9zg3ygkWXyj*DB3-YK-juYk_CfYRmjTfB7qEMEq5!(RkddM1QZyy??GVGAs zebO!4V~AjpiDPoi3JYXa4y=au*e=X2845NmVBR@u1fDJ2C+}ErFmNB>^viCE)Q0Ah zIZCh;Q8ZpdyL-1maj&m3Q9*Q=ve4y1wKQ6XM_n@ZVS3)K2DZ(0bTiL&H$(ZH2NCQt z&Q`frJ=5-&*YBvQ+p8_kcxx|?)$foKyKuR&quN1llzYa7)vt5e&2}oqL7iAEDW!%? z!W7|RL8k{U!jqXTp$lhGPPiT4WftIRD`~Jy=SY@f4ZiIhuH4fHH*xo|303kI^)P# zz!H;E{Kv-hHJo0**blfv`^YuFT9)1N#UC^> z+vK+NibLaf3Q8z&)ja}bDR@u?7+1=hIP4w9upD`7V+u5^dS(VTGQI5s4&OK>9wIW3 zz(jNEZ^pxHU3X`8ODwGl!xpkuy`qKXb;s*01X%D?VlW|E_-%WtyvIApRvq$PzI&^0 zfFQHHH!f@qek3og%TUtZ-ux;dne~`+v3oqc4PN2OYa8SKaK?AI5CE{wm|g9zy0y^I zsjROipVLODI>vw-<&GUL2^z9qt!&T3;~3O0ucGI^QOvO~QL~ge7K}%z^4a!LWWMq) zF%hq}9J~`NE-N)K5kHJ+xf;^JN42%Lr#4ofa+S$G51d?FJTz_~=!Xme9rX&%jXLD2 za5h*um@@kdv|1TH%*>+#`SyDIKg)-|vAE~|w#@J^KL7ed%l%`@-e16}gQu#4`LN;(*l+B+**5&Uu%AyHD3rN=k?>Exus#(0 znHC3UmH*BEKgsy{tg)>q`PbBI9sl>&{1)_^qW&!~<3PT;evrm2?AxZVGdZ4HkUF+t zO8RZ+k$Lrh7;1JZT&q7|D5R7Zo8Nd`&W-}AIs&K$U_&;}cmv`w6I3x0P$_pcQ!ocU z{B8kh&?r!)dB$OvJ>HdQ8WS9<^BIzd-y*hY-M*Tq-kt-d#vs64}cmR zk#ro3x0}6>8$NYLRj~ZunM@gUpq&;LkNPBn*&Ep(_)UUn-(=7N^m4}uW!Ff1z4WL6 zEYP-zonXYVc~!CSM)?`USv!^S1_D7D)yIcRhg`MZ>1TrMpw>0li^n0u+xJV&i`S3k zaaj~GRQ+!jG@~)Gb$2j>nyoaz!-JeMx>A|rVfghj>SjzP&ILBxoy4`QDKnH)PiZAe zLZ&thIj%5Va8AmqgnW>wqAkbW20irmI+#?erbR0P2X~9|v_Y#u0xBKxQuP^<*=jJ8 z?%sRoC+UeN1FqPl;blex`!%*<4N$aViX3%TZ{sd;jx#~dJj*%zO_GsnRT`*U!IVnC zg}?6!f43+asg0fW(e<=(0xOOa}@pn)WcaMy?_D8`4Da#9y^8WiBuvTk2N|j(Mru^sTuP=YT}-t znlHyiMX>GMB85YiMVn1pX;DU7BG<9!H%o28*!ch~)9VCW)GSP9(bqRzRzYuqdt+x? z9bPld;b`I0DocKN9ez&u%Anits6ZUr^n7}z&YK<5o8CW!?QBBR*d8IIFpCABbH*W%nBU96adP^M&I@kR! z=MB0C->}?CS;3Y}56*DL7a3QdHS`laJAFB&hK6r_U5nvnyL+^B15GNJBa-`C>xxR= z0MUvqKR;j1&_5gDtFWyTN_e&vc4iDs3{(O(gmu-; z8ln%DyJ_9C91=te#GcIaM~~vPup^MlvilOzruwS)^|KnN^F@c)V|KD4)in*Sxx-QO z^V6c6(Ul%M5zCjG=e4Y*SZ?o4?k^omHT$LAV2Wy@nrbistrx4&oQ5tI$P0L$f|^cs z+Kr<{zD9c)(P?;9RQpTH={B~;>WAzN2?_J_4GF4MbIpLO=Cp-{q(za?2keB@?XU=~ z_l_YKuizn&7(w7maDB-hQtiu=9Q*ONP`K%3sAxnR`TwOi>A z=i~d}exboF8jXda(2i4V#*-&Gv73WGJQeVPj17ZNyJMnt1&4dyTlqPWD<_zf>^ykn zxjx(AHP|6hu!j4ep(`{4=ZjYsqiArKjNJFnS#~XLSdp~!*H52FX;VVD4sJgYyRr`5 zZKd4vmeRqppqjfdM}}Yxc`h#>?E8lk19yQGoVlrGGqzn_&>0lfv4awz>mM@>8zi{31z)R7RuHH@sxl;a z-Eh-x?GSASxT~Q&5kb?cy(%2v7ta*)IDboX{5df+<49!a@YwIx(yD@C~1Kc9NlzB&|lZ!)PcZhC0F);L?I$r=Z^wKp-zWb}Ll75%BwZDTAFbXZEJb zl}Zb|l31YdbndVUzyj+y%OkBNB21QQ9V)%kV94ZM<0s;PS76S4b8Jdm%0>v=(>fQ* zf(GTPUAsGijNobMF@r@7vK)+BxsCXyj9D6nKHaNQAZV?<%hQlfXfr=zo2Bw$ zFcY*qB;T99t00?M3;TMNjaa(WY7oMLvk5DRiw#rGaBR9$lVnkMkrEIRZ>w z=2qQ4vS+#2TM2;G(83chAbrizmQ6Mg!J6z}ok5~wvXqkqyq9TzXSD%OaiuV8xp~9Q zPxH#uhtx_gB?21?nTDwZ0oSo4GfYFR;(Mcv7RniqpxHJr5?NRgXzIo;`mVj6f%Gg= zjG1))NDxU?d8$XaEW;$}czz)Qd#acJYgWWFUJ7h0OnQ)sD={JI<;>6EBS%qw5W-Yn zsR)>za6TyRquuYL8AZ;y-bRoe^?MoR$OTc3Oho^8$9JVMTmIG+b2PX&)RG4VAn8l@-BYSIk#3ZQ-A%lKb(k}g@V6;LV*&k zAA*P^kEqN->$w9FHr1QC01=K5EYFjj&3DNeu)U#|&$_}#Lp%{vv=OS|FrBjG+JRX! zsZPCF4j-VzSL2i&$jk!XXG<*vv1H>)PxczYAcv;mdU3BM@*Q^PQlgK%tL(rhZ%MH9 z&61-mSMtk`fPY_}nXJp=jebYJf{=XV5KpH@&qGzA{^Sj7V6}kMn_xiJFpT#t#{qZT?i_XG`yM7 zx&}W4pzm?Ipa)N1^VX8XAGL2e6exKHAasXJgx~nkJc0PQJ2P;FJ(~vNQmyertP2_m zKj@%6U(TXEySzTdf>6YL-EsS-U7e2he0q!z&oxN>d+<;J_AM_(c+5B$h-*^vq-&KRE(RQns9M*!PryNVRKAAd*Hh zGkU|Yo1WnSl7odP`{;cguZ&M@ z4n%p4k_FRWZ{}W1kZ58usDkGYVcScGEN1P>Nt&5D@X^7kA(^*vmqaX?(fHb zYIgM;PB^w#pl7tZEUV3n$&sa&%+W}eXyQn6a%8(ayLhFMoFBn_Hv_WXUvi4y;(}GA zu!0Si8{@--r5xRf|fc;{z{HG07|G(B? z{%1cf}rGFQiXJry&(Fl{?ySOeT-R$KX_T zyWA-YMBRZ`h;|sY8(i2L#H>RE-z&vMYRqVCk0Tij5VIbDQY&+m2B1{mWeZ61LWu{ z4n5UA+3+6i3*JnSnTg218ttDpjl|ty@mP)5U`0k_-|KaEMPrXOccd!Qn>bE5!RgkJ z$1{@5OE6QMxmyXZaIk+wW7T8Y$-4{BC~>Yp;xHeawZ1^DZ#)Ym(5>@PaybY1yz^ah zxi~_wse8H{BEAr#=(d}BoS@MU>Fvn7uwmw;L5v_%-VK?T7)XSgV7Q*2=2L`M$4VglE~$R+L{{+~R!4V1#a8bvoqI)F{cNubyP+q_mjOqV z{)=y+@mUQews_|WUW1#xw%6X@ay!4nvOBHhWdT-!8^&}G1EVHr1lLwY8%-m=w=M@B zlVNmWt3drivGiiuG}#Vf@&>{#mw$j=|5qjZ+YyE@TfY9YGx7fnyP|7wP757cVbFpP z9^V|~<>^4=ik)}pVgb!CHO{5u>#)bLK{6SZA3ftqt58PgMumf>GI!OAbg<)%_80;Y z_A{qm`KVxec$&Z4DuxuWz!YRs+cj3dq{KE2;*u@x9;ihdlr(w>t1v@wG`lJ z`gp2=j^TO7=(vgraXUN84r~=dy+I4Mb$WeysWyYY1*DgTtW-NTzqlRw`VQ~L_>v;v zQmoa}Bh+8JsaQZ$i5s)QgcJvcy0iESocN6tTI6}dK|REGV~e*PYOzo^VMxUZ@Iutv zI!TR3mLn(F@4VG7VetKzP(PIsYSWb3&G2=eK&1GfEgSfy%CB!^bf-dCL1SX|e}}L# z?!L09s7hSkwe}_=As*SeDx;Hl^{gaOm~yMh(%3&kx*A%Me12OSx--FWAb^x1OSx@% z8~sk?6o8(gjVa*a=nSJ+4Vt% z!+a@4EHec~w0JUJ@3cXwG%$Mv&HGJ;4G)hk9_GgK-9>Oi9c{vHoduKTS!gEn3*7;D z3x!s3NVHiiN7Ap*sYL$jd)59K{D2~U?7fVo@36e@nMcaxU2E&s6Q}zvlZ+(9O|e0i zpon}#-YoFcqFwO?KS(ixnY18mPP^O`*Pol6gTy|}wqeimSimf#e`~pWJiMOJA&M$3 z$W50$ykt_eL2@jRk{RS5#8zzrR0r#MewVQ~D?($jzcBE@kh`L85rgVQFNKD!;=_2W zHfXV6e$~v_U-<}LLeZ)@|f7+hyuk;?Mi z0taaZi@x%0m9b1iH`m-IRW1wX)yAf2iNz&OaMD~{5~Ive&U^R!C>5s6K0A^UyU}lD z5C_Jy4uE&J>Br_pB975}$j=V?*GWazt;zN7$nbbQur7|opP6)j1SK`);4CCAy$MvN zzed1o*+01DBf>nJ1oZ)$y*22aHFG)ny&mqj*i$kC5v<&IGebN|a@(%;%<4qE9(HE@ zxW&U^9PO7Xvr=4bu0TQVVkmcrd8lBMA3nKtKe^oXj#O1+bcUyKGLc}Ue`jc%EZ0}9 zf4F*@BG&vV757x?(F}}eNluh#b zeAkM;Ur9YMOz$vDfMT7JvECq|Z>Ma&NdxLIt6wc^-G)FbVz9FB3fk1MTaN9(!-AGZ z56ZMAVL+h!NFbEBTi0f`Xj33!Qbj;jZA%8OA7j^&9bqIMt_I2AI&GG)v!p8)Q8W4Z z5H+TI$P3Sp`g;(ADc)BEHPY{I;nqB`F2}02_mZ7UjB$YYsx4(fY_9!sh=6abam?1D zl(x$Kf&v&ow z)!3g9aIp~~D)t^eV)doGvwdC52PKF=e(6wRJ-Syl>m?M64Dy@~iJkSG5)$&{OTp{v z?hQ{7G`~JvlGaPPQ6W=`+z}{?~M?`Dm&rYVVOH#ol9I>33u)@r_$Z0#|CP zD7GVj$U;WWk` z8U7^~#ryMm=U}*hMjRx zl94NI1h*-fUCRw;lxOzNv(-Y1EQ|D<<`kkiEfpVfV;60iiiwLl^7&1ASO$BrpnbGQ zixfyl55(AsfFt?|uZy=zhzs_Hgy;{_S?Q02r;aY$=91pXts`0C4(rP_97J=kf+<2X zbQ&4^93G|05d(!_&)AX|ud7bp#4q0G(#vcjt$Y@t7ICwH_5ZBkBS$L?L|pOk37ng_D{ZPx(?SJY4E-3>oLhYU8)E1GZszJv&Uj;7!#)bSacfiS60h-V7!7w}+qwn6nms#^N+m2xk*zmsE#&kt8XS zDK1>DXSWx9+vC6g@EXpU8>sqBH2@~M2ijZmM$lN^!ROVS_b5z{67B%MB>h%DV^r)% zqU;48dxC8pEA*O4IX&9Z&A>r*qq1@w)jDszA0mcu>tDp}5RJ3?(Yg=Gf7CI{gSW0f zdva8wu1Pk1dgzCHMt!kE{ANUZ84&+!gn2pA{C8TBKk1MEfn4LS5(4dKHeSq9FOL2f z2~@vuBm7Qlq5IXuMq9_+)YQ<1%Fs;D#8BVh&x61JWij9%Oddy>T}L7o^*>Xi(QvfGm9eJmDo>NWfIQ1Hk~M>-C#M$V?!r_5fNhseVb0gDC#>wNH~5J&R_bp8 zZmw+`vpyL|LxB@oxtebc-s~D~^n4D_1S>31FPhJKGBx&r0yj${<_;Gec1o6IVoRTW zPr-I#1?8Nck36Icp1pg5&aeWc@m(aX0`*2>pJetbbFfcO$hP~Oy{?ftCys~$X?nr< zfQa0E0!7fa$}V1E7c_OaD#0$5wtBOgy6c^&QgFx>5X2hNWT$Ts36q|4=M>DNik`63 zr@1s&#oYbPf^?lR;12GoWFrOLihUcjOed?$)Fckk^su{jJ6xzsjiTzcC1%79{@5X_ zF~Jp+EqhbP6y-h`oQcuOlk1q7%JwO99OS|{4aRU-Ci-g{p0sd6YEeR+LJ@p4F=EyT zWKWL~Nl4lpmrl~*Sr&@h*z-|+;`mFyCp|xjVt})K?ZL4{${cm_+zoa#M+SczOITMv z;nNA-Ykn@gkHb9-pc;^!Pq5d1PyRsvVCuyUxD^hMYmgHdpn=yz?dWVLN1tv!neSJ9 zhFbM`)N1d&SaYFGq8@J>*Zfl#N`aeP`LP+6J%Fpr+kiFX<;dzkbo5VMpl7CIp`-hY zF8ox10^oFs%-^fTp^~`yJPS|-6w%mJ%Y^#wddmzU-|W2p9R3B)^25klFCLGdivJL? z@l6Z0uxRy$5GHlCdMdvpg5DAb*WgJQfEkwTi7CG) zw$CM>_CEBa45(j~Y)$iXUc8z1M|kU#u5p%M_&J||jtFVBdS%^M7U@R#fCSW6l+2rB zOlU@F{=1lv)4(Dj_x|Zry(n@Y(uno|6D|- z0RiglcxWsX-e$cFr7nmi0gGn%fuV1^iEPGsKGwc%+mC*I(n)NJ6>&kRGrHu+eVN^x zu&=Q7X7n=E-p>3!Td_&B@y(&B&ePc)q$X#o_X`p@+0O1mhu=CZ-G}nFLGk8P33JTg z`B%C_1#Wcq72=(huH~1>DOC` z$Q*nTp*xHk0J8>5oMLln*~PDHRtZZlkL5&1-85P?$x$~T?@HN4Y`hAq(paaoYYCCH zmb0NaLY*pMnwa9+v4><;w$6G_yr@r~5_Yt=4BSfvd4}{M>fn}P13{rsj8)o|26>;z z=Y1)u{YAIkv5W^F-zIx421T|)|`$__8_r4rAzv|xE zjT7@r@g;ih=ciD%-q>#C*gSnkddy81vmXp2sVBJ(5*tR9c+bKQ-|ODvK0}?79_2~~ zPvnH~NP&?R$Qqg3&ZRubyo2Q%H^HqQY|2Aj^EdGHBo5QRgC9&LPXl+$9%gWkPbcMb zTXCKdbXz+J(U<;Ccm)SJX!c#R#GKmvC=qVH<>$dxzcuzRoeONgUwL5v*m_x z_4SE;r0~#)l4JU@K)tQEoN(cURVyx=6yqNUBo!SKoli{TT||@F=PV_Z?L>TX3g4}j z1TB2mlb1|DS!9b(I@nQE_}-H{uKvhwv@qUunhHGS9{Mw7J_a?cm0=8q<`((j~IBUZSe(6uF6y@q;##{$@N1FR`W0@jxPknFG;S`S5{dLjn*LQy2L)~o zwoME+cw)wL7rKQ7dR8Gpa1G`59^)p$-^LStvAzci`07SN;GzHEA^UIE&`=sy$=If6?-K}Hz}r5q?i1Nx?_k&j zNOyxDue|28q4c^zobSX8fL&8O9PN(Wo!oSxQ_b{@`DKRvWh`*#iURv`PGUKbMDU;6 ze}=!01%7J9%K-^H|Iq#`wwb-O|JxMMNd$tCy8TCGvmfD8yTF9&oqa*SqX@#j7nzW; ziJv%Wu@@4J-Vh23*out^ienCu?1&?Awx+{ieJdYtOm(h$o3e&s#xR*hf5$MVg>v33 zNUX16A&^b%py*8-X7ws&;7t<5`RmlkS{(5>Oo8{c^g;y8cs5*c5TX!C^~u&MF(`7} zGwpx}6BjJu1Lr3%2R9+T&l|%o?sOX-L_e) z-%8VZr{fO&Ha{GmXnsibcRswS@sNfA<&SZM5Wz(<%*!i%o1&SYKFbk%y+IurzE_G- zzLvkIca67i+~l-qoFn?gUtEkA=6^}nv9athO2kx{>IXIzQn$*zq8O^Ksbo8Zshi?0 zhI8MUD4I3pU5v-+YfRP|P8{Njq^A`$)Zf@Ljy_DC{9(F=!&C8bVCURIrpP+z)N|5M zZjq3o1nSx;ekwLEcU+qf_SH6Isr~&OdBb8#@cA=!F2Zp=yCvE)iMZ&pO)nH)i7kD@4)pdfr_JeO?2KQ2NeufNl}D5Zu+Q zfRCqv(=(6dJ>AOl$o9VNKvp|-=a6OeOdCoDe6jj|Jsg6i$rjhjHRuW2PuVx)mc2xU z4meQx4O+z0qOU-;;&4F$iP-(dX+33obVIf9jV7MVCB@ehF3P7~{_|{QxZup|e&Ir; zhiOY+ml7~w!u^4CN6(+CAz_g(QNNw`&)htPIwkLGoHoRRwn2xpY!0`s(Pc~uU0bLK z*;6;9A=I=~l&o`YA!s7KgS?yg3_MX)J3AS3ziXf~5^p5+;q$3S{-hS~laNb?nqSHz zz=6&x)MBz|)EL;=HB^**E;bD>v09h?5U#WK=94{4g^7h(o+Pp>WUT zsgTi(E3z>pvdyC>ln}&);qTng96xUyV5_5~<~|Ardwe{ZpB(JEEV3?tg!mJayZ|D~ zh0w@9klDXsl79e1e_)aq3}o^5O8*BYQQwDBNNyZIIv@s6#?gFJ_|Riw{b&4;!1(#I zw;x_M@#6jkk^MQ!Ee4KqwKU9L9{@YS z5nxM){!6UC@nuq3SeaXVwX!k%YW=fi%LR^XU)+C*wv`nRNdSVIZN(3V=oFh&Qeo?}J3Pw|@9#H|KCBfQK%2{-f;nBBt8Xz$Aby&2j>tdR6z8+9^_O9J3?Pj%Q0L?3;$dYE?w{_=T0n9|r;v~<9EN`ij!YIja<7#_T{3)k`Fy5ePI_|5;=& z4eIWjyE+O*1hK2rkAKey9IH94qp{6tfQfHptEN}Jc z#QD6&bMZW;&HeRfm-&92fh-S8WWH`Q*!1-aWWsUMN&j>N(`ot@_-KNHs>~VUQXPu% zom#1n`$<1SQa@x3t93yXBL#`W)Xr;8=Nw~JtWSXuKrGJXmV%5 zo%7+NLY^NE*R6puZs58{s)#e>EsVReownRP>oWr;IR#4IR{@kVOPf+cEZKqyaMS>~ z01JHW;WFXQf~>w6t=vcqsxMXdG@RdAec3g*%%MEt^PT#{cR2t{r}p}{C$$^KXvv${fs5B&c3++s!xCP zcz^b8FJS9GtpxA}_Lqi&%I2%7#h>%1|Kss-FQf95Ar%%h9BIGE7yy^0dm{lxUAg1>x|HX8{e5^xW^IS^C+ z3qtrifc&Y~|1x`fsMKb*&h&y1UIP(Ax`FbX&KT0W5q?j?d7QORseNU*Q-@io$2+K8quwAQj!`|5|i0P(;24K^sdhoyBHS?)2+8J#&;X~;yKP9 zPU}z;?<4SUq`bxPRkNm*Ve|Q%^@DC=hzKz1PjNqeA}P)yUL&ciecOKI2$^!dl_zH+ zc_v8pp(S!RTneTT*ShEVTZ>1eY(@U?t8ys3UF&6<7@53D8S@XEzN%!Mfg{^o{&lQ9 zRi?Wm27oZ;GHZdpE zIuwLk!t-(7Qe zI`r6(%B_izD-cUaPr9zsY~ON#FE+0!uOop*srpeIyib|!H>Vg+pPg(z8S_YN5uM5jA=%7pX_#x-G=hGs zfp*vXb=73;OUf-3gyq2U}{!^J!e%bt7aT`Z1ch z2NX{kJaJnvUdR_S$=P_u+J?)f^XEW&I)UHPmV4Y<}xvyM^NhiQ|W1Rop|pjgzaKhkLu#PGy2u#1?=05n_Bowg-$V}my6|sOazUf7B zM+YRfdFcCnsE5(B`hU%RcRZH=_x~+XHc67Kh$wrLStOF38QEm-Y$;R{TC!!65z3Yo z%HDgIz4zvOUG6vTJH+Sre*f{idfd6)?(00yIj?ih^;+k3SaCm}x|T-v;76;ZDjHE~ z{2axA;B+pE(^SGz!t&?dx6jtZ!{0y2AyZhAl)3C3L5E@;>m`Ki>)>FmJVkt6Afn2n zy3VM<{=$`u)YRxUYNtX*x{ z!e-@o8JXKo;IhM+RfN*kIbhJ;s77jz@}kw5D!~8RbVXHH(?p&{>L%`97zMlQuwfJf zHjJuUJ%UD2uzUX3DEcRv^M8_|(=aiyw6HW+|0gd0&lD;CLJ>Ty-BRCD-^AErkKY4^ z6o-pV_BZM}`i9#2T1S(H&H{qHl+C_+mQ5f54lD$6ND$vY>e}s`95CWbym#NEo#Cuw zPcpwPC5xKmux8HP_%FBRKDOV8yQHZW>!zvuYNVZPb1g}n6y@nVeS1;MDbBL8mT&Be z&ZCyOL3+g)^5)^+42&NbKBLa9meci8th^=Ant!_AbGjZmw48x*P^EwI~)^ z(uytmf}zWegdq~nX(`p zM7%iP+&4&i7Nk0F$%=52dc>s9c(0l@x@I;hjUe7fRCH2n{+yN8rQS^<-)iu}S(W#^Sd`K&epC2TCwX9OQB z*ih`n6Kl9_*MZi^3g@-h-R zHv;Us;;3$?dl6Kn$j72p1G6d*mi6!73d=9@xn}br?}MZ{;{sb37faKqZuZk9%G3hZ zS1D`uLG4mfy4S_;=V#ZxT6;N#KB#pW{gPpJoKA+X@d^ojA}69FrjP!n9kDqw8l8;5 z9o)*shm^f16@FS2khW!IW7>tpQ`f&lEA-@gwJDr`!o@EvtK6qJ)IYuXcrxkXC&& zbVnWE6wY64e~T9M`1Fg8PK1GH77P_bNE!G8@h1AGzr+ql+6v6~YYA-rbcj+3AlYsKJmPt<9Xi3O?ghe{>`7YK8U>C-XL$i60s3L7L39+dtSblbTP0o)T0Tq2At6J-a>w}7uhI->_6?Q}*D zSts1-j1s!GPlmf_Tuz8OtKnj(aVA@|ehQD@je)4-;*E5n2>ChXS6BA<`7^9z!-h!< zO+9TR^n^YG zqL6lv5pd`PC}a0J*su;*5}@0)=550|WYn|=1Wv0Q!*BjNud=KEL*`a?MBYRF(|mAX z%tzf^Q%~PoTkDwi15Re{stI^=n5Li=reVka@R~Yg9uuDDija=}ooy_{Fg*XWS`RaU zxvLYfw{bv^V@D^YTZ4}}Z=HO-hx2DlK>~1?7#P{0`T(u!uiJk9ydB&)w0%1R^B#rq z{R8R489;-fz{vVP5ILauIvYK}20-LLEWQryT!CU!wf&$C0!GM>fQ0#fYLoBcaj-Dy zFfeg2(gGO*K<#x_24-4yc5OCV9X3ua9c^}PbtWCK_BymSIR{K_@*Q9Q=cMr6c0I(s zcd}GI$ma3AQ7hi^>&#h0OV?wO;I-9B%xQT$J72BxQ8D<$H-rrCsewD#Qkf=tXYn&v zzVy%QGSY21S?1&qVy4r{YO|s;1btX%*e*~0{s$_Sv{B$YWQ~qVuPNj@8(k)$h)*|4 zZM;!3pt^=wO=T8DGmTKW@#0S9#ek04;(**7!_%65qlGSsJvYqp^) z<2@n653l1|Q-*sIJyO!gdd1h0*jXt-F&I{1e>xAP7$_&kTBs}VrTu3%hb1Sn`{%5u z{Wwp#Z)8~idc8(tjZRKCtsYXAu|zhfHjdjWiz~l48K8ypBHLaW%XMzRjppOZ)5^2; zE6;Gse)qm`5xtF_*!I<)EIWuG*@~5^M|h-DEC-ADygF(T!b{A7%y9;+`$2bJD$ma| zRcDB_Jf72@P8?ekupCt@_4~|%5{_6h$1sbG-jlzE7~;<29s6ZG5r_WqfYDo)&v~Q> z_s0{ZKcBusLT2OS`>L6YJ-21$w9DLhGX_@S<_mlCrO(xI+DeVj#0;0UUbtbAv34ra zbJdnCE1YwwsMIzbOjFk|nAE!0%RHot>DStKYf5rfJiPV`P+Q(1+NHH9?xDmpv$^pT z)z`5dwT4R1OK+^zuAaW0$itnnfy`P|s~mnJB|rE6Wk5Y#udxWxgyO4hPHLGD@c^$7l?uEi2RS(mezziw$EJ z=r4Kt36ICWQeo`D`93+DqgfJJZLJ}cGh`#UsOqRahj71gf!PoLrDOV4J7H%o!dA7* zs1?0NfxiZ?-la7fyF5))sxznlHLT$JleT#G59Ov&5YMuC1 zagO)~HK9a_$;phNRcaNr@@~OR_+x#*iYUfYV zPIY2w-m0l+vmCER$;^6}m1AGqlx!O%ed~$}nppmfd6|XgQ`(eenNI(r@UNWQFzHrc zv^=1G-EOq}|E+%AUX1@iA)n<6l*S}UM~2A2KbZcj5Wn4VIwZUU#ULDT<-Qfn+zxD3 z5~UT%*!fo+>tdt91)> z^769Zn?5<6?8Nhxd*8H>Vp?*F1`9{t%nGlB`*r}EQGLCjUVY`%THlQ~{%X|f@_#WNr&EJC)rX&jz}1>D*cru z?Z(C8>y!~?3OY?=2|Auay0aV=R(SXP1a#v}?eDmArNwlPHlj0UUnllcovM4MD2^Qb z^T8Rbdof~-oL7p_?mgO(o#QgqW zCc1aKESEaV)wH|<)RxwNFox#eQ#5YfXp7Dp@+NpR#GP@UN2h5*DD>j;7dzls>+|;!JtZns;%5*>R)|B^f}Qq7PExTiq6%Wi5ckU=ouwA*C!h7(|baf-sY&g zS+~3Nu;0mBa#rPA?wllAQdxLC;vOy*AETzPq-Yp2?KdMwBdJT*|}d8vhHy4skQZq#ae&6_VK2SeNa;k$50?z-PW zDKU19ErT1@;?)*IczVejBN+CkzdVE6$t2w59u362fAiirH=$R2Z2==cccJYa_c-5V zi1pC$<(#sKeeUkTya06&YxMOyB~HWIZ=`**BP4CJcU#}e1OLT{I#%AYf? zQCO3_@}V^@r7)qDbv0OwWTln(_RLsLPr4JIcCzD$<}b0TP{UNd>+)dcM*-IBQgSEKR1~;sh~4cFVc}u_nn` ztJ``_WXO#<*I=oQoYOIv)FOse#oi-cxXduo!zpz{bZab$&}5y-z$~4?x~13~?|Yjx zE9VQIkWU*Qb=G6L`s51>G-IsW-Nlg9*Y10Fzizt^@rVhX9xC`UE>49PM5DK)k8guh zz{f@_w@G(Zp%K4N{v$*qnwSitd-|fAn z`q-xjGbu9OlItssB~uDpIrO|Ls@z6Dt8gXo1~=8~pUhPBEEffnBn6&Psa;4+b{#0Y zJCVMA;vIGLdMMxgtUJMq7#?AWO4=1F)w=RT@<=4vZ)}&o{E(W;rWW;Cv`Zk$p1IR! zg6uV~79%{-mW|;l>`g0)TCJZvBI|3UbH~Gb5M3ps%dD@G^P%|fCLc`yaXWg`?&_h? zO!FCP>#j3#4GRAD(@&eL=x?xIdN?L?x}Hy;K+u`HlC3+n@wgS?nssiWRa zon9WgITS=fU)DPm>1O_=GMG#_TFcEY??=wKX$!7HAmJTVs%HDZi@dsGWlNjU(`&fj zDQ(s|r*nF9O`lc05>BoeGNL0s!&8DcogvQobI69%@m2cU6qzS+QFxcctMGI^#c44! z(uA&6<3`E_B))!~E#x;pp+c2SRONLyG?_y_J}s*-3yElGHR!ROQ~z#N!X=@{c zl8$jxPu>(EOSIATqVOV;yPQ-pBJboCLJ}*fU*cxhp7h|*I0D=8=2G@*IQe#E$}(5$X)uV^-E6dM_s z+NAtWT2$H9W3bLF*IHwZW>|w9r6g7SqZ8>3Q6ugbAB}>N?Y3MNH8k13Nj@h`ZSqKq zddjxQZl9Y^FcjF$o+%P_?HZ<2-Gq_iFq&$jsL7V=`%nL_#mTn+nlT zrnlAzpUcyK6yw1C-M+z8nSSwB_}oU-v)C&3diz`P72*W-4#1br_a}UpyBv&q8uMg0 zs)C%xSP7;;Zni@EJ^mpnk;o6lsMPNg@e#KkRebbTzB6Bc?e%+Mld+OzUR6f-OZtr$ zXeb{Boozaqz|@*eP+InL#z51ThEK(U3oBKyjV)GJOy7{GbfJz_E0nr>=tlA9+95UR zSU<6<+@H-Q)4W{CLOLVj9LXHT>5|r(jl9(Ui471w8r+2^$X4GZhZ}-A+poUk(mk~v zRh5LCQWw|8`5=U?Rqx>~{o)8Bqvvd`17cIkHi(fZ#L2@J>;|{)Gtx8BSNBSa;#Uu& z8Z}%Cm9Xo$Hc%^@oELv{Y%wPi>-L+@@G{PdmT)4xJE>ujpVkEHvht(+^mv)%oSQI=j5WwI*a{Rd+Y!gz!XHl_6KQXK9hof&b@xk}lnXzfSZS!d-J zp7_K-;`d@r{M>W1Gb@4FVvX$+7a4CaWqn}{H+nv3rGRs}KGZI`5mWU-NNakLWuLHA zPMC$x$2gQ92{Pn8R!Ad_cWi<`=Y90@s_AGvgR6Q)+fpRBy{T12khvo;d^n?uITA<}5(o-3BmSA|yx-6OyB-{2ot5KHP9-o36oY@Q$ zllzI;d!C^lxn9gF71-XyIb8FEZ+p(OmUVH(8$k`Iz#xK9W+h@StNV-UG8eG>M#HS+ zJot^aUmW0Ew%v28YmoaIa}Y@6hX5_GsJ)A20S*?0!0S&$8IAx+;ELqIL_qN2q`N9A z74!l8=yg1A8v;ZG?K+UM;6s6~i|_+Qw+Z-MOMq8CD{A9zp2NM%F6SNFi|X3J`wPrU zs#d@^wah{3Cm^JK!>S074V(>p$Pb*Q0z~ugnOORs2FSn`EeOR42?LM`Qa+XI;2orL zW`kgvjl3cR6G*{K0gu1SIAi6>-=1^8z>lAeV-vgy;F+bEj6N*CH*0qILW>asWb_RY z8c z625gvnMAOCpuDZV%@-cm5nK#S3q`ZT=7h5&z5;Z8(pd5!m&NhX`1uMRQ~WqvqXJAu zQa_af>v;V38aj_Ca^5;c>UKWx)m6+EN8ks9Ls+E&Gcq*b>F*+3Igq1x4gpXTz~$ek z$_oDEH>Vg()dJsv7pZ(FHY!9!B1rreUokBTKcSN>I>ZNc*cHD6Lg8y6r+N(BE8$dpTc<7_6fbkzQRlJXcTjiw*jRz5n@1`q-dl zPJBc<>43BdQu`pM6FwY(GXNl#a>3)!@ABW2uxVJaO}Z=HPjSKk3E~nV`3CqCXRjq` z?Wy6qfAHQGFq(zEj&Xuo68K?W!3ZmapT5ot`VHv+q|}s`L8n`d+U$A~{k?`6Y$Vu= zdO{>$uSzZp`q{7nE65G(RjZrnjRSu0d&G6y)4=!qm{79d`+qK`(yF)HZzDOKyomT; z*RBH+qOHH*Zo>JCYm$(90a0H5G@#Kayd3;4@@Hhqft$6rFaEpIvv+s_x-=TSC;8v& z0rx^``R0Ennm;EXwkYO+P@tF50-Vg8`{#!3A3O!D;*feIzp6bY@YBTJw%-2Zna60% z2oMe@j;H6a^4&=d9A|TghHdl!0;%7-T@xCq0JRKTSXvHqhP-$;$w35dnp$=aOdwYH z2%ZnRd{W*Yk$*JYOSrthUxK8_Q2(SdgJT7`x;ieIwOrsEIoFN*l(eM5Cm6D|?6 zCa%Iwy-PWH;SqHmzfV7-b1s<3ychI|K*!SdI$d6O#MQI_bgnmr6r%}u{bDUHbe;2z z>eOIS7!Bv0$W1Da;W=Uw<1EXsf5ybwr3JNZ8ewf?CKr_MDMi!M5_07lzi$~L9{nOp z^4X4t=4HLdP46~CmTaJgaF*b;mK>VVYk21zgB6D8xHG0bWL})?Yu!SrZfYy~Uj9=X ztOkQ{f*29H!Sc_w9wf+i&<7;NhD->60s|{Q;ED|r1QDFj`eQx&Z=_DckpiDK+oZM^ zk&dPYF6x5tI~N)pwa386JJhzIomP`kos*M=QA>wWoefxA zVg$?3GIAcZxO6=3;HPh)+yQI>+7JU`KP-2Dnan*Nluq(;<|AOvm2`W~6;=@lK(vB^ z`mgkG9D@Z-j43gQOcIb8e97;c7^nl}3vi6l9k+@0G6{Cp{lEjn7B%R0}*@d5_pO8_e)xiyZEvHO0bbu!h^rS@C zR*$|6r`_Y8Bo1AXW`hYDJ9!LnE7E{X9&j!OG{Qq|668+iOE5qBL`H{?R8|4vUPEHgH?wCjN{-`cT5A?i)lMn4I6LSk7f?4IxsAJE{M?zJTTtQ{=wY8YS!JT z@b`ENdHQ=6_{s+ZsED6TsJB zfMx83793;8jiEU&+TN3^hTv)z>OyK$YubOh#{txZF?}ZNd<5OX|Look;6)dol)1Lm0kQmr*cpNJS<0$;8R8TzVz+x5# z57wm)-FblH<}4|At;4zrze8w%Xx{FiPyXh87iu)sp~JUs~N zpq;Nfdwq|qQuxC?U@{URSvzSx6;wUz_SbjMnb zu+IJG2*BbQdmbBacPH+yLxQ_;|I(1){$cpD1Yni=8wUA+lLY`p9 zfH(AT!#DC*dxGtmY??$0R`)p8Pu+8A}q+o%cg&bRb z`+yI9U_pWR=S9H+i~D_9r#?iFYp;YX8G}-Qg4O_3#A$)x`T=?-V}2 z3byqG!~fCjKiOUX)mHpnKRL9q_P9HI3fVLUp1Xg&AO@d=0ILYfSU9NvN)Lw{EE2HT S2Z0Cx|1gLl5Y`;MBuCY>hT7s}j(0H4w%ks_w{L+}{rDOE@6;9Qd+=>kl%Vl-M*|D%Yiqr}ldPefb_OaQ z4P=?T`A{tLT{Lm|I&)KhYv6WxU4JOSsLNXR} zjdwC;X0LI1d`M0utWaj>Mda*+g_Cl{Y@%V6<@>K67LmyzMLr7h1|yarhk4AfP)ksE ziERo5j_yUCo!i*hmZ$Hs_W@LOlb4*>a|mT0K?lgi|r>TPFd5aB(6A z>llqPYMtpuL}G+5@mA`$h3E;By`7sd6bwhg?IG-NHT76!C8dQbx2fjG-Jd4~kxDH) z@5%+qhu~2Qp3XbRNYZ#Cmp$~v1XOcR7H3`ev;tJ(Kn1i3$2fg6^m^SG*CnBeiwt_V zx!i7dm-+kHDbwU0b>Xxw$1$`u*J;hP4>yZ76tyOU)Y#6#>{OgCHrr;N5_DK~m-Es< zM9L1BkG;Z_LAvRZbpuJ#Y#w>J9E}9VtK7(~mnH zMrE%bv$2VkG*M>P2&u9Z{Xq2(f~~;zlt?1-OQ}1A+a{eK0;NvgG*4nu>0i@7QB73| z)qi6;UCFz5=zjD}A|sX)+5LX6icPkeH~MYZv;)};+mm*HT&`nEL&NCu5Q#QxJJ{M< z>xtA{c_X}x1#>*f^U{34~ln6|}$gOSbXt`8fKKTydK}E^0&Lj!g=WFE7h1jC>nO{ufM(2%Wj%(K{!2I^T*jgS((UN zwlh`NOSrDFp5J<^?{@TO!=YqjyUahmdw7*vyLG>N-}fX}x^~inqpEj*s5m?{Os;v| zyX)W3a-lftDort@QF^VfIazfN=TVm{-R+YC{05w_k0K2Hfa-aB(neD<7~CwH8KU!3GJ(9_x4}EUbHDRm)AJY@p~(v; z&R+I-sVAI!P=*!)>5mf``dIEUjr!L-^Hn0weB%OY8Q4qvet%OHVx zRqfuz*{Smo2{+@pbJcZaqS$?S{bbPI=b;8aFW*fOc-ZXT@-P!xMemA)Qzd;kVz%Y# z`83e{^x5ME;_K|gNLC9r$+DreYq4x0x^+kD=h>_$gX2txsE0jRevk7n?gtem9`~3>C^tooMM%r#P z!?6WE^tc*==DG6C)YUcbxDO^}Gml4F&f)J1FX0Ox)|;f~lZsPo#sPJ|Atfm5%C#RHLr^|4p zo)}0?l{Y(ju$1j~NeMktS#o1uly`6^JDohB9WR3AENj@|uy7u$N$$Migdneo!lU$N zHI%@9RI~Hg)!_0p%+%)7P!rHidi5v-QK>Q?L^bC4@MzI(%AmQab)7rCN2a@Nwx{_c z1=)R75PA);3;ozvb_le;Qt>%s-kf|(Af#mM+iOc-g0fg;SzCLSxyLBWMa&8+=`*`} zvR~X_%i2A)UJ>UYfo^cJLb2?;q!@|u<|ev`=&n&D|w2L05fNLlQI)zJgN4Dx}qx{SFqXV8PBAE)dp z1`{;nx9!jsJGz9Gx%&P9}K4#$Vf!lzN+#*YBJeya6qEz2LDZ-A+gI=^2SZ}7)p4gT4L0TvQ&J( z&_Qe_p}Xs?bOOudEcDUz@~l>s1YHJnMU%_w5YitMb?qtb=e#LIbBj_UqK@O~*oT(z zSy>IFGGPwK-Pu{L%0<|ROKD=F*Hp@!cNe+7HL(p-wOjPcXThW0jsw%hp)x|AyJGpm z%jZ*H^O$RMnrm3|upjzj1#v8Hvgt0Twz5nsuLfV7MbG)YFp@x~YBW=LOf3s#1K#B7 z4NR~%xJI%~8i(LMEpzac6ij{VWaAZOGU;%Uv~>}#tz~UweQ=KcJd^D9DTiowMmBjN zx$u~9s-7NaZ!A(zPhC88c2O;|p)+>loUN=4>qG%w+QrD5SYy0Ds<@@xV9eK@WAwh7 z5S6f+aR07~Fpj8dVC+D@teo{$vYbU!O3W;S)I%ZH1i3DzhCOE!l|6SOFwc92l$D}6 zDYLe*@q5Q^4|N({c2=LPB3)+!MX(~9)g{|~c zm!jC)iH9bkq(s@-MB|YSVHz`v+%hH#3Yll*{Y}CevZ|?RYJ0{BQ?5Ta5-K^f(`kp; zlMNI@y%)%gY;w!8Z4k=j%*YL>t+vGX&!~*P=*>khjE!xeJMWKntC@0;8lP|vQ5 z5tJfTPL3zsjC>?J%tXpCn2(jPMPd<8Q5<^7i<vMP_u;0zB0;yo+l~D)-8KzMnjG%6NXgl9iya@AEnc_VvuLx~ba6rlDjavhGTf zL-JU30Wsb>!nMTtVRywH<`a8N{^G2g z@!OLd$y)c>shPkHR+svg&jkoDX~ND*Ppr*)eddE>I}T=93w|oHF7{?QQu|*Yx6#)1 z^e6_xnS|OW*F@S*B!Z2w8(TWWZqgX)2X*9x+K-IgL7p1s*L|a?wXJv9>P4b zwb3-ViboVXh>%)F2wmq0p&NZ(kfa(OcUQP1bB}JAA&XI?7r`V$a|oi`WU4U~YYSqm ze-pwmR@%~_UYb9hcNBU@a_6~`aSaza{qvly32i<9ZK{$i>4Qo3#kRdl>Qa6#G`TX{ zar)jMs9bSwpg+XMf_<``T*x_rtSb*&C6%fq7l)$z1Uo9`qpQvl-Yb@H)lp3;* z5npWX^uexJKSHB_J^@1?LTpYif~iXi@@lZbkKLHfT_;zUb(_~<(onV*a$lRozTS~5 zHUJ!@)16UECBdEQRygf?cbqEKgxTr(BMi1>;nOr}6sEaOT7`En)_dtvUw4=Smwis* zSE>B3iQ=baaN3x9+Ljc(C$BBuIygxGVBP2+JG*3bn-V@|{dt1v(1X|He+Q*P?XR9p z<*y#TL!V_zn%%VLEzq8?wZkb)nacl>HS1ai2M@jv&ygH&|86%MuI_`}q`!Q|V$S@J zh%Ss!!aWU0zT!-L(!jl_u(6gXaV`B=wo5+)Q5V`N#iyYoC^EcHyYI3&&O+6=a^&Dj zCN9l&aumi;Dp+{tB#G|HM0Y93h$yFb$g(ADWGNwaQVHno)5$mZ!s0i5pM=O1tdRFI z9rL2iCL|-v4Mc^l#yrO|$e;3q<_S@+g9Rm`$)q!=rrsCLTkcS8#$)1WX{tmgszs1v zt!0r3v*(R1StL#&7dfno+}&&B%tb9+)^$c(bf#p7HeAck(@S6k5I+}n3_ zLIZ^Gn&a{-*LER@^7;gdP1Xw14p~JUj2s-UWX?wpbRZJegUT!DbM*U#NIju9VVGDq z2U5c|l$@eiF;I=XTa3U9!{i#B$5ikl8JxUiWH*jK0#x1#fc{v4U$~L*Mv3Vau!`7~n#kO6VDD zv%dm(F?HPHzV^Z8E`qIuUFv-cX!Tux^J`6Bi&Q%f@taWG>zcOTT;R4^mw;ESJWwn3 zL;!%%dNfqx=~`cZ05v)QaN>)N26#>Ffqr76UGx(0{-T!n6}WyYtF^)N5*nZyM*IO> zDlaA30dou76oBWd3?$$^rqjw>o}|ozUu9ZZE2HjTgF+^uo@?;TLk}c{zqBm|1x%Z~ z1dKe6G`}(?SNDe)x%#B;M>lVFoYzE!`u;P^`mf4>mypVqUhl10sbarNmZm*-jrZ$r z4c>3dt)|yA9bf#^6#nST@7+M5moNO(VCLVjw`g}2U_HO>UlY}bu<749xZh6Z@xYf* zl#SN#Jl|IB8=v#75NG8zwt2v|-liW4pRBW?0>0)0Zf;e; z{C-Dv5+47;ENtxWWDXw4*R>AQ9;K8oZV;*MpG70l+6(Uc;yOMFpc3_96wJ9-zn?!c zU}epE5|*c-BMBXx1$Z(Bx?3;#{~E_%j05mO%vJJictHz%f@8jo&gF>D=Y-KIv0G`jZ->(wh0Pa9= zB`+pawi=zVt7{lQEzQED5BSys-zv2h75TXoQnJP$W&p7AG}qvYnVXIaXf3XNfiHqY zO7i`LEYx!i2?>cop%nsu{uOV_*6Om+a5js$y2|FB`rA^ZG1;sd0KR)+l ztH{If8(N^`@1Z!k2CoY7`Kg|n6dG)l1FS$Q;(@OLJ1-=!!HO`4bbP;eTLuA0d4iF{UHg5X_*}WC~dF*#kHRq34|5@IG8F6n*8nk z&vowL7!-EEij(>MO8}1yRO?OM>%$K z@81M3p$RLHNCN;MLy~<27jbW|C_*4C07Q7`Vl@q+CAJK((zontP0w5A2@Cnb#nOes z`^@@t+QAy^1{{pE{2CN1Q*5HBd1~gQ($IpS%S}^^hC!x6%}(j4dqK6~ZEM*a#RO#& zggE;-inw#TB_`)8l6(%k+o9d9b?V#(89xpb`})FiXVvH+{Gx_){2~-**=|r{h;^() zYO_f-uw?E2LYMvFea-PyQoo5q?55G|1+QzB2UboxWjWihVZTiEa#&Zf`c{PuwV{Tw zu987n{3KrGV0(ks@q&1y=M!OC%Ys-cr$>bQtA98RStCEIb>^+gq=Uyv6>K&B<}{@G z550!}a2L`r(lN8rF;STVji_kB-b7k@N?Mj*PD8l+dt>eY>NG5hku*l+Me~aiA%Vgd zlZ9_$X%d|}vUG^DphA!}F3UCLB{7IkNsxSF4*FC@*gC2CUoR`>{$pYW$8p;lOIktnZE+iM@{0RrVxncFaQLp+>WnxAZMOuUV26H4W#WsH zgM|QZlPL^=2hm#xCW-TpaDuNBT|3uy03Ee5xDxl`L?WoQW=BXOs3y30ypcH%Wjr|t zo%%IId}V$2`_ZjE!q&QnvH|RlVzQ3(rnn1=sY5wc_zwIpI-{Wj6Y!Bnu)FbNHoxnV ztK;P_A6SHEmQLv<)=WkM&MJ?b4VR$^ViSgmjnN*B64WuZ#F?3BzXJp-S__+-E3Xq%Ab%ml2f z1a;`g?1ku%fZZ7-QF>BJ(^$Rv#2^k0`ohT78Nb8mK9OQ<9wPx{h z{kbW@vWxZoQbm6e4)<}1yKA(TYw} z0kpvf(Z71w|Gh*1zjNlxQ;ATJ^-}Z>49oXOQi@85QiyKIIgs}$oJdp2($LA$RF=}w zRMC|ySlPb?-_y`-iVY194pO}B9gwVoIE94`mVlvzkq8-}3>m0h$*IdiiPoP%w|(&o z5cvl(fIJonYYzVMn!^GB%75egKUDJn2^#ndw)i*jfH;+yp|-ZR?n(B*9uYJG#8Ii4QCYjtEEem{xxRqqGqP!UW}rF>gKP*M#0oeZj_AxtKsUt2I8`6NC=lGYR`+7(8~Pg?8?IcRZh;kgt8`-uC-Mu}NJb zW@3NUh>mepFPW=EuUA)fCC+?0pL!t2!Uul(gx!~5_N~~#XbS}CuSFE^HS^AmRhQ<} z0qcIab|CG(obO9G#$7=T6=BxakwlKMU0Dq+JTt+0`;i~zx@Tk^NtJUFWBE-!YS32e z3>h(ekw9$!UfogPU^A@$(OPi9Rqz^vezQ-lJ=Hg<+jAtM=GmdjXs?}wX?uy&TMOar zWp=kS#|dRF)u!{GTKZ+X;|xBnr!~?q%pLjWe{S~Yf*49Hp#nqxezbC02W0js{%TEJyBzEn%&u&U8RL zag@FoMNU))^{kN>wm|60Q|GP&{UzP5qHmwRS$8KEoo_4>1h)_bVV${FV5DK=1*Apk z92h+{IP+8%B$<#)3>ZySR#qu9;0!J8w}us8Kb{F!(Od=b&$pS>3T2YDp?Z?-){0!GO%zmny0)CFwLKl?XXVNS3m5cb0%ApbO@ZCJ54)ihm)E7_+RQ z^?iSpWn%cXUebIi($AT?+?Qq}lYvHD7rtdlg)PzEJ2GvOax*{8r0>S!AJiP5h#5m# zPmTjDNlfT&p87UxC*EBp8{kc2r@S;$M4PX++(L3QjY>`9-u+~mEL}L8OVwE&x&Fz` z>HNAh4tI`T#|Xb3lLGkjq+eWl{Ixy7>M6@2**5RcdVLb^mCrnHAkC?y%R^Vk+;Aek zoWCTN@?)8~Qfg)MX~j|IkS+{7^J zaSidq-PuZ4_T#j6Wr2am(hfyxjVK+4QgsuIA+~L!zG+9cD|C8avILN;K$pp$ol#^BLUW0F zISY}O??c@KdHlkc5(CX0Lvt<{$KH$B#KuFW{UDiz)Vi$h&tGhj%i|5FEv;N#PgS?- zv>P%cF@8jR8st9+)H!hE)Z;rOH-V`jNiYR z=KP7-eyrrg)0>38J>eI1l95O2{MWAw26WDr1$c#vSN#Sc|F2g1KQz*`zh+#l{{-Ov zIV1aL0QVVKh6T^Z{s!P4E znMv_mHgkqKl1Vx5$O3x+)@(pM4wjVl61q)lc>Jebt;N^+QnK$8P?LunPoehE2JVw3 z9*ITAb0ptURt_HCJ3ATM3?*GZq|DzpZ*$eyG)pN2B6i+ZbYjYqLS+JbQSqg&6_m{` zRhxQ$Ozu}3SABXw6nX%xo)}2)3piJjeB&1sPxwso#34OEtppF9J!x_&lp-jlrZbs@ z2qX7)y&*>GTS&)YesS^AXSEb!<2U(S!(m8|qM@~N2xOX2s6dpiAnQ67JGctARc#CJ zmqR^Gwk}H-{;}JV+(va|$LYI}O422_cDK3C(xDg&4Dt_;DRbN6DdI*JIHv^$fh^M3v)t2GNoxAko^j(}mL5*^F${-}Y{ zq?u1+GVsYqMK8J#C1l)m((HmNw@qD#0fWPm-=9WY%ejn8nc1l|wXcaijdDi8WtL(e z11J7k^Bfe!t>!s5mN~uPcrx#k#%r5$$Sm&Qe>F9 zyQ}$fXB2`G1Eo06r7b_jt^|1WGi;bz=~2NpmM>LOtn70#dQIS=qpvdU`abd;Gg2uQ!(}`STIQ?pen??JHI90QwsigvHnhevRG$% zxzx;PWy)t6>&Btg2l27q!yzLtUdP7u0wrp+a8x{!SFbW&0zIxms9^NtAX|{AQF%Oh zGI|kz?&wDbwgLjnPD-HNn?%(t?c+_t;Wnh;y$*5d)H3US%l4RakGd^sYIEeTvSGJA zMU!9UD1s=yfJQ#_w0NlGxfC_>ReMHm^LVFY5rp@rG8qgvCyT@;MX!w}_q<74p2h%) z*GNrnO1oE`q!WuRtJ3KwxSm zxE}4@)B+}RJB;V{W0R=5^hRGEhu#-iv^&Oe-+kH3A)p6((rjm+Ume;V<~!{TdtRqh z-F7Z;FWm+;yjdEXxaeGIhfh)h=?q=PaC5B}!27wW-&>E@*#<2qI1M{5S2iAcE>u`P zaa+32*euqC71~W&+?~u&3vryP9PVE)?5bBCn<-n3%^yG59`EoT!`6wUBqrUlSa+G< zO8Yx4$l2Sk8%eg6nA%(_UTb>D2b>J%5*7Fhi0g*`hQugW7?@^PhlkZ-dnGZb8Kq?DDA9n<1TR`?0|(>8DaVsJX{yK7 zH=bhUG)-J%cTO-PI11)yaa@|@HpjX-^eS4^5k+j!9U+YjytNMZZg~bIS+SS7B+iv z{c~BtvQq2J^Sa3Mpxyn-7<=v>G#`d}AHGz#(Hb{Huu}ynu0AaZ@lT8BT?;=Pp|fa&PFynK$2# zE-#^}NSyMo``ljlb93J<)jJ+GT*p3Xk8HSK_1)4|$sN z4H}iqWU{I#PFq$BOfAu_`C6;(*L0gFo-Hp}uY(~qs(!?rW6zV(^taUt_RVXViR7Tw zK+sHEPp-@gxO*+yF(6hmGrH8?U{ZS>Q*j<g@J zvVFKkg2T1kdHb@o6QZ<@Yi}$}(?w@L6$FVK1c^x^tquxl*kaT`N*%*c<@OB>!*TIm zBko3ix~kX@Yiy~%i`--$#3H4*duX%|=wH4Xus8KOB-Sv(+-aeM4o_v4VEulKdA zv^-T-m04x$?(1R1Z4i^G*12YIeCdlL_9JH9nzs)8uuU z7|+xsORCwd3DkwZjF9rRkG2}%rWi0gC${6P5 zZYW#J@_G!WrKfun^A42DezqJQouxpo^`M-*4>y~QlU-JF0{4owIw@sfsGOj1uY1LJ z=ABFCo-}X!(b4vdOtVI=-qDp{q#qQ>6R-HQnITgyL(-$qemcGeDR*zF$CGrA9B%~J zyhddd8^xaXu)LJUhR(U~Uis`2Ip?f?-thd#gON`WH&=x7WQd!6EO?QV`lgH6RWulz~ zp}WsJ0h1}LHmYi2j#EK_ohN=D$jn47T~*#r&0C3zpDQ%P*odi7I}Yis&v2HyBiNNi zeAS!Hlvzw8$gNHhm}VL+#GcSJ2X&!RS(~Nq8K)+XbYUCST(ck!>`MEg2K8l(jRaek z^#7#4aeOz~f^_Dj2Jb+Eh;-8SZ6t#Nwy4J9O9>c=~i&+6xUn>-8QiTGW4eM&`BYq;1 zl`o?f5S2^}s~;Rl6de0)^(qeuufje=pKZ)`>hkLM*3@XNdc%<_nAvWeR%0JfbW%+< z>8;ybF?Vb_7Txxp~GA-W| zcbPYjPO;b5Qw&CQ&%2bihil^%lkY3={2+KQ-&#)xZX(o&qM%I+2)iR+fuLI{yxj83 z@9SyCY?W_=0#Xu*&L)g4ICqwb$%M2hu2tJRJ1(Mh^S$g{=B?{b-`arU(mzuK7qX_X ztE`&uD=7Dxm}A{tmYMbF=w&SXH|VS`1C@U$k#8teQ`KLWIaAX2YvDT+?Ns|#B;}v= z_L`@#-OwidnA%nJ<%^JZcQGqaIBFzYp$0lki(X1yvilTMzEw;liW0+bN{K9wl!>DD z1TMmcLrypmrRCP?k4YDYgo=-_Mvn{9-HD3w2MrT@TEyBK|DaTnQCd4~*u?4e;gAcd z=tQMRKoTpjEi3^2Y--T#Ep39HCZDx8A9Eq_g}avJ!SIksE;y~$2!Tt?9sGwF@fnp!tGZ-Ga@bs|#>2dVgto4QKH z42yu`!&2P74elzAcTCipSBy=y}WzW!HyhACvX#meDQlXlZ@=gH@8ZW>|R zYEOQW>hbR>lyq%>z@F7a?)0#*w5!+4&po?oOl+F&c?)ALD`Hua%qR8!E~Q3AvtRsE z!p4Tp2j@mEYp>P7hPao(TgY!m#)<&;vDHvTL5K=Chl9NLXEwJFT)ne0sIZ zlYQi|sFP1EXUl6W?p&vYe+9z*2BsJKm|BOl1TZolE1A5@pgHqKC2VnU~FHc8~>uPm{#Y3GZ=J5L& zfG0L7_5cvBzfd2+^jYDLOfQ%?DgU42pz1`s#L3-n@})d{m*Bsz}lui&>fSnj{e|BxD6*AoB$QS z?$c*{rj>LhbI9v3Uh|i^>3ckS2D(89ZsLh7bT2`(*o9 z;So-91mLW2%!Db_ZRc;`%Rf^1rqgHeC687{%`a?QMc+fMTg&q7B7uwqWH%`Y> zE-C5&U?Dw^{0kq(t+K}bJ;Z-f1lVDqV!zzYXPN~gS>SR07S|3IFdHX7N-IA4C~(DO ziXpU2Eh`(u9yxOmzze2}_fH5?1Y+SA1j+Ch1i8b>!TN$HG5@zN?I<3k{)XTHdHecd z%hsBxfDtv8xYrjim;O+Kzqs&X2xq+}v?c;0kh8u?U<4BH02*9ZAGY2DT!tr70^AZn zIac>yGVxbU5y^ye?O$raVm}^<#e=>oivkE|bTWUI7yvHpiJS!Z_oCsT4#w3GYyd{+ z)hB&C;y5kvM>dFp+{{xj zj0hMBtl;}~58!#XlHvONpuzf#OL0*Aw<+<$P6Qhga2k8VNemW5D4@r))Ue^Wq;o?9 z7Q>q--CXpVpS9~-+rH9sLGX|_k8QqYftUOxq!%Ak`_jl}9w6zE<7U7Lzp-ke1upYl z9tzOo#^3;`-T2kL^hk}tWp>@?)ZQ>s6R&U znK1x>)ISIe{?pcaApX7wln;8oH7y%DzIGxKZmMZR2w!;g>bB0V6yY4&Dc`x2^A*)& z`Z$1|sb6KBtW=BtJZ^)7RE#TrA`<4}EOb=oqlS1&vmC1kR2+$=cfU1rx>F|FwU z@|#RSx`|psQSu)x%R4X(RU*8S?%D&2a^Gk7foPfJ(RBb5g1^f`L^8?K_} zh-~BumMkZ5k9_`D&)0wIq5B_QUw`@Np50&Y|JD8VCqPkZo20F6fZg{nvZr`2+-=m& zO^2D!u{V7qzy26%uuxy!PiJS3e%QbQ=b)~Bkn3(5^r?IM?N*#;A{sE!P4+lhryFeR z)3&qU_IiF<&6)|G?Qt{zoSIYtoPrFoJ2nX?+H|asIX|LH@ zHa2_rg9P_D@V_pU@*259(poY$wsayTl+fN91L&S+HYy<52sxKiWHdj7P&BjargH=@ zmeYGdsvMP)=CUz04Dh28uiJD()-^qa-}aK`e&D(AG~wpfKP@OR=knUA*UOz=KUc1~ zQH6IqgO~e^C=KaeA&64(aQfnaYuv-g`aS1z=2Y38gazEU!3o<{Os@0%0T?Rpog(Rq zs6}AMoUXhr$ZqSP<{gGjLL5b-n8d;b2pQ$|=#c%;*tIenO$nRp1zVX(IOakF!9o<8 z{UWm8mhc3%*La9DoS?%*rR;!rotYK%=;btIo8689Xn4zd`#-^LidG_5c8_P-BV2wDNT1g}O< zIo~cV&x|G8jn=cz9>8<^%%~M@Gk-B&nhiTtZJJhYn^16+VB()vt^>AtrawF`^ys2W zY~-?PNddMI5WWQT?YQVV8vZ;lFgSjqLsMJbdku4rV_BXEr`K{lYX#HA)mn7(QKKk9 zvMWs*HUwv+GoUlCC5mvQ^9YA(y*qo!G~o{0=D@mW8ULN0=TbHizsLh=5s#GIfFs$k zhM*;yc6-ANFy+N4mE|b?=w|r&Cn(4!g*x~QSk$sN-nSY?@B`Lr=&y8f;5fb3-~~|w ztTI6Bs*i|CP9=PFvIZ;J%uFfT%vgJKUQ}^)R83#d==0+V)pQQl5k6>h${yWk{P;M8 z>lSKey#%ZYWYRD5j2fJ3n~A#TXC1j@m@d%nxb~G)zSKD=*cXuaaVVt^YG{6G#wFwE zkw^BJJB3kP%)x*X_i@|tslk`(N>5(IYUONyrCMn+_(v+YqWrMf>vQBay7hp|K@`(l zlcwq#_BM3GzXKKx!c~Rw9t@CfYTMq%n9$i*uoIzdzt z{jW!9tFVPzU6xVAjC zV%|@RZuZ(4p~n{;g6O@R^4HfJyj~AmpZyARXds+|Z9aS%+gO{zOhoaf2~=Obm%#5u zN?b-teBFk*ZHh+f`~6+cnHQ8Y1nD;!_+jg}Vv=HK zEJ+9tpFe;$&W9j7r$xp)_lG^ z^!J`_a5RZr0CA1D>^)7sPE8R(w3rZdZB$4%As*o79;pJv3=A#miTeexJvUW;fuv&hlBN`Bsz1&aSYXgM$Zd#luC|)0=<5ea|{R);RLM2|VXF22Xna1(*EK z)rIzXzb5lPY0cmEYyLIoc@D~Wwi$ne{QjKth`1V`neS7mcuK`6EpPCvIXNB=>+i@W zmz)-mE3nB7E}2zR^Dnj=3m@#!71en)!<# zk=^&vf4!Kep?vZcKMJ!fU*WkmD_zSdf`6MszHIh{8foSsetjqCs5E=8xE<@--Vxoo zwgg|cVvFlCRjp@^SfWFBwgfQ8w*fB6CVxF@8Rgazhjr_$L7h&; zmbk(@ygmS^9(pP2;16R5YAsE_#=}Evh>>)CQNB=4%Gn*jC_G+V@CwtTSiFM7K5B-0 z6X^el^EptQYsW~IFwZbQhIoKrnzP!woQviv^w7UOVzU^GW@j0IKr-wnF9d)rb~RWU zN1QD_7@XsMVRO^ec{-ssrPFvW^+}pHA!}(}UmV&~7<5?mxwZ*#3``T#;>k;1>dimi zgiBD`8fi|`LW~A0@11TP9u3MhykQHDD7)EJ#u)9aoo1_VojGQ6`}vJ=A5D6U$n7kw zX*me*yy>SMC#kiol<{W}k3UXPndM}WL1i_6&pz$!A*;T+3qz{T_jYKESVfT4?+~9~ ztEXR^e>1&9nlQoNi|6C-(nO_4%fO(=!lX~hLQl_1$w0@VL&?g-qEE@n%F4pP%3!Fk z%c}b?))y>CjPx|De+BZ<(NQum|0PY&lJtL(raw{cm34vOo8Dqzge)mu?{xT9B2b_A zAEN)?#!-Pq|KDzm{{J0E6#@RjZ6Y=rgTHWge_yA7`Th_4_;*m`Kku=orDdR_WnlSp zkM&=@oPPV?u2V$BOg+2G9S#N0eYo4F6AGh`LY7Txy`He7Q?ux$hjkx9gw%;GXKa708ABcHs3-9y=#GJ^2d8;AuIbPkwXkd0R5 z#i4>e+Xi^;I66c*>P9(gS$eO=P8B$B&P??6WtB&ow&)LD;Lw_x$F9kri_5iQq&1ok z^=W0z&F?ie2cp@(71m-7H+6`wxkeCMdyP?yu-qzYK~LIw|7hLlQ&vHD7W&0_F&-J- zSZCT~T-qqlU}~J&-y=LDm3vu->zn4BOEjT74Mp{u9}X4kDKoMD>8|8kmD<>;Q>T$f z;NC1{gQ7K?lBOB06$MNz*gvh>A)+Z(g8qGHuHG@umliXiVHqY^(9fZ!=g*Leo{lq1+ zEQZbG1~`Zsf4+%RPnSs~<7$d>xy&Bv{GbqQE_xX^5J@!Fk?%fmUNszcto4Ga?g@vi z#f7CvqAS-E^}dMy>%od^$+sHpWCeT@q5VoaqhVPm&LtOUu|Mb) zZ~4r&@YY)1w7wvW)_hZRrD5%P(Y4go#Lj6Byc5&CC`xKF zfiGJ3S2wr1yyw)NFrd&X23uNT(B0eU+SOg&PX^{|RaM0JBb{(o2=9p3lT6Q5QwrnB4c1 zL^C~FYz{=M*y8wFcYR=vsA1$#r2ZU(udd*G&KO1IP@L}tR8F5M7mFJp9aT`2)KAO| zc;6#Lzj_%68ee`aVg?_3oz z*3%ntJ|!TApPfFk&m{+r8jSng!li^-U}pYpa&igH%=kx<*^L;<#RR8)NxghkBu+Y0QeZ;{v0y1D?E^xO_WX(k8riUq zEsKlIEDt0mn2FU?*UADrL$DSXdyl$`_m1vzzAL%g^4p6SHZ-%%E=)%MIL-%(PB4aw zQrc`Q8+p<*nRNvs#x1%G@e6a^tRYKRVm2J?(kneKM>S~v01esZYPoDq0xg>J_2h3+ z)3{xY+_YySDxUU+il&#&F9^~Wtw0B%&MlrRneM0Xo@Zv;99L<|$8O^}RgT;-$+DLt zfmlzV70LObmW!h8YOSYciRdvAgN*@ef)%5bS%i@j9@_)|L0Dlw4~7zttzjUahGW+%ta@fQE5f1lN zs<`Xp+Qp&SyQXip^`;o;^{R;DTY*Zncx01$BKC)BKl&;qw*@kI4+8hdVJw93)FUAZ%>KYSM zg-{G%@;K)}_HjS75h16+`txOmc~i=0@6Nz@&lnZeU$;f7M-BTCT*DqFZI)a#J5Mk4 z>u%Cprae_GFS(CZrKeGIZNcYObDkwRIu6#GP5U1;;%T}p;h-#PIrl5#%vg(^fx10T zT8WP19j{x`M$~NPx0xu9XD@)z9(6{J7u>M}vzPNF?#`VFB~5x(ZS6_LZaQ66MjorN zMe??}985NIwQlv1`_n+sWlr0oL3&4~`*|4IL>awOv5A`f(MTBX7uwQ;Sd{!B(pvf&x7;jW{F4+(Fd%m`@i*=xd zbc*U`9sBEKK9vCpm@F#HcX8J&et)={Q!sxhs#(b!#<#rsF zHs55bwsU#C;G(fA(lYBZ-7V#MllOhFL)z17RC(zv#XRd&yjy_ndabCYv1UJ1lA6L? zdgyT`vnKm`;p}Fm3LdnVx0vp1Ce@s8mDN3sseE3O%N^VMxSq!5F|C+3EQH=-Cy`|# zoqUsxlU>c}Albcs4s_m6N^?1}2R=QP$i*GhtzL|*Y#zI%<1_1`2r3o6^$1>+Ub;Ws z$!&4$J{&#@@GPZZ*w6N4xsO*@HHb(A>1p4fE*?u6g*mvU_Hp)QF?#IISV!)_`Cm~y zZn+CfgtrD^w1B?ogQ9^)w~Mshz8<%rP{wq|m2JHSVi%jOAqK^cq;sz7CugyPr4R0_a`cEsygxK#B(Zg^T8X z=4jM+8ZN`VZO{}CN4>Lg$=BxIGU+Sv3p88ijsiQ6^?@rbO*ys9jvnNd(d*!Hw*QyjJWvEM_E0(+=T6cKfP zRbWMOgSwISz{=z=vvZtET1Q>MAXk3U8E6}di78@%DdHe#$`_#`E=IOh1r*6e%+q&O zt}mfxoZ*V!6Q_E+*|&A9Sj}3O$IYqkbYP`3^4SG<)PwhIZ8c-ZN|wg?qXcZu*h5=p z8*F>0t$$F1d{gh$W2k!T7*Tp)Cfz7Ayu*dXzyu6ivpcT4tVh4?yQ+88?zfslmzId=eNNfvQi0F}e3Tp+Gdp%*C z6p2Xg3Qzx5MFHX6y_HPO<#%c(R3)KucCkm9whEYV6mXtO?l;k*L7x}vt#YG^PGLJwe}2! zrKc8Qy&Jbkv!=Q+X)C5s;t*b^00^_+?7%tlgS$o0J%JH}=AR6Dv)UriY(XMwXTnNnVj zLs7O4&rQaubjcwQO3YN_l8GB!RJEcbysZ^9&1F)a3|pGq`HLX1P6w2<3>6~j*})di z-nY9=&RZ}UC1)c1mg&Q(uXEBo5g+)gK9n4UGmUSItvpo7 zbm+tp1-}t;GfRe-gcWZbNBMa_QBlZspjA3MEoUvjO;{LXWcTmjbuuaiWDQjhgG38H ztQdBr7$*DFlI;0*EAmj2;iZ}LI$1tLI4BDp$5^UBo|p`ps#I0U5ANo4Ub*Eg0#YPk zUXi1BBZ|$Hb6YN-ExxSOlBAQL?4$~}Ufsk^u)@<^*~)eu-pvuKW>SBk9{$YH$zqlI zqX3K2V(8+D8Jpm6Z1`0P22fbv#0b$u8aF@Z@^P3ZB}DyQ>2ngQUDuplbUazyr{*|i ziE7KYbf)VF$weN>6W?~!s69Xs)eIS}JNf&dKHnuK)m<~D@i|~~bJjvsX zA1uuVnfeKCbxK^sj^n3G2v#M=4#GQ!$dYYSy@?Go`w2oIpYo8@&2Kq zf#FD7Bg8pO#wW5f)nL5390{x7F1>_iV**s!xR4RQiZazMB0@`grb;UKO|Xp$i$DdvkCv{&9kO5ck7_JoFZZ^8Gwl=#Q7s?{%t2h9DwNUGwDVO%JxVP1LcL^=ORW^;cx8~iiN zt252-Es5u5<#zqkY>Qi{jN(1Ri)guI!uhHtfa z!FlwK)5gS(%<7(U+@<22`-4Y^fB)=`rZr3!DA$HY zwTrh9yt&b_A7B5C^ac6v4zs}z21i6?y zSx}ca8XG^+x0iS#?Vz1m)hx@B5eSDB$@y?+| zFm}GHsN29e#c{Nej7-`0G5EM%!PQv9|au!KjV%ZEEl-rcLU;o!5s4mEoD{YH^kE)Um0j%ywU1J(co%`n-n%V$lAEpX8nRofA3c z%T)@=xA3*hu1CGPVq-H(D;=exEQ4Y3lz-=;r&$q?pV6CJ@9s#4cc276%G|wIWG`j5 zS5deI%!GdeT_$*$O@}ff{;b0KXew%eaC}EdVEwK3C0-gV?Nn^2SMCcn&bBwHu69Oh z?|J8mF^bNUj|68Vwem|JyuywcuS#NC0-W7o@$}0GSsw3x>uRp>eo&u0>c$#^kRYlH zu1aj`&;M4x>h_%!&4tpyXMD9{1GmBRS3;+VSCcvC<6JCz=Q4txWhD}pu-8vqD+kD! zXAG7%GNV=u(lm$0N`=P*1@*V2*H??tC;W!l`4*zP6(#O?(c^MK?mqeQaP<1Ef#=hM z4pQ&cJ~sqHeF$Z&s2t*-6nNzY&u=+LFBVZsz3|mi2z=l}c<9opq~{7lDK9(Ps?auH~ETYU_4F9QZfdvvDG=W0dzn<>Ife z!O3z2?KiGute>i!iwx0JxCU{O48}M}4;HL^xOq(_Z=^`!+Ul3LK}~UwBngRd`#?{n z2?bMMM)&bB#g|(Wjj;!E6;)n>4_xDx*q7Oe`yfqO4dE{RC}+W9ni)tTH(DLpGQggl zA6xmB$N8ioFm`LBZU-eB3f7JjqM`y(yfLb=O*xuuJ38RX)b+goQ5fkmLkeV(na?yI zOfK*_tYzw8(iI_FlaH+TNX#iN_SclUfXCV0#kl)W3)DZs%>Sz(^}-U2^PdE&KP*r4pLTg&@Z5p-Si3;ryN8hhmdnNDNCPT(JSngRG~zv;hj^J2 zdEI)2=V&Ns?fFi^g~_f-5IC+jO&s5?@Z~z#Q`DwS_WWppmBXksv9SLW{lV%sXWQ%D z@fR(s^gmJQ8#u%J?b)A36eW=BO(PBN9rpe_sh-ns4%iVnSc+~^lOSm05N1?d3v|8p3rEQ# z`N1eEkIC5d85i^Jgw*=4ixI|Dw>yNJ=VAduVh{J1Qk|lbI^YFPyvfFz9OWgjOkaLC zF&eq_Oz7%sNVH6wlUz2~E=gHRpx<4ho`LyhfO*m*4;wN~XTd6w4`sU| zMvn&?=|z`k;lar)qh>HS{+Q;=Pl&HOXBmrg1!?3i-+LeS^>;w`VvGU$Xs|p{uVJ6R z1G@jOhS(nhnU;nGx(s?aw@emS z_|i)$nm636t5fkb466pIVm4RLJ?D!Mh!e!gxzoY8bv?CY?jDrwtwhdUhNrB249`ij z@qB`hI>r=wXS0JS^}C9e%7@hLPhsjG^Nu?5y7nGJ*+$7jE)ZP<+cbNXLR*Kld(WYz zo|1J|LPHZ2TR3ZuAfAdS4HYs0q~RcTgi+rx)OP-}jFG-<7D^M$qptj>yR?Clh;tS} zjWEdFK$B9qOL*tFI9J~=rXcow)`4f;s0@T&) z1qNee#PwGZ!6xR6SrPBbT2ehWHz6|8%tY%ljxthAyI;GXZ0VNUAXg={N|6;z!BZvf z!*lh=aE=a9c#kOZ6vNm9HcWfG5H)0wfBt+pKw^HKvaW0`(AopkDxInA(qPCiDQdF~ z0Ph6XInZvNuR0}2otjNe4o@Ps#g7mNT4S>K$!6!F`J}@46L=3+&aYM{nQ-FPFZ<^Voh(Kg3PMDBbkP14t zuwHd}d+sShOT0R{O%YjO=o2*y*^o|c>4qB<_$6Zpqu6*GrS=Dq4!|iq81o#CR9IPI zTX_1EZE)>N&K!AGSK{ljbt`p^^BtS}+(pDxuQ|;bj0u``pi5ABWs=w$da5dqea?>_ zhHOnwXpfgdzwVB$^-u4e0;e21`clhjcW0djbv&oL*+k1%(ZO7xkuOahY@>Bc$zyC& zMV_1T`JTg=BDS+dejX<&_;q_#k)8bK$gn*f=i~m&v!$Jh{Jn!#Y<6Oe8~pZtkK)eD z2hG_kD?O2G_SM62!Y+IJpM!v&o-JRnAxB-gqC1EKZePzm_xUbA5~ua7FKeq%k_nMu z@JbxKS7KbR!Fe39J)`XxxsHd4Z>Y4IY{E>+q=AFT-Z+Lpn>$+(okTR64hR!cC_cY+ zTZ)|q^D|t-0xDd!;yBUrMQOwkf0TH43||hs%unlRM;CV7lPRBDkT)4-bsWY;!_P83 zommh+u#itCYS;P!>PK_5(EYkqF`N%E*EV0G#Zk*d>WBrP!f!fz0vIJSr zJ+PHSwLuk(10XmgV=5xm5Awo+hdyj`F4k|>cLu`yp-;~sQBHh+i zPnk{DOGNn`HYLhM6yCCXY^Gg{!V^V~3?-k#rVv`Xhe5RrOP-`6=Hq!Ri|EY!>jTnH zbk8NfDsLpSjF)%1l@q);9+SL;cfP2-)dQ;AUQiOPDReVGB=fVt_UUn+u@5w+N$@;@ zij}TziR{B{R`e;4w^&4l&S3SNTVF`7rWB>S-|HWS<*`MEob|O}dYG@I#C%O;tDns# z*AqQ}AE-|AigFhzzZJIJw9p4Hm8P(^&-)#897O(;4WtLDa( zCnpD!h>3oHH4ehf_Dnl7GQ?wAtwPk}0J!JIR~{ZB?6EneH~dtuU$1^~ixvlQlwrS2 ztE6BI1;PvygWJ`9Bh3$pE77d3((q0e`O0qQdkkSzGVk8@#KyU)AS zr_WRS>cA_L@`yN(78MB@=EVcg-yO*B}+au9*5;mo)+ZheDSBZ z^n21!pH$X3+-BOefk%R@^;1?=3Dp~V^@d$7OA zYuz2Z|IPrAspcDCC?WHJQcjm6nvzM#;v>5WGa!7eO30&2U2#%fk-hC^Td;w>cSGNo zjG{Su{T9f^!)jx84IK4aPGzu-$no|wyS+&Ft|W^8rz6Rgq_$OZPNu7H|x$16NJ z3ReBH>|-+tvrzzWtQ*(+rhjPk2@QeM4iXi$?~gxDOH>;?vsO{y;%F`D>=U99G?^pr zwD~}6cVtxwi_A%KoI8ldjpQF-qY}E|c-*4)))_22nGlnt*O=Ph;R#~2PXpIEo2FI2 zG(e`n}}?(S9zYq3#Ch;9i}grJFvFoPM@mIP7K40V%pNKW;n%A*~oQIEv8GW)6G zT#n-mckBl&b%pHQbs4T6K28$N9jCzyAnH#rlq#t$O0INyZ?R*gFtX1yTYaE{Vd$?R z5*psN&nm@>UoD|~ct^UYGj})Ia z9dl(AAZyMKQ19C|v#}cG=Jo7=jHqJ~GuK4WUTigz1Q+Px`xiJ36t+ zcg|f;Eu9n$VX3lcg^kfiZN|6WGumlZSfZP%vy*ytdL@bO(6$&AKQGte1vjIucy#+PzO=aDvscZOiy$j8TQl2N!_6lLlG zAq7{OnHne-Ez87fk;HgNOa{#+J=gPvc$yl!+|$g@Zj`b3$TdZTXGU+tvB+k59>#!4 z8?YG#=^}X?rwAZWmX?7WL~%c_n6Io}Q)qg4Hk%5_%~*(dQk5}VC%f9!cASYfIuuPS zO(}26Eayd`#yh*Q~ zP_h9Bqub#alo+|sS7q$hKRug&aKt#AACq0%m}V_U5f?q$C21eF1+wp3I9yoAH?ws2 z<(QJvXTO60L|&zezbi9v)3QFTn;Bu9A@&*8?S`#X83oIGydzRsK{0<&?=O`-F*NlW z1d*z0s8yk;W!fxhp8}H8>bEjk#)9ssV(1>1G_bq8v|T*B9!T2S`Pn>^S5;jQGf1`y zvvcB6#=AK#wYZypnafesRFFuWWFu24CEv-_JYp;c-;}TpQKpo!cC? zg3)}swN(=j1^7ptHw}k<*P^s|!l@UR6uk`f%9PSK;WY!ZOO=QKOkxgSxBVQM+Hk(PozE z7ckq5%B;$Zi{K5lAvJ*u^boxE;mnhl4Z4ihS>|yXo$!J7Z`7jE*O}tc!ZXdk0bk+sTaPc*L>>ilZGiWsekFb( z`h)n)!(GPHx}qrA@2qVLERDb1mfFR$M5UeKF>**JdL_hm@$hHI_sZouE*kx*3ND&a z161W3PUjqvmj}B1>5`f3w`Ta3VFs-PRUMEUHmXXWke${7-^mn$x8s?<-@X$4g4|9a zN7D(bVZcF43a@uZ?RS2oD*COq zKSF`x zhj_tX{@?|`_CM~~|C_?_w=MgHvhc ztg^D|$6O2&+v(Ueh$$EIwP#poXL67UmxHPE^ST3AC-g~Z)I4x0v+v+?{y|hDZBtGWm7M_6?-3Q0o$$uFw=@uxVq>a;@VeeaR8ueHH*NU3JZ;d zYxjZ=r>ferEC<|7o#mW26RplZ7oMZEZyFlPyC((Q2j3uH0i&$7X-;QgAMVDSczKr4 z)s)el?&wcXa+aHnY&UFeRC)GAzx0zAvYn!3*gDv~)bs9SBJhNEy9|8N5_2Rmxj#pi z54Che^jCC6*KqI@>?WI5xSPxesNo6PBSood>dY6=&rgbitJGNCJ)4#w_2ctH@{@;O z{B3KB3mJv()XeuvpR?;3sZ&If+;h7(=4W78jP#RjLm~5by;=|QDS67R`;qrHL zyX%Dxw?YmQP8xk}(=(}TMo+VWgraR)hb6-Gp34Ev>B}FX#ujT_#+_X>91d$mmVnyj z_9mTE#M0S8twkSOu&1{-cBAAhW>duU@o|ZvheWM=E9*4h<{P%W$^#@E5W;5`@)W?r)YsdMd<8D>RWG zxhW+7h>u^0vu0yBN=QW$6eYlkP0D=MwOTt~++=2~?Y4G=d9{Apb%uoKxO&J>!e<%0 zy6EE($@&#ifQ4(^RCWy-tkj^!qJOqb6j zuoO1DLFou&^Cl?=Tz8fnjkN{o)x>b4=FGaU@fH6`coB7OQB zSwuZHx^H-dy;$a=h)435(4HxpiXfzcYN9+D&o72HTFuIIBN2=>?Eh|= z99z9v?UqjkbWb=>Tb}8ft~wgPM)*%<@QT>YuOPEkDdcd$=X2=m0rLI9w+>IS( z^cD~M+5>jV+Qxj9i_1j@<1Df)V-&|Lo641yJd~7j_?h!$_MNU_I+)~H%EAh3>y^xN zcvQaH+Sxs|yr(@KnJ;NX5o@;AKbjF!Ztx^0BejSlHc#3ptjkhXu|!#^I5|1>u6CFd zkCFLWiDC)jvvN*GzUNc>4Aw#A@o>xg*93ETA!dAZ8cxr0O#K)zWbq2(d(~reP*{hl zT>AX`piLf2+)-s&dtgR6YnaqvSy{y(9Z-o!mM1%7=rnkMcQ7-?s52V5^n~g;d)%(P zHhcP9&I*oLi{Nn}wAq$|cRf0CR6;^ml+(QXGz^RJDFT$iCCt-iP@JR%yeTx<@3Mp- zJ!0^YZO5N3KW$wGP1vg|GlTiL&bC6FHwqiqu#wJVt##F&N?Kd6_e*%w(!GLJYo9Ettdg|^CRQY5QhWmoy`{N z9No&T#$rl842+h>HG5kA*?s*JS8Fy>)oF*;7WwJpWUZ~u7O%Bpzs%3V_PSWSTQ^Q` zjUK4qH0uE-nxOugL|%sZKB;oll^yUIWe(xutO~vVEwvgE9ZPr=Eh}z?1)DAxb0>lf z5OnkYlp>8IoJ=hUCvjxz@TeD^g*W(m&j#C6y|J?@*!FB>JM64w{D8v5X0P(~)!fhc z#Ar;u@#elo5 z%)qfBd(HLYU~x!`4&|3mQ71aFt-6WaDWQum*mNTuhU2&X=9eebAu#DPL@12e@L=6lbYsvf~JZX9vocrGsW z)jtodtmQUS_n6ww38;4~S@$An2e zVVzmw&2aTj&MXpKUB=pIGCYAX1vsuJI_yu~9x}#nYNN zX6S<`Zg!s9an)Z9V(6R6^C2Hvgf|&*OMkkQ$8; z8;`_w6q=nr^xY(zd3~7sxu=uOWp90k4~fdYlNd6uy9cm6jUaLi&}BzP5E(5EcOLnG zsKDDpjfcy!Aw7>+k>#}~I-Q0Q7=drJe8v48=oKB{0kyfi(p&Bix;dTAKk(Pykt?4| z;v0>z$u!tGzMW_xtA4i9oOAkEM7N!uvb5}3F^_GrY?T3!5WUS&=gWeUm87z6bXGf< z?@QENIEqb47x}o|-GkN#1-bWQ>{@GU@>5*TL{yIOsg9n<*=-Nqd64*4$*MDbf*^jV zuPz)zx=^be!#`F!<#;WMG3QvMX!Vk;+*E$4EWgAqgjRT*9seKvHdLM&ChM>*y(B<(f=UV>dOYgE?o}zSz~a zT7QYa7Ep%Xzzqi4VY`i*(L9EG-T`M=DJ%kaC;5ekvkT-~7IP1T=@9;rk&znk5>2B_ zW2*YX^mJQWX$waf2PMEHSQ-*Zq#0=-sx|$m{bJ#`Cf~qcQHK}C_P(wco5mWJ39Jik zqF37OHOVxnn@?6R5eoP-EK99_2No_e{O(>k)ewCrgI3eeOttoHvx`aqt>AwmjHVN- ztNN;r8EvZVw1wPJ*ogjD>cT*c z@0EBp**3m?^4W^_LR|;NEi~HNuR%A=&E%qnTkIt$b_neIfBs9tw=@?&uO_2ozoIF8 z(j7#bVVjxC`4&;rwS)slMGbso@kd%Hev7;ycHy9RoiH%boE=~)*U+Fj7u|N@UQvAqESv*bzV)hgc;fDlHl5jAqFMiz z=wE(nB7ai(nIHi8`v9E6Si=1WM$nvP{jQSCrodnz;|vE|)cZ6+M#4$H8eZBtN)i^v zWyZC7Tn>`u>>p89O6=wxETyM>&O;p(G-{#E8Eau>&bTt^ID}WLB-`fe8ypOhWgLvo z%kFtvJ5H&T9aDWw-UX>vR^*fl z)rtRX?|H$jzhk5$8)`KFlR^tW53eDo5f>K=zag(a3%8y=$}Cln+ki#ikc-2B&k&%e z$7OV3oBB-`?DwOMf9i_;RiWiyxc#ZnGE%*=hV;9PmswvO(-ZV!(OlatckZYFSy*Q& z>+eRBE6bnEZeH+M?Tpv~vmm8@F&3AG6Dz|;ZrHiTrb(%S!Qz9~(S=6Kich-sGJWHS zr(j*PVpAu^dJ5ocKfX7)aR)nXS{XGP$w#`CMcNOtatv1mWU~FYDy2m~(=+UB8vXgcZN#K=LgP+bHW#)%!M4vt$fI_tIK)GXuA-QqBTc2W zMXY5vB!4&mLv4( zy6*P1-IE1Oa>tClhHTXA)zrJ(D-q=46R?Gky?<%)cR83SbcNxc|&g0mpmf{A-TjjD4=UnF5;O9e_}B>ryQ8Ew$&f{)S@&*PSVf zn+9c&&v&o6ZC?3u*M5`4Me7!88tF92ZQj_-N}sn?n_?s47g~M@Vp!9i-sk;K z>00DpkC4`?VDs|Iru>tpi}L)#qM^x*Gk`FTM#7^c$v}c zw5VMvesG(NkR!jG_ZdsUS3L+OXJu0kIXAK`NspTM9|9Vj$~% z`G^)4yP6|cMPh9g(32}rF=0V>{mN{udZ48Q#n}f}AT(p{`PmfVftvfROLr<%=iKS8 z7-rEuRf?Q{!C)vI9O+1>-LVpSTQKGgmm`H~xzR12h!rJ6dLBCSCsOw~teC2;C{5|e zV~nJ#&D297yzdQSGLmDfy?2dtLZfKwKU(ln&%3dP;OmkLQd5CKnaw@Qi1LmP%MT`6 z@fk#xVKN^rt(e`r;5DoD+OB7Joz})%7P-)KtJ+K8TcQk1TSK%$uJxlT8g7RRgI(=g zd6`W02~K%6Z(BvO8Pw`O6+w=WBUY=(65KySy119*od!_f0%?C&!}@=So=`l=N?^b+WET|n!pDNSKsUNiT4>e^}{~gqnEt2Sec!r zMY0euajPl2)mHP=y9^O9{KW8mZ!ku_8RiY@njPy+P^Hq>8+iOx2fS@C^O8-Km77R+ z5hdRGv$G@5`mIgaN$qCoqE~QWOw+A6+B&A1WhdHm&Bc#}oUWSGLsuHw08jBTATX6& zRd{FC@yPDlVytzRjt=;V+0oQ-mtd|nww;??e?MTc-*Nw`iE7g^A`mt)vx^;SL$gTqqt0TXtX|;Ou7R!UY zmC>@JI(PZA?!^u#SdRRr6b6(g7ue57ZVmc!&wfxa5&F{F zW};5eI)S6+u%Bc$I4egU43}8Mw{8arW>`mS6V+^D_qaPD;sry%uqOJ zSeUdq%@nec1V8@#H6#R2?r>M1nu)9exweC0ykCl$sU|b-+!kt|k#{({YuJBo*in$b zaoVTBFzvim@41>Xo5t`4yGhr2qS}*Bzw0$Q>XZ?voO#_&AX$to<%;htVB7^ljUALNyjZjhds*~MXfSlI0E{VMRjxm5P(xM7Jx)mYnhJO%$ z)=I0pvyX0#1Jkze;_#=}tvN!kh!Ayw)K}5NcedP5Uf;@c>4!lHrI9B))3A7T&$hjF z*wI6)-c=$mB^u z;iR%X8-x#oDC8_t;=8X;?JgshMQQlXMC+^J#chu==GR#6!j*k9ea`nf^Qnp28LoMS zAgk@<#RuZ}^X!Z^OF>H2c~uW4$7?3Eg0hg!jY{Ly z{i`q;`SROQFeyN(Q8gJ4sHdltXTQ_hQZf_BnTO4V9b%(-XCn=yWIs+W%VNyU#y2CW z*FH{K*42~lrm0Cmy0~Oap$X+0^%bAY8}*gRPv?iSS0q89>6yvdT$AsE37sHSMv@HetM9PROBthesR?~&pTYf`iLaypxuC(6&% zD}$%Dv0haJK71F#i+>Z@)g7ebL`y^$(>8l>-s+>4Po*O~CW8l_HwT=9P6V-YC1U+& zb0#xen6O0&CyBfy`kO-96qL%(v-T#K3gqZx{AUAhA+6K&Rdeo5sDHr1>XIdy&FDqM zN6ZfOBAgiBY*CtuD3icwH>Y%{jU-|!Xeq5|HtXs$_}p5XZMsZqBxkF8xm$hE?{=*- z_zBQu5VdAZj#b=}Ka@2DkeYV#|6i9^Vyv7z3 z+iU7vhuF~fySlr6)ecg~agCuT`=fA?y(}2ao4DS&U!WaJyGGSs(A9OlJC=m)ZL2XA zw@5U!xImQu9% zoj%yan!B#C_3+o%Jtrr=iL|ln9g$D{5?l0}4DlWFyABXbn9DMvTN$P$MkKsp%?d_I zV4LoD&L%phD~D4^88z$WOXisj!}l28J&hKjNal=Tf8eB&Z=J<8&J!NXbyPD|>1I~y z^OUKiN(SUx&}JxzIqV2WbVG<$eZrQsNk#E`1ZF>J)M}&W+V_53oz3q3xKdOIeEJ}u zz}d5qD|WpLcd__Uj@{BnNKbvfNHunhP)XjstON~2Jc(*LqtfWW=YZoF1Six8lCR8u z@ODXjYrn7%g868zKhrmRV7aMKFl$;_kuhrnSf!w74HABvQBJETFCyloJeJ5Usxjx0 z>06c^`+WAH0@&CvyP1 zP2Vi_oSp(-v-I<=z@>5qr!fOTMg+m#w&s(0HDtn=jf=v$g35!f5hj`2i~1zlXC%GN zh_MpgHC%apkeFJ24=#1>v))VfIJr^@8@ToL7L zM{?Zb`E05!c+QL;PC)B_Vy-hMp=a!lm6%8ZrgM~GPH61QSO2nKJ`iJ4>apmVC1!t^1Sc;C$V%07XBi!N<)dO6w^f5Y3a;Ddgg;uXVBWIOW# zn4ZHM?RJBcjK(<1PsL;vFnc}q4A$o;apLw&cf9q@B!T%*>V zEj|%jYVulaMx)C4Lqy-fhv!V)?BxK+?18QCGzq5#-)s`jj-@h3lP}Sjw63XC-eX^m z+BO_O?SuBhO}054H)(RbxQOafVGb?4Y7MIV6dwUZk%SW=ed+zqPFY(Vf(?EuF zjzd*`#BIFYv&atIvxpU8I)*8H39E!Mr$r}L3sG3nB8z!XZ~DjyN&8Mv`?(jh9xo}@ zuQqGU4t(gtuV!Ouwv^Q-?J{L^pJWmeQw-cR=+tfI*y5uXV#xv*TXZ&RTkL>@$Xl2_ z6`9NC`Z-XwSgzvh0m%Eyw*kX_5SdCHwyfoT2^R9>{6^BVcNp)+P;EPw2TK^I7-o4Zarj5~dEs4F*c zOD|>luI_S5aDsb2Kv2EnaB?^}IHKxO$PzB5tCRNh@zm7H>I>pzYllge5VF1XK0Gt5+^pSF=KF z7KSpo_31qIx)W{O{TK~E2clXrjpgo@*P8^_W{rp|di&BW&W~YJS`f)I55HLMV#ry& z5*}^3^KDkSYQ!rtJ)M1h2a)P<9ia&g_Xdimk=jl+g@Ak5-6{%i+Y0U9$ zQg!*-y`6#{l3iI!@-7InMM*(a%+5tVlZ1Ufa^I(+FP-%WWTUNi&-Tciiqf3@yw2ot zM>L}y#GU(Y3_(E${?o*i>K4o1eG@6}1XkS_FUk0&FoHi*u-eStAQ`DCRG&>*Z`VqY zI+0(UoSvMXj$L2Ng~Eh~G0OV$X};huUZogDNbFI}50pgF^ zCwqH~(i|jP`!-UrRh>Ycy5l#1&z(2?8JOPztAeowk6l@H z1I?v7Qd0SIqJMaj&>MCBaj5$D*iV5adNqHxM|Fnm9OVB)-tR#(D#mT!NNnfKgRh`+ ze(D(Ynb$@|)=V7fJD_b#m0SFo;H%UQUC3{+=)aQwBf^c@RNtXm7FBd^ zqxs&!O`TYY3wXH(CH^q|nc*+KGmi%EcXt00@#~R_KDQP6^U>K-^S`D4ne=adO=lj> z@0FVQ`fVWmE#0R?qi;QJSj&@zJG_5jyjRtY+Awv$*2)0i8g~(g)s|(JJ7o4xLJ};V zuueu;nRJ2J%vKT-P0d#z(_^7>6$YAQMx^&towWDvs3w-=o6oevtCD@o&`S#1adDPT zPK*Em-U@Eso`msQk?Gj@_)RNIosqtY();5>wMpqbQKrCwzHsfzqz7S}?L0l!-DSKi zokmdTl+fPMa2J+W6q4z1dpXw|TY%LJDSENN{)LnNUYlzuwDiaMa^H_pz0h*{mt$0a z{ds1EH#CO?)%WjEs&?eRQTTpP9jlFt4`#0875Fm)LcMK0w%7WXf&c575ugQk< zny={J(67I&U1qD0X*YDLnnEOu`YVsah(ObUH79zn2l}TFoK*X+<{Q}#5n?$Fe2E!YiW{joX)WMF`-smg4rNoG zA<+9h;kDzOAsKtKCO3n{giS9mhkHPIjYxUj{n)l}JGV->$YC-tUyOmsr zANVl%1J`5V8YR}%&><}KLswi@m~~5m#>uv!VtOuTaArZLoCDg#zGlrT zp65z@{th5m4a^gU8r)#W?J5>r;cqDbrviA^ypQ(=V=Wn=pT@8?19yhszzqlDT*3e0 z%wN3U#u+Qf(9-6AJoYyqW!d-ph{ye%&Pxa9?_?NIC!qh!u>U@B{z~TNU&*lQSzCkb zL3S^HF!p~N=EvLGu zX1KWip(%c?&>c;EZNaD&dJXk<1^nmoe*Pck?=A7C0{^8E%G5fn`X3VGUcL1AEy)~u z%}b(mT$R9Z>IdRt3T5r+q$;Gt2wBcep^rNwW?8%V7B4#R3-ubjsLz`)3k^QYO(-zP z=>a535uI`NJ)IN$PIhx~(?to7=)_rUB)-fHPetT?78%0@rZA-$t6Ac8*3^?U-gUlz z^X}SB66d5BW0Z$8F}EM%nI|Pms8jmTWVbZSI6s4by7Xub!*PDY?DLAZve=CV=v*ot zmLQgLlEl5s-4-;@ab++#wjGSGaEr-?@p^}|4^hXw>Z)KHO1!F>W%BTTdCel>W%}#1 zYe;9KWr9bRV*`bPZB1#qWx`YspG8cvZ%aB@HJX(Q+xaUN3j(nlo{$#kI)+&=a`Vz2 zQ#>GTLAb|TsdHBnVwvDr1xpiBNj@;RORmL3AV3a5#wo~^dJam46EJE zpa4mkNqpV%LDaUOhhxjU-1O(pwqXmH!}keKE->Q8&=~W2jyVyS^8DQchc65kJ6@yG zE!*hr4;gexr7JZr36tYo?sRQ$&%(Q6(r04cCKr{y+TK?vHxp}X|E4kls<9q*ZNwTc=8b3IPL7UeUon5FZV*)mvcultW} zNfGS3>)XW=><>SCS!@tFn>+AiqV7Wo|rujzxm*xA{Y&$s8-(HJJN;Ab* z!3%@a*{+8~3`3=|5SgD6XFi|L_Sv`KUUviFCdbNKt5t1`d0Hp5mRQHZR@Njp>FG3i zQeFgH_N;^6Hln#>N*MIIFMPoN%IuUVqc}MY)9q+}adcuFW}{Ss%N1g)lIRu4k%mcS0iWOL$67@akasOKcN+^ry{=FMCf{3K#$PRzyB~ae$(Ng2 zD6ha`L?-v{aP&f^lEyXiKxS~ulRH`(8;&b(3oC)tZ^|dW0xA`+I$yQwk|vBZ$t%Td zDjup=)pCM@#wsN1gn8%UO4U?&8d0Mwq37G!p9j;qi(R1j$UP z?E&%DjQ#O>FB3oE&s*If{5q2~6p?8K+(-8k9Jf}9H)~js^y-V_P{b)NA-vKlmE^!2 zgaP}qeQY&-LJPDw9K{^Ne|T)#q5U`&4kf(e{S`8C`%05X%tM52c;Pg->q-`plV+oD zJUHTm*PV~0wyXs=-L`C_GwK?=y8LfGXt}Xat~JwXqF(ii))6?>b4=3$34PpS+m(@k zt=&?qeJPuO$Kd=r)sk(9TQtp?^WF&zHlDa#A)m;j>ydbtQ=3n1SU(o{Xjv)Kt=FY; zj_uD=Pcumx{PQI&^iSjX?^Ewj1NrBx=l}0JSjuuv`_G(-XDpvySTJ2&|LybOrzXYk zOTZsf?0;qLSMe900KXS6_+J|4KYRoHQTV@n2mD-#eieSPAQntg`2Md8;*YxjwjiQH zcsWp~o&Vk%KNrM*QBe5F#=m_ciAk4@T~^tlV`VBta094kp)sngzgWI#uhIVg`dg^3 zy14$^K>COF`g?Evmj)BO78snr!&vQj%o#^3W?X!hJ^BW;TK0cfy9%(VyQL3Ew}6zi zl+q2-A>G{wNJ@8?q@;9%5(ifdVcmhg#3^3D9fQR}I3#*@*>95lL2PWe36WJOU9*PVB~0=Qe>rt7|r6t3l$>5&ST%i-p8jYv(WSOzPcZwm9YGIIs*aGO5; z(3!L~i+;45u!{+uJcw2hPJH%aI(Lp-xFpABANy!97neVN((`@?MS1tTh(LXenMYn# zy`<1|N$;Erpm!_7OFUMi)TQwl33HOMQ8PWBvC>(HgfLj~L+bY8oW2iw8wU2ICBe@N zfkiwAG7QQOwY?s^2ZOBhn86B}&4v%nH>%rcjSwa=hmr^b<-D1U)FnTjtu5qVxpQ>*T?iF8@Zu6H#F2${q$S-M0iwvjsU>~jN^rMbfm zDs)F@eBjbKoVE<5d@z?n;_hP)e_SDdNHpyH-esfuu$@*B4X@>qET;B)jOTOy*i1V} zXC2dNlnlrdL=E6A$Q%Y*L6M)5)5m$(~GMzz9cni9k zTfZt=5L8bTk*L*}0;aG%asA5>R^|2mLPo6-+*r6X`b|9I)UEu1%Etqp!LNu$E!K+d_66>yz?UXbJ^^M{HlZzh~X*gc2C6s`2Uq1h&;J?$Q zZx-=2d9xBdnEw z%ea4x`4!Xs)d>B68F#-g=F=^KL+7jLp86Ch&;vty`Q-b%dG6{|!vgHhFL%F}Tk@~7 z%zw$d8i`wyGJ1mbD*_g4yQ7ewXPED{9JMzN9s?2#>?x2f@ux^ZmfT+|Qh)#jK(YY) zbWq0r)~KsWS&j>%mZ%Xe6Z6TXw>%YrxJQRc^Z1S6%JKj|5~=9@QFHINM9LFS-t0uS z)5bl<7hKM2PIXx(ex3@y(8JqP*Za_W>Lce}iPcA}IVcLTh_G{;)d^*s#g?%igRWH% zxrlGweo2ZgAo|J1Led`p(I^8Mq=39_cL4=Qi=Fj)e7adcEYkP@l*oQk4#0iv+|=`>hC<;Wg(JJKq&ioo?$ z@9>c1WZey1R^r2dMTDw7U^xx zw_A=H+A<#rgc5|aQZte|wWZ$#79$W{e?&UU>D|O-R!P^yS@)^z^puW}*W<8ZSmff< z{*p=^4q+;G6IOn<`|D}D!;R=>0i~zA(Eyfm* zx+GPDv>)w`j@x-#S=w#A(OHqm+B>m=_pn2@q8mY#Hm;;uSNw&GHN~ z1k_Zp`Gr(YS|wv$<$?$qyGAu5EiA|csQSI2eA5;m75lOXV2VD8pRejF>V>iL$5MND z&R8e!je%&{NQ>iBOL(mA)qZ9(ST03ZOfCNm_1N73YCuLH^2QK8-RMH~Zg6j^Q%ho2z>lSM6r^x(3VNPB=e1v-_V-I5@ova+lD)8_L}c=U_j; zP!a+k{lQTF)oIG_4P~e0yb#8gFqLnQkFPH~ku=Y+iCJ?{lztq^osHZ)kGHx?YUrmN zDH|j6_%Z?#H-ZoZ1m@bjdA-)AIIKo^!Qp~#m2ulHzTaaflukw&Ho?t#Y&5l1Go9qT z=Q^Lfug#t7F7EF$jJ^dguv=ySymcJ=%`5V^^Y4ZaO9FjY_EeHjF;aKwGj};e=g`HP zip3#Do1_%4Bzk-t6U ztpp7&J)|_8A8Au7vFU}xcUi=&VqNJ|2;J)bCXO1t=&l_b=3LwkjT%($E}RJC zN%gm(7^IHib<^e*$@|=aj7SLF6y~g_IK5n5E#6LuJ}(@7F!o-LyK{`$dLHkD9)76v ztm8e|E*R?Gni}*tsXm;dFl;c!XMTCD;niSjf)sjkSnm*00g?+7QR)<22;UK2Un#W} zZuaQF&5~(?PFa{mdJ91|>PaijV%;&}1~Xjg2G()>Vx{J_*){N%vD4eVVTLnPn03r9DS}rjyM-ga`7D}p>!23qx9qj5ORihb% zyPB50k9a~gg$>!{JC`02XnQ;cQu(UsK|ZN#4QEu_x<_syijz^?WX)D~-Q;3fA^i28 znN_8(izs(Jy7`-B$xt3ICbvnV4w_vm4ACwtT8DbnK8@t0(pHi>$?>I?+wELrgj{Ut zY%^72qKdjSPI6JjZQZ=m9uIKS{9SrgeBu!EAKp;)66u9NlE+ix&Uu)@kvLObkxX>! zSsiJoI3z_aqmT4$AAcr;kya+Vf`QJVte~|#LPf)}z?BVf&Xh+{kuy8szCm}zLmcsK zCo=EesW%d?@)Iy-k!*i`mr_!ORy!F#C6&&h88jQp4%lyZU%0H#cK#xlP zSL^M#TPp-t4YSjokK2NPVGaaRA^pKH|FcH{UBR-k(6cjLIo0{SVdl1+7e*bt?)_vg z5i9@<&I)Nf{*7o5T7DlEVV74Zr>cWI9-`qr8norfFR*2b z<7C3`G69y3KR8-QA=bQ})N!gKX>ycJq6Xj1Dcen~$!D$56$$*L#=fAf3U(m%&fCRz z3P*zxkgex8ySU*CoKR(*p(CAyB3F#@uCp=B>b*d(u7y5#S&-wl$FoAg#{Gy9kCj(* zbi@>X{d%N$TLZ$Unsl^}kuBIXOb}axR_{^SDKH{nVVZqI!6rKh(TQ8|k<%X$T~sGu zaLL}jmBz@vqpT2^l|GBw+~2_y!O;6W+{Ki)DqeFV52)z)K66ZBChA=C-L8vc7gR{` z;HX-l6-MTsiY94{_4`)=9teo5dUUs_*=9N4BWIB=|b=^edv zn=tixi4L2b-8L>8uz)R9o{GK}OL_-lYFPa!iXK{y1U?3xLvNxcC)l&m5Br|YhwSV) z+pZC>ayI)M(q{wDdF#bgi(POCJzz+h=BV^cDB1X?KYAeElN!ag=hG8yGd61MQjS?^ z8j>a3wwYiN((HCuTx-j=(ApP)lzDJrJgk7ARhJcMEq(MZ{oThl zrWL6s8XIAT5O3a;nyZa@9ZPi@J0gxVQdjkpsVkfuj6&F_t6bwS?0*xIJ<7VCZQy{? z85Slb$%e;lX5yd!0_xg5Io#uueYA^3XI9*{xugE}rhO5yt&7Qa=*T*#7s0Iw0r>sK zB~qQNTsjX=Q@n+#r+dsG3=t4B?m@1xlQdtp{c%Fj$!YSyr1+IRg-WC?l`6LZJxIY#%9^oqn3VxjbWMCPqMB^@+j zS(Nv8ugUc6xCDbEnk>^7vG?T=5F%~w%QClZOrhO^t&#GOk2=tPunrl`P$@fg3WM+j zb~~cdSL*@F5<)iAzzr^CiuSI)=Q1;h-D|!a_dNU{T5)&+wdj2puop<(vs>uUn*;Xc z0b7cq+5Eh0hQpe#@16Q?Br@$CUQf=`Si{Pj`_BrntKN)V+ENL?mZITF-K#Nr3fR(e zHR*`!4Af%9nDq$-A?aE{>AKxG1Rrqj4zKj~JT08hnii!V+pa#>%X12S6ghJU?hA)c z)>IS3P8%s5K#Uau`%Zg&b%Y97@IK5K7mMyUb4<#og&)~#cRZip3tnX1{~U+(eN`!e z$FQLe@pm4tpIFsz>)qcPEx<{Q&VWIq_WD69u9|?}^?L&AMFO{>NQ`|VvtwQqmE%l{ zO?+h7brMEW;_*RpJ%%5kn??8X2Ap0UM)R>T+%J-|&%O8;=RBZ3;V=M!N zdAzQ8>I<4I2%4~HsH3eIPCl`Q42bFt1JbGxgaaOqNq9T=d<09`OXj*oClG)dSr$%eB z=;3LNGeLK)b!A4LQAu2c@nE3OZTNCmPl*mLp*~XFYgxB*(BaK>WhLNS-Q=d&0r83qDuLAf)~h}D_wYt2PBZHyH8EpiYfwtJg98*C zGp3+aUpHZ&Zi+0PUMqF>vyz)t)`y_Pfxb^?O$^Bhsev_^QO6XBF zIEYsg|Bd5T`A?49lHS)g;JoXl)oBOZK}=*{W%#jc8H7CX7a8s+IHKqNxP8`Hz)R>^ z$5X3cqf&fYP*PTsv999EubYS+ZG&39kJOLU<_^Q|B5K@KFQk3it! z445$?nbZz8d%#T&BDi<0OT`CR-iCK}uRRW!YCuD)fAoT?x%jicEJH-9Bt-|2Q)v*$ zlMeXHFuwT9SOI@o#MY(1?3})RdY?QeT5R2I`DUgnrYk|a4d6P0Lg!=Jp6X15gh^)7 z_Idi^pkHc;FMBW0UPEmoPU{7!6BUYpnp3UHl;|UBu@%_e+JRa^VYpqK?AQAOhm^Jj zz={cH>0C9$M7-z?b{2BEia}Op_5d=^oUUS#%k8yVk<^yCq$cPms$kqZ=aA324RjS@ z&?MW9^&6!L@AUiS>n6NCn(w+k=dvlDPbD*1y-WouWA16$pM(5p`M%c^c7%5;LG`LUFbu@` zP}Z)w&ki$xpw|-t=(F_yc9_Y1c9``UDs=iu&$U1XHntF-%0%c9I972sw3X#^ew^=L zDkuo^yg2)`bU?RgAt9%|Y+qEY#a7z7sgNZvmzq@^({`M6fss{C==oT8v4t#qMIA|| z7_nhUauTA`LN(QJywqSUdyAtqW02}`?(hM={IHbXb-mU1Z=!u%-zz0xiIm;E%OI&$ z9Jur_d)kV5WGBcFd=(l^&+FRJ+;SE9Xvv1sTnKH<($e#dM?<6Mndd*K=^`%rtFP{r zKZQ2^6XXcG=li$U;{WSo$u54_mk;@&c3yL=)0)H=;;IQ~svwqp3hYq%Zv1hF`i}_! zK?{8HW8A@KTkwmRgA&!xM~5=*s>Fpbe1X@V$ArIdm=7m)2-B*3_99Pfj22J15!bb8 z#mqb$2RJ&6p_+TI;~i0MEj;&_-B#Ut&i|OsNq?y!KZWkOSkll=_4L;Hht3YyZs*y$ zQ3W1>TV4Cq<)jQBJ1dnMrNjqn+r-|oD<2I)Hty~jg#e4WBKGV#j_a_uyvZ}v9z&SC zJLVIyau?fgC(d?}#$%Esp+~$OWR-oWGcApy@E+<$G}+QRl{6Fwy6A)ZYOqfEml=h`|#_&EIoWdV>=MwjZq>bmI-)fj1Ef+tGtdm|@0w>%>#cX6(D)nqTXp ztQkUgs3a-C9O63`x4q+G@@~$?aUnV6@!)7e@B{3nojlq|T!y!_0UqKceHf>A{5FIW z?_x=&Q^F8VnTc7w=a@&rHgr8-q&?z$wdac@y;n1B_l$2F5m`XNsZiPJ8F(q_1IXhE zN(p0$Y7D=FM&ybYWiwpjOB9SNIllg;9o z0Z07gdQkiKunjX46_i~LSccv08NTLhIKGZ;-p`6xY~=W|G*9q)KT<({Lbl-%Mu}cJr8vUWG_SWO zS&DdGR)^<(#a6m#!XpAvafw>^KHfukw%zWa6$hz?BB@RWs~4lX!^5%^%*-&?`$_zD zi3ydS6%!agCA}55z$I!&Al&wBxjcoIK@f1`_aQNlqY$mvhL(7#LOY0dALXMy+q31k zyQa&{$#}I04o)%-MPU zJ9vcWh%VvjwJ!xKS|A_ykHXOw8LyW{I&sfe1ESELK>p++&$wdioAFvcjMMv;jCy?eIc~!#}sU8T_lWj*>XZli|dD7 z0xh|$>EMsMf|H7&`pIAKxEFa9lRw}vb9?>}M$l5hokLtBw@9ekd+3h!J%%SWc@#;H z?ik7SZpwd%HGq(WpvxSFbX5!44RsACBTP`j5-LbTDd~?ap$$o|_K=rZuvQ$q1p-9=QMCfH!t)SG1_xY80Zv7WTkWZbc z-O_T=75Vt;-B=OdGD-O24fB;zs3r;d4xu&F61D8VyM9ktpv>91>NU2PqM}@VccYQm zjYEtVvIUZGnw4C7xbfF_iF=|LUc;6om|LF1tcH(rC9EK?D_HcuH5$x9$`f;LB5PV%iT>at;(=CoMqD9jyl%M6 z`M@}M1Lp2z&rk&#yefo+6aucX#O+K!KkDJ`whnLKB?^X3@(Rl;IrEvCTG-O<{g*At zkEF%&VJj`|H}MIcX%fS8T4px-I+_RbV#*79tH&3<>AMkvR9{t^r6-glQg=TSbt+yRTU^(US%K&YKXYJ)pXrhpm#3w+2Hj3-SBelh&=DIpMg_<>3)TWe4w`N@=O`vO8l;ZkXPlxuW6j{PS6&V9UVB~?pD zDQDNN?k<+j%yj>H+43d68Dt^A@b9DQ|L?K|-g8+SY_Wz%>|mj$@}Ot!yKY_vAEG?* z&(Q%ce}(~9F#izd2P{&4ysvTfXzCYnhmA^~BSNx*G{>}q@q(e~HuxV1iANjgj4{Wc zgF6@qLBKPk`jT;Aun||RZ_0;VKaPJG;oAEt<5S4gxMz==d^B9M{0#~d=j$wlTb2g+GdyCB9NH;XK_a2!lSLxy)hgyG-eqiDmb#$`>k?#< zP1LYZlb6IiQpO0Q_>Mj9@dwkUo}w7-JN3I{AM=z``)}9}W$QmwuhZ--xO?-ynNlx$ z>^lynR{XKCfGyK0{?z!`oT4xmi=;H|e$V?V<6?TG*dl~7Z+zezb8j8o&g%lxe}~W1 zYV6&0LrNk-U0fUb_Cx~hjHWIZ4Nouo2hMKrYN1soSw+G?cMXpMpUvEMc;A-=1`6oy zF}2N=MGmRs;(Y>lRJAR(7N!sD_i<+@?NNtxHMzudqIvsJreH^)QF`emynJz^jLDy& z<-$f~qPjJb1d2jH1WGz32t$fMI}5#%AgJ+uZ8j`zobCs!6go;A$j_qfmlu)uHbpWv zVr{Y+s^Y$hxbYq52308jM-sSdL35eWGpggwL%yjB{KYkNz0KTmp_ynm2MxU56=%8% z8?*o+?LN=>RHaEaT2;dm7D#k;1x)V+z4Cu;3f%!7;Cr(Wt}SNQpq9z$pV3N9zCc7sB#0S89ni*pX{c6 z!wO}L3naavBL*$ zWorkMy2D4v;nm(IwH#Jw^t=8Zp7&kBxT|>g-8s{(haFC@IWAXyva%B0*qlpw03)(h zj)1s6y4gcNO|Txcpo}9@Ml=U^o&{-;x)}ge-?i7e;4eFRiyxon$&%DTkV-2}5spPM z-`_j%BDJozMrXV^s&VwF=_B#Y@&4WhLiI393oq*I`81Ijijxh_hfxRp{X1p@;!Gjt zt1cqfx;GBEU6+LmC&OX9rOro60x+rx$}Pg7s|K_lW(`hai5WSS5nBvQqU2<4&R*od z(#376SzY?j`{{G6;rFxVHBY-BZD1SH131cl8hrgd)axI82C+Vu8xvp#{wF5wx2F-R zHQ_+;Lbtlgv#1E=0AHbbWRsN86m}i_Dm=X`jSUmtboTt0K}N|PK=i^*nRgzPoh+h? zDsr;2y}k0p-sy8Xd}mM?fr*i94m`;tNlqzJnUseeu60LAY?f25QBhNC_!~I5aN400 z;5)tD2b@|{zCv58KINMC-DOZB3Z{6M@DyxX+|%0hNv;c{NdY%6D#U@{g&XyIUWzb+ zl6efTyBTzemgU}AWjg9iN?jKuC5?%OSvr&8C*z0`6(&kRb;^PfgnrBW4F|lqFx~t)U0@*5aZx36#6v|HE;^FY zE%rgCxf5tFReI0)icK1^)|5-f$J-;bedDZ!u|^x}P~ zK^yy#fQ%`WkwxT2QjM>%dbi~5od{TrX}IXFd5fdP^4+x~&pLDW^Nr@JrmE>rmB%}d z8+9lp6jjQdZ;l=wxEJS`(3xRrm{;{OQwyvLXPA!*qd2oPm@#*jnvZQ5BH*xW1f*09 zc`)EVH++LwyrRm|&iYWf!Av9He{|I@w+Fu#2Q6o1FDKEwW}lk}X@O*7F5UM5vw=UXs|F2^gkUMnP^8+$*0Dc#SwhSYAH zG{S_dU2H@&&J@Z@qMQeQp1(LVkers@Qs%rH9(S;=^7AdhP>7{yVX| z(MlQ{IT~J{M4VS0=5ijFW|(Y|#jS2l)s;!|zYqX-+k<$u-XG6~o;*O}Xm`r1%Lu;T(pLjY2Qc-ui?oOYDoLK_d(987efpjXbI) zr;2W);t(KxyJn^LWRQ~4#;pxIW!SgMFH|<`sf^b9uBiG^e;girsPO$AMpdqlu=lN^ z_&Qp5+aX28t?C_wBVIsv%H5!ErxP0w-q&koCX>``&KQQ|d=ftQR$U;7V(CemN6IHp zW)7r5YGugtM^pJms;F^3kg$~8xN%9ZkmiUW_LO<}{e^gq60%^W3(f2#0@W&r6j9nTqCi*L} zs^2fW{?49#Z}G?b#GqGob#3&GO&km!{0Fywl`@(UFRk=0_@7(^ASnd?cAYmC zq_6YJJ*{vEq55->#FuN^mvvo183hHj#YoS<`UUE3;0Cq!&#%({{O6bH5-I<_zx zSh+>GP}tIv9`WR)7f?775By+qN;}1`6`CJ4IX+shVoG?{J4ifV)eM=oHRa3+uGYbJ z_Tu=gdrR^DS%puMYT?S_sQFYNE#_*c6Rgk@mSSA}`&S>jR^MmyZmkVBbG{l%nzto{ zW5<#T$BEwCh|C3C zyKGKdQ?~d(fo9vY$n87jt81&$o;XuXq`vV$3$GIIn&610zz*!Yl&2b1)~)V0IQ_3< z9n*KukDo@I&>g~=E9^Nh5RZ?Fr-Y(7`#;-2=LL z3Drc%Uhg?)Z0BqKThr&@GJ=UBit2|$fpKqlS;cWjv?IJ|50!7LLS~0YaMp>6tVP_M zT62SZ>+G=RA8_BZk-NfGb!mW+#`HZIs~cnEyMPE>i+n8nWj!})gtrDC7+Lp5=?hsL z8L74tZ0Q0d*0(bpDNo4djK^dWBzP3~%w`o!DBhPlJAfMz1o9Nw>LccSayNE|OdU#1 z8TWXEtGi4i2QH%$vUN>pb3&iu4tJDK-c~Att;uo}d8YJU{rdDkQU9X+tu8ya6^k_< zy^fZV3#^AVGRQoIb%RK0XEKXcR-@=Q>dgYFU)i|O341`#=BIR=(L0!vWsseAP+E=a z4Ocd*j>8?(uhpB}ZgIO^Kh;QoNPDPZ)HBGNRvj(qJx-t*>tQc=W~v zJ}|YQCO=J+36JOQwmy2@2cCZDgp0fEdO2Gms;4XVp;%8EC2lz=iM)#6g4J_8nJm{?J;bmB$(;bTHOLKDyU4ekuy6r}Y6G(Usw(oXBW8`;YrRS8%MAbQ<3 zqbz-l3oh>oK7USB*4!?+@=~m8L&NF?p{{a`M?q{aD( zb^bi{{=zaV(27#uX{B!M=vHWvq7jo6qZV6`cc5xh+?Ju0qi2+(fAgA=zJ#$z(aIhU zcu&u`DBjiG*+~u8+96c}wS$BdB8fh#86WUQelUS`iOg zMOfF)G7|W~n>VP5%tuG(+fQ3-mQwfX3T(UF4GmAsGG1Ud2NxR!sy$Ojm9FBD31lBt z&@!jTktD->)LLGDpOS(5A%g+JDXnuoWn0EQems&f^1zebg$h0Grw+!H`Mvjxu+44M zA1355*As=(zYKLmlAWP&MQuB({3`oSi#$PUUriet3p}Q)8JT( z3c*)~oUn5B=n0>;VsKtDCnRIjLk8DQ8*3MhoWa~OCoQs%j7j5}WwFlHm&vnGRQui7CwYXJXUtabD+7);?{1&Z5~9#p9NQda?+&`|)|GOZw%gRQdF&A8YP*~r z1WTbi7Nof~Kz({NTu}Gi?0g_af~UycuHmE9DMgL@-os9ybW$B5joP`?{WpvLf_jl(dxox6c?A5o0f3p4Hu5(^?bReT$t#??>sre8ZsdhVD zMU#rhA!_9)n3i{0$E{P#UqjmUV3IiT`qJPO`|+gFA&qN#)9J9>9cPA|MgeQ_3G&hm z^6DV9sS@0&GIbkB%SqR*MT)Uf=y=4Yz98s5A$_rRBj<;!dL2(n2}{bA$#`-f2U8mL zk3WhFCx~WJvpo=$2;!7^R%OFCtDzQo7-O^(9~ldJ z^Bl)Dm5bGia3sltwWrt*go~@?*ymX%=7d#Vo74_ORwplz#rw8uE0a0t;CpnV^xbtMdwk~kZ6b-*xs5BN{d4%#UyGx zIhYquZj8?Eg`FbvGLOf{)mR_n%Q*S1vAWA`eA1m;LBP^F33fy#IQD4B&wy8XHCc9W zc2T(!QrGFczB1;VMrf8;d~$eiv92ZIP38x`Acb0wjOzDq*4jH=yHbH!62+q0t*tT% zp>CwC*rv5Te4713h1nX45nJ-Xx?Cl)5vJKiG+XY&@CF>3Q_Qd&$wfZoWu`dKg1n)_Er@*7dQtQT0Lfe1cN$ z;@JK~Z*yS3VTRlGA7=xD3@495P=#(Ytr{vb}S;eAp;C!}IxjqO!YWmaJUd?7i8} zdOb@|d|V#YqwdM2ZuvF<`P8bkKlOLHxxd>VqC7<6@-+5%MhWfYQwx>f*yx5#bMaok za$F9#(LVR^Tqudn)NwJ>QliercrW$h1a&fow`Lx8!)-^9b0Mb)uHe&vP2Z|i9HR5E zwf!WWrPF5WMwsVjqU+j%lFnWDRHqis=c`O2c5TMm!FydJhaV4X!YE=l)Qu$YI*#L{ zJ?T7$<7^7wWkHP>rPlJ|NuKo_-yhnHKdZlo;<*`^eLT3aHSSSv6|R4vzl0g*Q8>EO za+ru;$FuJ@pNMi%`(S%`9KV-~YD#CXKnv3L!!u%~41yDN+QU}=^Ce{rZfs<74`Vm_ z0(=6y;Z&7WdqmB=Pwi&ow9O4vrW;Zp@WYldhK=g1SA3qj;;B@hzd9dXJ7j<6?2=k( zu8CEj>~@e1q(jA4Cr)z*Zo-*qOd-Ia^8#6U<6hO8P1AG|-i>uFIXuQ2iTs-%!1AN6A**BYJ!EcOIJ7@rRj{9vuyDX_N}Kh z1_jq0cB)A&`7PP0tv#3B*%1tvJsY)dwDdeFn|73EyE*=;SG zlAmX_(B8PI*sm`B&8}z4osuO}R@yW!`#^Q8PIo~%;iH7x_~e~8t96rXUHSz=SwEUM z1xPx$VVs>b({VYkv|UF&zf;j$ZuHh0UEQ&c+S;vS>N(ni=PsA;g+quSs>KQUS_*L1 zpEvW`VGNFjUq@foCb-du{RO3jYf&*R>h>||b52>Et%ug7$E#T+#EceB&#gP<6$FXy z+Nj!PCIdHBt+j8{JPzv<5^3cw#?%R>>ml+q%?S=8z{TGZqqfna7jlo0Tj9!FdMzuSWK-p8u?x#e(kJ^XUPM%Di*VBV-~DghpWy$JpP1eZPkyAs-5mXkylwZs%P{k?TD z9(uZX)G^M+m!i|e1AXC>af7&FvJp;#2+e!ur5_HdIr3|U16HB@^qfTCVoJE4*Sv%r zH!sh;QISYv%8|w0s0l=jw{{WPeC6VB3{RjRH)H*ht^L8e{z4OC+TNX zz@jNB$$L8+Ue&0`u|HHMV!rHy00VE)qTS{LzvBFU>WKak&iyt0vWS{kq5gm>-Psi# zIg_E5!Rp4Tdq=f`1(}MU&|0bZ%V?4(F+z2u4YvmB!&`-H)5N0aFpf(o%1Xr>UqRLL zSr=DH=Q^735|pcAJoo{# z%rO1*R$LuxI|uW?Ds58w9^}NX&@NH*v*)q9bBy~!UlE>_nmOP``);lt?sxT>a z`69D51AGMwbFH9KO*@+ymAqh@*Ck9nsn1w0;+bpW>s~`rf~P%!7qK)BsB^)yWsZ48`|(s%n$#O2D~ z*c#406>E8^KsJ2<-ItTNc~{r<-c7=^f~RD()mCjI5qAgkb}D>DN>%O^>LlG2MfG2WYkf<(TzbQQRE5_B4Oj00#9KdNWmw|3`~R>4bD&qFQ&^En(CqBcGA z=5v2R5E2^WH-HeNs;}P z6Z_{{a4!Ds%0M1N{7x#cblzv@5MX3;I8J;)%SXie*4;$FH@zOrm~UMEBsw7Jb3Zbv zUQGaR(^c4Kk-*pqoXF8-xhfO9Y^+m&2op-^CEm*~gMmfpvM)mIV^ymVXwBEsfbp5< zUVB?8wjvcj0S>>Li;o34Ux&#k>2H<^kwFk6uMP1o~iN4aBT%CIAG!>DgKt;1^)cfv#iT9SSkgnob#)r?!1Juv#uZ2be+6 zmG`-pnrP_4MC>KP&yVQGFaqFU2|;-)%SKM5;KmK%kHNwEz&Eag8O+%)efduF&D;-ty;tMT4-uL-%m%sV)=($!Ma2+R9=jk;JzqC$)?2PGu)#6*e#g%<2 z!(d>MLCuT6Ul`%-GGLL<0c8N-=G;-h|3WHobw&3sZ}*^>x~9GNI-Z96Jkt z|F6$~5#{5rVC7d(T}lt|aL>PUi3arX5wV*A-~yNzV@edjsxJ@9`p;y)z6DYd_& zlMCQ`h|L9s*F>-90=k3YJ-z$+v($NPcs_E`b}VD%?7Lpka+}H5mNCQOzcocErmo!z z;1m_HO9rDvi@DUhj{!W+inJV)`UKc8UYWDiP#+OdrY@0ueWb&bKZoPXo+;({`UyY) zMk(hr1Na!N^vMBvM0wtWHam=pkuT#>cI_N_5CV+v3A?&%^w=d^+{65lxtlc#8S&#X zzF_$Xy`&3J>w`@x&YYpO1rya7=oQtNs197na6|zU1;5Jz{I7{$@inkM7#1|If!PvN zzDj*MC@>c|#d39RyrNOL{t+??--ui(9<0hB%nduRFn&Mx2xvVhZ zWOruWhVZ@9R*IkQ^*pxs_G+f!SaV0}N?$;R7)4pD)o=>J++^UYwRI-+ftl&yxSTJhLv}~TGoHsRRAG6$UW)U@|{$WR5k|k>j zElHv$b$l6~wKGnA|De7XA@hDQT@U@3su?rJO_A`tgsnYS;VQFTNLEX=1hrJr-PqW~ zP6EN4r*9V^okEmc9{Fi?M-kwV@4HVe5pLkDxM+z1Ul+>m9{Y>h!F)2UObfZn{eFj}9fEy`LQBeQED|(8bJkT9_Ol;9|WH8chCM z2QJ%V1^>{$9z{1YTru*5S1bCcS zLEK4SL$CgECw(5K%Sgzd^5F|Ej!0FF#++H%r+`|k1NwLs*Fzvm{p&CRxM=?p*!SR( zuMRFVOCwr_uMV)!xR6hOi|gNK$X~_>U=+Zwc;F2vuHT#}|J~5s=!--$1H5}?fOk*j z%FxK-Ty5$vSwK(o`?2}rhy=j>a(upM5-Yy!{pIs2I4PjOEhr!UM+5W?++PgR zO@!LIc!0J zZ3BxjP_F&2kbPs@FVTG&)5{zRmn{1FxN6M6*+9^-eX+GaP4#`%Z2#Jr3121nKQLyO zLMfueU77Jyqfz=pd;f8%*{=is#c2E^5PxGdE*aZb-Gf>KSPA2QY4RelR{b{YY;?4L ziHqSls^UqNBh62nfeXfFD2;9H1KAi>B!}Kjh`A5YxqlDmh2npQ>v9rc1WG#7GqNzyGcq%mz{o0CpsP{~Atxl`H7}(jYMY3EXG4{Sw&sdEE;CIm_iBUOunR>lTJrfYJWv zzAf#oe2ZO3s8c41wSrUv5c;O`%A9<$&7x+^%o{I z=}J{|P#w7Ccm>yg!L*+$!8fM;K@l#QcD0H*D7c;m7u9oNqMfb${D(n8z zuzgq0tDO5*up1Dyzu?-{3df+hR^6`P`d=~a_jc{)i?plzSBVfXkUCMnTgHE}LzlvF zwMx{NZhqO$go04}GPi;ReEzLsd|gdmz9N6y!d&j#55m5K{aPyMcO&zK0Y65#B3+@$ z9S%bCOW7cRLP+7Tth3Y5>)o+zySBx3#6v&-#h-OnIgv0y)5Tl<=)?sNBld7kuL!LJq> z0_{FsTj5-*^Q)5iRyaS-1B)B2Ug)-MXa0MhkKDBt;T zwR8(8l-Zv_`Ob&m<_-T_J9xF=259R!@qfjLpWZ6|TN%6@3ed_H+*f2!5O9-xEid>p zHTXJ>f(pWar3Ro7IIlo>_h$&d(t~e2_@yp^`-o(G+0f;yPT=_BW#WyIw zTwZ*E^6$bKpip-I4CO~&{ChNh$&Bv`akWSWD1yMGpD^S13h|#BhO1=-KpXE*{Ym2> zhT&)G__g(K1Mo|G@Ld+JmJR?#V4e1#Bl!0S`~`%oB?3SpAg2Ew!p~IlzhT1FG60|m za{r9rQVOps1jq%@;upDmdFUM#=!;cCWQ zPz0DSe}dpUAO13b?(YQRJBq8RFF{fCzy3Xn-zUQSyVkEJD+Fylt@yXC|5h6RRt7-V z;yB-?>dSqbUg=L-|DJX3?>cui=NxF~N`6NE`vCvW@cnJyt|sRKZ5*!hCw=?3e1jN0 zK;nNaF!G;RzN?A1K%tcX49dUw0D|Nv>ygh4`Q=*VyP#jqaRZ8?zwRdt`Htdu`u?}0 zaW&@%Xyb3+{iJcw(fIoeB!8C5f9Decg<{$86DZ&L0Py~oOR~>=_>1M()#L}Djq^7D zP2*QZ0c0lr+kpU8dG>EAf4NCT>-ZCZ-(6h(9q+%)zCWJ8T;2Tpg0v2Di5VdKhl8EV zUC-4k%%EKD1|j>a3(bFq{`=Sz&d$##g_jqZ0aE|3*c6wqr2aBCuV;xtHx-z1|Eude%R*_rJOZ_sw|!k{e&l<|VqTw+a7m z5Z6J5P8VE84GoH>CP7xAZU96Mh=?mFR#7V)zzG&>7_hQ|3JPB!oS_Qbbp9N~0|0Od BrG5Ya literal 0 HcmV?d00001 diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8_parquet_log.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8_parquet_log.zip new file mode 100644 index 0000000000000000000000000000000000000000..230d2b6945a092623df81b5e0f8b52961b9fb150 GIT binary patch literal 76217 zcmb??WmH_vvi5`ow;2tdi>Qhrce+02}}V9X&Ixyc|5>Wj*-8 z@Dczrv~xrRym$qC0RX&uru+6E+2Q`dPRIDal%N&&uL>aklLUH}KSF>I6+~YAf7H<9 z^0h{SDEIuK_*Wq`G=>%yx<-1MmRi=fMz%&4=9*@DwpzMcwpuhg);fQi`fTpMnF{HK z$b$mx!fh@8m)Rd7N-rJT@_&1F2IQmo@874VPshZdO|MPO!l=tg&7`YEPtDFqM@LP^ z!b-=&z{fU z_i^}#a)I#!nn_j!5#^0jpy8tdFZ|Bwo-=s@nOW({WRtac3vNZNt4}I^P-281TME1y z6?Q*+A@QR#?{p@i&8Z@{6Urnne}9=CCjV?8I>a2f-(i6e`?1;^GTx5J0D#iJ8Th}n zkN&qE?EizEkdd&m)3g2C!e`fe{C5k*XhikDfB&wnq-wX8d;X>_S*mQ_O!Q&9b$_qv zChCKTs6s*(IM2;vwZ!OXv9-A-a?!QN$!yh$vhX;2WoR*dfgqajLNh)~I66Ej(OU9+l6FJfW@^6C1D{)wv5!^`d!BRXbu0kLa8Nuq32-zte}%7o!!>n8KYOE&rg zzT1t&;lnb2s(#Iu-Z@o5O3r++KOEv$*UoBasy*Z7_ewhpfL)knNyZ`?xiPveKI$%gE*n>{xg_U2-$5rc#?+964E7kgKJB_ou=mPpd=hY5jb> zB&XXju(ZrIceF7Us2VYdYv$F)8F_~$c5KXBUDgu8Ug@}jwQWP-Jfpw+-t|c(^Ze=~ z_niU$ot)~NJiGK#-qjsyV7LRr5Y)H>KAl9W)x*ro*DieW0%Y8mwhb$zFOn+`)Sbm{ z?gi(jj#lahA6hUM&8Ztw3A?+8Yq3Ue`D~SBBeG1!wW5ePST}%-Ik%Pi^0f2^#jabY z86o@X4@QVOQjSk?u*t1pD5%6e?WG6yT7_3~qM0FNFe_ep=RM+a2r{5?#_iWL&1Tje zPK9yPncOURuw#<+q-XzWIU1&Lg-0f`66Z=rk)W?A@nG)IgerqRIP%p_OTJc4yHyl zw==*;p2FR-R~4tU9&(78dQwN~@t5a<4B}yyg$ro5ZuqALxYv?F(*o}}D75Cw+mn{x zc9%Bc{R$~Hn0b}#L1%w(N(^6rXFokUK5ag6$m(A)dk44VU*VzlBQ(mLLH}gw$L&JZ zkJu}((9yE6dOw;So2^e|G(hV#5vQ%Ws&V}eq4y60bIa!Z2MOjXpCp3s%F-(1qOoD- z%2yL5)+#?wu~DI`qp(|epA{%qJ`&uwkK(T}Ot{IMZ@EdK<$69=Jg8t+stlJdBL-Dy zW-bf7NxqG8qHhZvKmB+`iX;%?aqjFzW1vJ)8~vUU#|cEZ=4{SW50lx28-7G6MEj|p zDnnsUNX(Y8a!wVE*|Ei8KMYxD`BKzx6h^(+bw8f?{ln?FfQNAt8rGoQ5lc5IcZ*w? z&+&?*<30C$(k*-fR+F`Z&K4I}(+KS>Fr{PL=a*l~V|)vE(EW6>d%lM;`CvqtnkUhe zn_n*ZHI@e@_pfU=DFNz_c5Uj7xKVff504E&^s5sO>#(L97#OSRw}_4$n-g_+d&?wy z+6god0-bZ$$iI@uw-sBt%w8baynYb2MZ4)B+E!Y8IVw}n2-l$M3Uk=fSKe?@zm@fV zg&Gl*GzO!sxSVtpL`7C}*sf{|jA~(wXrB{>MsDI`a#qE_q8(g#S=w}(4c-h0I(Dhp zAqdn@$C3xhZ3z&}En!@CoLBw?H6b)&t*2kBTOv0(gG?2#kaW+bfEcdO$J%Or+O#U6 zD!kQQ!m#CT98-rI1yQ8a186%PbDzx*6N9FEueD`iCmRdIUi|BtC{UIwWbqaNAgF}_ z$o}h^_ee-F+XSfwifj+ew5uc9>#=hNa^j+&8tp#Jl}w{G)iP zvpneyvC%@cNQPhS-OVE>ClmYb@Wazc+l|{On0?y@m?@e{Lb8K+lT@T4SU1|WuY^Uq;Fxp#giVruT9@xKQPZ(a{1{qM_ zb)L3zoO^+hVJ}id)#!hIANp95p!#Wt@O@l^V|>dr1knd z4P;(Q^ty&}z&c*|iBYM%SuP+Hd{e#3Pz2aic@ z%NoTT_=r8d?dj~46?fv;kE#E$llEx_!NQNo@LTTVH`mrCz zu$-)y`-1=O{p3coM`|^F(Tg6Y^$iY7XD>WR=`X4faIkd@8<|=lZ#q2Ss@jz3Mk-J-V5q;MDQPwgQ;AtDm{ke;oT{)=Ypef7 zxu8az%*YOm-jobnqR7EC7QItbgZ@S7D zTurOp?aq#-WgW(?x{HYik7%-34BY$ZcxWx&9s{MC^IA7=haP4>X*Sw|&1Wg~MxV}4 zR}kw?2JV-Ac%J&(T|Qug*>C4Ia~z(o3y&$E9znD#*cV#E0bvP4pV;Wuqr}>b-4}1N zU5+y%PTiI+y2MzDEvL=h!CMgl;DgB|Ry?OMr<*5Qb*~_cw#x{&ESMI?El~+;^L?FP zSXJ9RxElr06qMN9W=O8msd_&^dq4+t1Wh-YdTS-NOw)>Nx16X1i!}qs z%Plht(0-uXMX0oF6I{<`A~qj zJPv|YS1U(X&K{)NXxP6;Y}v^@cJ7V)&mN(wwYZyOb__MT-CU7`aL0$JHv&t^+^t+3 z>3^YwA@3X}fH%82Oi&kYjwl-hvRhJ8ePOm)DoACB{tPUQems?dl=1pLY_Om zTUolZxs24>bqQ}v;ccmmIs{uMv1CO!o;rJ&x2eRMIi14jgY~ z$vuB$W!clEi-Wj8w;lPC#1jtqA?E zwnxzY=4okhN*ySioygZuO0hUh^9gZu`Q}7+_7&OH9fbG-LR<;55tJ3bO90Q}*D`OF zd^26FG%}KG&Xji{?L*6VcH%dT&T37~BuCO)igM}-@X1ef5rKmAbTl6)W#Fpy> z4wh{G>SJN7>5{VX=LBN*7WX7iC-vp6<1(;cv4V48-K*~I<1cAvY1+tC=M+Gb={4r= zqE`GbNK+0iBlpB^a+Vgy2PpBPG2Nq zFLbrDOYvOj)T4hz!F^J+~k=zVJrFr%$(t03F z2??Qwg_d4Bk_fjjs{JhdI);Q4zk+T#!~9Bi0C9EwW*H~dMxz4OopH@R=>)*Y4k8NRB~dODPs8Gv8}L$mpy~@@QjLvi%%N~PSt4w}m9Ud6 zt0Py`FtG8_89ip3IlP+vdNHT9q2yE&S!JS1kwzMzdLjCvpF02Y0`WpB#luR{i;b;~ zNi{A8-=QFc!Y<7@N5)Ob$tt`g;b!PE#78wRAL$0yoBHurZ|u6IwJ+$FQiVLt1LYe} ze(dCeb8}gmCV@Xo?003cs)fPesuh1xmH3jCtZ3N<=LQ=*+}<>=rVZ`Wz8h*}U zGLmK-KBZ>O%3S#^b8}&|avQa#z}b0ger%VN<(17)#=Hn5%GdIDf*n}P8VP>-XS5gx zVA;M+Gp@OeP}z2lE}xJ}@_NEL`S1)-Tc3=DVt>|yHBdcywH0sMu3@2c_%vtQ{-JNI z+TTDw+Zz?bwvPWXmmkj}WPUt2X1EY>p%T6hA%iMSKL1om`x|)S4 zs?hul=WO}0eXz)cpRDyqG9=A{|d?59% z<3IhDL)WbwXcEkU@%ji(0HZv2ZpH`K+wo9aMD<0g%f~3~R{6dHQ?9++-eeT@CGE^S z{Q^$r&#I{RH|N3h=zT3S-{_?uv-2swhjrKoo4cmqw9&l_bXsT5vmv95H7mI%We+R) z(#GwK%x)Zf^YD$8>gwSb0VwL1GB6`}!~MBTPybGgt&DBC_M5%zv<7=(i7YG+1sYJr z=d)Dbjq!+yS)6DJ>zx~kgnW%ms;tfTu@b)y(JD)At@=7yCAIga7u%;LO)oAyjUj6w zD~2TKsU*tE>T`f~qovR)ZL!6*^LCkpH42c~4>1^Vxs~Eh_=2EL?%4Di27wGFhKlBd zqV3yhsUm~vX(v$B9yhsKqrb8w!A_x}#*Spc`crICb5iFXlq$!|<4-D=5u%em`-c>f zUm0BswfV!g3*P!-H_GN1nyf9LHMkAal%bPFl|9J?sU)XPlCs;;n8g{YW~y>f1!~_( zg$b9!mLa2>s(xRv%u%YZx6#kYXo*zL6Ky@j$d+YZw(GnOW?|u$aw8yZ{x+}|`FXmi zqQO_FR#9+VGxV%^%%8 zGlH8EphCAWP{t-o0KoWKxRQ`fIsD^czK09zJ6DZ; zwKC?y?}!A3G={^S+rYa6Z(w8!RrUVBbh+xpmSoFG;RU6Eq}9+Qd{Pv*heVKUTBS-s zcJ2J}C-_Hl5ccpFY$3IGwl+>qce;~X7oW@zQ<5&MZPcB{LdU?<9k1fo$O@V>n_KTy zWZ6d~1v6hlQa{TqB;$}Y-*eFoP7;6xqy#03Bw|Fevpt~vOc0l}{nH2F^-qr914Mpb zYK&q~lKZRClPN(opi>r9nJ7Gek>sIwFW zK)`I-h34rltcdkY<4Kdjf}= zlL5XuH8Z~b%Alq_$CD3|!HTf|)BMj@HnAm*KL&fEcQ$>2bVVvVZs2($b!|F6kH#9+ zS&st%FxJKv?~yG&4P$k(f_9{vsOzZxAI`7UGCVbms+v zOWogN`dFfQnzoZq0-<8~_Lj3Ekk$GxZnt2U`Dl&L)P$uikUzJ67Lh8c!T(BLjF3vefy7J|8bWzP-nJg@Odp;3vl(D-^FY+p#nPz$l`K^>rsS zMIYVKB&3Vr!($hmJ&KntUkl7n7sH3Bd$v(%Kq3Xa1`i@cyUJ(_i_dq2{2w-I;Q_n| zYfI|)-<6@h65}1fK~$^OhQ;Mw*d!(gWFs0shu+Uo%PJ^93fD;Ei?3pw1#citUqfvV zLoPW5{;~ve>$BvfdZwivUwrnCCsOwa0p$KyxdNc}yo&UyZ~Z;y0D|#c;?IDTsNm-i z`BT1{<9vn0*FTr#9=`zI!a|YX;m?b(u}ud6q+q3KA*Rd>Ju?~vGxqFZrYyZ;K*#0* z5QJHcq5s7m{h}@i@(A&1D)gTwXH+RkKx8H<+2`l_6@|Ujp9$x=WtZD~%2tL5*rQGn zgK~ax+VQ934;$pli|{qwCA$7C9>8;`bC?6*NpKAfAz^?~=YjMR@I6}sNDSBcJ0||- zB+k2XDyM(0rnDFZp%D0;ks|{7-gC12c0weS<0n!`-@W>dnm-pKWC7U2Dm~z98%$w& zvqxGg_^i0+>u{RiLGdi)x$q_(j3Rc|n?SN5e-(r98>MVQJZt#h^~f1V4an3r{H6Ty zPd@>?;>GDZQBDE$;8;p8>ff&yMhnp04#UtFSIi@;01>LN5!BpG67Og6|0ui@>l-=% zUi8w{KhXv76u(sGBbvE)HILZJzntQLyjqT)+Aon)PlYl!P{H`fDHJj%d>B@V(XBB| zRVd;!v1AY1VThUED`gORTyPQ#&{S*?TNdiq9akB4_L-4W(VtK$ER;~ONgUUW3Z14A znSf97FtNPHPR{YH9pSkr?O)wGpN^PG*s~+KAKz3P#w&BFp47HeR4pt~$m-CGFC5m- z8ZTiy98xHdQ`EDIQy#RX>EE7-80E2Y_aaDn~$fp)Yj{DKD$%I_TF=ehZESmw&TUNZv6MWN??o|zXkX~pP0{* zK(s78U-j5H7qzm5>JN=-DlTMM;LDjuN?bw zb9>x}(fXnEhf030NbLKfIvV2aofGj~L)(fvdQ?`T%dQiD>P?U62J#xGWai4-Ld=kz z_BjeN)DnUC!GngA;Ne!}fRl~TqU+ENCf!z_dOMmx$-7H*lGgc=>R8Xc#92G>^E)%) zoK+6j3x_F1Zk3kHPEFm4{YfUD_VYTa7p4yUi=Eqpc_8MO6-{dL)n|#UHB)pr0<7kd zTM2b^`rWzuO}{2`>y;i|uD2Ohy0f#ClT&qgtEH~&9&85jecvQM_5J!Zkpk15zj&yn z{mEDg@dG(P0SZeRPpsbsPQ0R-D>nf#0u_u+FUm@ zn6v!FI7X5hsBeR^xD8rcmNswGB0$3RI`$46Xw{oka=Eok6xv1&O7y%!*d9@O^K>CxLw6nWWwFDkUTAy!{L;nTtuub7 z)MjS>x-GfWzoZN`v7Cf+BKZS6eqW5tYFBQr_{{3RD`3}0PaZFk|&pxSnid6A@9?h@X=uJ{r z$IF}Y1cC)dEd#>vq3S%$yatn>*s9C6uW#Qn~li?*FKAU!F1;m&W}Ca3!_Pd zG651GkG^ALe&{i`y95o{9`Nc*u}8 z6PkXE_U0&D+fCBjRs;v1O4*mFHoxoEmu#3~4zuYP_e(#qS!2-g$`-(=0RdAyeo4vz zO_5bUFzL9H>woyt6@>Fp9d&0j3itT2$CWDvRkb4M8AD5MDhP&6;Ld>dU+QxWvPTot zeaezWs2c|sGnDK=`LPG81FvUyHg_n?Yb#hz5D9<4mhFrjEDi2T2a2t1%VdpM?7uC4 zs9~7J^=!Sp0tU;J+PD@S&w7IJ8F9yTfe%tCXN4M>+orTuQ=HYnfUl z8;PgiW5Xlmk0q}%J&iqmQ*LL6{;T+;=3|z_kZ((A4cWa)UN#t&KlEqKEnQsBRdyP_ zH)Tp-=SPtae>n=)I&$FAK@XZ%G)uC6^Bd%T{t^EJhAwZx?J7ZP-yt88e@*=VXK?xN zRQ~?}EOQOfBY4sw0ou9B;I3Y>1d;RC9UN@H2EQA6O&JX)q6x24s~_AvSfHss*e2H? zpo-XNv%k|MRqAQwwYx}uKGB^1*CkaCm{<^WiucAr0Mn#XW2M;^F zH7w&$U3k9Vi{j5==8!uo3h?p@OsObizKNo=6x`1Yp&G1Ic}=NM5(o4O8ISN0o0)HDU zA#`4vld>{tRey>$ePRqtVsU|peo@VA;7L63q*%P1m~Xl9L<%j82$mq<(MJt3yoY)t ztw4>?_yKrKqG^)t9bG-WY4j}#bl*KR8&+FunoFm^+j=2T!FiT!#)XXcO^$Pbd>8_L zjXmEjc^8!W@|&*w#ztr|%oL)Iwl+}U^g`@706kr=NjiRt>n#WYaM7hsVg@X-?xR$ zc_c!%_Nl6=-x(aTiI%Iz<>I6=dMJ9sg9jAo0`YO^lny=je7Tgln+0`w|yVEW<00sZR5fxWCr;A|yUiI|y^k zer2VX;IuT>(76KF6;yUY>yRzaUdVkKWeBU+AbNVT1S+OQb=IC9eRuzhryd%dDpb5XvDeq%PHGk(e8-rfcV9uw`C z)YI)jExAEtUzK;?FjGZs?Wd_kmn;QSls7lHt2{hQ^WKD`=6n|1Y=obzyPL*Mz$s_r zDW#V-0W5PnqcnqUc^5Ml9{0;*xpnvPNwvlrjts|zzdlXRp5o4raWb7hBqcvsaI%9F z=Nixg$`*On_Ct5w)0)>K%~yixQOn?+E*H3f>=t?EZjX*utC*{BE)546!W>+5W(%z9 zNof+ww{^+(RQF{otGEZw#cOlbMZJeN$j%!J`1(iG53(x$|yz8wkr}Iu!vdsjbrA1Z_2TR$JZMa!AyFfjOo89O)gV!OqCRQtlr(o(OqCPDn zGL~}prT3dB=rQ~4fnO$?cy6x|)&~$R;8pfpBG7hA!NP7o4AA#eBU*Fa&y);)XbEdG zcXCKkCl^|X`A}ji8h7JsEVglVo48D7!p0osk$T_3qTOCd8X>i=khtHb*SZ+G|VWyJDU9}2+ zFsOCwwfRCzAin7n^VHa88bH0@Kw4qF^+wfIu!x!*LKrK7%O56Iri{gAznQ}ela7|SMpIYr+M47|2 zArw#A2puUmu&?DN4JjF)CiHW&zK^lR<(vswc)H2vEL^8S#Wt#M<4adXS=rcw>_3>lTk58AkcVWapX%eNNK;e3E9>(zS&6fnI08#RNp$|p)6g# znGK8Cy=h&kU-Poeus_pjvoI?CC{vB!>Y*c+f1~7mwiXn*JQ1!Ec#QWJdr}?Gk0fy@ zKD&BLE`*((eLvLbovf&{u}tCnw?!t(yjM!?X}YYb^euWHoBGWasM*@1WAs@#WMo5U zVs(qgH@-?S>gVMphwY>s>6#S&N=ivcrqUZ67V8(YSDFajpCPUqkB&B`&)wcp4V{_N z6-y#w=jmOiD+mr_NRF2A{}CNPtUT(NF<|XOw5m(LOT7I)=Y7ABn1+2cJ=eT0gXl}C znCTRX>6tCMq$5(DLi&DV#g;cVqK)%O(T?YWNYe6T*4H54AKMe}BBK{oqH(pk{E)E| z*jNRhsR0NVG}EudGDN4bQYPf7Os#q}vP4;rF`{A6R9ak)D+Rmq(=9 z8_+t)ldv0MRg)wIPi|T-yl;ZMk4iBKXv~- zjdmF2WrlePJIjsXFnyh(Nu{)@1a}!4w%Q9Rpnaahug1y2`}#?sVhv4Wb~=Z%nu)cQ zt4U?8hl%ws&l!qgiPw}W(W}IPFQ+r|&gL0e#H!-x-8AqY>A8;m z^z@trQC!TGMMR`?S}OCqqXvF_9Z)T+D_?{J<6cjgIzU&$^NW);2u)H}i4ANo@r!4-WFC!FWeKDg z**e_?XIi^#7h4B78~?aimy?2baDT5Jler~mg99y?bG>miz^IuJ_gPbzP4sffCbewK zX~bBnE&&|_6H`KJkJ((r;qiNptn~0e3HfD;ouHMnxqU^8o$_eGh4#dCY_1UdiDj(9 z!eu2TF1&gV#-N~pSW8JP*Ju=F$zWnQ=fhR>(Z+@^dBJMf2TtXu!=dTpA+=xg7|rA&}iWqk+V@{tEtPfhn$> zlHU&a;gf|w`jm@e72iWWamY{%-P zDJIysKPEH^t@bgiFn=o^hw~pLS}FqLq|@i9*f2E;g~cw?k@1(E3{M%Q(w*VhG@FK! zC>ZH5sS$<0MH&@oI9$8?sL%;69ydddV)*cZ2uKgMs^~)8f0RgG&u`X+7O`mVn(iN@ z(4;QAIN;kDc-=8JrA%wAgGN|WQOK25S$H}zZJa&t|0@-EvL5Y%MAD*2E88NlNGR-L zRn8_yH{}q;YRR9yq|;>IBwGAMMO`AF>4Aae8m4G!t_`P{>H5p^5;FCDN*L%HYgb!Y zJyRGcuo*>xbRf*t26gxIH--8lO6kmzcn5}Lm-qH|2@d8X72gNP#>AqAt)tS3gY`S; zAfX>M;{NrMd4bSPyik%s{=uh&T|DkWp*W3!64AH;brt~}()S;J%~-2CGv=&Ogq89U z4Kp1bl3*5puqzD6(g=RiY}n)avQ=}|EJ-WGuIk;SC?~3!6Z_Om)5WD?hpM$Lw978v zlBg*e$8sqev$P-yybhFBC=9kLbntiSJH&$>r02{l)Nv)~y;TaTpsZ!~FS=0>oWn?7 z$75~^n6Q2|T5td-VePvuV(qKvHWF;vK^mucZB4L1YxL0q4HnPX-CeKd0HOey4>bI23(~akm$DHR8lCT zG_b9he4TEJy)F#dL^koralo&sN7dVkOp|T#MwS=#TO8O1rx1F_r7J9KZO)wwTivZk?-m4vt2_ z;(WAiLAKdDtH|nP(KCf;2(lrnQZO$6HE}Cb1+{ZYSpR@uanKL1qR-+9yWm@ga(=>i zj!dZttH?n7N8An(u!x2Fouc~EyM}rLnQq5|^X|h;8-=8377=44ryiv!hx5B1h z@0KYNxSvH)Vkx8{omm-allbERsE87^oS2x%KEyy}pdiHB9d!beZi_Z5=oke zlZeJ3c4<+eyRjjDW(P^)wmV^;9RAt&4R{p-rKt!o457!ZBqwmRyvA)OVHleIE5Qoz zUiu2@evwXUeuTNi9*pw<-z_&tpcLu$>{Ii>Sz!QHXt!J&>9+EUa z4m|Z)Lj{3M%znC^iAw?iG=IT6?K$mnG>|{fkWx=vkA(8)5f^7x6q3b#^}0+dZ~d@EDr#H@Md+Nnb^oiLZ1`Ayo*-CkVLlzbl+@v zmz(+vz!?UL9RTp;g`Ebd=I|f*{RJssJk`k7Op1km*A_H;o8by{W(E=*Qy}?nuYf09 zUYB)FD$i@yb%t21{H#1)H%;E&PS%m%%zsw+w>7PS0iS=63k&PoMR-UDq8Wqv_{{e?tj>`XN`(knUsVKeaO=ko_~b;f*M@ z^zyZ?BM$9zyV@)Fb4W(Kns!7tBh2u3M%)`w4{(daLHLIaA_K(-9~ojgFC7JBI6=@6 zdaw-hIuKasqncsBP|WXPs;;wdaY?eEhLFAX{oIZgax3_ktJf+wG-^?3BU#?eYCu%( z-RpbyB7hBB7=rN(hPoJ!z@l<@>3w_s4bueTZT!c#-V?M>jp^uVzHh}l%_coX{(AZ5E*eVSX;4Ke(d#4QHY@0q0 z&*jhvd+k|o1_fAyWplt}n~Mjy1_gYEFaxOBcwt|kvB4t%yu7)dEw`M1MtRgYAt(>r z*FPxFSr{+&zoI;F*1u7nHEZqa-22fl zH$E57@G#ow-!Ax5hg{p8HaowxYhwH%f<$$%76ib+hQK_aO#0SR$mn1ggpSj&HNe06 z#0>&?0(ug1|6(3{W10LssG#><&lW?<+QfQ4JODIB2boBal7JqN_k4#qt9N?l^f%5^ zFUOYfN(~Bx^4Adk>4#iFylYD_-En7-nSp7(@&eGu_70MGhx>r^_)LP2^*oE=>|M$J zX86YgZ!bCK?F)BwNHz#?Oyfw?g|`WjQctHg z?PHJ0=|*UoR{!qz^hbAX`U7x2a{$1zE6bZ~$Pk>#{ulCVzRa(DFZ>j(KxlbCfBwvX zqIo^5?pjL_CW9_Az=mRmO%*OeUBMjhs_LG6nk2=yn-oACY zRziyUnD8t;OCN#}o%s~Ley&`3=DI+_%q;lE4G5Zh265W)ovn``FyVWoC^P_0;BXn_ zcn3R>Fb%riYyD{$HTe`WW9GD$e=liW%&}1Q8T$|}oX`T0+xj_vk7x)kz>>4p?FE2< z{MC2R(h-3ASt=`80#vU7#+&xQKB+JG@O5+8r)#G%>7)JPnXb1VRC46Hh9V^bDCdxhliadbimAQFF$tDWWUR0Uh|7=ldLzx_Vr<`g~rANMaDy*YObE%w8HFi$o6}wtTtTyL+?oex`b|B$`TwcyUsrF+WTOvfd0i<>Ukc%Z$*$f}VJh${16%o%rJE31~OoXTrnFk?J)lDdwUYAYYxZPwN=a0cd^2eh3*8r2VXJbC@muX6fWuSaKtGz zA|QS!j;K+^;zq#q3ewV9?{Mr{AgK?nmg*I&QjG=`19JGF@eR^{U@Y;n#jP{or2ER` zdMHHF(~Q6fE&|7VF}@mf26?^x^!<9gb|W_S&}{UsL3*HewjrD@1lK*)j)*V#PoISlh6oQMdjOPc6XV z3bY!xbj{MGqW;0vAt>v*@L2{Pt)vPNa=G?i?m+ex+z$Dn>UtkaMZb6?yn#{lzRh2!)lUeCzxlhp5GaowV#;4}HyJ^a1 zgzID~uy8j{h`D|7`BOsg{C-CB6N*8AZe5Dw+6l5G)En-NqGR882r!+7^q-dH#fgnu zD0uet=c{_WBMphs5q^CDW^wta`6z#hpcY#|g1wQ_%NtwM^x&vQSL5wY6?i54g*=jv zVlSx&sj%=uQC2Za&Y|D30M_h`^KMx$x^uYO9+}iCXB1y&In8itM2J(mDZtu(uIuE- zX#v^pi5RwbnE2WI*1)Nsqg(M|G7XHc0&7^Ka- z?Xm*ceM-n8k+-z(YF$TWYAKT!(^=nJ`_ILGoMD9isK=JV_!CoO6)ij4YE{5iw2K9y ziv@3WT-zDmy2ctlyGTiI8GMrZvM6=*#6{ExulefSIqQ|X=bXQnMJTTuISzs}k4D?b zA>xupZQU#e8zO7zu3Fiy`%P2Ji@ZZ_Od!du)Xe?W-Z97MvB_f@tkKJHzhR@{uA8MM zE!lO%&ymyHN-XFMD7o*Bg!a^mwl7?RYJKZrvxYHah&Tir$_c;8A2Zwsh?7s<&)j>u zb&5%==yJbZXy!fW;4kiQ&X234eB&4AS0HT>bps_^ReM}=ZRck$>Hb*iFF6!qC za7Q_acWXK^YI#Nf8U-oy%V}dbDsSw-MFYo&@ho)?Dak!K;kvcq%MLr$+UV_Qy3aZC z?q#j%jdmAv5iGMAd&{btw4st>#XapGUbT6?ZQJ&K?R+A8?E9UhdWSN%<`MymdYxB5 zJ!Ga@X}b+T{ngk+j$<4X8#^}dQX8$neS(ICRwgCM@0}YW!!T{C*DEQdlMSmnN28Vx z()Fc|->FrMyO(Q}7{4z5{8oaoB(^K*lqw|(bn=@d)<9bz4*JEYBt~TZ21QkHurqiv zX@bJB%iFW2))mF1;EMp4FHXHeN`Q(nJeM?O^$M{C1}f3m0&lTzK0wXq;;Yx!l6mdc z30+2&z|U2)hP?XI@?=-p@t|aXoXLgIE2?z7p?CbDDsW#GZ))^#3*V3;+@;0i%zD#4 z;&ZdF4bbcj4#;(?PT9!`x8}W+y7yp1?CNC-FS%#B$zdyHW!h3kcAT7zv|WE}kyAa9 za>@!(f0Pl{nDQcWdx?W|mT_EK_uAPeSvSoO!_vgK;&}9h*jU5v$bBZ8|M~%UDNAK zAn$X1OB6wyQPj6)b+Mg8Vtq~@Vns`v1L?IoF=17-)N|HsN-QPx=xx}w=rF(Ec!^7t z-sV?I=+)crl0@0ZFXI_?!!#+cd&YLxeet$|0i)xVEPhg+_<{|bDIceq0nUsoB~0(V zgH=!=P{z1Zawg6s0tDDQ`}^XN;Tl!E214gxu*U^13YgD&qH#$W(|(zs`lb!#k9;2( zR!TVBdlysmlki+Vnyn&)Sr=z%FW45}Cz327Mn%`0mD<0>VUQr%$~Qo!nF#mT08MLh z3v)*UZKQ=X#YKYcCdu-Ic^$lM!atc=@`gwr`q+H&@aH}?Sn^_2_ZG?GK| zm}b@FdN$6;&P*Qxra@9ysypcn*5$);X<$UyovS(Pmpk(GbfAp1NA^n;fj2CjY>PvR zNmOv^VA_RQ&&ac#w{h;p_)pl+$m;JmO@KD!Md=@Jeb4uQIkU|A=gjgAWcSMLf8W9K zUqYTwUjN54%dE8jhi8`mpxDn_cwk{@Cjw9b0HLyfgM(RV_1GC0SlP5#sdZQ($CjDs z*jT8w7@3%<^|V=-b+p-anROWHAt#pUpLg)E{^uP$f5XAg_W#@No#SO>TQ&Jqz!~dD zi7%?6s`vic0-xrDd~=7ks-`cZ;_W{XzS7l*2le5aZc-W2=&0sw1g~fMNN@Gz}E#Ecfn~UNT zhKn@PnoYg{o10El+=q7y2TN5dwC?=Z3hn|}b2DqDPjl82*V8>Nl0(G6ZQqIfVHTMz zWapXDV4%!ZT8*6O3@+PhhxbtLTLc{42wg8S@J-*VGMUz2?<~)V<1ucG1@(@CH)C5g zmJJ^3Z9Lj;xL2PdcFVX<0#CtDMG4Dnd7DSr29nANX_`)_)rZ`bCz-P-?8o&UY~Lkj zOK(6sF~^rQYPW;1X=%VF^-}QA^bJ0KFC}tOXIG~x9%z7)$iR<8%SfvdIL@7IEDPi@ z66KnZ<6&^LjcAnCdFW-QNqO^hz5alQiyKi{eLLbnMUF^8dp}vT{ph4XywS3FHgA&t zo|$+uaM1$a<9dreCqVLk?ZH5j99~MPb#R&b^llo=X*cFjnl9(m)fXOSFj_W$J6RY* z=6*OJhT3X=J=YLW+6L-~puWFNNatQ3O4d}ipF#8xJpxZINC>xfj6<()ovoc+U_Wg) zcvUI4+9jN(xgN37G+v%C7YX52h*`=$M67a@w#{w_oh1p>Uyp01C^oN}o19|dAIWMs z@9na>?dc>aN_@7sf;NatcU>F?KlNo^(w-!axq!LJ(pU}5?(EzycUvv*m3KAo$8RL+ zPjAay?{gZ%v)u1H$7y*UQolT%8Q$FMtc;*x_{w&6bs;|8fDA-~Ph?!|JG^t+xN~l{ z!{5|Y(Q@ryKEQa)Ri6gKKdK*g5p|A&_sDq2DHrVA^gFyCXYiQY^wqym0Ee`AioU)Y zUGWg1D`-0nJL!m2S3Rn&{#ww_k{QOf|5`7k|a`G!yJmykLEgg4%YaS(Hx2 zy#q{co2lCk;X>tlJR(c0x81fmbz!f+S-a8iG23?*a9qACQ)6y4oF!v9ZQ5@T244!( z&Mw(6-*2t3+8yC7Rf7_OyjvYs8cu4}gRdXED~0jdYj!b*!POxDP4}mEELxsgu`}&e zR!oem$9WrYbE-~FuZ5yVgHF$2M4SB$8Y^f&D62>z0*U7CI!LSsDCKusyy8BaH5!YO zh~NRbEKb^Ay+1HO>tKygcRXyzqDId1!*;(OD!>wg`uu3K%XzCL!hO9X;Wirj z>Fq-en$XIkjjOWd4)-qSqmYh=4loSNWsBf(6WJm;=Xkno;=R@ofq&hH9oBMkmpl`T z9Q4?2rr36M6bwCH-z#6ty%*L^gDSgT)@Hem>45+C;fvx5R@mJkK3;|9z9ud!T(3ac zk4xItTdxh9vd7$7X0cNwZ=<4~HidUctrE+Q^Qg2qs;+4~(>CpstYx3~Q@FC~3hZt) zxtzZj#`!)3c-%g6v0|j>9`l`TCy36FWNL5UMh^s^*Ay)H+ZNVxb7hqlwAPl=v`p9D z%XJOq+R7@)B;xG7M%f>eS&UvVe&EfEw-syrxfwe(v|yUs$}Af*z``;;W$-!4SVb0j zKk2%sCp^4SPS{w;M0Sd5Io8jYxs%k5=?vqiizIRv-@Ysw@1;>qJ1qRn4p*sX_GHG8+58H%?wz`{M zFKtSa=g-yTF~%Ghh~&sXZ!dJNC8mu`)>ilNd@_}B-y0lE=d5}R2oU&np;$Eueku<@ z`Bcok8$srAF~A(rJq@m{0H&?F=|3t=cifP1owb8(HV;m@?8cVLQk@?Zrg=cqH=}Lm zCmb6G=fXHwL2IR$wS`ZYqa57x%O2fz=yiqe`=zw88eD|rTrCs1rAd~fYP&8L<9les@l4Opew3}KfMP}K<=22?a?)+|biMQSh8(h@cvC#t&aS`0H;S=Zc zt5rm6k@tM51qQa+sqDkp;8)p@Mn| z7h;uk5&U{=zWmjXCKjPjFr#L>mGR=Wl#9v3ulgqj3I`N#ifPIadgWMRg%39~ZOEx4 z;z>uvh!sf-2Mw5E9*|qa-!&u&*{}?$$N3O6Tzk?bx|UOzA{Lf?-Lk<*jcCwKAc=Mh z$~Q|XxdqyBBy4sTkK(*mAXaQ6JAyAfrR+DLX6UzJPnRTSCC-eu8fn}6aG9W9jv!pk zjyGPnimj0hS4fLB&XYY+IdI%kZr$~nBg&QTO--qWx{AEx9R(Gc&tt7Dnw;by^_b{A zrr9|`D+Bdstix-RUFhkEu%QU8JxbPury7Ba_~?8s;O;;n#nZ`fDOu(t$M%%zsp>oO zg>zhUaY2+j`SXu$hSrYCBH!FhuCysjQdIDUyFrUuC;yMPw}7fES+|9WAVHJh9^Bn+ zlLUghyKUUvVJEl~+#$FGcPF^JyE_|q{WsmGPj{c&_rCkac;l}zHqY%U7m-sm72hQeQ*kIK(BuQ@8@-q)lr0wQ|g8csT@atOw@ZT5+~d zO9mTWTU`wf9%Bk`8%@uEhYm1>mD5;mixOssySrC{s7?kvod;H!dbC+>Rl~(oF($kG ze}JkuO0Q`a0xQfiIB|1*syg+d4-v$Q5v?9n_0lA(t2BFt*eT4=w?$(k=xba+<&OJn)gi;udxK$p)|IofG3og75VrvNrWkFn2i(f6{Y-g!M^jL>w zF|UIVGmdciX<9Z0OT5>w1!^$5Kulqv*}L^?Wr-Pb&(`abct)BQ!y$&O-*fqmT-}jv zPdJW7pY-U9XjMozm}$k9q5}$kDj&+mXuEEU+r*};2M_GV*p$cIb@nRx&XWdDqul8K zY}<1{{dtOT<4zT4-ZMqqtJK*`&`&lWk0((L{!E@`3oO44?k%|Y_jad};<7i3h<56v zQg;6!<>K;XG+O;=a}#~i=|DB+=~o0pCzZb?UF2ym=Vp6dG>TAeRTjTN>wDTbi%+(S z#04du*%<{-ED}$^=h{&}-~pxd zNLp@BA5$E43@?!>JI$>=#zU~zb~A=w0|41+@WpELC)UuW} zDJ#u691r(|)H_lZKGza;3{8A$p$SQ=_$4%jMxzf#>dH=6YtE8mN#Ne5K;hs{iE6zG zT?&H5`_&sewJ>RDbT@QCb$insJJ(`)P+CyQl_WRgW&YOI*6c+^(zd=KK}Bg@osEs1 zxzu49i6tWmy+KmAh1S@~=&j*jO{cFZQ*EmRe@%Ig&nNOMKsJ3lulD<=NO=cE5QG_K?S-VD@Ais7m?#P0bm%+w9yAHF!gy)S>+n6iZkY1a8? z(@^-X%l3bD+bz93^LK-iNzVU`(;^enf&hIiwZ$-0y3R!NyEo&x$*=eW{~E-B#FF1K zhhv4w@26Vobmu{Qj9n1Gg&aHkC-5!2`Wm0a{(I(M{vLEJ_`rO9`B(r01%kd(`puuU zV!`fI3k!9S^;;MJRd}Ws4>BZUM@)!Y>nnc3@^5@3$g1S-6&~e(6|w3Iv(ChZT8dbp zK!p17y~X`6&ZS>cgGlY)RyU_1eip}21-I`<#pS3{Va8q}{a-N%1lY)A>`8g55O*boT|E)#m%=>7Le^s*LxKjBq z{ra;(sFjO>KU(=$s?5S)AHZXJM$ms9801|0+?086m<~=x+!c^-W?bc;pY6Q($VAno z!HF6}Q@Po$oWxrRVR=cj0wT@vrJ0#g*|KSgguOtzx&!z8b(r5@llc@Nx5}n-X+u z(i3K~vw^DhvmyLfR?jnzp*Nr|Qv`uE+DQMy;RAsn^z_)6xEO()bb6e6oOG<5T#R(O ztoodEx(D{#f+Jmj+Ns#NZ=U`_$Np}6(oZ;jSq6d z+YbDC@4BCkbrDTNDSUaz$vL3-Q<4&nT{IILII@1|C|%8Uc+3fUpG*E8hK;xkImJZJ zEE#T+9GPv0aY8gyQAom!7DrD~pS4PXC5$|=z$OIj*5KBFVo8*DKRRUH4)(aZTR*#L zuCcoEXgQc@^OeSsguXXYqv6gQ;lS}=^Fk(7GY%ePoG3OOSPuqwR;#*8w&ez;7Wot4^a>ielSvQiytKF ze*0+;EuayAa2z}cPolYO?uD^V%(~Bxo9*)d&$-7kTaQUf*7S4j~R8`vEFPUd=TFD-s%=kfEkR0 z^8Bhn@{NSqP`Ui^dz6jujEKY#VbIqNK2a#YiywEUZ2g=+sWtA~!2WHxo5J`mYKpt3 zZ^K&-L<*@b>1<1d^jlpyH@@Ln<0<+6vS6SjtL`Y{S#bV{HkLVRgNKUx`BX zsY0`l5tP!l9{AIixiB2*UAw+u;98oTQH_qyHpyP3Al`pe>ChC|eC zM@bGV&vkV4>+;(zkp1Tl7Hc+MClldRy$-wLa%NXgQJWL@{rkh4sfD$1c00?ZW#kdx zsl%Jd0t{M{|in@}L3(#dJ7K4!wohs8e z{=}NzJb!r%rwHo|C8v;fT~IV4zh7uvt@NVEfMw06mi3zS;1L`;y%xPLli`~{Q6AL! z3pyAihsiK(+dqHw^eGqXdNz8ws4nuP#CKBXUe+;N3c1eNV4rBy#v+WX<#fIeh(5W3 zd6w)*yy0DaCmvnF6ZfTQ#Mb5t<BJBu9(}aPt$R+?k zEb=Cdy8r8}0yJ_k*uuH@3+rLpwJ>aTD6(0o5w$ z0b!)vb}K8V>MvBNP^Fvo^$63q`GI>19{Ld_S>AgJVu)Yg2~mJ65wj>>=FVzpG3-~l zNNAZAPe%g>`?_+ICs#%Wa@6r!GW2g$aJjGoGyD%kKJJH2X6=JQ4Wr&ZQFP())#wj}2e-HQ z=(1{yBpJ_%^^w-fh#(Mjyy4ob?ieTrX7VRU24q3j`+Mg;HQrXt`iLX~mvXi8%iV%BVWbHn4sea8@}0J{Dw3jpHG~MX-%NJakTN zU+O4`C)c{kRrkAxCM=YphWz$dcsm!<@@ltcO$VzZTyQ8?SBx`Igrla?l`PE zCinyp!r^!O<7&&{4lYM$3DGS0=qe&2$GFZmZ@*}<28bQV^s<Ev z;J(sr8&q_KXE*50DSL4JjF%+X6$YN3*Y(L_^upol=;2dyH4jAjxjyO`Kzw&0zRM)@ z3Ro_7B6}>hv){=<*F*D0NO&qtb&Z}2cU?jakhXb5az2IKf2sR71|C=7fej@_cggKmecg@7q$+Fr7g%m zg!1{m^C#%7E9B)$eZqR(m14^E#WyKX`q%+|GUIb7u2RL@Lb$} zYEg)jC}cbX0UICqyDQveP*5^voBV^^Bu(p)k>osOPQC495X-qYf<$9wT;bIs*Km2m zmE~7ybsB$kp7pSZglna)80_Jrmp`V__DYVAWP$OA8G3QnsL=U;$rXg|`WK`m&;;9e-}4aZ9|Mui%ciNXJQ5T zVOMvHBCZyEtVgKn=Gwr~h-LK5itVs!yEbyZ=Xp2@yg$;pw|>VUj>Ru0zDC$~Jx{KC zpX$8E^}ZND>X+*5zX03w-Y{Sy5>YdL6>Zqs-kcvsqgez|xSM-J>2s{J9~@n;H8#tH zbQhuxoV7yg1*Sz^4yY!HC$%^}cepd>xT7A$S0JPgywNnYFTNY!v+mj!Fqv@90EH^SeDk)kCzFnsLlct+;t+B1}PJ=jc4P zs5m)m@=?TtCSRpJ4W2wB!J(?NA(JRO@qD%VB>9F;U|w=b(PW)=9+qwXAJqxbYmQBK zOpKQW@Um`ii{y9gA}}2DV<-kPV^v$VlnJ6Tw5nTZ@CBb7<~u~gJm%4=0Be!s_Ib-+ zrAwaGaEsPux98kmv=*P$m{_1}CtgM6n=GBz3pJY1J%T5x#RiV@xX{JuYKAVCgIQ9; zEvn7@%Ah-I?iXU%rvtyc5ICAC>!I?O%q$4*J7vnNXGrtbSxAStCiq4jS1W&<$?8qrxxR~ZlSx&nf{w|3tz&)zJ?X_J zuZ%i5RqE{YfK{~auR*I%;rJcP0Z%OLX(jTEqva%l*vb9`Xn|@?nyeB)?^s(8c$=iw zt{RqPoq%Fy_gYlbrf_yYRBozw(^?uQuXKBQPD~*&7n&YP=Eh&zv+YG{3uAR{a;#v_ zGMmv~h|te2f!|zJ|zfA$z7*C6w_xSps03LxcKIfL(k$y z1%t7$AHt>~ua0Xu-uifli*E#&ITx>LlMhq5=$1Ab@+&KGHs$4cS4JL!ad^5^d(RAT zSuP2F-ODsBp~R$VoIpwg5zgI!e_sr9VNS4f{cZKQfCq`O`iG@HQn|*&e3zx(V1k+{ z7IMa`tC$FM&z|enB2mIj&KQc-NKoT|6@9+pk|-)UH(kd1;+>;Q`waSvn8+U63o8$} z*z{^cdCv`t0t2Z9AunpW0;j`@Y(U|JI3a-9-1@;rbA;kKA zV``WvHo@AmF;|7c=qPinU}c)n(B+mwA5dJC*WaC6>?x+(`_Z3?c}u*$@r7oHfp5P6 z)?KiDC32Y|`UQ8;N>q4#Ry|_ElUrI2CYh!3J$IeykM{7X_~3+n;v(=)O`VJslHFnI z+@0Z%b_86lZlELXZx7U;3n}s`;M5c#Sj_Y9Z?YoKYassLAQ1n4p#B>a;tw75wXY|3 z*S*D75wcW7MD$-$elgwxV$FR`0tPNwH`$lMOwXR!E5FqqX5R5i=!j-#eyPwwUgKIA z?G?5WPow^9-$oJKb_1-->j$N1Mw?48DU_zoMs`hhJnI58F`< z9r2@{ZBnn|7INGBNn!h}ywTMk9%n^GB)6Aev2Vm%Tv{8jmDHCI2Y{E7E#IIT?cFw* z{dfosIa+>*fXleE3MhS>*$jE;-h?zY%Jdv?qC(`wOz#J7WVi#PC3=ahxY1iu+`G>+ZUrr;Y*rc z^wTX%vp%UNoijuAX*gR4?_4-zQV(-r4fUAGXgp+b)njRwk$`r^F8m2|TRix)8Lu+p zQWNaHC2o1%CGNmy1OuZ;kMqs684TD*bVEYae-0IX+_GX<;lNF>qOchG9KtoMmq74V zuEPGRieBF{sl3XlDru6x@yQ5T0AvGFA5bsm%P zOT2cNE?b?K5Dj`h4Rf+>JLi;7vw1kp^;PYa%=rL%PU|n?~uew){!Y|g_MHov6xwr|~`2l^rhOw`Y1vYTr;6*Ss ze7k_g4`JyM(z9*Wc};pG`jW8_cTu^dJS4Mvu9soq9c81e=R`zq6`K^t$#s8NN5LjL z39OkQMq!cwLC=eBo2ngpX8KYhrwy+IuD&;}Dc{6=bpdi-y0k>CUN&ULRNt9t7c!2W z*G%yd_YJ&xo8puEkFLt4(ro(Y=<S42Kf{XK-q%E5V@O+g85;%nbrQ5-ni>|7P(0Ry+5K@sv7yIB1{!O4E z;tYVTuOuCuTD`6%z#o6vnXXCo*&2Ew$BV-ih6lAv%(O2xbFj1XA)vOYq9wg@F)N!F zJkKLx0TveLvq;GvJe%X@!I`UCSxbw0LH9x0AKJQqpjBilJ7O37+S^`Gvo%k2bC0d_FmfXK<)IhE`(v|rG6FHF`l09&O`$i zv!J6VXc}a11+ynkc`X*jNO!%lfvH7vBy7AO*(}o*4vZbx?^C0PDb{!}2R8B!mY`JI z)4(1M(9VG_CcsSy--2#cqot5%r*Bs|)H=Vr&ZpGH^xq9$tbANIGYa@#Qu5R!S~MWn zyp?tYZmRGgno+(uLL}apX>Bmdq4XmK)iqpAcqdited972@+96s!RJdeLlbCr(cv&6 z>@YKY3hI*=>;YLeU%4Dm5()gw@qF*Bir~p;G>=Zak=reGZ&IMwQxarYe7hD^LHJ^% z#?^%5ovqEvhHxLtW!$(hObNU4huGa)AVTL`{H)cYE)Smf(bG>103k%T{!C^zuLcam z_97ljf+?z!swjG>rH9jGkD%n?2Ib*pnW61l?N`!THy*~JFpvwqYr;gxmW*<=w9A|C z&b0v5$d}g~P;>Vin70AY7KNS6kH&%`2|*5SXHHColgLG|Qz4)3jW=OuSWfPDJ{^>r zGh}KwJLM&y0jDD*->ln{=m2evgTYjDx6x#vnK!F?dI)y5r?T<0DONTKcDr7@72}C! zEEUDPN%M+5GYd+=2?RAHFN*G*)%_l!5+y(1D}`|FWs?mzmQvq~V`bblRVL&VWl4A; zxS6zia}o(RnF5U_D_VA`i3P`akI#}u=WCaUKa&ocrTD|_t=$Geu)5p>t%t|m{kULA zoWFKebsN)=mll}?#gWxCOz;JyY{`QSgu+C`O&hs2PIyr7djAE$P5a((u5(&bv-vS>()nia>$#8BB&i%{P%) ziS&VkfsxMq?}`yB$XeMloQYL!jU~VREDgw6ZEn^tsQu(S3)N+qchL*~#M%R80E71x zOeOa8`U8F>agytp1Qo|_mbjkaIDWM$^H+eHsaPq;$bK6>%bR!nINV_LO#w8~2u-*j6Zw7611I{Iplu;c^MEOuPR(lTUa)8f}k%=qRfGQLVocr%-7xTYErVRhHI$q z;$M6fdd30G)a(F;=%?2i5t%$;!cMVXEs8J3bLBTyLrIMbAtK19IH`*$=f|%T7l|gYP2^;w zzKiiv-n7nA!Ki{y!~VeW2vWP%^R3@8YL2hDY+z>zGe{o`6iE6OK2eC5Q{hXh?Y!|6 zWq&W7e=QrUmwCt(x5h9&il&ti7`KKyI)>Ia8DQj^Snye?q4paZ=v-TMnPAwKi3 zT)$X9WWrBEP`q^0@10Ty+;83y(d>!(iN~kUL&ZVKTZMFdP=Yu@u+cw~^VC_88SwJ; zVMH22jaLq|joVo!zUT@VCsEW$HF>xBNvWanDTb`kkEDvUeG1Y2h2bijJ1x6c>roU| zSm)d(T_vrqp9o#=NC>uCi#N8~8mZrL@y9l+z>0o1Yt^>v2(|Z5(^xHW-Gn%S(i{9f zC+$hj|4K&evPuX&xmyd|7ID5JVmj}f-voS&WfaJY)xSn$u|Gr9d_#hDeECIhN368J z$cOd>@ts=aDq*~?6^Ezg(X1m z*&(DCAjhJHpd!aT=}xI;{}gu=o*M$kQ@(=@x0&9n((q#frL{$**Sbbf`E7ogOL_xhU#O?nhwcvqCkzh*q1+jh4_BZEGPEYTlgv^QuX96 zhD23usH#lRGKchYFNBl^ADT(uX~Tt zcn$U7naz$sCnHr!q1qZ*!pnG6@9v>bJLP?7A`T7D`eB#Cr6D7rs_b;u_78iH{Rs*X zhKT&8D*cb3!2k6C|IwKF_W=J7pg=!*w!t5ufZ18w!%f)7`m!f85+`tJ3_Y^(oQh~F z<2pzJ$EPAX#zsok(&%Lxg*XX)g4nxs3yr&hA>=^K_vKmM5}6s;X@7zO&jwo|=h&NB z$m*QT%Kz9*h=ak9%YccCliiTcz<`+%!p6{JqT|%pWus#>U}0u7;9%uo*VTJodGVWF z_SgN@{ut`NFoFNkVp~M9)^LE(F(gF<)aE{mFDHt{Xl0dsM2#FIlnc=bDxyP!`3~g# zG^Xz~ei%VcCQ22sFab&llKYfQ;N*(OY$S_PJ0}pfehh=HYrKkvCAc{u+aZ;^PW9sz zp6$ibyriRcl10%7)BTNmj=AUE-PyzA(!9sfm0L=6^0u$>aEu? ztMhw5$Rju}Sysih>YX}+&^;;&04-Ury|GA>;(HvV`-FvBajIy)(*Ql$NfX$7{kW~{?%7{XCLU`s zj1L-~j&*nK<-3@VvuHL_UB0Bd+PK{#)F|I*1z+@ndpj;E_LbN$)`n1GVfklV_mLD~ z;qD|1I9rg^wTeBDH<5Tyg%9do#tYk4Km~Az1#s+u>A@|P7L?yxhG!R+r?(wo{Mq*hwe`$hWC5Hub!GO zcd25;t)8wg((X@Pit;xfjvkJFwp49rqGWq&duS1D%YG6*P6}0@#&|zYf6k*icQz75x`Mt9=j;KhK|Q= z9@Qv&$o%P1m5y!@oIf!OURH5cCN4X58@EaW!41s~tVFML*YLUkzs%lbRnRC+6yhg3 zN)n%V-`HuzYhayS>d2Nlj8{2HQS)qJCZN-79jP=IuAWrqo^nE2Y*d0oLu zKdzspB`SGpaE@koAE{Vx?fg39KmQgwE}rW$FTk%>#!k!|Y8$FU6B| zXSfx}RlvLlpQG_y<)-55_}3U8f8F`b-BQB!Rsjupr{6;JE^1PLTI0EJiS#4*ilCk$ zBG2W@$tn=(<^l!OU}t+nR8m+6MxFJgAc%FIZf|)IGw-qBcXf4AoF$5LB(Pg8_Rpl% zuD_;9DIVMce+S-3)EH2}jT&!%lo%yvs=PIASWUrk=l(Z+xR+;05$$nm)JCAcar>-uSA8T(W zT)+OIp|ed&l#Q(>Wn#is?yT3XDbT@1d%4a0tS#)a+0PupX&9NCm|qx)*flXT?yJs9 zET^bRU052QADKuo74d!3?h?*VR>IanRkl09d)(_%1tRo(S+=&n#w&T-|0Q_1!TP|c zQ1?`ZJ__H|*wi#IZoQR7psv5$wPUFx+;wA>p+zOiW1~`qX{qrnE?<7Sal1RWGw5V1 zuGv-Ia@v8Pn@%Z&L%SaEaA&3ta#SnCvMQdLbn4Yv4GBlTGWEP{4OOg>66=7}2!EKa zi%U_&;)7HG-7QJ-x14$3i~>7V>?WP01BA9f9R(=jyb;umT>_E~XZeRL4C-+q=)ghr zz@3)SAgW2%4&f^ zWwj25*1am#ncl&|!42W`j?VT<*18Ia((N==;wXgJeMQ)PIo@4@j1F1`;L*jOFt99j z=1#8G7kdjLS0a`Uj#_#*$;KZi0A^{Is|Ny19Km!TK{V#u=7`a8Pt7PM;%& zZV1_WztT0)S8VJLDWmtOdo;iLC~fjm@$7w7R3V?$AlsyVk@jOTW`NNm+ zE`QhsKHJSi^F~Lx$Bd9^1x)(3vI45nk)?yag^y04;hv=`G=tyFxh-J$ak{cTCjB|=4Ml(-RVeexva;bZh!iL2wi)W=5`d9MPUKCj@+>CYvZ zmxq(fES1c-mxqePU3Qd#a|Y9{>~e&koon-;Vq zlq_)2`dZDH8E8Ph^7MRb}`1(vGEva1FXQ+{c4woyMXOPs)*>?xdq3 zL;7*Qsx?yZA*?`d>BK4Cy^n$>vVpoSETu|Zd*3gYi8rHv%1{_tGwb-Pwtq!X+5z8a z?!CE%wY))wr}{b31(hDS8lLdALUXyUVs20}4&_%l3qwUDrnk_c1_sfA&tNnHv z;%mjdo$@Mb=w}H_aT@5=j$xMi!XTfMD_6qsTLrX>W~bNrf)A4NL9z%xO#y3T8;Paa zEY)|R=D|E#vLxD>$AQpll2qML1zg8O9h<}CQPZiVo=6pT(6t0pAZJEwR{h6vA6}C z0MC}$n<~y3WvRcMbWeA!Ke0Yx4}`8QJL?|+n5z+}4Qo$&hQ`iB{?+ht17-W!4WirS z;9HB#AneNxmJq;LwAc;ey6HN0v5Ql@?!LAlb}CmvyzcUY2GcN$utRKWs8}}T@b#Q? zYEmX|%h2{LQ>J-pQv}gz@s%H!obgm#RzGvrc3 za#?TiLVbVH(BCz1q!*kEbi%5*{-HwsB^T&!!8L~EO>O8-kli0-O9`s#s+%i951~ zrgP{+ECz#zV|Q6Ot$#>m2G0_LZC5Beac#=R72Q1EBD4|G*sT_w-+wkZd1Z!$?|9MN zn$ZCM2^ULveqHw+I`eGrsi~mvtxM;gMFf zi+gKU{?q4Kz?zn>lQKU$!Z$pXAxU4@QxM#I@H1@cR*$I*c(UmZ2pr$c%WqP|Ykxr= zeci2xPCjEJgKB*jIF-8OSPV^|Z3Gw*$a*>|e%dn|CKy=h=J!IXF(M_DdZ#|Dcj}8(10X>qRzMuWh?-1dUm=F@eW zuROME9RK7kK&(e7kY%p)`D>^iJN|!@0Rql)3!bz7$AH|m7MS|C5+y_B|B>=*KcvO! zH!{QD3jT3m#T`uU`phw~;(in^h5T^2<^dHVLoM}SJa_FcVP)i0`ET@* zzvTUMbV)=iZRa=g#xM2pI#j+FE5pM_z0cf|-@-5T+5hbH@!a)x*Wa%Z)YrgD=M0D_ z)PIJOu3boze@(@Z{IkbjIZ6$F&t8#SExK_-v>MEOzQ{{D_h+nst@M|8V*+jJ@BRHt z;@_iq#55GYUAE`srzKZ= z3m>m!*!ei4cbi$=48S%PP5BB|^$g?RB`>mY)W(|-a9!ueEs#9GMRfTJ;`LP|i|=q3 z?K*#Ad2}VFrayn-aFkF6n|OIxlX3Cl=ND!c#%6e(JdfryK!%n%HIRvIcAbfCPSMpy z?38)Eg|4`K&iY1HCrg<+X9`5~*ji^6ak!hi9y0 zHzIZF9~G@25YPWLlJJjE$!|vXe*=^JThR)?J09&NX@zuk)SX7 zAE_R4^Rh2mc>Hmz8D^v?z4OdJTZ&_AW>Sq&S1W=I`(Vb@(u@1BK16?!?4OBMQpmQ* zDWC+(ap;V_wHTfG9|WxqHO`LrhIk^*fp=H{`(tG!kn2r}tGhp4MU#I;)vTO`f=F34 zhB|Zd+x-MIh*>AFP=;!FfF0EN-{}8r;)HwZ?Ad)%>$=)pQHvod|@h`dfLB7V{N-P05 zVf>`H!3bL-$D`5Tt1sf(N%vWQt-a)gnY>ka;l)LLtHfpNp}~x*-;OO+6fSM3a7rqz z@xK2H(P7%gj1KoZ2&foB%obZs+?aUyMz{>zn{SwXbM)0)J=-hz6ryOg=HL!dw3^!b zUD2xgS<$M9X#n@VicarK^%C(9i)sp3JJU0OYy0n;YZ^j(00w$uI2-!p$_xG_hNFAq zZ^5=MCViXf>U_~0E4dHli>?QcZUjHx2wn#WtHbf3h4}QEDm9qJi%hs{dX5a@4=0>{ z&WMnqL^ESVw8!GHRR#L*A*&aM^D(^xZ1-OsmWxM?ap znAUaoMY=5{d$}aYD3g13K8pk;-oe+6qat4~gcd8k>$UhtI0prb-W20-a?Xr0eFE?> zv2sU%@yc;~XGSVWE*C~xhg2*p?W|oa?MZp`1%ZSUBw_?t?iq>%)uMRSyE}*EXTjfp z#)y5;Y%kb(YCm)t{KR{3@af`15y@OBSCQxSk`-Vi=E(~g&4DZy z!56Zdah=;+*OpmVmud-h!G+*aL-F1S?+guHoqn5Il5J4-$n|V63v&H25%L22iM5e|)+sn+9(bY>i(GcA# z#c$HuT;oo^9&cD^MhPvNTG&jyoIrJ(fyz?JCJZrxrE>*PRyX(E;+`IxK2ET%8|_H^ zqcEB6!BT053-emkfjU)+b$Oaj7mpsGt23I>Vtzc37%uO9O*@3EKh+C4$?Km*@W<0n zqj9yDLJ#9RV^K_MD?U4Fba($-FZO@f1eg)3DAr8!0cRhDR5rgej# zcei@GXXuT{^j->4&UYy@3*{qPe;=bOi`U5TlWS1^7~bLniIKNijini7Ovh+`QJ>qz z`h@0s%grSDtGSU{4@FOVS3PK$DWYrW=<^~ppOGtuoo+#z^coGel7yIH<2Mw1x zV!SS7%eGS)#}l0R-1E(B)+I0BkMS*=-oCrRPkXFyQ`~ykg3v2_6Tw76H&_c8=g&z9TI1|t$JixdA^L5PZHW=>DaoF-{a0VN(tN(o{oE8 zC_666$mP6SiY(^%*%MyyE9!=DsO^{u3HTGx>p>N|?||{{cL?B^Q=QOs;84q`sGtb z5Z+I(Ep>>Z)nuGfi{lvOyfUal*0DI9Q#ZRwdyh!qaQYL=8TuPO-En+pZ5$>GB$xAQlFUHEcS$*p5q;x`^XdRRBdP8D1GrI9^INuy z6%boh0-o`7Nhrq)k${3wU5zv@ml*45MlQ!okYHQ8(&V)NT)UD9edCPtY12SSM2QR# zOCY)f_bWXmfjGTe)IYIM&_c^ZRf=r2zDJYWT8~^5j855rAtv^#^b=t@3wqWKC+OZx zCgKK1G<7S$?>Fi5h2zg>`uv;(UX+BYgj>JK}#kPx?PCQT(^(OzWT2p%AR`Ha(z@ zc~sfku=HM0WUahbFMETMRy(dP7~RuFzZKDnOhif=AzjaR6I93&Y5pPX8*8-ToF8ve zt%*Q_2@aeKB9Vlc00JqU)T{I;M!J!#6CZAY-dFOcAF^zs4i|TABBj{HUK<@BaS)hU zwOF-0ft*?%F6SdE>+{@z%u4t?bk!$bDvE{N@40rR&4|BrXLDGHv63aKslS)(x)>OI+tk?Huyi^9q>d9{Edj;ji%QNemvh(Y>R1Fm^Hlnu` z8RQa0wU2Tnlb91E5r{7u9*@r0I#<+zNoTDjzAF|H@_OTQJ0^RgQ!$z;l;iH;dz2#e z(nEO;S{0{Lq^gEAPM5{QMGoGI!vtujA{DeI#*>*${La#}(5}M}DfmX>*;Hc-@VG$>SMY^3K?wJM`mXI z`iZCsg2DwJa%yF;hk)B=gW#=}$7}JaC)3WrySO9Gd!rUklN*YsNB^{C=cR4Wlh&l> zN^-l!IBsR$WDV`7l>&z5lU278_k($k?Pb4V*tslRy|(oD7F}~ zBl1sRL&oY!a_s2n6@TqUz2M;eomz3)RZ!xEusP`dPPzH!3Usl<|9G7fY8BdawaTyI zbZB}<2TfytdO)G4tijxd zNs*KUua<>fRZ+OD(*=fAI~UYtvk{zES6yB>=BVE#Mx6*s%k$3SS;4nRfdQfF@&d}{ z2tr%M*~f4MUaI)_2fnb5b9HwEo4Ake(+^L!+N8E?n6@ODai$LDiYJH@8^a$5*``Yh z@0?lRC#b-Oe@v)~Vu1seXr2vr7X(Qz;e{@8yJAl@md34)qPTmA>bpo@>@S}U>@V4$ zjVUiVPxu!X)!i>OJRL?19zFG^1BD6H9%FH*d1cNz=ch^@Mir->Z?D^n0<9Y@JdBUSGW#9; zN&L%qbtoZFsu)JQ2X%wz7CqBh8Q;|etWpYpVrO&aQt3v#ttzbqia|}dAC2k|;ruBX z>|S!e-nJy(WPM_0HR#^R?P>p{V&ByC5;oKdu&r*ieRp?W6_vPfx8QHZqq{s(V_EOE z5)tkJekggoZ0xyz;CV8#nT5mDs@Mr{XsO~b-$#(nX?{F2yTl-t(B`3Pb4oRH-nt9- zyfGVIswmpe$Cc*X;M9s{U1YTh4nF^TjfUr6^yh@o6b|>xJP% zXYHKzS*xR)rOw6q41PXSVN%y2Wk8Md*6OZu%We?I;|1(8e&~It-=kX}ige4((EyLF z?>A#}sQ7J-9YrxwU=Q8y8R*`E(g%#3jFSd7w~ zccW0Y{5OwUE|P`&x|=W4k(4!5X4=cL-4Ax3i||}6ou01~E}|V?LAHt;q9(js0@a?J z@GQqK|EmxRS_64(JjR^^$PQFHOju|ANHw{+rUY8Av>t|k@;?y+;P32dp_x(b)}9eR z#Zum5vNJaGx|}t*9?iRgNXz#w$y&4+r>;OHl}CgSB*|^4Wvy?WBKFjc-vhkyVy!_S z_o(jfvPKis=5j`i!f)R%n>nqhcu*Nxof&bgJ)^d|zfnmXamA;3VeM&ASl|g-tP>7F zG^jy__9*MJCRA3M#GkIs@+2GZR#* z>@yge7IR=N&5Z>okDD#x_7uP9`*6kRoX2RX<#b`%!(Ln$Jf;|63Zx|6K|OAT z5aB7Oc-Nw+J6Oz>Y7o11((?{ASH~|n>w7;(W)+9BZ&ib5@T^XPz{5cVs&1~fyKlf1 zUEX5Lhr*f7(OnmQ?ij8ye$_@+CCwIQ&<~2P^0pVvY-)EJth7gool&w0Q2e~?AZq0; zp400U72~JCf;kUG-4lda55i!{^Gu-OL3McNR^B|osqMO13h~(YBS^*#$Vz*dcu6at z2u8Z_9X#vBID+w&Y{4sfh$<*8G>Z)g3`ii`IHJQwSimRU+&`pjf`&FQJ^HK? zU9AQu&&*FPSWAzC%%baC{7PcKzw)|CQ5@Kf<~B6ZtC&9PG_mv+iN;}&ahw56E8Wr0 zrWhX>>d@7Yp<7g}Dsdw|-?AHWEKv5qTv7Kqz|Iu5%~Vr<>wzqE;*h)U|6}bdpz2tb zwF!`5!Civ8ySrNm?(XhRaCi6M?(QCfy9XyY!JVN0COMLG@3Hs(H*3v?nccg(rn{%A z>#ORndUjbJX{V*#kz%?!chWi5r3SXoF~y5^(-8sp$_v(p0yYTKmBn~Gh~PYt+;99S z=;yU@xt6TA4LEuS2ZzzSU3&tk&Cr&ZY2iGuVARkKteo$P1MJIbmPpm(Vd#r2KnS3X zwo4AwndIF3#a2HsveMP0#%>d@?rrA|oDR$#PcPiO)@zLquogV9B86sU7fC9lTksG| zQ#n|b3=UgPgGI`>rs0zIUw1#@W$HclM2w$ovSN(Y#PA4LPWUvLLMp6#>U;<$LQ{0wb5$H~NGm5n5QZ$eifLtkn{gR@p+b5sVZY-2p zd3JBM?LM66umAS&N^FIWmwGPZm6-B-9at-~itI07MrkNILrvZSa)@e|zuD}&2)q9; zVx3sZT6>_GoEp!WC4e%qJW(On@`klOd>PMC`e_HDz@%S@2gag)xejkL$M{akE0Jy$ z#uqil$@uvJXy{4+PqgV`@*6_`y%6A|Q)gJh7>M*sg((m%)XMXYHGq4e<`w|XVC2lm zzc<78vVqBf$H`n&69rts7=u80(26y$)y7Xly|>n_x*=I zK48o=(O~2gw2of@vA%XT`N8C`-?!f1Jn;Q`QHbk#`Pa^kwLF98R{M`*&7hBi3TSP- z@<%uQ@-2-L=JNxL33&eWi?yEGWQ>Q_@*6V}01^BctNjLADt$l4q4j#?NU5*j&L*!F)Yb8oEi7Y@U z8*tWAQzM@G&`-!Ndq|{KP@YojP|Y>5urQYxK@w|`>?}dN)Z9Ro9LCaM?+7|FaFu^0 zX_kA`T2_=rQU8+QluBTz9!#Knerc^*UyptwN!?NWxc-3GRi!wGA#JbU-jSK*vi@aq zef{>feKys^EP5^RExX$y$0A}IpB1}{DL}a<2;+C$b@VGVPzR|`x6yrj`rR6S4$d>< zoJ^wy446M#vG;!!o%c^6h`+AbV^!*}x}Gk^&w+K*g=sIRX|Hmj?+zGr3Y9(N$3UVO z+-IGI0=GyzIwPKSR_iBQjy)doRsvjc&*P1-*o0v{hgB+@%rd)ebJuE)I*8$Oj;@{jEpTtc5ses`4xbhtY}mQ( z$(%Mmd+l82QObEh*J5h3Udvu`sV>^27j45Z4L2#@GX_VccxvtJuE$1ZRG0%+y-7hl ze#~fa%UtHAxi{Iz4mm3p(OMhvzUn3=hBhi?=L$vHeq_+Jx6gbCfG)vk9rCTJL)+16 zw4{!O{%NK1^8)=hqL?p20DFoZ|2LlTFPD}6?ESC|<^)PTyfXxmOllhaWULEvh)j~?D9#a*~r*>s6 zgxkx@IYnWgnP}IqB%5@BNbNo=!p&Bu7K3!lV44sag$X>_|%v9>#sWHh*Y#NB8)(hzA-r>U$dz5qAN(*7l}i zX87ac;$z8L`MO$rCWRShCVRrKDKR4>R8=88(5L0)`bq*u3Q8J!(#3$?dW+d+6_n!5 zznV)ylu;?!L1O({OE#&T$?-i+)> zwVe!>tS;Cqw07Lr(3GyM&{tkm5;+?Bp}?zw!r8BeTl-~ZT*@sV+PF|2ASxn_oO zd5DvvJrN=xRcF3(Hu)~60+WLRQ}`hwyW(a34e!(T=D2qW*=Jul+LE>g%}PCz+T{Z5 zmZRJhSoa(EFXOiq84vR(GU^yA31;4Bl2CD;_c27s8LtER^&)+^q_y{MUi*ADD!m;Q zAZjAdfEGEz;btAu+Viz*WEs6XtvKR3T^N6digURrw}9>b$dy)eJb&ulVr5Fkb=MGz zt?az7aNO~@x!HpKbmY6ZLU+AF(_UV%dG+p+X8-8v<9)`lE8T5{#P-3>jFOeQ&8^uH zM#_`uQ#{Q`>(#~BQnP_6c6^^;!n^(gh9;fve61_kHf0oX&RLG*6Mkq8yRC(K1taJu za0ImL_haEI7;Mbdw(Bt9Qj#PqZH{ke9-ym~O?GiS5^sCq{e7^xqr?fpigl|}S_An` ze1=FQbl;U(Z%?))rZIi!Nnk16OOYcDRSEWqvPPQI)PCNrcO-#Mrlg~r>JeGN9*aL9@PlikLTnNvAn71E^vqQ zsU8(u3I~1R{nkAN1*EX*=}<-4@!dCuWOlIj%RCaE2W{rbPk9^o(2s_sC~n&YY7LsM z8p{N8OK$saixfO=U9jnHHZ!y1`^R^K4*mpO9b4%X2|Q==(njkdt6axgD`z9-xxo^B zLhSqZR1KQrE=o@lPz$jl>ZhL%_cR)~>Ff`EJDk7vj4#!ZXeh8wBAP{(J*LXxjOjmp ziG0(rQT)z`;H^44_2g^@)>G4QK|Qy}+xuXbhW5*mcU%_>4NuoJvWt$(_NKrU_TrJ~Zj$4R(6#H_cbvKC2)BJ5D8P)X;Q2ahvf+XBi zKG#F{9<$x1FTZ-|W4NcoY3~_-mc%g!nX`A^Dm&R-%z64-1ahRXfy<3iM+Sb_X+_Yd zL*~A=yp7TE!D?wa=b;W9Zd=w;v<97T@GLw_9*R#c6Cu$e=e-K28zNv*9tZWQD-Pf8 zOR~R6@8~2_VQ`m=b6GPMjL)1~Y zSRi^{LY#cN-b%6nu}I&|sJVfKX_`Ck>sxAwt=W-9i)HQV4jQiNv!v*FK7&1P!8^8r za}h-$5yh9WiCNRq7z?b`x4Y3Z8b#^)Zd#nEmG)wV>nD;IU0Ke@`H};fUlze8 zceds@3#7x74}DN)xa}YFH~V9f=WB-hYpR!|GhEmyl)4rM=2g8PYrXLM)@lehu zqN%02C8#7y*i^Q$%uK~%sMFJUitX^a+7Co??TtFWyE7gLmSD*9eFc#)DxRKtfNA>f zQ(%mQVIaO8eYmQPKK?xVK^oP7;GMQrYk|=~X=cahU@4dB{X+T;uM@qpo20(YIEBww zVKpMA&*`F9gn==^(zWJYA766pWJtHu(!22qxTx&;_>^Ijg(nDAPzQ6X*z&vVzi6*j zucWlo7Z$hO`8u3@=~p08z2`SLgo1QrJpbM?PS6t(X0ZO~dS=_lKl;EsCiqf+Pq`HM zff;j;$55~$13u22tFvbAo|Aga%!dn$a9+I~F~TFCkaYQ$o;dEM<7IeF??)-ln`-!D z($FJHD9HL|!o?klYG60ly9p?QJg9^?Fb-F9%AupR!!ME|D<1ay9_lv8;3vsV$@%IC zWffJ;SkZMxVL4!?TXRX*!R%vLIStm`8Wa8U65w($I(3z84pUKrqw$m zmgR1@^s&EGod=C0bxU~I`odd#6>F{;XO8?^`1_4)2&RI&Gt+h(831#sn{)4Fwv0yz zj6S!z(iEF7M^Z%JyW+5VG`RL{-F2Rp-PtThYaL$^xqzMbH@(koQCCMOwOWpp2m}sq z+_-Fc*)Iuo27KV`4xojBgmIG?@#?d?6g|TBHD4&T1)!W4f5pb8kItFah)`DvTayEi z;jRBid8*gjmGiVNHTmX#umL%%P+%`rNs`A=3s1ANOE3ax}2j7&X=fZG|SjD%-`Qc!NZ2uR%x5&FdvTN6h?db!+p?v?Q%!D`S4w`Iv8H zhs@^Z=A?t3?2TkA$t;`-?@G;RFBAE^n;}#PIpp|)GkFbLrnSe1C=yvfdwlV` zZWlg67=j&v#I1>HG=ZjGlXazRPrO#EM@HlBl>*Z{oQamYCfsn3UtDRUlDD~V0V&Gb z$`A`O!~k|#8%>Ik!Z7U-)pOx0pQ&VF6$MJ6 zm6&T>;|;9le6Rjx??|w`gJaGTGcwMi%>fzuPF-DHS*g~1J^4g+c}rPYRRh(=#le$C zGVn;U$Uyz2g}a7TkrfEg+FE__=kMWxG*h>_BtX2;rIe1p+PAb1Wn_-PopI{|sQ|2@ zYds850H`x=!_gNAh*rTF4E4Ji?Dab1E4!buIpAV#X3LElU^8k1Unl+8gg7(lzKZO* z*h$D~yz=+*KzC&=KdSt3DLDvD^y9yuKUo<|0d)h>Ocetvy~k5T;Q-M6_0YaOVfigu z6Nsh{z2`fA1K4Ug{@ddq$iT5GtMOjzFQxAae^fUW7HRYRWRe;bp}(UgfE%4iENzdE zA0J-*R09yJ{`0f{S|O12c#0;=!E>EgpS0G0DcIT7U-0#? zd(^*h)5&M}_Io{_m8i7=rM*rSk)Mg@lQIbX_%RHc*4THrgEqbq89DYuC<1Z*x#KMJ zZ+5bNa02M5ogW!d^NDf#pS$X6l;b&l>u<+YP=(}6XM;v~CLr3kO*_y1ZyMq`#qnGq zFqa`D4XE>CQ?B&+a;+ls`A^7Tnbh~ju*Q*mz6OFJ%#e8nBX#s~BsXM=PZ61j{P}Jt zsg>OC?fQJ@`vDeevzk`huKXi;tk9&!!E@j%t8(eQTXa2x=IJv8^n7@Mm>4`iz86`) zk?sqid`y-(z=goV^DqGrpZ*fd*LlI{-VrpL>Nb@iM-K$n=H8^o;YRPiQ}La^Zy%lQ zXBPCoUH$lnb~w=!&~%+8JqoNq6+vEM zKPo7pAR(b7r(hu`Ad+29Hovrl0C^8HGg=fceFA$h04|Hf`IM28DM^annVtj?TK{+mZ zSL_+lBRM^HTp}MIHa-)a2S3?aUkinPVXy=DWQCAQ*Qx#dhTxT3UL&tAUbN^+)o3Lz zw4%vJLNrA3GQT5e{4?2gmeNG3_Kc$Ge);quU@>y(v+)qDaR1hW4Vc1k zX(6y&~AqceYVBr!aMiyjE(O#vMjVMAREL2sPKOYZbr|8Ggd{Y=_U&)C31=U+1UpM4Df6D4~S3j=c#OM~CL`A_yk{QY^g0mFr9 z1h%`~qM=%Wd&H(zNZ zfy)L_qH?)BM}Su?*VW6Abhjgriv=grae>vW$Z)VT?mX_eYB+(uu#EGUGP5PHcA%&n+cEHCM#{KZD zL`X4cU?P67FP0^75XlCpU^0CxZ(38H7^R7FxHqi%QRPJA^e2G&YKB*0;pvWX94>3_ zb>F*W%I(JDDY>%vN5Vovl$0#9CDINm85!Ya>m-e6xkTGj!?j@DLR%Mxwgpu!Wm;Mj zA$>u=*>#iv>yw34>F%zbT|KIPMW0^rX^pYup*c9)H?a}^>&NrD>}#w_LK*mz_SxnP zL5O}x8D)-W0&1WA<{MQ0L7h@!OH)@i}eAbS)M@ zc@v2%yF*C{rlJCY*EA1I6&TIG7J^=WbX9Qv*|vL(c~^xu+jYq{KFZ6r!QVHO6}jmx zmU{UVX!S^y|NUqRL>5`ooLawivDTurH|>ha#s_S=+0j#5&T>pjWlnf5Jk}`VdG*fl zm|Sb1)q~pB^b4*K`fzbB5({Q9wh8=EYA?apG5W`#WPnrhEn76ZZ8R1?5gRVF8+AyFx;+l3jb3e>FNG(IOPxhA{1E7KzjM5^fH3bM`u;*Z}~LSc{Dw+k4#IW zlraOAWErOqgQ#r~y2+z>Vp_-)9r$7TNSpK^Q zegwyF9(-q5y%Z{bA7wLY_|=W20j>xfR+;z4<Lc?`^$56eOykkja4GbGN+sN`f~4x$m85ZmsyOXK{dq>AIzh z@=m-`ZeOF}6EJW9igDDsGV#GOaY_zE1jh_0TeNQoqZ#~%n?Z>BIeg=Wg4Y~E0gnof zoEIwVQwM6nDirDU+UBSNl1C zcO2F!Nmwq@!a2BmQLEvW__P2AcxNw9OB9fq)>c7rl%sm1>e-kxd=beb#k0)p-b5QA zvkleE$=z(U+@wM7!VKPYJ7>je%W{h?LA_LpGM)+ZVS{Y+^pqF|Y&?gL0g^{%0tfN2 z%~K|Xo>PIJ)-K`|DVp-AFdnF7@MI0~=N&fY9Xf-NFlEw^CF&3Kn~eBnLyIE4aJ4!C zWG-34fR+Dfw9y*LP>qE2+bE3)gt)b_5GQC2;i%$ z99Zs>Z0?4>38-HhXK9pq<^0N$MjYXTNm(seckyh8k_t%jsFmsLnCYrhPQqHv1Kj&B z-&7{@Ua7^*!XLEEH96_~x%!e}wMH{5b=A>4%A33i+>Y*z&8uL|;I%%a=t-XRdiWRt z*n6SekVjgi{w743cm&(niaoq?m&`3k3YT9=I3Af}RWaX9tsBofqjwpPFtbahjN;vL z`K}14Q~hc=`;5;(4QO7Ib~c*66%lwNnfGiJBT6GlcWR6Klc#yr7bBcqT%f%IDkq5W z*<*3I*8)bieQ;s0CcyK3G9Nvjm=H6imRuk(9Um2hAww!Bdz+edv0D3VQf^;C)syIE zBZ)+{*m|=FcQ3rWZ(KannZ2eQjYD#cFk=o^-(^2q#t`){#JhX9Mmy%Vv{8F?>K7(u z%H!Gb7`0M|z4BPpQ8ZVr^{`cWSGS1ikWqdE_4{Pzc?KxjHBsZd|9c)$4vl2M4BYtGk2oWc;GgiGPXbfEJ{)VB)Kb0DQK zviv1%HvJIvOjRP$(VvVpmLkyg;9|rFa~K|lq=Vl;(91ADy+9R?qks7U91>5_S3sML zJY_%+4%l+q3m?27V{DEGf>B8(lAe<=QbII0Q${O%hoZ#~PyQIYE=SX~$L%h|$9+4s>_H^p#QEuuE*atGkF2J8e%MM*={> z_^oApbY%An7E8U=ci@j3&KGD}q=imo-uP9Y=vMTny;6u7gr9aVa_0#{K(dq6+44VN zz*dj%>Db}Vt-_j$(>Jl}eGgtfC_W%rB+&6iN50w>{n%4Dx^O+UXHt~fJN?2HV$^kT zvXGlFDSl!$B#lckUqY>$+#9D?1D6oDOSP;-bUAz140O!g({r&1#>AfXlW6*cHFN)@ zIQSP=kXliKA}vleLZ(T(BFS-joyNzk+ID({CC z7DU|S#tmq5*7J(Dx!Pz}bv9C(bc&jKPK@N3l+K*++&i!FRO6sp6M*wIJLlZgG)HdJ zKP3k+aFLiX$Mf$pj-~^jqP^UFf}kIsvUGCFZM&bizr5bZ5h)n+Jw7H+dA##)KkuzN zxGmp0S_prmF?FFdZ?1vq8M-?CaKnH;8(C>IgSC1!5+I>oBizZHT5gc6*j5OF;q0~R<2~HJ~t|MF|%%bE#5XMo*J=n=xy{WJw~nO3WK<@t~uBv2m9u& zEUfw+L>x-u?f&`HDBsF)C_`c3P~m2dK$e|+(wITlf100vhUAo=5;PFQ|^@xSL0 z`I%!{20A8M=ASwKTM6*P;Br*%+{H@-GJ)YbGoQ%yFc%jokTS#N10J#jm|K)h1SSA; zOQjsCCBRb1bqHkhh3ziE#T!@CvBNglaXTZihjHRdy}_6ld`oFo+ly)QRtX-59d7!Z z1O#NIglN2?9#%_9U)f*P=!t{-==KUmnu`eD2W34usuy6>C4!Et|(a%@(_{D?;w&6Ol17yaho zf$@zqO@lVTacl>XL&>2A1rObA{}dud-^6=hyX(}MMSjKEYwKIw6)){qu$HqA{RT^d zO#dZpjjCtO2!^OjJJQ2c?Ntdi(wo*!Uuop@tq>_zZLw_pHM_jlXKM>VM%fKdL>)9y z;dIPDu3P!-viyZDIqS8UwMu*lN*?tcx>R<^%#hHNNE8H$7W@!?9q?`QoJ3|J8AZ-5 z;R_96H44Y}LP@3i00-dJ@D@K&USGGma`XKJp$-visymUzm9J4keFQ<%=un}2ii-TA z<;=W8I=OI-x`9J24|^7Xxk3S0EoM|9MuQ{~Vdjd(+8XC9z7q}@>OQ0Qg|#_u$GIVY zA@Q{-2>R@89RxD?Mj7+yJhI*)lV(_-g73>vmXD}DAi)WTFV)G!Y*A0{mJ{Y;+reb= zWwLg54QmqNM>alSiGIQryOqkMgH1gOJsquB#qqFhU3Fc(2SoflJ$`=ZySX2xO`o+W zsIy|ZN)zaI-n*kJumH0hadK~%wO@1$XDm`9@LmO32-wbgYgT`{HGx&vf!$kUCY7^| zAzAwfOBf3DHw!t@q4PxZfq? zC%o|>+yiX&9OoAu=A&-NON2?)aFR8K9~t{iqc2x?S5J)!V{5Sa(|6O!^x1T31(;6T zG4^&8@T$qTl(_elm>-BrQL1@EK4GmoBt&p~F|_t093g*0h9op{rf}?nyLjlF0#@Hl zEb6szEiy<)@bBEOqR7-lh^ehSauy@miO=x-@OY0=t+*)K7`2pim&I7jQbLWSv1zN z@SM+p4KT5t%a8oxIz?8=jU;zE&l;=kVrJsPMe>vqDvYg^Y zdfUcJsf@2k88s)fAdQzxDd!U;I1>X#SxjkpkgC7nqP$TxBfX`iY~SUBVCSI1^@w*UvEeeSs3=dBH zH=yr8^#s$hfpL(AjkcloD};p?Rb<6>2Ak@Du^?&v!zB_w z5d_ZuH+`c-H~|apY3mjfJ?6q@?g-rMK?}5-(&kznCbF@{Ppn*NljjTUafo$)rNWp zSmACrPF94>cU?BxHX;y5A*izhczsXG%}+`A2T7%&7!nlOeJ1gomFnfgNF4@@%N;?UT}Ks+lKP6g462#czo8?xGgLZNhrFd0n35W!0aC5~Y))B~*wDB~JBW9#LLYYL`6&n*y)%it_`HDpC&0cgpEt0F$$9!#Ej=H; zoiN=#tXtjR4>|2zb=su2a~5~7bFy=nb~t_z?{P6#mIeq6U>Nn=hUc{sN$ znQA%S{O;`e9aN1vhV%^r=y)8!t$)z*znJ3F~$CV-OvpXhi3A-GXh3cqnkKR;I7 z-1u$r-eerN-VMD3T>L}wlKV~_y3go51GTQ68~ZPu#Cyx;=eUr>Lz z`h;k``J(XXbc}W5bF2-PM07z(dvQf`Nr~zEX}5>MQ!)&QmEmkCgdCnFOaaY;PrDdC zg6i6ObE49v=^fixa%W+Ebn!Gnf@orl(S~FpageCK#6HKUxh`~{YKx%zun=E`xQnBW zh8>lp7WE)Oh!JfKkUc3%i%Aj2pc`+4YN8KBy|5N06($2034(@5N~R*f8)pLVZCd=VA*o0esix=xd3=*~9ubBgHDwWXOEB0vczTHfFQbd3_yxb?!3F^gA=pj0) zeq6@rtU6)0^0C&psMozOS}<;jbaJV$DE;jT9MUAH;iuLf_MxG%z`KN?p=#7Un?|t4 zIWx|?szx80y?wkZ*6^qOcZ~c=uyBd8@H_3w?06*U@e`2z2fYEae~sB5o(=xja%@a; zQ-gFyrGeQ)&GPax>{&cf-MNNW^xQWAL9b&f!t#RELAbtNGsu>cnYCR>3(I3zP|eS4 zm^9Zj$=keNJch)Fk2soXJ}Xn(dcYo4m|tDJQtV@7LyHznpKLtPw;=wsYYI|}MvUNP z=IHFwz@zyo0kge5lLTJ;tIbe}AFlK?ovowPRt%t-3JqDM!0i@u4eT znjw-CN;NF78W9L@p^f!zG>>Mtuw?hK5s<3Xp>B#p%A<82vrs8fxWmdO3+JWhq>NZf z)dtl-;ie;aUvPMTlI$S#AIPRB?K~6MoHYL%FzEv&7Z1 zY|Pm}bLEYzfi zjv76NY&2RRHoiCs;A0+BWQW6Ia-4iS*T8&49XKr9QZMY`i0ydA`S{qSuR(T5DzBEq z2_Ae=mM~T0mcOGrXRf24>TRZyzx6F)=jO1pp`S4q!n6W|xYj|GuP16{ls_d#4Sdg{ zF6uq9>>8^F*f+yXkiLYOQ_0Qx1>!xI!$Qzuk;#zWVlBgIjf_Gn3W`p=a-)eDku{I*-4r1D0msQk@`G z^Wtn@tnuQFLx{3MwzocMXrf~a^i&pND|i@2YFO!dLYsiZ&+Fi>7+L^EyD$jhNDg1i!4SX~t>c_UvKVEE zH_KBvTo5#W0hCq_!XLbL3+EL~9GA39zhy1@wh%O9zTK#WYR*=D5jvBWi_Oi&*806` z3&-P6@fpwlOQHOo`{z)c=ab{l;EkWrnE&=){t9pWhQa)U`|^)akblGxWKf5Wv zJGFk6w*Pl-N=uy8qu=L1&jxEk7#DLHKr;v;|D$I3W!CkRW&qszPkZ&d_4gOHYMpYM z)uJ#`11l&xJ(AEwmRwgAtw~HarK}zT7}FZAH_s#ojXMx_bqeoJI=K~eg zk28X)pn1)ASO^SEUD#aGODN9cIl^Xe;wJ(hJI03~PKX{!?2eF2tbSqdGl!a~^_p~w zVLeC=e4=s)H#c~}#hpRt01+=)pObrQ6`c5sAVPAc0rgrP1}s0Q)JvhwIxN6|JJC{qQYlw@Cbjps&d3TDLRL7R zP>k18ZI8C8Zn#q}pWOQ9S1LGU2%Q7KvZy(=DA;Gc)DReeJ<98j0&Re^@ySX#qx#TO`sM}o!v|d9B`lMEb8C0nQyH6YvOOKLC zwA~VV+hbZ-)C8W_?@dlthHSKWpn^jfS1Rre3N-ZJVT?}IJ~=ODMJT_g^;1r-E

g<>AQt4j5r>D!~ZiMba@dF6(}QnB^OvbA=FIM^eg!HaU1pTS-q)UA2UbL^a3dXiIzR1aRIEInWt<%+ zhMXlEB$aXHG#S<7kczag(rXY{Gp?S#8F5q>AYgW2M7Yt&r2cH-YZ`E-rWddjN{3}#u)Uy7F_Yc0clN!z4AbLTn5H9 zp&^O;RxS=kQ^v&@d}G3lqo^9xLVm@3Kutw-{_#{o1AiH`hD3&=9bY3w^pY`9}CKA7y>s zmzdP`Cs}>EG=w#pDtYhM^wboY$#Z)k34-?M(}(rxLM2;+{P9xbYg5&rq8010p+uci zRzFZDF$aVn#k};H!yD4hQA^{%aNYkpB)z~N|8kwFHX&Fd1n*F*t(K3}Zd>7y*}zbR zFcZ#M!|~;`1b&Rg5(nk{+JQbYk5s!RW(ITHaJ;r#o?{9TG8}C!uP;(xwdvA#qqI?h zik2{2iEcy{m_)sa*NDhFpFv?41=$r<<=t`Br<{2uv6@Gd1s0uf&j8<;wNp?(h&OV| z*7G}!qU`%U>bxn|+;pi_B_=FXErT%77g?A7^D8W=^%j`Ii%hg4-0VNpM}z;(PZQ{8 z_2IKj`PK{&!x#be&!*ykYmYzLrhlP2dK5S!0l^5{@Z~Y|-rkJ8J_;cOBZ1fj67U=m zAHZ4Fz$AFF18^w^q8nib3>*{Lal$|*!NBjKChtgc>7saIiNjOnP3R?p5p1~0&gP{) z2c`5~kG!wpzzTQ=Gn{IV*GP~9N`G5&IV`*LQn@d|D=73cd#!EC-jkLcHbY}yjgN51 z={;lKw?T@DT!F?U4y08?>lk`Nsh1y4(PUqLpcB~NGeTnEPlBso$^#PnS$ zJ4jft<1k^)OLkO7Dn3&(7xa8ah|I=FMcN<9uoM0zNML1sBx4Z|7XPC(L8X73L zn`f%COjYW8Zlx4v+b-dRRJlIizFkQ@THu5+gI84e8l(62v;j6^j%vRIfWyvy0XD#d zlSzUM@bS_@Nre7uP$&YD2yndmABPx;S?T-rs>zz3kg}^)II^-(@MR^21$@xtHxwEn zn;@44^kJG-+|{EP67v1>W9GsX;HDdM6XHpw#33T;+|3SB4&bM_sSTsyIaCkHB3= zOwGl-!D`!htFZ`We%w!+fpZYNses-_0e9;6sO&@k?n;23=Sri(eG%Hy?ajHb+Qyfg zqk#+ko0*HpwOk(h;+>ftmc*+r>)SObz00V&Lz$UVp8`F*5MgI*`uqVf(AR&4f&R=7{_E86Z*gy>A^%AU|Qa;G;}IA@R64k(wZ7^#dqLdRQ~gdVYe(9GnT7-F&U7r`<4U znD8#IH(vf+3_D8<3{InOs6`rRN@3x|Wlqz`=x^N5v`dl$9lH~D!S@ynQb8DwwfwCi z>H#2-MkX*S5~ZsrsXcL<$1;%5uUC_$ z6{V|r&^fgi;NlAafrQt#OL6(s#PRHC37b3|`muuS(?i8r1|M1xXF!c^aX0<@+_tD0 zcjKlbGFR;m-2(4O=aN2-BAUp4S=#Dp@Qd>Htf5ktGF^5k+mGL9A6@i<+yw`iEe6uGkgSYy>V1Ck3zG#bk_}( zrGAz3LYCp{&PxqM^B&ayCPwX$6lnrQ#(LFs9QzG zMRV@myuw2}k_FA~mg)4s3594BUJ`nN$^57w2uXjO>Ce!ld%)TE<$XF!QjH6qFuyct zdL4>$1!1Xr=XDF!v7!6ex};9=c9O1Kgn#*ZF$M0IEWceI?=H?`N7$V_p1O{W*#K%e!}dGq)3`12T1nDtR%Ny@smb-9`%0U{p!kqQU5QZKmN>b{zsMe7qN+TN^Q^4A5v}h z`=9Gx7poUmEm9NjGQ3phqB+Du5-IX5yJT%0HUu( zkK7e1cy@QRaeDD)IYXVxgfmdEL=kCNrt?~<;S(i4p+z>GbV>|yH15)!S{`XPA`vKq zk+GML_l!9b|5^emmzlSFx}F)f+^l%8e|G=jM=}8^g30#QHlx>kn1)F)J74)3d<8&= zt0@>>BntK}0nhn>`Z?0Sz|P52f%ToOtVB%UHIJ~G>wu|Mn257Em|RA{&q@9;|IOY= zDT>#OX_~nEK!?o|Aw({y3C{vdi`TR_M#_( zU(IqCbBPp$^Q99CGK5r};)1V^#y-y1Qg8VkHzv&7y5klZiIZLr@b;5~(p_<1DzSa= ztL6Qr7JB}+9wc$A+-?Qy!%!X>)xNBzf>V8T(oqsP^*eE!d`A6@DhjTW#FBZD>1e8U z&IBR|v>VOCk#~%?Oci$sGdSBjGF50r~ zyCE5g+j8|Wr5Y03BVOBrH1Yp6wpZMOM6v>~P?IxIfdqAh};=wZzKn z=&g}C)~N8DaBEzr`CWcLo3L=oHP|3QBLmF24R_8G>9d4-2! z&Km@h1Rcs+m&R&SZBCXgTcRii6-pjzG$yf5f2F)(Gc9Y-=aXao3Z?Q3kh&L0wvpZ$0R%sgoa|-C6jTJd?eiXdYYomgEQZ+g^@r|wtSHdOi}P={=e%~vmz*JB zn~o6CYntPU8+(J)+Tii0ISC5`)WXZv;+0@@$fgwDk%MwjD87f$c36fjHFa=&2L<0! zmkHzPgEd~e`y++wj`6ElDl;}}0~Z(QejxR0ABA4%pQm9AyvLQ1PG@ zTYRLDV^<+zO#Lx&5xB=thj)B%co}+-6g2vY^w;h(_t+bpFgtV3>T4V0bhm=0&g(!TfB!4zN@Cc$Q2zhP8f)V89q| z{T9J+^do}7gn3I65W$cHTm{FL`vie7ybOq7kiR|Oe{9HjSbfLi;r?yoJoX(;+%YWD zR&o*MY=q`A9R_T?te8vOu%dXZ)sJf2ozX0 zNyRE#&NQO;)utFOtCEb~$hy3DoSe7HSdm=NdVZZ`#J}=F{07GTMfLv_9;wUfb1c;GgpHyiM`MW@f>2$p@c2iTe?|@e z5xDyk+47r~{I{(BF4>-S^j>m6nhF4%n+C+f|0!tsQ)tEytMiZG+)tsTe>*!>9<$mP zgp*P{4kE|HBm6Y7u3Yd4-zNqmTFaR^S#le9HPJ0+P}Y`euQjJxv*MRPj$pLbnatq% z5`l%#5(^#y^+*(p)kc5L5~08mvCjlwsLE+4^_!!kqaky_V=A#TsuZgkQ5Q1Imv>ib z#Fl!cmrM~Lz0vg*LbUS78FT^Q+`2lVpCC}+_+A_v#iG3u_dAt=8r-2+YQ`X#6h{tx ztwa7Y5L7PC_+ie8A)Id02mE8Uonu#Urx9?7ptvEZrrAW2_}T|)a6L9QZ~L&kly9_~ zSSy1NMYzL}iRT%ZE zINCv>EO#vUupw5@!NC9@-aYNv^LCOn_VdhmBwdd~RfR6PWn;Nb7icco&55JL%P}^i z)_?$)(vBsA*SVZpl@qw(56w{VklfMFxZEOz{TN?B?lAaS+iW|I2*fs#Sk@Uo3>j4U z{(2egfc;xmS^f_jp=FQhVQ-?c0!PW5)&WWe$~oUd1Oh`6DydOdLiX?4J+x9Oqh z+i8CWGX(QObvCZ0TD`c82sdvF&n&t&?;a12Ftewe+iQ9?3w@$z=gnRiaQb?AW+cK0 zJwXpcaq*y@s=&zML&5<}2b$iC+Mc+>da6a{E^K2UpvRk;=ne_N8S5*H!o7f7uj5qY zqNHY%eZw&dv=27XgPm4yi|!Jyd2cW78s3@|SFNp)8(mTk9r7!|s+&|>*mJPgE=EK= zL?GU9ZGzS}9Z+6pVhGd6joPx<&DKm4tUk0JP&&=4*t#B-n#xmdQ4T2|mUay*`6icO zWI_OvzVzG%HsDYx(5oB3!w&Ap4z~n*>2UHus<02Hp+(lvw3)qci4ry3X>`6j5wFN( z_*DFwQp^?KlUQQd+J{z)$cr_(G~whlND$jUG(b+k{9tG$pEPcWiin?;fBzyEzoahc z6BIbb02y zV{)qo8Jd6QQz2SlBMW@(8Ns!2v>4@-_y5a4Z){9Sa&khfZFJyTDFABg;y|_4wqw+S zMhYTL$9GvOMII%`;%IaB{UrOB&9%P{ky4+4Ct2Ve5AY#7Y9%V4?@ z%iy;e%McC5GCYhA=O9wUD>L-D9V}FSM3ZMeIiUGNK*Nnh<`S{!bbJ;9%~ehaH$%jnDx9OlLqSOjMKLXk6{u+bu`16t0a zb+L$2tZ>pbj^(Qz{OQ6Iukf`pKTcTiSp6@~1}XPiD-uS(yn?4AF8;KI$GQ_`+h5dp zeX=7rlVMP|_Ush-C%i9H?MJyouVFEl7q-Ne#jwwcWOJFtWsz0nw$|2*J$+&1jGp;b zg=5V{ZbcG{+LzDFAZ8W`bDb})9^AH^KZA;)@w$pe${Lzxsw(>o{RN#ZiR+iQ;Q1f$ zMFug#-+2ij=>+!c1kT-x=d#2-C79ZhRJaWrEb*{q?Z6v!pe+_AW{wT0U@7ydok704 ze;rJnu>N%>*8hJ71iB4?W@b|1U}fUsq*vimQKV;6Rbiv&QsYpe zXX1pbaItZ4ad9ZK0Cl=I5=nAG=KP!E5VTJBF2I09l23ekaz9VNbst{zd*}s^dT@%e zhr-)i#^+Vf>0gpGPcdmpLj2f75vet)_Nkxo@$aJj^c;<3Q4tFu=hY(#ejFyoiyfN4 z>4a3)oIxWa2M50)Z7pX92Xs1E95U7>#2#LAb4KStOTTs zPxP0HB0DJ(-#{c{8DFEsa{9$0=d-(CV)Wjgh@dpAyDKH7ACpjdDm6OpXrCj_8rv<| zSWE&&2Uj>w4HXjOf3xSIQ%h9=I*p3w$Hp^4$U+r_EPD-g|;HWqht4yDsxgS!mmHr@=#xv!v zjG8G?>T8yaVwD(XtD6a8GnkQOU4{cDWC;i@77bAhpQto$Iq!N+6PZh|CEaQiVl@eS z1DZKnCEi!v(X(A9^cz_zY1ax#`wmxaWb`aw3SX;E8{%&@pKl)Xw8{9;k+Kq0Dn<5v z?aQ=`_Uo>Z8XQ}xnMo&lzcWRn@^G6%>O6ft={bR>Y!@rSk>t^bAN78O8|t^LKJ1di zThncb@ReUIM!XGPEBH1CZ_r1KB%0|pj)XKU2dhpNnszaAPv(y?G|wz);#seruX;Tv zpKVs;JI5)7clz-Fd19`_lVOgpDL9D>MH%BxR;K2`xeEyg>#E(f^-1UJe;oE(R)5j= zEal{+lBDakP6vNc9dS{O6Rf$aHMz>nJ(!`{8EuuG8M-s9{nNc4xw{@%Yfh_MXb-BC ze@rKTn|b|gqSUvmuTloY5BUvZ(82>lEp+=sKAn`}_s{sk0@N?JnP^O@@Y8)@+Vxsd zT=rwjY+F|w=bC9(K6bizLS>UIOXsrH&x^ewd{co(oL6U8BdC2A`IBvK<^*KkPBIR9 zkK3bVDzg)xbyq;DLL2zry5pI^*;02t- zrHP<6lGWJ&o+`SB>Amfk3)sM7i=vGR;P^$B~7(P`zXKRM1QyaRoLYn7q4#19zCEf?h)Q;AFo%B*+y3T zo{YqUR_c;wvr2bb5Nk?=R#DGju0)Xx0TEklX0XJ!1W{)uLlu&y zaJ88Q-TN=S@wA@sKboIU;$@BJOTO{Wy*rI* ztc6s@%tp)hNl9K>{d|GD%|dX-)SNX9;YHbdnUq7Z&Q}V)9er^`ts-<{>{0rvvK3V! z*ov>FU>TH#%2xEc6pRY#ja#dE^FT_EKRfbMlePL%2hHON&8Ty2kES!!IxBl^`tZ zas&?#H{?yc9?1&AYoxTu4$INLd%hspX60Q9!kRHth|kitFkd<&mlT`EOhbo>QQmf@ zfQPs(h0((HVV!gQY$N3ll61tUnC}xWTc_KnlHSW_e&ws@vPhqc6(4yS%=aibx0x7W z>Fm~}JX=3z^Fi~L4X{xOCo7eHeNm)k>bo+!;p|=0P^J7nV%r$vHAVUPJ51w^;iR)2 z#|0Y=8zM7Q%uju;DHC5XR=BS>rHTF(ts**5H3V&ZuGhD;tsBj4k!AV9NrCceWi$mh ztagbdBEPYJ}T*$ zY-q3wA`wMc#y-IGJX5PE8_5=3Ku*NP$s$=!ie@!FRe*cNIyyB zeL`#IT$|BXQSou@e1%Ek*!t%7aju|WwsUZY{IGNEnaxE``RK)tS9DB%T3PeyD^Ga! zpG~yOH~p^3mtQtuochk4lvsp53Ex74Uo-S)b+-XeP*mrEr#Ux`uW`l77?s@Wi}r4= zc`!D;;JQ@Otdrl~_;gm;x>6v4%9HexhgSwR*QfflPF<5s>ad?wS?Kp{6eT+5uH5K7 z<#Gu7ma(seqhUjK%-x>&`|7c?YA?!DsOK|t3`7jZPNsaZc2>B>KdEq^Gx%~i)2b7J zYae-2+^3Z8)O0fPSV97qa8l|A^|it2$(=eG;of?sKvQgu~80hS>88!Up#U4#T_aOPOD-HE_lG`A+uxWGy7p>oj{z^0ewS z5YT9(STDF&=Qt6nMLoqkWfCMb;~_+*@5WW_>QfQNDwBehfgOJ7(d`($GVkIh%g4B< zqV3YuGYG0{s~v96KNCv3`G{D(*`sToR_ua!$Q`O#oTP>H*^76p`>^Y*r6(WD;2nJ( zAvIKQtZXz3zuJ>CBP)aF@g%|+A=46Q93}dq)Lj2#!r{ZrBZE^Cq?5XYW3>|LWF($f z`>fjfV_M<0mNz*3f|9!@fSXL;q+RAb2W|^|pv1rTc|}==my_ao2L0<)g6fZi$1n88 z2#Ki<2^RAsp7eJToZyAu>dC5UvaHVclyVOj9!+y{bs0Q=?cRa8Gdgl6U0vPtz8{6l z@$|V!d8EU2hVD*vv!$D@EcVl@sdLZ5xYzsEETLL zL;DGfE-bjbKKvQEugl=sXLRzk)3M_En!Y!Krw*=qUzZBGJ9c+$H2?OA_iXH4Y#j&> zHFuiE?y@$Sqd+(ySyHz#T&qYfi;0CntnaoMVP*eK(XX;$;;X-C`>JopXIZ)zjhL65 zD0$Z6UChGJ8MnuMG9#=LFf zQu9tU)=ao6bAnzc<6O~qTdiCp-pA696^5{XI9`0W|RbZHEcdNQ=#7+TrtY_ZvUyJ`T-sPr+T`IMAy{P(6bq@;7h4PrF9=Bf#fHuVpDb-%ZNxI#T0?L`vKtd?~j9>0U82$6He%F?hKOw ze|N`s82qN!QO+)VD&a}BV&yjQcNk35^Ns+Fg(TzDTUv&4@XP<)3j+Q$3LVjd1`|G> zodRRQlBxslvk3LJ^t_%uJQI*Yci>xp9kIu+^_-VtbYztvEW6Q;WZ)N+C;j81nR3CI^%_0({UHXnrj1lSSmABAuzA$=o2{nCkQ5e#B zlLGe!;#iv2*#onCK1%j(YD$(g>f1H6dVsmb;dH=90d{5f0*r62{qBn7!M< zBbNcPsn47+D%?K$YVF741R%N1tMxOlD6AGeqhf<|;B5iAn44znpC7kB!wOW_vEXzh zz{_{?s*^KYub~Xx6)Ma z`w41aFt!qLB2J0^Ie79XVTBn+Z@F(^>WqEIRFptuLmQKGk}(R+=O!6QR9v#e|I7ene`5A7q0OP)YgLvIC=^JFx~f~TB* zO5jCl;&W2CT=0j;&PpwpLO1$cc+eOhzdwK zyOX<^$BM`jLJal<9VXx=R9t2+m}5}>F+`&m?8%UBG~1(uAChghO5*SDtTB36 zXE7}FeQrQg11oM$-`sb^BOOnLd zi(he~71+-Z9e!t^V*gcYPU49{z5L@}4?=!@o&3>Jc@TeT^j)D-mZ1BX;4mc*yy$b+ z*fatSUyhMHYiTLv)9F!OX&z+L96?I0Fm*cVyTs@37OC4E; z{*Rm?)k#l^obuLgR)#E}R$P~F{s-o4!<+e3R>gb9Gn=I2L!;@B3r_r)&>F$V&+Yqe zAu6OcdH%AEkn0(P4$@}4XH#an?%o{P!Kp)+PueCga;;&Fw~)D3kc7w1;Tkp-gdfWY zJ)I$ocR8&^cU6TQ|C8Jgd32rVc@pIZfoL^tq-72U1GZ**OGi#OUs5XW8~q|~u3&M7 zNc^Og6|UTHqssFI3Tua!g~YGt-CM@_%H&cohm*Pp3uX#)2l!qO7ZIS9nHxK*Po4q$gmyMfgG&|33scW@hA!Rfv6NJ=smQDI(#H6o%UOpX zjKuxwR3_47Zr5*Z)d%+)l=ra3IIl2_pER=+d(5dS!u4GHKvZZ4Q*lRp%TJH!%poDX zuMgcUF=Llz`@de%YFem_I2CP8UN3c-q6k-TfX2HElh7E-OY!4{K`@4QfOweX~d;PSgjLFj&h5V2 zRL`wvBkV9ma)!RR+%>UIA`lNf65)F<(xO05U*}+kCTX5drt^Effds+~1)W3Tlm~5F zISjwdyD@$ezkYv-A|o8{sxkH@!H@bE%3jG^Jxz~rke*Ig7EC9sV*fI~#M@^-(x)VK ztC$dDg!uYBrr9d2o|{$-s$YiAxRtSXRoAKOkE{sO$`ZakO)q*faMq_~dOV&#Rot>5 zKNqN(b>JY`p^f8Ypu^l4$u|B+hk+AR@WO|U2!cO>9UpL~1|4%hQ08E`)fAUK0+FAKS+$lFMpJq4z%_C?49_uN$4 zy8~x|4z!Gz2;BxKgDDg2ind!M$nV9IA zIN(eiEO20JoJkefR@&a4-`q~p(N||+-rR257?%KWHoGxndyfnI)!EdiTjL~@8HB)8 z5DIRe*qFgu@XTB&F#g{1+CsOhCT*j;n)L3vKClBvvAzQbMRyT+lz^c7ufY9X#W#R~ zN_os8I8+BfC&+%~){coH6v!q$cC%-DG$xZoSk3_R6PJtKVxI!S8LLz-a~muU`vo`lQHszLo1?y9y z)Shi=dVH~p{|vYWjU8@`UKSbMe|<>nA%oj5@Ai=C z7Pvj+3Qm{GEe=c60-W0^S7mu*aQoyO3pj4Fpc>o%90B$iH$iLA3^^?j18}l%hcy%{ z{%`1*SlEHg?QG1P$kxD24_X7~eym~VVghtzTGBzr^*@eG8`=Yo02+>r2T;$tl68@r zV%V>Qws?R%>-s(8+0veU3)D99#EtMFGK5559$aTBw+uMMAvccD$NZq3TkzcjT*5%z%a)h3_JWB z3DiV_=ZFQNR>=*I{f!J3wm4Riie#xiK4OiruER;qM@aKG(tSN`8Bk zgn~V~HPZ#3+r#3w$#2#S0wC||0XAsPn`;hy4_!>Bi9xB_br`8Y6I^@bgPz3M~C=fuVmQ2NcCzDhd=(W!NJ`bBh$o=vFx#$A4b+t?stwVW$!XS3&!mj<_9z>CgD?kyzvi3r-9s&ei z6WCca>feY0xo`p$N)jq4+oE9mOJvr5Bmr6_a=v+J<1;{|*3BZ6C|%S3v(E!wav$c2 zoJSrCB_A-I746UfC;$`&xupX8bqSzVA}7^^LU@M?!VV>bNCIVw&AknBXU;k( zii&C!jDVonDTpmLtSjTLQrK2Tr0+;<2}N zgYFeMhX=H8#}FvULyT)*PG)Z-+wLiHZVo7vZ+k%5T*HD+iMOP@H-v2^N6yayg>a<} z1vemiw#QWP?~1<71LTl;D1c+#C;&jj@jIs%)YI%N<+3*~wxJ-0%R`}j-UA9Mi~kqm z*oJ@{77v9W)3X;hcE*KoTHbz{uru@>ieLd1f*lHhIuIT(&3$-W+nT*7dJk#) zZ^PbxH1W?B803e4)`9KTA%V<5r}kLb0sRef7&Ej}HW0GEJvwB4YrjJK``MI1pbcOX z?f2o2U6({RH1JPS$ZLBS27i;<=h>7$sUdIYYD1{~i=#8UZ|8nMCiow0=YkzT4snJ` zk78(>9sg;-GoV(x%4x%KgTQt_!$7@?1njqFp*)1Tzow9nib8qN@<&tu?ZMIA(4k|1 SQv^}qpFcGWhJyw!7XKgGwIFl= literal 0 HcmV?d00001 diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9.zip new file mode 100644 index 0000000000000000000000000000000000000000..3a93b28687189aa2ca950c63eeaa1430efaa6c19 GIT binary patch literal 68838 zcmb@u1yo$ivOhdPAi)9&A&}q@+;wnw4-y=Ly9IX(ZXvk4yW0T4-QC^Yo%x2GbMCwM zym#Nb>s$X?Ywz8=ySlr&ySk@${kle00_Ft*;F;=vD69Pr{CR@~U;`NF=$UC172p9d z95?qy-jDZ2c8-Vus27(|0Kki9KG}bp{ncOD=@|dR5`rTCRRF|)DuJHmk3xJPB49rK zA7aFRooz#c2={zZ{A(dJG)9(|ddB)%R@&cejctuBEws$_ZMF5ZZMA81zv=#?sn5;* zTT>y)kn*4b+B?$W|J7`vpUP^4^GF{sA!*N-(%3eY%ENS+H_3RbXp8_49xU&3{3Q_bj(c5Ko&Y0GfP7ndM!F?dPZt`)_({` zcz^y`O~WNgrl^W*tky@8pUDee zXm=FD@=vHN6;L#NwycV~6mEatw4_s}5cguWv3$}Y@XY8l>hL>reVvGhbJy}TK}NEq zsWe~$GD}mZS$^4GB$6DWZbOuGKzP=iyrN=tc@2fu$fc5pCMxxVZg!pc8yd}kcb4c~ z0r2&8NHm2cyDNIINP}uz%pQ9Jl;^oAwQ|Ye5hoO!yk4G)GtsBW%$H2k)1X>auzJ4B zZHT#mdwqs-+;uj%wj~(utV&#cvQz^=TE~Yq9Kv!{_yi>C~i_OynEsB($8;X@=?2mP`5; zIa0_M6r4Me?h}vI9B7@6d{aw!%x_1D_qiXg+K2HPc7_uDOsC+dc3p;OKqivs$5Q?n zv~`;mB128S4z(QN&a15NUl&t@ z9JYh3=b76<%h*_#Q+}C<33!bLhaKC8cc<64hFa&NzQhH1`$M05D#eIn0wU4Als~5Q zP#Cyp@`DY0>r-Xf`ao4=vki(y6IfIjdr}x5i%wHFtMeo2NmV@*)>M6O$_NQEK!#e_ zRc0HWpkROM_{&`M^n!H8;E!8w$Mc0PSC%)n(<_vBEsXNZ9(`On=7`O;JIv-S5@)n! z#Vzgjv0EW4+b;#qTOL~6Bucax*L$)VBc% z4qCoUIpHn?kI&O)=6i$#MAstGW#+Qe_G!Lez)RKmX;U)9u#d*R`nn zSkp6-p`s+FOgE<`X7N>}PTJ623~G}&R(k8PO#|D3u=sJiMr!B!$#Rf-i__PN=t_@~ z?PzcICQZES&nZV)?HpQs9I4>(t|PL<(YKxHh2 zS~mCT8bc9@Pns9|eW626;3kX5b-V-XuCtL+L^BT817<_2+!pE?>k?`wy5*G1wyT^G z?fRL+n$gI2eM*F_pI%ELk+2x_ULrekP8KFZ^9ReIJmq^y~mmx{wSc2jUZBr!dF_$lrK@F33l z@u$qDPON#eQ@f=_08P$waS!|1sW@)eUT#Cljs3J-Zf&^R-?>V1F}^e%g*@8NrM3ES z<+QNz)SsM|Gp9DKg>336X*JHi*KiRD!XPy(UdFZA>GN>gMh0T{szuOBE`nG57$eQ{ z6QDa=6DVOKf*xl4M`c!jq50d-_yG~xTG#y4?r~^6k7;-gJ6Avt+e552acPgrjTw=E zr_%+B$vcqMXaCGpLa0wjTzr7A?*Lqz*DVh#!NZn|iYIA0#w{?Ly}>Q3n{1)6f;ks= zF*2)bX#Oc_H}LSCEsq0t?Z?IW3AJZkv+dtITT;(?&(2ca?yjzDTzNRthpSu>VpCZ! z>NZo!6ChJ(r`BV3PPR(C{PHxkAK@o@@f@3U3h$)EI$e}priTXO$I>sRI;nnD{d!=IR>Z3(>+uJ4 z0ot)*5VO{d>Q%5-jhVZ+wfe$TF6xHI8I#~sO}GH61xL{W4lV^naJWfM@_U54kKS_C z$}|Bm;?W6}>WDB5-9a3kwznoS_`m;nEw2@4^tlDqf4|o0L@prPho6r z7RAmYew66zN}!dd`SeZ|6N`(CrJ-v(R{iqG$g4n(Z=6TnS-wYO8daK{)>Vnl%FcC2 zsP`KZSUQDF$;jI09oA0_<4c|8vrH+O8&hr8dXa+5$;O8x+52O!yPfOtEN9F4nVs5` zI->xo686;PV^a#8w>(+wm_|6?{CH#!uNqb5NnRu8#$W_+R>`VFI9X8{cI(|JRem2v(X5>`-qtaA^qM0OC=}dy4 zmohInNFpfwHCJhxAhxVoFRd&(sm5N0xJH&{rUIU9iqgu?D$e;@#mN*oMW&m-m4wF< zf&M3Kf|fMa+o1P>+rjR8lj3Z}ne&Z%oy0nN0`RfbRHem5pZ3kOvNmt`k}24-GyOTX zPpuCQmTO+I&x#V3AIRk$EA{2$kd9VMu@uT!QnFaD2S?R0YdsFTvZecQ4bdE1lf;j| zgo_O*SQ|SDu_){(WW3S#y-P)K0%c-xM2R8HaC`!hNx(V@SrlUv-FiglF6I0bx!&4y z@vxqER+!i|fdo*BYt;Nja+ui}b&-NfJUy}$nRFm2^40P)&Y}=k*f_+zd_8DXfA#XdUp}X?-nkH zd%V*%B-f~FtiV6ybyt_<5cbVhcbJrW+b6r2t$S{w&J`twy`dZpE%o?}9_P#)1M zI0*09zAv=4PP8n#3@fJTkA}ztw8#8O)nTWPgYe4X#TtUL($-GbYrYD*a!|ErSkn(> z`epJBg)j@-he#=ZZe#jW`AW%@k4>3akN0|=m=YVBrde@y?sDqrWkWlx(m-ypFs(T8 zj*Ojz@$yLYUi)t>D84hoOJt>~=z z_q`SsbF`#3wmU1``Wf!O@@U=$JtkDDs%C$?&iI-J%6Cpz*q%DzdQ>&98=DB57Ixv7 zV=%Aljcuttfa`%Vas9bGpM?X?G}T@IY5RK9b~MC2tJw1DezR2S+w05fIaqyF%QV%e z2+khL$-2Zxg2NlDpV|7!n;5%`m!f{=8g;n6C2o;ZL@G5`71HFjQze3?G^+J|g_B(v zGh-i&Bu0OFs3%xRY8iTUB&@EZqZw0$$Kub8;Lr-0!c52XU>RNAWPb{TzU-Em7&}O! z>pitGE*4Zn)0ns5i|X|-ar?&QsABtdA9`GfQs{6jEMCa(iq`;#;&w>s$LmSkZTNYL zxsDsx%k=Ljgyic2d$Oe@TW>$ienIc_FgLMM>Q1xg#Ro$$<5}Qg#W;y z6$y$l?N-~^`1aNbMDOGo5!C1p9kPjzJTjFHK(3j3btn<3fM-6)H-C2a0@}S-qg12e zYkt}XTe}gApWd5xP;44bH`5;#>tIot3~TjPPhUdYtvZWJzn`_;G?}3^Mj+LqFFgCX z8>xPN;ceiEudH?ylWUf)&|EgVPZ|C|c84{%82-AE?qIL4uP<;bFgg-+HCr7WJ=M@x zW*h-FZG$&km66*_>bpzmABbjUwvoeueB$g>h~>YAJRyXyG?Bv&SkaS|=`ZqJXq|Nc zaOvq3!2FR2__h$G@Y3_$`c!C0YXT>CUWl2x~Uc5C&R0cjjyeYUlxZ#Z#fVBHM0OUg$0C0v@YWSm^G!d&0*nb4au&2c# zsT+VHHE6)d8-KwO@0UtfBrHq-zz-xYIzW#)=wlg>P6-OoI-Nlh{m?~tr@Ly)Pq6Iq={VB^xVO=csq5e4G z+8iPPz<`O&_RJ5!DX&EW?C3AZkE3(mAT_126w*uq0D9bW#2Z;?&5N|z!+a~h@F9#1 z-tI&IJ%2RZY`~ApI%k+S>S^BC?cCImq_t6=5bbT$p7#@`fCj%By!h~Qb-<6pv6ah8 zNa_d5$E%!iT0%Vni$qAtfT2Zwlo(=9g|+lBNGdO){RosEm$WxDZ%vx;v#id@KuW-` z{PiL%&!UdVkB}broam`(W*rvzZApP}Kw_}F+n+Vk8LjDA06-7l0n+pPAH)kh`6>xu z3)q+V<6X{Nt^nqjSbXH(UmSqlP?+9-y!;NN_1B%C_9#^UF*ka6LD+B*1#4$C^Fm8V zfn;3Kv+lxl^Nvj94bxeF3DBc|o$zOkyzDD1Zy;sp(KS4`7ec(ltG)05hKUY^=LUJ= zb{>)cjtBf`mcF{d*|viMaQWJZ@uIA7vHp?pk9!Yb)<)*7QX355hgzM}O8{zDGd`ri zp|t(yqOy9jAVuO&TTA{U!(Rc7KkW2 zhXBnVdSCHzm=YEMpk}4~b7-=lXd0g7{Dwt(JU*H9yZMk6Cx`rZuj1>D*gyl)Vo$b| zAUt|!=@@sFfDzbuwLwFcEe=SOCo9MTQr@=&gde4_$uy0N=Hs$iXbTx z?^J)6h=P?fq&4__IUrmDb8AlkTi`lG-_wx&HC!|*hA?uQp#Y-vA9Vpuq31x}kNp|v zKC_O2_|L`vML+E8ZZH9;{`ECELRehP zgfkViM^&gv3Wum>&Dm=gEhav2SwvR1nK3&RN?EtqYmFaGPih8Fe~LF}Z=C!(5l@Y7 zkX+o*RaE&UZh)m%VLjqgQ`+q?oI_c;3c~K5HxY(dD~VQ`dsg~F8Z9scd)@fzwuxww zrLnf_i*9XV2yUT*ZgqmBW|N|2wZwb3>P30&(h-+W;nQ?AH#;M5`B*PKyzf_Cwe2_6 z+rcpZYWE74-qBWpwDT+EBK}vq*T1(1{%^KGc^XkFie9STfnoU`DQYoEF)FbQIeW@J zg%cT?&vXo*>8i>Y=&Bj$m8|UGA@6hyo8m*mgM(D?y#rF!u&2n#A(BYcNRm>LgM&zQ zD>?O9@1yi)KH5P2wx|AKuX{Eu8U*p1q(O#N6{34`f0O$^G}!$gcE#U~{5Lkm;%F%& zG(JqfNKvv^gyNr3npm5}riO+r5>h1u->DwttD=g3AmOO&T{)0$;4m*H#{Kr5FF$R> zN9lC?(^bK%rE2e$)=YdwT0%$LiuoL4VCa|1GM@86#$!BV*H}Gn~|8C)@+FI&WkDWF)>SG(g5KHw6w$D5(eqM%F z@4Oj1;46sCHIy1fu!Ik~6Fs*1N$%21v_YK?yYoq;xCthmP%|?V`ul`#BPr6xSP5Qv z@p32jSGW5Qqbfd`pWSlcS@rpNJmFl+c|PkiAL;T3?N676dJdTFBB)o zJms?~`Je=@D!0Sl(Is$Whk5JIwX+TpXvTH(Ci=>=ASIemNfhG{=J> z$N|m_+pYyiq@z)BBfWo4rI)3*H&BmA=g{uzn*n|)h? zM%>`**RMJ^W!z_y!wFBZHN|*WdKWd=n%@qkBw8EBv_KN9;dl$C1&FwC!{YA1t&2dJI^iQ1{i-1HXf?Ha;15VmU00SOYN<544x7M+ZCj6RThyBj@cjdK* z8vJ*>H$9*@E%|#=^TsUK*$B)_d@2XvlY@x=VR~xcs&(*?sK-*RgS$@~R5ARMoh94o zB#S|T0w_Pe3r=TQY*^nXF!7i!F0lUr7)G%6ec{J>-4O`^vmabn|41k6R^EWX7Ntuv zrBaIZ24=AI6pEg8a-sVwUCo_9x{+SJx!i}^vf1qx(`0&@TFxl0;TO83Q_JoyH#-7w zWhu4sZ+iB`-AhnmMrpHI-mRZwX`QLbxSmz!oJmWnurp(;9QZ8-&*H1H(lQ_k)}{o~ z#8vJoLI)I=3v^?oI^}zO(DL314HB^Adn$*!ZuvWpe) zG9Oc8_*O9~X?Q{Z#{bxs>ZiKj=2*&tl6&RWMRTZz=^4Cu>F0ja3RqY z^$^m>rtZhJ^|DVoOYTU(x4{nAQxpW}@N`I}xVZiRV&*?gvZ&WXqQMIAU!S{uD>XQ$1I+=ss zp>+eVCdKcisXa6r-|g`hq);+@Jk|_4cFBk?79{^whGV7->JhG zaS9p$uv-rI_pQSJqt);CR^i{Qd|d-tlOfh0CCBO@uJVM=_Qll;#-bkgA4H3{Zmp8XgCw-JOk zPoX#70;LZJW|6ac(&cdjVu_MDQCU${8%LSWMntI(Qp=cp%H<|bV_(Tr$pW}KO*@HF zS!iF;2aL+muj+hnNCPb#930HGJ&=2&x|uEA4TTvuBfL>E>OdpqR-P705kKKnOfA#b zZR;*7evPMFH04{$T*9S*>JGYdyossH`>=kIO3P>-J{%8Ju3UKd*$JX6F45x)q1(Re zX+n^VIXta#^PAuemG_WvvW3u6Qqt=~!%>ARm^y&(Tr#x+} z*?*8gE8}(>UOmm#ke1pkEQ(XR9n`~7no!MlJJVFl#dKS*MeQczqWi6*s$}|!amaP& z>bM0YAh}zs?|pUEb>MQDiL36RH_=Ni<=@50=5*ujG0-;?V>(k=J${W>Z_s2l-1mlK zOlq22k@GZ#NC0_`YN_i*GCa;HBFPcN5AsQR^1Gj zZ^Rci4&vZPH|5>k_KqSu_giLX&!g^9($21%`y6{Lwhi10Ge55_z(-;D zt2!As*F4Qv-?E5AJ5=~KgcFpNrm^>Wn)oPdz6ZF39~B}vFr@`lVvVSwgo_7ZUTR*< z#jf!r%vV!bDn}NZNUOBEMc2B5hnHWQx;J{{8y9YM<`yY&MpZmC$Or?gkt8@(T*lLo zy9=K<43?rI_OCi0xz~MDbBlS>5NKPc#eUw><#9VdA&g7ahR>D=K9q8lxU*eKg66@SR(X}yc?nrtj7ys5#u$5ABi zjlT|4Tcb>AUHeE1oD$8HQ3syZaxHa-uFJTxFHqXUNnpdsy{#j@#PI@NU3i8~)^d`U_w_0Y!h7&zJQAImZy!)=mmD}VcOz9-tg^75gbNqJq^ zgny}eu-VeNC#}w{M!spuEKbE~;ObW@3m1pcJfGUy4e@9b*412?g%(fl@C8xLhvfUg z+mpxknkJERep>v2A^+SbeJzzH>Rn%h@^2ReYGy*gt6NVZW(M<?b-6f*PkaE*pGmJDF@Uat^9!LUCE&+LQ!{EM+8J#-3p2l9%J)yF zp)?j!D-*jR-#c>gn!r!%+KQ`faX!N(O6KOc9*;AlHpLIBe5}~=&14hkEBj&%f7hcW`?Kj z9Wu3e1;JL7PmxN~IQOnXS5bSnX!i)$7AY5!jeW;7Zqs;qJPrKoU!Ls1m(Cha7yMF- z!PPcIG$12BY$~@_Ev~r}X{B&#GD``IWIAWb`Y&-`lHIF!%52=6?cgVa+k6`c!oh~7HpmefkB(=9)afa@@NMng{Ptp4CImSmq9M)x3$PLl z@zi2pc5o^)Yr1S5A|1rn{O#83o(F6k$XPw{W5wyHM1+e(Pb@vDl9 zS1@x;{!G%X_#}EMB`P5*KKN}gU^XGjpj!1UZF4|+!Jgnie$aI3&neF+*Lvn%w0?54 zj_(5z>)9TbAL#FL@(bQ1QwE(KqQjF!ibb;&A`X|5HT1`~U-Bgrx7DZq_&KEE?_6c$ zflr*=2z_a^ZLcoQ4xHQ@9Xia9vbFd&nJMF;V&xLA8)*8P?%y;*>#Bigxz?z z%DUmg;(YW(zOd%t%mRBJbvv1P*J?o_#rWyP6obkr~i(N zz7#EGKW&BKW^{DaJ4FM<_t7)!zVPC?Vz|rqMW>91jN<(hADr146=u%+D9$R6s+J3J zi*3SUbd=shc%#Mh`*!!JxRO*5f`SfJve`-yjMib@4C{p%DQ(32HW$3P2@5k0jw&++ zQwwdK?t(JTiuzpo5BJiw`(ov}J}sQ;+rAEpA|0&8cpq(mzO9gQTVFJBS|8cOw9Gtp zDu%nAcQsJ-Gh=NnBmT6BlshI zAUj&+mM!|~CJ(;z+ndp`r7wLcmcx-Tn_O29Me)RJot>uxw@1$jiu1;WauK%Q@e%qGr`eQkPvh7W389J2~^dn zvM3b3P8RMn38BkEP^v$s;W|+CjU)J0%8wM`FnY%0T|y%?v0H=Hsor`cZ>m5**kv=W z&{pz#@^D1di7_iJ#i4e4hC@Q@NdR+DU9 zj>;b<^T+yTtgBgt>81o5_Ew&VzqEmiM$O?P7(OnC(d~G+l`kS1KN$>EC}~!BXMMP? zXLnNGDBKik)+{TRb4Y0h&AFqhAJtB2 z$5AL9D?VrT=A}d;c{nk7vQT!`$Iue~zE2pF_NZIz4DuKasx-DOLMgXfj>ct~vM5aw z+3JxN1yR%j?B0=2kp>#KN*W8o^>qpF=)a7Z2TclL5!sevx=66)>|W}l`v$UYvW*c) zXMC%QBr{($5Wh!oQ~stSu9J|~lEG|XVXdMPxS*0>r9_r!n~xE1IV}>_aM?A0)fGCB zp!4?PQ6XC@HCihwPjVwpxaHxtSwP>x(7+(9ETjiJK9#$ij*+qYlRV3pTQz*cvOaQ` zt*n7D@7@?m^(3*S5{QAD)sm=5;|~+Q=hx3WhX{Z}`XSO_XbAea45dy;1o|@(fZr2( zWe>Ze&)KjFR91xKd?ru8woGo@<3ssJ57mjM0EseBpyM$VG45x$2h?FExR+Cm5Zr?| z3B{Y|?)Mwc!1oS;C7$<2e8TSczF-{RxD+VqZT4p)a6HAV8=Ui*#Tb zI&BFk=_UgbDYN4RUw^rJvtIXxWb_~s@6p9OKP3cv{~`OOR1M#aB|cf<;Q#;O3GVcUs7f?I=PP~bEK2}O0BX@S`xQ}>4* zIVI8XV}Pl?ewTL+d-HyX^%-de0G3;$vG7IJ!-9!-P|b3myN##SIu>sYSO$$-I5TY( z43X-YR<3UbMSpc$1#5SIUEdydto|eWABlh5e+jLfK4Ym6x_RUN8wzdp1s*}YVgUUC zh0;K`+`NXsJMWb$Mx>dZF>Bt|PzC@f4EeX{&lx?3S45tnP}C3v$RK*On$CJj)0& zO6w{lK;3|2&X()R8lyU#FO*#+9N-tYT#H}!u2bv!e9r}ymHri^_H(|LHvs5%+LUYn z>Pejo41$F$-;dR}X9(8h0wtN1^#wHSJTqY1-;L)1Z@M0UuYCJ!coJ6a3&1ZFt3w*AG8!mHG+Mi72u;=6^ELj8^_zD9Xjrvp>}#cBRT~yi=csN6gZjf! zp6EFW$7>2>+bsYR{F6{O9G_8}Uz>uK5NH?DTGW?5e_S2;dJ9Y+3uLmhjc zm7TY29WuB6PR-<2nr6=@+W{DsRkf5WB976pfEUP`SddYG)C}`LMgn=VtDh|RaShF* z1FW>`3SZ*TI9>(*a$3_=cxIBdvj6b>{0Bi#fNLGDS)~Q|0G=AwFoHkG zR~*mk(hK^jKZ_KS;FJyN=mP%dgb|p8BPhU175UQj<;=-2-{dnjIi%{&tkltVq+c2rGU$keO#imveUQ+wFl#No^1Kbvgrq zO+C|4Jx^^aVXD+2oC`_n2-vBLf&d~y9%Q#q3p$n{pB5SSgo8+YB+MrIusqomh|Uhm zP7z(iynrGVTlxsN5u3YQDM4(GYKB_rSqSkGkZ}-%mPH>8o)M{Cr46kQM7BZBM5z_2esnBfig3QlM zJ}WrwoF$JBc56vHq$4>|?b)j6pI4D#2&p=33#n3{WrW>2VXafebGnV2Eo*jO?{?bX z5m+o217pVzsQQO1c73i)9@!-Y)b#9924m9ATJSIV$dH3u3-I*R&q=nDmm68#-o zmg-|afuF>Z_OLrUm&TSCr+)G7RaJ*m&P)y!-4V%XGKjQwkGr;y-5;3kXgPozT@wZ3 z`iREoLJvFk9)mb$8hL#t@6wD_l2S-MfnoicI@n!*+|F3_jpeW2KEC(hJ6LDf$K<^j z4Ia#WwF|Ras?)8LeBJP2i7L#NK$z>*@Xu2P?V>jpHTqFJADtNKPiqqIbbs6&_6%6r zRzF*}W7Qk4pgM*WRIpy6UEAJsqs893(CE99Y{1RuD`~#jY&lkH`*EAQ=j-$1RxS0Z z6oDQLR+uTdW2?Av=w=|mOLlWzlmtRXeq|TJWV( z52kA7mCHa7Ja*x%ewEr%=Y+!DNy@V*WpO*kLw(z$^PxZ1)N7}W4X;%jkEl|lsVf9N zshRU_LC8sbuE_}vir~3Y_LKWrW*)j}(yP8ZriUNyHdGs#@9C}AEA8=Sa2%Lrval;t zu*xitAK-Mx@!4itj;KlL_paOzW%)5oovMBycQwy3wE_~GF>+GlMmoo0vE3VV@3jY9 zKi|DbSxRs(<>n9K32PV#=_J8DuO>a516|Jc`|9&YblTG6+^al3s&EvK0^*V!Ytl*I=(ka<`*7;de3*Uh~>GW-W zOXMd|n=O}hyms*Hg*TZWFO9fB(Y&Nxj^qv0BKdMVy|ahYY*zPD7Z^uj;AM1CL&2cw zG5kmvaK(1?s+19l0$rZBYUhq56Q;p^bJ*0K{VROcN5Nl8CRx4XIjKW!GbN$yErFAUq}5}YM&?7W z|H)-e$^W8%aX171;c(^$fq9>=zd^nK&giKbAi4g_;q2dZz+Vn$dcs=tH@zj65ueEkKukF+36^?%|Mh$GENlXA2W}5! zCjbN4VE)S?7KpI=%nbSvF_|HC0$He;SXgzbS+(_n)O5P~`fPgiEcCkiEdPtFbiZW< z{zj($jhw~NgQk!}EM1XC#Ryzvw=Wko1uN|BBOi@k`vr9ddEa~{i(!y-Fc=^0g&blr zE+)n}{ge^#ii^w{okv?2*` zOvgNhW5m=LeMvZVam?3I4;wnBNJCa(rU`s=da}%#p2Z>bqD=5ZCZAeWoiP!^S*|Ef2My5s5LrIac(RzTdz-_{*Z4E$ zy4)+5OALd24GrO6T#lhWx?#Q8r!P80f3QY66Z#=Ve+H0*J(QH7A^E&T&|095va(vw z3%2)R_(s^he9B6h2F$|8vf;y~K$nvp&h&Z+iUJXA$`N*74#9mm3&x45C-R~G(w_Dx z1IOHWXjM12>X-r5@%_yk>18J1ARAGSe1W{8Q}DsEbCXew`LO?xzhgd zsC~<%iMqU4wUcG<)|KiE`yv(0SQizsjVoUTdSHj?3B1R4cqdlCqN=x2yaG z%Tgg{jAPV=+Vk4@%N{nptq$8SS-VAs$s}Wrg3y3$X#XGDMezv&lIRQWCu>_AwT2I^ zr;(W$q_6#7CL{2&EV7R74F=v`Gv;c!lNE72Ijy|cpq5O}HI4Qv^}T4K?5EV?#xRkI zX>m&}DyCYl2Vd!G9S$z~m(onS7t?axofcW{Ix(EWBY`x14Gp))14GBa_bC55rzkG* zVt+wCMvVS`*yTT;QxFeSYI@**c-ZBCSONdq!a1JnS;xusg9Db+3D3uxyI$oVFUhhL zYmY4sxA1td6Q0j$DkRnumd%@g_SrO;!Z5EGH)+;%`~lgr`QkcW4k3E%>SM4kt=?~b zpsDZuXwmp;H(b$>@)z;-cEe1j*V**f&Lxg%TKuf7B-DAnZ-X){T&w5D#+OlY>(&5^S zRDR;sa+GzWI*rXMLu}P253z#yd0UJUf@v-@_`xf>n)o6e_KJm+G16h<4?GqRl{7v^ zV2%Bd(aoo(Xsawk)p|>13o&GykGS2dgFXUp{PQMakgKqGXTD9t@##IKgI~tTjk}r) z)pBAcHgVc=R}Z+6SEpkN(w@}Kf0tSGLI!Fg*?_9z1?En->bdLNkqt z4u9ZsJ^N!5x|JRrNp9^DEk)bT>yKA_nrd^$biLlmL9I7U-7_P6GB5yHV9Xpp$NQlF z?f3w^w1X)vM4@c|{!su?C^lAYRtBK9HZ@S6PMeyEUSE$|n_ka=8mPmnuMcEpW@9tZ zeO{!0FUJ4u8vNG}%ioVr{hOuwFO51`8)M?CF6h~bof+x{Z#I1c${5VZ-dcP%iQ0L~ zSetiZ-x&s@;(dDT0G`Dja1rfoYYo`f44s5))nJ*&VIcI`=9WbJ3FfDL2$Ar>3&Ht% z3npeBcqgM+nU#wN$H<~=bIDYE$XYtHAV>>#Jj0v9+M&xPaaVO)!j4_2Fh$e$U-QX$CH1nblfYe*CF)5ifVV zJ@p$f4B1grtB+xGnzs?k&xTfUIkOsPJKp3-wmm7hQzr6N{_VA07!CLB&266_49-^6 z3hjea0tG?FQF|^*`Pd+ULUedhm5QjDb%xg9+9TtqpOs50qrKo4a#kP_IPu7@$X`MQV1w zM#}KGfB_9N=^@P@%}gxWIc6eEJjO22IliHN%vqCA)XOxb~(>I%qu4 z8oBS7MKZNJWCR%H^6-W29Zlsb?i&Z28+?Cvoq$+3@nGD7B^VT6tDCdjP3&-LV)>rC zsbL(0lOMxn%I@a|yWaG6fV0kIU&>5lQCs3A5=*`2B=K_Mmv8oK<1H6c!D`xuDUQKKb;Q)SH|D8WkXX5z%jw5@aXr|ixp9wnmwKp4a) zEW7*OC!FLQp+){(pZG0~%V?+a8X{m_@ z%|+((jiHdPH|@bek)_dIo@GfvsC`G_USkHeA~p4CPQhG`-;j}pd{KQpv0QmsIuUMo zCssqjnq4SgenZY(!Imhfgkox960>-}z%*=%B}9zcb=bUn5j~#U?zUH=*=eY~8*<(p{#=9)`4Jm>EEL!cWSdiEzy z9uP(D%ZCicMVLK2v!I8p{HM4P8N+F*#E8<}Tv)fv)eF(ZU1q7-aG$QF;RGLMOR*b9 zp;ZkbwhvNG=MgIN?FM-#luVWF4I*`g%&fTR=bVI4`VH*4co1hE#J2AnxAI&W4WB3I!(`-f@}yQ8XRq; zB;*z4m9lCIxZhe9nOKyV7wzqs1Wt7|nq-!D#T2BSD@;x(PBPVO+ZOGG;yfm@bi;L* z=6pqw{#EpP$+}(^#54}ApZ}ih{z5;Pf?Mq! zLMS)F6>mzmF5@em)ao-46m-J^*Jd(T@JOr5l?h(b_0ZgV@KpJv3nEI+029R<9=z7ph*QTTxo*T)$5R}gP@8?i`#<_wBVajM#?^Z#+uZ7Kj5;v!JbvG276AQ47Z%i zv4qFXra7(DKJDEO3Wmq()=_FQxY42c#_H^5WUFvXT`>YUcDB4!UveD3EoXP-0XCI! zdup}KTFBNVM1HWQd+T%*t5$2#(tH#_yHIM4L1?XuAu+H3!Xg`$tl=!{*^Q4x75Qq}NC8IwuV;JXlXp{;w`y#pS!5hP$El#`&*K@ z7$d(J*Ri`GQQav%!1P`i)n4xoWGblEOLSjF8iB~F99A{1W!QM3U09CpM2j}78n2$T zw?HF|v`cjcJn)=gp4iw0CT8@#`hiSaXf_FypS-5h(A%}mM;a{+*GC*vxZur%>}KrxNX(d+PjlhWiU-BRBg_8zs7sr-)*LBzd(&ro7%YBb{4cM zLK>F8h7o?1dNpGXY&l-HDIuptu*0#r^-fk&3c@a}GVJLp)3 z8X04FQQmrp-sdqe^#9}SEx@AM+J<3{h)AnQC@m=+(ybsOE#2LnGt{6UAl+R80@B?r z0@6Kn$IwH=FvK^g=NwNw@B4iJ^Pe&5(E{`J$Hh9vWrz0K*{^A( z@Rc)zs(^GUgEJQ-Oeq8XO9#UED8`yNa0|!X3%hd>$#%Cqqh~6qprp{FdBtwq-Gi=*Sl>5vv z^p)yaNX7v}ppSE*zbi@^33a&L>KboX;1E?XBHlgl}4;?R@exVF9KQoRy0PwE~RB z;VoV8QGS|vR9PFr;H}VYHvnTdU%daoYTNAU@RFV)OS2JGOOs>dBC8jghLa?s-K`aOlAKkns7s3~l;X7ta9jKa1VfQiIKC~r&8RPN z_;&#ijS??AJ55FGa9az&FwpmHwI26t7tX@;E#DhtP*peAyJScZBP3TY1h-dcQ>6 z$?wJtjEURURCn9pQK}jsZ@$JZ8(?j`_Rd7MwgCk+wI&}8QsEP;9bg_h*|VWRKqk{J!O13rq$lUHgRkgu=C^gQ-n5{=OFw7$AMf_%zD&7-AC z)4KNJbxF1>Ra>mo0VY{yj?d7^rZkPa0f}c!T*Gywe0u=~9M`AxYIDv=zKE~VSzK}x z^IqQBbUHnsdt}$#D9^)(Qe<_lti&P8(wfJ+)m4qg$K5A@m(THvrA`27&*SHh|EgUN0WEj7o%1YYi* zL`X;yRWZJ@H&T9}FBp&SHzaf;@$~OHTtwB|E?Z~B3C%;YDTGoq^r7iZ10)*aYNFdn zZv(#Y*WC0&cqpGKjVp*5B6QD4#Z2>#Xow^4j3MdGC8W2{4?44N3N85^ss@PO2q3Q7 zeNOwm^!>|D#D?z2g+J>M?T7Sl?hW)+!RPyJp|~_-@CNVi5ar*@_Inu#NZn8Ggy8QH z&|=wpsv1h$Y`OR$>+nYv5^=_l+exzvb==@!wJ)Um+5CGSBqZFKm2)`8DPmNA!ougu zx?gVDa^KQnwgfR>NE2jU0XP26Hz4sO?fmyL63^!!Gr8~?QWb|ptldD{b=&yO9Yixb zC3V$x1oF4`YjI8AJO2G^FHu8b{q|dZY>6LNXri1Es}gz#G^C|=B8(W<`w{IM37UW@Ef3@{M zgehF=X-14;p>m0y_^oRDqx8T%KO^?U50$4h=TmS)?XD{`IIJ{?9-qtU>vAdZg{pEx z5u1*(?zocf7;(V3a(iOk<4k95=f0sSzk)*ks^H1G^75wi+{~uhJg%wxRYqw-XUkMzFI#U=R+MRMZc_F)n1({JF;A7H|Bk5M#0#-?+ zfN?)D@fBK=Y#p_|2~DuPtjSHSoN0vGl)<0lhtiLn?ZdHbQay#Pt|6X)`BPV&J9`)X z7@FQUz*Q*v;=|sFv z@jP*b2sCBvY=dd1V$-UR{Tbds7|(_RVTYRdpWZ;2&Zf%XxB<$-@oob%ly=~3)EpGK z`Ky&aWjsh!f034Yy^UVCDQ*DSK*}*Zpq&khiI1wurQPW-!P*N3%qW!|t zpeEQ1T(|gfK_{%9YSt*GF;g zmh{-Q1}+p1?{d)bEk(+0jaK;Zv8id(Ti&j!8_ju|{~&m{hAxNan7^Rw*Vy(x%Gnnz>7vO_?pAHgZhcyEaowUMLonQ@$|rqytb_26vx$=G--P6{E@l*==d2w2y=u=^gwM+L<-&9Ja6jP9 z--DQ6=rZa!EVfe$8)D(2IwWk17&;za0C`CTIKt}^qeUeTfhHn%7AUil=Yt zcp*~@%`I|dZf1YP(2r&)I|*lN zYQ$zCt~Y`a8&POoi`ewHkZOdc5f+d#&+$D~3lZM>>l_LlMsat~v0p5Uac;i31-Ssbe~mcYVV0b?7D zh$=pb7s#pGttlFVdC@P4S0OrYP1k?QJs{dt}LiB}qw zrAe56I_+sS{=B$#yE%?aVb}5T>K%x*@af@*Ngq8UO=riDjiHb}es6Fh8ml>|App0`CLSn&z8sTy-=`&ib& z&)1~UExaWUnFDl!8bfEBRdr8W@~kE--B|gz@tW{edRWCba0jqljLSSH)2(PC3Q~D3 z*YcrecLAbNrd2YpM*9oS`p_*R$GW5s>hk=iLecY0eDjP9cJ+@jnm0uC+2p7qs|TT6ujc2Jz!-`YE*8HAcK(m=|N?u;%Ntt&aH)G0e*$JCMcX75HZ?%k2dDyl~5xZs~tM%On27aJ%-T^ zMXN+yP%2Rvj;!1>D?a*j0gOklr9>!Nb!oX(eG6?PrIvSj7bT}%Jr;+7bS~A&ZsAf( z=~Lsbk{+~tzQw(P%mHxPSXq5nWkw@Dj#@se9m2V9*w=TLp{GKm$GLQyRREM+v9`jf z*9t-T78FZmD^n#|lvVH$ytZ$6Tg#79gV-DPiuCL=^p}USFx&E-DB35x-j(E1HhG4| zWsWG;Qm2}{coDzW1dzxv^*npeYkqbVt#b7|dnBq+-%`SrOov%~esQRBAG2EQk&Y|i zh-6NX??qE{v_2qO--tn%S^Uw<@eU5F5Ox8(=h6C*=r?{5vyxA*`k#`ln!a;*8Yo=H zMS49%cLl}Rua*@g2U%Td=D0+Y(R63gU9C?6FHF(XS_2d=lEMOJC0@FSW?^!ZjVORf zpT=ciW?C*ky5jHNdrr~Tgv=o}{rQcbr!Qt(*t69)6m6fUfPMo_EWIVGf(m}7VNAw6 zcEYONhufuy`(`>k?dGWCN`+L>4cKD1x=828n>H^`vK3}tgexQu+mRCH$u!2|hWE^LeN#m?H24DPs{9!zlB zNvY+_DSE4xHc-0=Sw2*Ee4L<(1e5ZpIPdzb;apmX1RFh{eR6Ks+4IG5j<|MF3ZMk& zdwF;tO>J(6G{O$fk|jZ9WWIb>ml5OgGK=nb5z9&0Q($oqoIPX611cCuv-3br`u1(p z>+>^TW0*WiAj4+5TXgJ@Lyj~^TTZv?d5*_*Iv*B`$+d0V3yG(*AEZ)>CZRs@1XpS? z7`9micnnfxYRh{imN=OBv(~yVO1_i;OOWvc+c4D$$AI6|NlVxp)><-6v~a@cpOKBS)6e` zGWBIwKdwYj+11-udF8H*9hHi`_l`ejYwXD|CV{r*;)#-uSNHr?sz(#c^sKJePG_;q zs7$>BId?RPnMM>2oKrk=%OV4hM&Yz*@t&Y3>>_bph`@zm=JA(l4;oGG4IeL89Z2A+ zsk7%NUIl;$NKaW205r-km&qSDc98KEZD(`7wj1Y{3(=aKoUE;Jwuh)Eh(n4ux3x;L zIm&AGrih%)!d_4IWAMSfi8W_aJi0|C9mz%B>;F z#G%xNf`4hHleGG`nz!79UeFttgMWuP2mG+(8*1hpF2eK>xPX-PdIbpP7|GV@N{s}m zA%W=;g0x?tC1SO>_m}hkz1WbD%nAN3LU~)AgVEu3t0gUMKI(?Y{tj=e%l}UF!twq< z$ajwY_eO}Lp?Q~@+%M-pi#kE&2$a9pGm`#7v8_Q*=l9DknqQ_5PYLHzB@kvSNDWuC9+-}ihp7%14QtY|Nj>UzX z6!aV$%k~~sR8jS%b0*nnK5OP3X)57O+32rgH0!G3sy1a8oTcsaI3_87 z5HNntu%A#*L)8y$C|SM!MfZMV^!guzsv%5<{$rAb?_Bdua)p0Mu<-vwP_-Mn#a5Wh z+z_&X_y3d){I}+)H%SoK|1D?3-;*FzA?ybJW;u*Vf?!9y>WQ?+Pl2KDV~I>l#oDT3 z^`(t=_Xz(!%4EEOjt^S;C#ncJfcpWSx+ztHgw|bavqVV@b5iQ}G|`fZ-@@P1h>&8X zi17JR$*$kdMnT77a(-<_e(%6eC)x>m#3xBh^lkz3>axRdXXkY1^cuQbJrClKR53kG zjtW-p3=K$ed|5!RLJ~l{52=h$KsDyFk$y&-IIV%(*Tw%Jpvp*7LW=6(%g_c+=a(fl zswygQE#FKHp9*fiwhAKnGIxTRjf@M;l*TmQ6t1z#n0Tn0!ts-=gimlTo0<4Nn}UF6 zTBlys;Ltns52lswl0Lm-8%B(mH1yKe(YCB@jgDhGD`(%WfrBn{bzEPPVk!_y||B~lUmr+Aj zWj=?Qq{}ma(e$P&OR-0dBbeXz6|10LxJu&z-s$y(9~Hzhapsd;KiQmHzj+$G9o6=1cxpRFcT)o<;^5go$;7=^*zZm>@lDn;OXN(56r1O zp67dj<5ICp)p?$PW&YsG7X;f9_KEGP!`(^cc<+(E?ZmEaq~x3$?d3#+aI zWG|=Wexyu&g(|8_Gnp!ly0c#r_BD!We{FB;Xl|r}n$u|rtY>tkfsV648b7uxrvub8 zO|Bczs#z>9s)r;2B%b(tQV6rVWT|wjgr&Zl`ts;=<5HgT@wfzKL|4|{-Dq`wP9N5j zy2^Ra8jFf9qRdjcFm2z;;%#MM%ZV*hNWteqp$)--Mpz*?GyhSdIGGz@#}_aQTSeve zTqyR(Ca$$@pDt3bZ#+5C1Q)w+4ggba7~YO95gTMr<=VWm~4u z2jyhbieYn2=SQWCsp?MP;r7)swfINf+5-4RMzwZUJ#pK4j?7^xT&_x6mIT@!mjf;r zr^~e9aj+2Cv&*Zl(=@epD95<0xkD$5utyKY0OqV$TM9A`=!l36*hUE#!x!*^g~dgO z*sbBguhj=G`M*>(?HO%q3O5@_uzl&RRD2?E-qQhTx*k+*EXX&Wb*q62gRi-0DZSc^p zYdW{7sw9nWR5iCLGy<_*PkJl*QqP_$Cb)n)CadF4G!p9Nc2$0P3O||ac60yE<)$J+3MU&(4y~1Ep;g>6?RC3iH5P4@&1f z=lz2E8Fv@Y-cX$MRyu}Uu4{7(!_O*-+Aek8cuk|N8qObwjSGM}HPPL9nv^~WI_?x1 zjCVZXFldI^LeWlPl%~u)d39Pdn^0ohUG@EIR-LXKZS7fjr-l^Yu26)R%)@I8qIk8` zI+*JD?wo@hRoVHWlU#SOWn#sSJs!g0WE&#w`X&MrD3x$ShcS6 zH*>fcX$^Eu+;Ci3c{@I`Dl=ja5v6~BYqa7?M1ESK+s6_vm9agK%UpOU=4urt%7&U^ zO;~>3CeG$;^nj^@s4ZW!tE0^)097@w8r~x&exSyro!yWgs9~>V|4T{fg?#qumxZ$Z zjJo{{yGXW3qOMicwyX%5<#wsgpz_*x9|5eKk4xN?KY^N`A1y|Vt}*#0>W8h9bnCvE zI#_aG!mq*r!-Wv4yF-ple`L%zDy<~zHG2%jZa`C zxm2PflCiG=INIoub2n@*40CRzUwloRfAlR!5xEhU)3k_b6kG09jf=jWJuA;}l4Qv3 z{FAV+`2JCEODt$!`Aa~C$w&AbcsW^1EcAgz1o*Gxq?t%wVI8CDZ@epw(J#K6l*GeO zyErG!dN25P!Ygg(wlY5m4sV%@y4AH`RpQj9Uzw?(ingjYz*>10S6Kq=GH__9)3&~< z#_48{&s<8GmP?*zGtK+5q*?**pn(r9P1ajmHP!7<;KKQMTL_RZ^=CPUN^0CY%B9V#R5>4VvrzH7 zansqHkD1NF^S_ks@(M*pcR>@|Xl@11>IL4v;?+2f$SWylEcKNn15=fT$mX zH&ff9n=3BE$Ou4$*3G?EMTAQ$b?-_vPAgo!Bq^Q8Nz_$I%n3gWn{AtnMb`o8=AR|c zJ+zV?6RtATG}LFb7HG0}dN5j3SIUZeS2#?Nr4v)k$DfldHL%g9$fFpG@OO zpHviXRB_zc)K^TyjF=8Pv29yMEFLb2_yRUCNHT?Yd)_J_cs>Pymknf@RrVc%FBDlw z&s@_m7$0cD?|8Gv9Qf4dJ#$f=vX0TNW@qb=Rhr^j6)V7QK7xGFKbRSc!KGys;xpmW z5o#=5nsPO>niAAI!DhF9h{%t!!Xv=ZJhSjE_$ZK5N->dZyWc8HjfeMP42XCx@^u`h zoD9eG;~EdNDm`R2Wq>Xb3Zwe!kG2=$X=|5g9R#R+J;G5R3F(Axiy;MZY|+uT&ccm11zr4wSH5ZT4*>Q}%Wr(= z&Vhv+f83*G&RRPQSMT(q>{de}SOuZ8tpwU&mKk-c^7wc z@>keyZ+L&_?SrelJ4lWTvw7byHgtoP`F_)5sjT`w4d3fZ;wBNpacl5J3`tLJjr%5! z1Og^F&ie6w6kB@AXdBG#yd9^4q9G&Et+J&3GSelhhhOYImR-2vASSUzYUD?kpWh3| zoluonR6NC z5I9!Wbg9e8oog-;t9sW`j%JyuP&-9bsw}gXavDEZ*jmbBJ)^MomB-=WL6A@Sq87-` zwd`qv*=uIuJX=c}@^XQ($(Sw|llnT1MEI1CmAku?6wOeo85Di$!A4P2Xi=z`?qHlG z>FnMGMbDFnxN8=NjU3A-H=N&v*I!lSZ#4+<`w&b0Z*H0YrS1HmvBiJ)^!bsq@TUvu z|D+-R#Vr&6HbWM0=OiPtvc7!M2)9+fQr=ut%zf~U-%hU;AK#=TAV)V=%YpI9hr6V1 zd-Q>KW5JnkE>H_}j@Cr)?Ng;T@JxQ%$WD5x$j=&;0!TgO9x!q-y(2-(totY!B?}&B zwNyqLd2GjozlHLc)fwxP6NZ|ITtXctjX^0TK`dUW(v#P+`*)W}RU!i|3GzJ`ha!W! z-o8L#{l)C2!o>HfAZB2mZI1m)eX<8 za(;9Ru)v8P%0qMb74P1(w^+g`-hHEwn8>W62lX0Mgy-V?Kzo3KcWX)}^USdn7dnX- zB!e++F6I{Hk~jH{TV`Ko6T&Ss`RR>Yrl_S)?+H3nY%Pn=bU~u%RvPA}N}GPw%n~p| z=*BHmT9^CAEwd^=I;Z;ppLfcmaa}|zVX^|#rOVCjY|2it)~$Xj^w~M(ag~cq``CFS zHMICl1=H99cZ*8XN#j&}aQQYp^D*%U1gSN$N_^wZhqzstoA^T}z(5L?i{d1C#) z5-|NGz6QmAj<2x}QgZtST&%`7^vxcpr*-FfBRv^Ou(mHdkrCgSHoT%25(AZL^fvhcqRQaWH41 z_qrsyD)N4=`(P{)N(OSRX~E-l9`E`Mm+&XS+C^U9kUK!v;)YnICJkNriP(mw#w#S5 zqcTC_m+W?3g-MO(IJ_o0`=h{@p-s?omaNu^kl!4ivCWLw83{5iLna&!r^yomeZ4Q% z9lRj4uLd~^8IxK0eIKuV9d#p4&#B*dSDb7D;~KO6 zbBpiMA=dKE19zWvSXrG-ist9N!{&F>lc{G;tX2pbF)aM7-;Z!ad{!v+t#z7-GE7U( z(*Wxi@#{@)h-J}A?N5mP`yaU>{(BeTKlksy=A8W*-sSuL{lDghczTyKI<8Pzfv_o!h+>eS;kT12Gu?%i*<=Jyk=AC)8q=DzKrqotseVYWU^H>bFpc^6~A z@PTA@#w1g^Ml2b170-fu@}z9`LAu)N zrSWU|bzTL79fbfsz-t9}jso7ALV=Xm7QWkow;a$29&(7&edG=GrSB2#QmT2b)I2mv z4)C_{H@=E$wJx0ac&U0Pebzy(06<5^DE8igK69hGSNZ+I=)S0D9w{O}M3pn~XGDI8 zC*Si!7(IDU&dyFbI4px}m?5fWSJ|+I$e@Ol9@|4;o#+#*Uu$C0GMLW$3F7xL=22$0 zC{saFqP`jmlWs|=3IFJ@leBq6%dM{!A2)=d0!=3!f+ZJ=0VAfJETlo3{@Gg{B3g8b zX>Gzb9(94^J>y<|T5fv>FL*26JPZw;Y8#qr-L_!Uj%VWH(_bkUie^{l}OY|Lwl~ zN0R2hzVQ5vL-EJ1{2v^g|1D-leH*^cKVxQaz)Asr2u9?2@!z7kbF=)Hr2cR2{ojAw zH__a0X#J;(0;n{GFkd9Xq`38j93V}ltn}@ioCo*7AZYAuNd{PXO2c~AFFZlyH94WT zcy*e_YWd~4h=t~Bb4%@M<=CpC{L!)#ipr@cLV-%{V_=;2FrkX$Ql=^JR3qR4JMts2 z3X>@8QYb#T7Lt5@J>x#mD|kp<%bRa+LXY`HeG#=S>^>Pi|c2zl+1khy+^$0_;bhMVP>*WbJax5-wreXak6X$N~)mNAtZDzIt#CNfM z@sL4BP3Pdc{qY1h_;4hRMP2Y<#V`}7$_QJG%=B5s>1(J-=U?Lv{8A~tI6!(fccPDy z=||M;IC#$X0yR}&Gx?F%WB&71%4)3Zl^_XXSmpKAaR`STd@U%gX$_oP7&psjzP&aY zeBjjN3lfx)J+kY|7@N~ji(+yl;$d$^L zY2j$iUV9u(+Y{(by9<9~rv{k!+Y8=1;0&`NR;#gnHmn#n%X4+K+pky^M}Q&>>kh=R}XShHsFH0euk-tk!2r=7c=h*K4GJ_=n(1rVo_HkEIgFEXiK?w?bj z)C-*qD+~x%u2zvjKi6HxS_e>qD@ln9>JJh!8kzvJm9P_J^b^nP3?>|w-3#UAW*t~z zLUYY^21yw4y$hE9$h76#%zaUZs9#X0a7;Thj})-5Z#MilTcy<%;14 zkn=q&xLT0VtinG{K=(TDS;<4wg&j!*db1=8Lg0d?6+FabHeLtspg5bUx9ih}){o?p z>NX;NtDRsGY&O{%yj7~p3m6nkfkfXX6LQ#CF6KOe*fbsIc;@Vl%RTnoWr0qX2;EI} zn}3Sa44z2xYP9i`Oq&Rs1+|sS6~yC(xC1BqXq#duvh>+K77f#`f%B(@)n3O%X1!^i zkQT$h>$MtDo@Z|DTF31ybGpFY%7Uau(4w{|>s9vs;^^Crwu;p{ju41HUMd9fw{W!& zopHJvevw(7+jlz4`pGm8>uwmv_1@C#7P{O>Ecw}JIWiZ4c>BS943XBH3&UxmU6R6w1v#Q4KV-rWW}e$4p&Qd7GQ(b$N+(ezAFeYto*| zWzL%{sa{5QukwIsPVug18ajYXGU&knA`S&kd!&2w6&RLqxE=43x@6W5xWSvivc@~`cy%W`8 zYd#-__ctx1%r*k>5A$uWcgtwOO$^iSrfkJ^XLHL-sgo?ONjP^&rCo&A{Q1mwGG%gf z_tF(y+Z#4B$gk$h)1N3jE@FIrd(yzgd{}Xgm3*eDyum?lVoGu5+2?AYrTN61V#99B z-cU+NHO_{$J#=vIOX?B{s;`t=U^Ny7%kI-6!J6vU#KlIeBBCKpEzYv)vl)s-xLaU; z+uRaZ%+}sm-PZnvtL=VW+m}hf_N+{I)Qg$*y_p(965qHUS&}+4eMd+`wm6d`$6-<% z+K~Hp&2Ps3&-!-OUsW3?YVfTM*lt;DS|p`CB1n$V7WjIc>5{J-U8^T50rH99Cvq~; z;Nu&SIbV=0g46HB|=z& zHBpVTWKu}TrUZLR1jWPPfGQ@dO#~@4URy2O7#vQ9JYXpp(KAuZ*^7-uvr6ST%%T^&Y5tUY}v zOjsDSZNup?uCmjY;>fO$AF(*`S@|fwk?ZACw*l+vjBKZD!I2nA)PSXfz=JrPLdL?o z{;}h+x3gAJAo6O-bh3;LuL~IS7*n(eh(}J@)fX%t_f4|8D`R_qCF%4J`05vE=OXWF zt-!3>3X7LnAlg1bBVWPVk5q*3d_WDXhWZt`DIs;2A zUbH%a%T^>rDyT64;!&a;ID1qG8-FQPUVz~WzOSog`-NIhUWS@#7)H+Rm};!8kTJ)1 z-FGZyw1}14TMxitFpQoHJKdl*K_2G0_DWRM_df0}rKd|^jtJ&p3tC8QQZ1*MK*YIC z-+-NFvc9shKm-^Q8kKG+D(6S7u(L7~$XOj6!f`a4n5M6oycl8SxApSKWWx2g-4L(Y z4_&p?wl*ZhozQwt;j0s#t3a<0%f*jtH7^?Wiy!myR-5!QeOf;AtX}K%;&sO^+fU=C zf*XXI0&OC~`&hY4i|F!-8qUK5XNLJz1Iv$QHS}b41u0U6JHK|R_}5bhjpNiB<+pW* zWYRlez_gww6%2rHb*XfAaT!Nh%N)^6ds0M6)p9Dhx@J5%T_jfxR6Lp0AYVOQj_wH! zQ7mF)h<6xo)KXV*oleKA#UZYJlUxP`-+D?GtT>k>Q%k{Dh+L<*#?)FiX+muisDiv2 z!xF9RQtq{D0knWx^0iJ*>G6}wx=?YWjrFeDFooJT9&c~vw){!|;fnoa0?8^`-Qhzu zUK2%XT*%8l8i};t3@?Sf?ePn{`44T{?jAs;@ETX8jGD4%rA-D(qe~ZMWug7NxQPT@ z5@fnGM#5dvALh33#Z~>w-r>@Y%6`V_o&!fYy0}YpUI1l=V>Wu50wjXkowBX=ox)ekl z-(G$hy&`QP>J39U_Wf^Q>t=d-`n?gbJYi)1cP+q)?Z~-1QUhOe0h0R&0vp19u|gHb zCJ4Rw2f`L2y_X2;&^dmnXwYv|G;_T-DjGqPUsW_=KheLv?9w^&G~`aY@mxe$`!V%a zw=F=s1S=4OdCQ&f6fuSmwdTJQ^2hC3K)o_%NZ*!=Es*xO?QDON+H=B#Y~ z!~U@aw6=XkDw0_rj_A9MFv0tY3Vvt4_NRIUsD0u~)Q7Z}y&|NKgao@-b^PNt2sM|b z4I%asOY+#nsa}F~jw~$V- zI1uyu2efF8P2RME=G03V#C{V7{6x~`W_|w(Emz;PTYp13>1{;+O;4Az z@aIH?uet}`G_)<@y+OQCd7wD=v*Wh)ZYjbB?>|CIeggQ92jP3Gk3W$7?6JV~eZl|H z|E;RCfjoZykN@tkqLF@d5KPzp3078T*_l_ zRL7_I*d`OL%!&rKW*dy+=5o)CtgWS_>gZ$gEDQ6@L)-?6q_Xm8z|t1qq@~AH$E(42 zjB}W*b{q8Uv-2%Lka!G^lpw_0K(M6DE1Bf*$Bw<*?bbURY#WoDxqbn;E1#Tpw$B4N z6Jn<%aicc;&W)t%c%$;9j4dLuHjcK}4108iDxTccxo%!)D2}~t%Gob{N%p7s;zryj zj+$eC^9#HA{3GP&e<>6AU*vs%h%f$w%Nk@Ih+l~eLc#gy7Ysj)N}25(Z0(F4oXm|K z|Bzwap#7T+!!t_8c1q-)9~6f7inlya`vrL<&a0ZcoF38PDx1cf+o642mc^CzGzZO4 zy_OL3_4YE;IhdP?$F;`mJv#RV_r(`H>!>)FtW$oAiac)U$oS%Ix`0#N!p*|lPcw8_ z`CQ$)oTkNQc0wZ16Z}(Np7=TYp=j)Nu&1}yPD})gq8Wr)=}J1smJr0mJ#N+)tY5EB zz&1-^7Z=z~R8DWWHCyLTz#Z~izmf{Mdv&)Wo3qHSM`1*qo}h2Sb3FBAZ+M z4DcIo$6Vg9{V@G+iDdPFi)s#(fsj>v(SsND>BG@6H25+}jLM(0GH)wuCfs{4X!0QF zi<;!?q;%tcT$0{RLVNDWW&Npn6vE?7dwO`iHs7W{I~bD@UmY9{8)c4H>VKg#clq0YeoV1{dB~WZjIHhd4E_5b9@ZaX#cG5#@vo0N z)~|TUxX`TXq%_2=~ceb^NL6Lu4GD`Rt`|F5X;{fgSr(9GCc|Gy;i?_1IRUobhDTN_)M z+Zg}3%>N<;{p+68i|icbhS-nD5E&js{teYX_SRn_t0J^3Je=Imn2n6Bob(NCt*y?0QccR1Qv6=O2E+TW=xDcz@8$5iC%d{?k)4E7e|S~DO)opqM*q0dv|HBJE!#h} zO7eq;=<}rt3G54`D`Mc071TSnpIW=jP$ZTX9G-HR-%r(`;5{5YJ=z4>Oa zmy^1)P8zinBlDLc{h8lcz9S$kvQgJj1SA%U*xuiBqC_(DHyOeFSi2qxs3Ga7M&x)m z%#;eltJb~NcIxWUb`wYs?a9}E<1Kd(cM44_C@>}*P+Rp`MZv^%q?e5D-F=-}jIP>E za3OJxZnO4a<0vC8jrMAod-Gu_vpA)sE4R8Mcm>rDmosAuxjE%nU&Nm6(|0T(LyRu-| z^vipz_lqLA>>e?yW$_GCQEx6MPL!^Swea$(-MTe#?>!#xgK>)5qun6MFS@BzM{QF8b?e)M zdk**2{Z(EnUS6a2Q{FjsO&pPDOHHAArs+@B7nxhCFI7j?lR^vZj~&<=d1jCd^84#v z5lIoGQ$V_r?hugfMjE6;x0ejFjA%K68Q%6dT5*As$e_tLW&(!^T+M;4+M!A%Z7vA>kisL55?#D z%yOR{w4M&hP@;{lgD{O2Mq8Rx#G27Fl>Rw9GvT<8m}HJ8HRc`P&$T_z;FKN`L%u-! zW?UC$G+;!AN6TgMRTt!ae|?uxBmOAS7Sy=ka?VL8O#joY_;Qe;ED%*uXxX@p%pRDv z2a)8#!f!C%ZNI`naq!DN*md34j8aF_-`qI=HXVAlFL6+!;L3hcYdXW{05;=mbq>@jz;g4B6}!-*aed$^}&6uj3rJEPMhME}-g zg?E~d$r^m@paQ#MV)?aA0Pm0upPS3v%SuF6wth9Elz|CUVg17)eAE63g!BZ{gA4f~ z8n{{cBda}wOF;y%v_S5H0PQ#pPJ@#nz=rS%z+>>oRef2@;+m;NWoW8rY^ZN=X>8=L;ZeMke!JUr>V+1b@qO`CR>sXwZM2CeSbiU@vK<(k$$c6YCsD$tx~F9U}+ zaME=l`uNZt%AHz55)2YLxR)6$NPI?168t4c!zfLH3*{qtW`R^<2TnSS)%ErYQf&y{ z<1dRdEEi->Z`7daY~6%n!?OE`j>50XN488;$)c4g57x=Nf~YP5Q8nZ_$%`}NaK^w!`wpm>to?Q> z+51k?Pze<5Q{=2~ctVpcAzo^{m>+?o0n2IG`f56Lgv&Oy zRRBIpgc#;WX$$S%wpol<>LVRI<{4ii{-6cE`QiRaZUl(fE4Gn(FPV!`Vr1CFm?vFd zQMA_?_nT((wdS_@cUkoB&g833`y6zPRj;9tJX;~3jG%K_+uUYRO7$UkU!M;Rp-D(j82A=LIUU* z_+tNK=f7UXKYRbhO8(LNuZ)O@_uGUo4&Aq|3pYkwUv^A2+gC6*q;VeBJ@82qqbK8y#RrCrfExo9Nr%?;I@3hG4{AjFI64i-hUo zzWMyKS)S%AM0%UAUaZ{~F*>xlR+|xQwp}sN2)XdlX^79HV5*?$4qYr)GYs8KKZhOOsL4p$UE09R5%WcL z%ZF&Y9XIIN>n2mLYC+%PVVC$mIFlIi1UKU8g~E9Eun=Bl#TLu6#11SWyXptC;z1|H zm;-kvm;9c?NOZTwS@n>nqwpMLcU5)rg^=@WF8VX1Z{?V5r}4ItNW##E$+LQV!M7VK zmVHa|vQ^*Ms(md*>l9S?V=#39k*bM%Y=9vC0;6u`5Wc6umyd-yX{%qX0U_tcZRZ+H zAG_)jg=G=GlGYQPF8?(qz!myOl_-xCIO@NN7wdt&iUKo0w@7hUA*d`LwLNcMfN`

UU76<>sOxC<$<{qY}U?!-AYU$ob%A))IpfsF|c8@DUG2gfe>Mf%YN? zydJ%Z{^T(??CbcuPBIi*g2IXpZkq?nlVZm>{1IPOJ82tJBazga2Ew@sxB=x4z+Tx8 z&*F5DX`DNqoJ7UqR-f+e^qt`D>30o*H?ugosJdEMJ6X!Q&NGMl?hI$ykolP%TjA)Q zH7=U6ny^Y7qfW@D*-o``-DLC(pZd-T0HfReS(nl>(=f5n(Oh?1+|;GNXLLCE_lkM^ zYr}TfP&F|+BhvgS^4UQT12ZU0*H#o*$s`6%0q(S=$lxA$(N3Hy7tB4P0xsO<-4!al z^-73)4R+P}^l04c$5Jw;sHmUC<@S5LL(Iz}>bwWuK<=C`w4lTLPK+q}kihMnikSt8 zeyr{CuIBd8{q*<~2@?fov2 z5*X_Jwu(VK5ou(}2p2;LKHLD4~Q_1Ou(7Tlh)qv2Eyb{a~Kx^q9rsT?-D| zltH;%iBci1m#a24RA#AgDTaHYH|f0WqeBN1!GZ zh@pM*r4W2Gwnm824H2!`ygf>(&~x6}a3@uU#M8Q8;Ro|(I%)>rduUenHnF%gDmwVb z%;Xu<9BlAglM>zStyGg+-|#{UPsTi%|%lrWU*li9pQ_ z+X}YNZ9@`MQ|HGgM-!WIuAJx04*NXE`;wy$iJ6gms*JHl`(d)_LIL?B(z?5pE)Ncp zy6qH453s?lu`zvTj54%V-X$=IK9j*}r(1y=W8h{)DI#bs&rpfXh*DN75^eu95Y4Yl zB|a~hE+bD0INOo)g0y^N*w=7KEd_pwB*Slbi$TSzw;C#?F=h8~?qk&$Yjw=N>rJJ` z%z-luyq5-i-Rc(nuSUJ*7X%(6rDLS0x^xNtRBXRkR)11#W{w|-p504zTIMxf6=w>_ zySeTiOOo;B0qgz|K*2HJrhcz>#04;NdFs%naL?q0rXf>{10W^;K;BP`m=YzO8A#8 zz}I#lFoQkw1qCUP1x!)BU|IZycaHaCX~9fBXo{$3kaxoF!+jXe>3~#y5Md6Z3XOy~ z(u!>75vIootNeLfToLROXSLD=S&yIq{_Jq-7*T^DJxlSn6*O!pBFSwGa_;>RVU{P=xY+P2OzY1dOkQRft9nU#X>;jq|&_JP9(E5i>p?zbn8fUkRLz7>!d z!F&lpTTYjeHGEV|H!+3*4}D6r0gn#~_YP<}+X!$Glop)H52Qa7jZGuzBzn+@=Sa>| zn+2i#_>1CnEUekae$~aNMz1*M_%o2?xZ-gl2IU%Z9vYwMR%|{R_GD-s+^5hdXG5ZH ztQM$5#=}Pn=%gfxBAAOSz7|_J-hQvm@}4^q2#Q!hl#!z{(_;)$O*c=3Arw?UwMZ@O zu~IEaU*sLK$Q|M)1@pEapT(gPudcB>-ZQm1SZ~gZtC6^HiWpQM6wlXKABo?o z?yFj1SKUUv$;YmH{AI!bN4yW<-rub&?iwFo>aS}Hj+U907EtQ&x<>oGBfirV;OepQ zsa?kpe<v)zwMnS9B`E>%tZsBW(Tms{vOe^b^T^VO(4 z1%d8LKzt?Oz~WqF?>Ma1ihPgh~Up@Zpr@*>Sn`{d!rM~!F`K$JCtS|dJ; z1GwQ~0B-mII%YU|8KLBODct6Q4$i>)y^jM%Up%fs)4)y#-E9dJE{dsde)`&T{<&)x zV%;d4LRP`5O?H0h0}&%%ED2V5mJETPHikr^WUW+v->b`Ji=*& zH+$rD&|NT^8%e@y_Zl;Sj9nx1c|GVeBoxq1^6V4C5I04Fdn%7DJ4#*EP%b=RXy4BF ziwRIKMt`E(&drn-7D(rF_2~PkwMRF30nhKz2ynv-0o?Gbdq^ZkZA~B5g1D6h9m#Eq zGent+y-dKS0xG+U8c~GCFB<`+^m@-f z?S-yBx!#TXY2GW&r01PP;)`+1P((L}l2@?9Pp^jph9v~&pO>W?Q zPIp=7g{-&B^|!KV7-ntG+%A1V1K>q#7Jcr-msX%m(t-7$r&cr^A%?OmfPErM z+na|LdY}#3h=h)hg=hmR#ua8?6%1XR?(1i7&o@uG9`4lyNN_2fp9$I@hw6S1@60pO z@v5(Lc+-Phe_!=Y=CyiFYEXHR+|lj;hi~ zjuiq?f`BKwL_&N1vcuxN^=Lq;n=B&ys(4b_7kId8#tARLo;`B|qu38@f9+eAm8FYC z4cyuH;*D2+buLETHbQ{G0DS#vC;a1L)@_^3|5t+{Dql1Erf}87dD!@uM#62t{?bU` zFdL&m6gvZFwIRKSOwrRygJ`|8#311R#748Z6A^7miCsnX5MMF`k!dTs=vkWh87Aa$ zH)}`;mV*DOu)r3HxW;TM)|el8HmIPcuEfr zM5MePi`>=g>H&cSZy#aysHK#07z8`P(jpfnG16_|m724e3BgaJHPo1#q!)~bqZe(p zdYKSWmjL=;EVRt+8NmZncXA#u0=5V+iJmW6UOYx0#-N|!`OdEH<$`6)G3a7dSQmH> z8xZT;aE^_jB=;!87q!y3D!z^K{ECcZDl}Gy+F9=#O9j!G)#B5 zFBRi(P~p4hQ&oV}58C}kqMDoj)}{AyoVOo~tP;{4FvL=THy%NF7-^7pg?O!z+ob64 zSu&oyWi1AXPC7#y?vp;lx4IG|=r6&5RzmjhQ7YZU2*y6pN_Ysc65xSWLa_iB9>7Wv zp8#43sU)2&59aaM09Jw+WTANSr&?@u^{I8J3+;9|t`lx=P$4dpvS$W`E(-3?ky(P- zJfBxeJZRy3ckjD?9PdzJrk>X=@Qa3gy=$pLwbf)m zqFRekLAj~#9Pm}tFnzI9udn0vLyKm}Kwn9|{kU39oP*M4zemTEm5zI7N8MA+)YYd) z?<-9!jj?oVn);+R`Ep%*9ZSDb-!#jwJGwY-Gj?y7<^T0H{bc07U1B%8*O4#U{;@0g zQ{e2MP4@4Nl)r`XE1wrNvxT^la*&Mh-dvUDd?wz_RUbVKi<$s1ski~@BmQ7g{d4W@ z8d|_7bo8`O{$o=8!aG-_$YwSxh#01XC9QfNf(I;_HSdXGkVByKvL6EN^k?+27-Xnd ziWX15#R^jusFK>i;;Xe;hm+|Bl9?dl3HTzD$|T=~KLS0%Mh|l6A>K=*7|AEyaGG*r zt6-z=&zy|nn14ivq#95~ez(PJk4u zKAdF(QWYbOhn5;CJ&*Y`GGZV&C%eTI)z{yt){>~i#FX9j=E11>N|4U^bj@-pZJJMR zO~7*uE`t?tynzc6chXeYoDc#P1x?TK%sSi=Z@|7DN`HAWmsN_;&K(M1SE-P*_PVJc z(Sy0<6mB$Y!X^@uW^ugdi*p>3+wLwy{46AHNR}UMF|~Prvpge&ko4uqkU5t8coiqE z6Ffmvr$R$M!2Mpma-dDJ5E82HNa&@5xo%u=;#a)f_9gG#thY<}-R+MhQp;_;B^@Vs zx`GQ`OIIbKDA+N0NV=6It@AA(L(J*H6ry<{xswy9s;_r%X!}8Oam1?ycn%cmem~zp zd6z1Yu+@auyiEMQ5JL0AWm|R8yJ8Jh;{p z9fv}q@`D@~;<1D~)SLL`kiIEJ*Y{Do!xLmrg?h zx&U>UL^w*Gh0G*1hVuosIkpnViSBjRMCsy75#!g*jLwCQAM2Z8WX&;^*?nU;;hdgA zcqp&Pt~=~#H?ND*EqL%i~2-Eu|Um_c)C6ql#nOI-OnQ;$C#JadUM~?8b#toezZW7X{}RSXZy%R z)oY=QQ8)Vru5Qz=hoVunO$e>*b(w)FdD~Sv*l-hKdnqI)>1|lsca;4aAR7aBb4e2&Rk_bH_cbOc~_<}I*z6es|MU-*{L$1yI;gsbep`A&?L3&MR zSNB&z)7uc9(AY8`Y_Tm=31i@RsqloEH>37a<;?oC1tYLAmL9D>=rfJj6HL-Xvl*+! zI@D9orhgyLo8I! zs*ZNjmI{pBR6J*oQuu#ZmVeTFw|RH}XTA3;h10=1TX+Qx7<&TPx<8*Mx2fc-efw`h zPro~uaZ{o0E`*apy`cxJAAN-+y}Q5EX*<@w@6-P zNS4b!R761W4xfV@XSJDQmQre@9<^C2`tt6CBX}Q1?{Q%M9WZ7%b0K4{?vr7cv&scy z^SurSsQ1%FwdhGu2PY-P(T4_1#_sT_Ici1dcE@M*i&S;m8}|IMi4drS9S-2QSR9^y zQgHn{XhyR=L`5HlXQ`wa&4mKX-6u*bW9X0Z2;X}aP}LyvYd!UV?Qh|)P3h##a7)KS za2h}dg_G({?!xoDgjO@$3Tdzvhf1i9of@g)|I`0+$a8t(Tm}jInB}B zQ7U-+&uwx*mK3Yt5DvTbFa+8!``x6}%7i!Bv*}7_uT{PbD`6PbRfMeTh=@ zw)rDN*oIOMwW;`*+FAVZpGC>Af-H{8`aVg4Kk}l~`%o1UP@Q;J_M7e}&sN5sf*G3H zth6kDRyUBB7@=Wn)^d?KlyFfHE?7)YeP&rzLUVMMza^U8R~>UHSWDTz%l0O)m1;Ym zXHd@riV3oZKt~#MC~NE6i>Y(mD9Xca&~@h8jzS_Te90p

!Jr>Xs4qI@+ymJF-f1#fK^YJ*we2% zBI*0mF+N3&dO=`)UqJ)J0vEF30-9mLu%v=3+i42gb)xFj(e`5Tz&mOBYuesy+1|?f zN!9Vp#^kx_S#3TUiqX@6VzM2xZv~Udn>*cnC3?(0={lae;5=N-v*)Z-ZOiQ9T;%Jxg3`~oV)XQx3?uEg>OF_=}@V3vc z8PSDHX4mkoGgLm2L-mDMpXLD^jkOd=!^$I+y1!K>Jt>h0q4S2sxH9k&)oFXi;UaIF zRF7xx;(A zd}FA|`--wPgKKSPbMU^(T&QQ0kCn^yZr)dIKkv;cVb$r~Mk9FHRIe3S6p9#>N!eqG zObU@RGV{&U)d5q*_;49?{fIE5iBbj{`lGE1Gjl*u%S3ZsBe57B2bN09W?}?lMr+k5 zuUCl@j9l&a4@zA9yCvXepEl=5L4`R! z$Hi!R)^7Q#c!DJh`#4O-NRcOV(3CA3W1FoX+R2+YQ>Nf;4GyIwRFo!rrd|bO#XL(` zUm<0hW{FAj3kJ4qR&@lfciiYxMu|<@+6Wib&L&@NW5;80x`_LhtCpKwW=;DL?6;yULYY75^8`YYVwfdWjl^NPXU%t7|ZygiYuTz8?YKK1? zO-Z=CAWu+82lqkZQ0WP?gd(L*HS!ylvQ-LfXAph6_eA65-^h`Ly0+xkJn)xJ+|wXS zTc~CH$9ARL*@^B|qvnyvt0M&!=55#sKSm9WbB#Vb(Q?385zv4xrY=~0sOZ++0|v+N zF6G6LNg}|tRw*MM_<)5(Nsq{bUIV>uUl3w_KU(u~cMoQ&EpY%5j+n*(>Y3qWxFny( zbTWaR6NqO1JEU2SCa_>x{Z~RlJsw-bHskx!Yz&93oW>)j;IDgLVp8234&Gs^n(Zs7 zI6nOz+`f{1N~ZIHe{KIj_hAX1#dYG@DtTqlyoeCh7<#810#s6Nh4=XL4sW9;vEKIQgitxdER@;z-y zZ73dSLX4M zQI~Bm?C~HwbpN^H!0`~75nxZBky=%_@67Z9nsWr94mCCgMeRi%ks)?nwPTEz}?5uc!cIb4s`b&uR#@mclVunvF101H`5vJb5{fld)fno zg0T0m1qFQ}Trh0C1KsXG73EO8tVqN~93OGXT#Y~#A&|bq)g-yh@Na0mixVY`GefhuLeSjtP^SORu@aH`r14Aoo zGfVr+`#yhV)715)BB}yxnjS!y@z$eq|JklwvuS{K)m2>ow`)VO(`=R>St9H8gDg)^ zd2&K3&Y|@628*Dv;x%6pm6roH;h!_|iT9t;jeUB|7OSI@Vb1IGNWTk<#gg}{R|EDr zRE{c$#hiZ(LYnx_48!}xVfxLpNwdjH`%N+NQM0zuM|8-QNGBY3Te}A^)zlAo%$C8N zsKA^h0)#$JaxJpQo40%pY|{=T#0z?>Kg+otdKY!#Ufm*&tivled8Fw;s9MQ9K6l7Z zlMc@GJZlf|)05d*7d7sZ&4?nQC+Ubx7vaF6k%>dUP7Vm>tH0w$)29iDGB!a&+%1^( zJCcB-)ag%vedkd@+92KY(Ql`-K|c#Y$SHxaBZEMNL`?m3t`$xj zxaHJUktV!`szw6F{`r-pEXPFV^A1oxfIF-jb@U*KhJ! zprYPd*HE!}%cSC&&!Kyb)Ut?LNyqV>Cb?0r;i61F3=V8w!fwM4O!;(3yp~0;i_!Rz zP!#O!EQ+{dhB{GC=iFkC2+5i#JCVR>rc6M$@^gd6i9t|Tk=^Iajo-WIH+n+&j&DCS zRGiN>`N%yyWjGUEjAoR(t;In0qN8T)zDXB)pg{!nw8*_6!&;`*u%Tzp2SE{DWsym~ zo$4YCnfkgRcDNA28xVjSP9-N=vd6P=2O4L?Cvq37-H$h&Pc)A(*J~sSp>W4i=hvbJ zoHKI_yV*=bho3O`zi6iOk6f4c*X#TQJ7kgJuU92cq}NGPgxf}+eYR)K-e(S%xiONu zu9&J;fMN6564PH9vf^_Icwi)F6kSZ@?4a0sg8UbT^@Ay_)0jsYUVUc5V&VNyPjV`W zQQGq|yaskH?>tsT6g|0@aN(Ss?j6S|z?Omzipl}`4lc)x)yt&fWs`P@trRIBPPu%!cw1is3Ef}95Jf}_xa`0DcF}vtAp1>`b|1kZGg2Fr zQve!S(t&}Li4v+&3Sx1+5d!`G1Y|n}zqv;!l5v(m&kIrXCk3!o>R$>S81_BpYJ!wr z(U`!>-tQ~)_I}(<9g7t^MWz4m%RE8Gul@hlG`h_p@>A9SznVs<_(spKOe3I^ZJT|5 z^9Qs)nMSvrP5R-h`F}-L*_%>w6PYpiBr)*3GVt|BWVhHy|5wP~8zDU8k|X}tP5T8c zzv{oepY>OPo1X;yMeycNC(uq&KPtJ1ZTQ|F3a(;c!k%G5jIX#N=)bQGU%!#7X{l{s zXrrt1KV0s`!XP>UUJ(JlaIQHN|H(ISElt4BVf{k${}@)Sx31K3YfP8d_EyIgY+%VK zKo>tBNI+k7G_+4Hp#sZTdHgr~z_0%tm=fXppVQF)|7-!~S_`y{myNLh4UCBABlW;y zbifzc%|@vHp_<0e%itO&y4IRHn%0^>sRtY?@b#~I;XkOSxg;I{9QNh49U#?Og8(II z0X`wz;yWNRH~Z6~*VAU9)zqZaq-W5jWMF32q-3E3{K}}wz{tR)NvlJvqen@jK?5wb zMaz86wE7{M{#|GbNPvF<0~FeFl97w8-Hv(^(w6j)0lzT=jr5h!D4|$XNb3hql1Co* zNZ#fcL=rQZFo@w%(%ijEK(O-E76n!5sS;N?S<<8G5500+_cmbj;S-Z)wKaRBLc)D# zr&kUYhl7(1`-mwiJ&DC!rVs$(6vqvgixbwxas5=ePMVJ)85SAnGIB~}DTFchMmfkf zzL`1VX8oA$q7F%#N$MJUB=q4TUuLYLGv%}jMWp95DQ4yDwZ37=nI`Bd(q!|PF`!c% z)5%1Xsp9GfDr1Z#SxZ|Jb{&_Om&3umWC_|J+)EfYpMlDkB}b|Inj*P9|3y(wl**>6 zO7(%Qc_o-2Y^jxWSF*Rlf+$9Tbj;4y2k(PYtLh9@#`)0G3I$kl&Q=!3!w-vs5;1J{ z*}P=w4#)4Oyk_mB(v?Fe@Za^C%A{AwhNKpRM0K(Tk4dFw<}v%TTD@*Jp&JaZXs3Yh zT#`;3Tkl_;V13o7(;xAcSV`Z4SvMxBU-wB=WB!xKRW^S6;&9|0^w@IYoNK<{EO3s>lwa>Yso%wX7Iqe6;pd`o^2KY2 ziau;jMZ>dSX_Ji#6^-XOs)faIG@@vQ4h6qmQQyW8^PTMQ!3RyN;=ybb5QyR`y zE88k>Yt-xpN+tz{CbDbRI~1n66aq@~W)>36xU9^VUHdC89K3ibh*L5o8EN%k#?){} z>qX6?_BJEMzzJ<<**51I#|rPD?|FSgayEREKyh3!{GlAps<88Ex^kwZiDUym!|p)m z=P-Ya#4IfP_7#)|nNVF*PG(`QlT9fVoVzhQJ>8Rb=92S+-?B&BBAyZ(J63)>%6)vH zyS_B=fI5OvYY>p57?5s@i{)@+*$mdHQJTa2Mdb){O?Cfz77a6&7PeoJNxmXh`Msn= zu5I0Z8gyDcCVx`I`Z!h;n(Nzn#+72*{0XvsG}8KqvV zj&d1K5DKSp%$pfz6NhAM&oBmJs)CmdWs`gve^lJEFQEn9fIdnKCS_yN8MUeZy1+>~ z{rIW8T|hU-jN@sZF`wyvs*D2HOql8LsbS2{Nl?KX*f9cSlgAJe-XHC)5; za4aIzcAS0=U13&vJ*%$9Fxq}7SA%Q!bTTG0*QwE3ZKjbiyz|L~f-L8r|Ds8EmMleI z@AC?KD1i@voN;|Pm zVkNtZyngbA_~5NG)1&IiF~A5-lNrJD89M|cz-y(^^LXzD(Bq?Duj)>8%{&t&WKJ-GI46+OHcbKi<@l* zH>(f4B}|5@E)E*`YK9H{=9JDhA~meIq)Ahnd%acu4A^Zr=QdEeNezS*#u-`ujp%DYc$ z5LRtPQQdZ%XVdqN4nIe^XpNkUlYAIX_*7SEy&ei>X5=zey!+X(2zR1r>|w-0;QU78 z>-_+e)RQ>Jqh0|vW1@GYR_qeEc>P%jCFkZ18RX{&Y&kXOnV{sB z$-))54(wcAu^Fv$YgQWb2y(Me_r16v!2_^I4RNW$lCf>sZM1D{L9wchr*XjxEQ_O~ zxun0~^yDlNxYd{_r8wzu=Y6F99LEdV0Lvi)GWY=mry#~=q?*fgVpng2&zES#(RUWr z^=!3oQ5`L^FC{z9qp=yDLu7lg5xHh({%Q58f!IV=D3d5ADdp+JF4WiXnxn=su2DN- zb3qN$Gh_-b8-tSh#Y|D!`HS5;F5}4H5|K~K>2U3qvnbi)oZ=nvmSd$o6i=O3Q%Que z_7qteW=Fon`D1Krq=~GT9gDS-J)S2gYR)oYmD7of7}oqUo`7jPoYtWhKgiD+Gm0NP zQ31B-vQ-~kpzt9K0wwlNLPVt6|Qig?;V;nEiMFYF%xy@U?Ak!1N)$D0WIr+MP z$Q`I+2zD#5d>J{4bT4FGL-9{c$7fUl#n{6kOIY#8dTo>W=I^l3>G1kj1ghsfqhhCrcGvZg^E$>&)ImIHa+)`03%(ciL6&xL;8;!7KPbRC(Q! zW0GwbxlT1gUeZD(9HStqxHaoTTlP8wA5xd4tj(j;NG(NnF?~oU{XoaRwTSqzw(_}? z`KGMy>l668Fqe+HEJNOXYQT4HGiq@X9fX7C1Ya-q+H$t(JTfW!dj- z#m`IL9L2#o`L-Y#0hU^p6vo#Bro4&@hH`rE-X0gSR%H>Ycd*61!^8gR42qxzuW@-6 zuWuK6ZpTC^XwC#oQztN^B}tW3Ok*c7NQEP&tdtD7Gq2Ipr}-c!5+!Rg5ofb$p_DE1 zm8=|8p3KPOhcxco+oM#{iY^LombAE6deKn~ri>XW30vFJ!Av ze6y;Uecp809u12h+4$HVDH>6~dz!dYWvkl1foWLvD60J%1?}XhAGjAYxy8hxI3xrc zm=t6drrCNUC{Lxyr_yY~dq3%i|!02R^Qaf%04NhSM|vW};wX8L65GqIBswXs=dz2!km{YJ+1b*cv|Va7 z#@CEPqOc+n?ofZZLoT)sVNzO>gq;ibQx-iP64+pj~wFH2V^Xa z)jMO21U$ZcAkj&Cnv4kT(PtmoD4!D}NgU)ys8vbp;+Ejc#sez&?H-iDvXtAMXDXg3 z&%Vmyu6OdB9A_MyRONq8Q?r~$5~erisqG_x@{KwrP_r<9;ZHXVCKTDHQtvCmHJl3eI*BBt4OXM$u&f)K@wD^E-QOqz6Y<cfBQdcrC6dDV?GBUoUX2kFCT~xe+rl^y!9M~m`JCT-=Yl@0q18xc3sN^2Axy^D* zU?dXe{!cP^mmMB-Q02(>Fca{15bn`gWJEXEW)lHYUl?GbNN#CH zLcmuI-ECPwL4rItDBMKr1C|z}I^R_$FSz z)EE+kyS}#)4CMIHC^bk6KRY;r4#BarkjgmyUBEZ+?%SyVQMh${?);v%QW{0U6!3cw zdGyvDkm$SY!O%MF+id4NQ0ugbhMq_W_981nHu-0UD68dcuqXn6&fwo`eKAH$~#a3M7KPf`LEfOyz3{DCvz2MDwHcprRn+^FYVJti>QXJAGt_hrgZH`vbw zF94r_xXlT5n6v`l0Utb>ZEg|}JPr#+>@(N^LJ*TfDOwP2vUm=#zzDkumha!G-Trz? z_x;^J{{%u*mVgUUe&-*+ocgE}1E?7;y}vvd0!IP4jl>N2UNXMv5|9oc(}&w*|I}*d zu4$OqU3M$F8?W$@!OgC85_;Mo^)X=X-tZ)SPu1GOUMUR@g5+J&{TMOL$URou70|?z z!csNhM>3>KJWLCs>c#gt2I1yv$i7w+z6ZS0`@W&SMgln9J+7$N)=z5y=csNzJx}dK zFaMShDWDzjPKTX}3sfk2MF)@umAgDc*yi&4c!_D=%ctxLk4Y`=yBC3h)Y5(wvjO2I zp0>p6fimd5{E}NpEsI6GZ;8wIF@FE7s&<|cgqyn8`g*r`q~8Y!a-4EL;gQksL=L5)GDTG{1iOGZV&MS1<{(YJedANT=X-y9I>_XqERsKh^T|M>)j4h=!(d*gu% zhm1BL!n}LnAl!n}CG5a2#4j>?fd#@#e}jw!%o?5{ooyGWwcpd%KmHhmPFO^}70|kn zfORy84j~yZ3+^t0^S6(xTIJ2`H1HtMpY36G(Qzt)4FBWdMlhFn;AUNCaxE2X;Jp{Y z*uXEi@lQ92f`Eu)i6=CE&c)X$y-({190j&9kUS2^%A;18777dnqtSV8+CoEVn zzBe0hW{v~kSiILgEYI$4&_TKN3t;<)G+aeQ6L7X(TZQ5WEzNVCn8pS%8tix|G6$xN$Zqy#R6ch~)%{53rcIQ`IC0AWKaH zn}j>zv??x0!)ILuJbDYE%W3(_GP4L%HUJGGj+o;;OHv9rO1Z>@0a(<4C+O|7fB4PW z7c)SdUEKa^>OtPMb`$z%^lsYP>o^8C>m#&?yVmN~2&#WXS6i{Jh;gV=(a-zvmNmGS zaFqf+Ez2sTyhG0_qO9K_Cm?ckyB%JpLM&xN`jtTLFTEJTR&`iUpKq|22@D~lF>1gw zIG$EZd1<|>9;SI=e8wIy#Mq{J|AFE-3QCF=!HAl=y7J1G@YQzmIl8KX#7R}n!J+m& zs?RKtT<=Iil4ma-vkWc@^=0Uxoqs<;dgTNQ5MR<%0u1*#;6-%P3HEEB+xL*+FZ^vX zR6-PFtrV>t-7+m=l)|FI6vA`THso!xTM|@KG;~rlg?V%|MRc?Z<}aZEzthmoi*$8& zc2YpMc8C>0Zo|R76@{gQ6%`Zh?1U|yPAyA*7@{+bVFh}vycE!RXC4v>6AcJ}+5y(R z;=geJUwQ9-x)y=nyIVYX&!|N7o{Zgwx%>Btc?<*Ggu;4X?24!g}sVi z!C=sE2C^guf|1U2HOuKj!{}8(vwsvR@2Jy#uJQ>DD<~DzjC>48G;vXv55$BQkm%2J zUPGl9cZz_2GC63neYI3UACcHQfInN&3=y|EY0n0#)WLN6@$j^JQ}+33nMai3r{(z( zlSv<{hbx_Sclj4lWy9+lau&K)8dAA7SBIL}at5Pjt#I#h?H5h@{GpK>8#RqCD#Mdjy%cKW<(x$Y-3`K|%Y?{bG5#wI!;-yijj z4Kdda2M|CUJzx8hv<-{r5Nv5q>k}J8*9j42e=k;rY1Wc&yT7t%xFMiDOPd?@8O6lC zfbenO?rCcRa|2TvpN&b1A{ezZ=a`;$&#%*-Y6`=|PjzL;qN;saN?b9D@HzxWK3F%HYEx~`8Wldxutg=Txu+X&B|T?o}2w<>SC@EFOeXP(1z z(o@q)p*?cC08e>cM-8hS3!HCQm3RAYeN2K%g+;3~3Y*t`)I*x?*|C$rBkFx<@&qkh+g-v_GtFtUYk>A zpb~tMPi2+&2Yte8zdaGb7*-E-quQ4zRfI?l2xO}fdb%2jIl1ZtQEP9r>gE01wUMLD zQE{=Kj`BqV(Gw@S#(M8S4AX2>>?JKH3Yc2m1-d6YBQ&2FZRyCjBCM#rpRH{r*^(R) zOBswxL_Fh^;WnO;EqdHgYHtIokK@CcZ>0^Je(r4G3=umR8#Cte3PW>=QVLW;;q4|k z|HioXV=|6lkIcMAxa`T=^!y5(;6yoR2+joq*<*qByDjb zt($D5*{9l9(eD}HimMFab05P|4RbOzE{`(7o#XYVIX)q3&9GoI;pp7AbzuS$Lj2RdGI@;cC~VA=>Z8921rX#FdXp9qi6pO5dpJw10O_?bspePK*R?aseuFmza@h5yQc1b zPx$?#DWDbcy{YfPqd&Ly?}PU+H(GPQ*4p=Y7NEgf5+42=-pw%5kC6flO82Il4gN8x z^ydKpGoKTT!J-3u0lpM|4glES6M)6=gB*YXP*LeI(QDGt>9SDL>(SCsGSD&6Q?f8H z0!pDVGSV}%JYk^Mq+tMrm9Eb2{0Jvq)~)>~+;pjy0N{Xm(Ek-wx+KPxdKZgm05$p< zs|dN5&keg-*Nq6Zh+?su!mWeO#vp1F~Eem$v71N zkpDk$(NHo`(muI1H$M(){xdf}83!0>{te@NKlEVxg(wzqxNP|ou)^PUqLl;8<7?pl zl4gIx_MK>dzy*|JJyKhEPCxCw>$v7aVT!p9A=VYzI!eSbsv_RTawX!G9L=M}V{A`Xv1&u0IvW{7JMwEEpispv7&Cr~%+E%ROHw+7Y1U$#rln%$m$} zOn_4kOu96hlnk`GI+TEu4tkVKTFkn-Ow5cdEPC2N?L-MA4$BQOb9qJm-(uzur1N~Q zs4~vYTcQJ)(*L3>2N)CA(fuX6UFyi|Lh#3ttbd~`Ir_!wyJR>Zx&T@SVisxUebqcK; zzRzIXLYQFS>vz^?e`sPY{EH1PZn4d!lk^_Po|4tSV{?l$Y|e=u!;ljEk0_OKJc z)BhDys}?tD@Rw|JWpCURQ?%b#BJ#xc76WGn^yiuhYU`78y zd;(eRW*z41;Cvi!fcwiC{gdMPPPAVtolBwtP41i3pswSJ^t*wJ>VGHNt+V-))Ba*c ze_CjOH#hgMd~ZRn)q?^|!7qn;Wt7~k26Z{q%Z<$IYpvZ<8|rt>U$NhHu<$=s5OCkV z^6m=kkE%ewH;HzmN&9~{d81m;|K2*` zjn>*#Ov2C&(iwS}%PX7u-0H2K-=mA3k4rCzfX0iXQ0(hy*9zg%Zm3*lz9kLxJl z!fruvErZfu!T2daehKAr$pc+hH>-GDhf;eRlq-V#*pJ^{(*r<u20DjL3my-Y-`SrwAY(F^+=(U-Qxux@KlW@z>e{}wn^uHYa6&Kvh z(tdsPDzSe8@ay6KMDQ0X_hu&c>j;i+LvTg^AN!EoQsw<#uU<|5&Ft9MJO7$+%kZyt z{+rt$zwi8t_BWFWU+JNefH0nqZfx`OzV{(o3}KxuR{QSfyXqqm{BMuOj6 zsQx_-t|s7Sde`f{Cw{zz2G@GOoPcX2xHZ}9-^s$885*yn(9OOD#kHCEVLaW&g#XJd z+|0yy9l^zI2(IXG8-Sm8w|+khSDbiV3~}5LL(O@=AO8=F`}+8QcK>GrTtT>*L+?5P z1V8;A!vBy_?|0mA#gjMl<6VcccN>(ex%h_|*LV`}$$zuu0Z{x>E?y(ZjZ{?EQ6QH5 zfgpdMnCf=~xq@;t&D3=$HGhI~i}Z3x=zq z-%Ry$edvsp|6eQDZiFxlL(@<|L&-4!HIt;FprB$2c3}sOmKL~!hTj0(Q9wh>2-v}~ z-y^c4gDjvt$xn){U-Ik9#Xma!@tO}rvEB$MAUsQV1aAawc(VNCbsh*|vQ0w}E^1W- z3KsI}4n**h5CjiME`CcBL8f8f)6hPfS35!fuz(?T!hvVJegTzh?*;KeIcHrCxo{KG z>c`{K2;gdZJza8j=-*^OG0DlR2dwaP9*af@vqKt!#PTxEXi09bf;|3=2)f4_1VWtu z1p$vZBZ6fO0@c6Q8>8m``<;Q|$N%&ci~)nphUqnl9cyDAV@3@ZAl4DCGSr;%rjN6x zdsBMnB3o#OnT@DuQ9u^OxxH~IO4*z84UEgVbC6cH&P0>k0babEdt-rrm2*FF$|}Gk z&UkwH#aY&v!zj>H%Vcp*Q9}e^@jB7yPQd=J6LF^~+Mx5`Pc>x(@3x{LT1A6?p5@tm LU^@7p_Qv!EIlZgz literal 0 HcmV?d00001 diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9_parquet_log.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9_parquet_log.zip new file mode 100644 index 0000000000000000000000000000000000000000..1f5d2e8de56d4ddf834d24d2c8b14b685042beef GIT binary patch literal 76199 zcmb@t1y~$S^Dnwda1W5+76Jr!cL*LNXz<|f&f>0tpuvL&cXxMpcXwy8JxlU__TKxQ z^S|f0^*pmPQ{7cl)m_uu{hKa1saH@4fM;rm`J(>&;;$cQ06u_;fsvJ7Nf93KsvUf9 z3&g9BakzhoGcf-TH5kSIrvZrnrh$>|kCGQJ9*j*P$N$!h zl^O>RB#3d(2i1Qnp`$alveGv((6-jGwKuUhv9i>*FtFFr*Rj{3)3eq4r>oEI{_Po% zVu-$%088LE(`UCaA$M?qXQLfoB^;(>002j2FaY`gd>%WK9-FQKGaIcgvpzE|3%dd2 ziiMMdmPL=*klDbHjn#manU+DDftH|9`XVa0$^3L#es-Od^TzL-u)Mq!3i~D4e*|e3nV*Bbpi^*4s;;0=HATM7}7# z%F1j^Z6~bP)IYx}6>3bwmKUyO#M9N&3>c`=2ECzvS03_<+Uj-ZR(eCJq{{JNmI(~H z@`q8DzLWbgTPC~hhaDTHok3=dax0nFRqM36>9vQxqz)ToD1q?Q2q7Oseq};K2P%8>oL0>qyY%fDNzh{Qq*&_!k zhw)9yIol5R^*TK2<%Un4s2iJ#4<_+`pl{)*dEZiN1S%YX`2l^_>X7!(_D0`VIT?#? z`sw1pD8mg#!VR2~_nNY7ZK1-6rTKLQ*IBhq0{4x&)8fkMa0k7}kY&ARU%XNCgJT!6 z?C?>x_PZ}V$MtO6#V?qr6C+lf?lWr&ANnwejIEf&WcOGuwI3Bwm(C*SPcSE3i||UC zG8u399(%=<32S)rhVlP}5v0j=fAShJ$9h2f6I-??xJa z6h<_%k(^y081hArSs2fyb}^1v77$jdu1uUQ&3U91n%28_Q{T`H&qbNw-ns4h0V9wY zjZZ%97kz&#Lyg<{eZ5*$mOpz4y<>RE5%xUR!U*(XsgijKolgIz;(=+nN|Y-UXMOpE zXrA!qgiXmZD{@lsrie$eze1e!sYEncL7`@lTRt}+T+{F>dgagjSv^3|?0Tb~GBnU`GkCT?98>6E(CwX%=IFRm$cF7*0e>3Iu!kLSs8B=}ts=3Fy4%zO`>WKRxIx=kELH zESOI*4h6`8MXoCQPkv%_pmKboz1+ERSpM^shqCDk#F8 zO=Q8^Mos4(F-rFis02j!o#&-aB`Z@uz*={AH4$;w9!DubGgy32ih-bsoJZd4+5XBp z&t{W}M)1Cksb?;!h**v$ds``Rv|`4SeWT)hg*%Smc0rGt*~4aO;Nm<~#na~c?AY@m zXE)ObYn!+Jz}{nObmTg9b}V;eS%!Uoq5Aw1t0xFimZzD0(Ch+}!_JPnIYHSWpWz+r z4u}$!3~nXm)0fGqif~i5BF*u&mkY)gKdhy;%PrT6iQk^(t_xdvUhasO5W{9Q&3bj8 z=|wwcC42&k9mmIctN=&gdFh+Tjk67i=!ZQ(6WwqpAI761aArB2lc_%qr#pwAJ|Tu4 zvsC7^D+NihtUFm9&0%j>G@CH_)r@$!WLAwV_Q9H0q$Rpr4V4{LHy1>7H$`e1C0%+- zX}Mm7YE~zWv!|qTtRbqOm82FBXgLmEcShr2SQURU#VealNVGZAVKcfUN{0uoID%hw z={(+@l%?!l4{z@&^;n>SJ%-Lb@K9%?dZs!ZRvH%^D|My@3A@D1C%7&XL1Wq|3ywsz za4mie3q(l|xx8CR=9DG)7zLsPx zSd53oy;@C~)ygEQIWV>1>RJl!;&B>U8W?|6X0h=7A&nsLs2q90U_~zZKHXx`H$0Q5 z7Hn0`s9Eng#O`Tc5RMD1S@VsssL%rHPn8g9n3~~oI7tNnjhHnwSWcX*oG$ZU8Vwtg zk+qy|`l6%m&qOTodRYT4m%aE^#vtVx}d zvV1LZnAhL01U+9Jhgj-ist?4i*XR^qtrd;h(va-|ytEyn??5H1DZkZO*TzGc5(B;g z&*9M%tkisP8cz-+zr;LN=eAKT=s%%xpCMYdxBuQ<@POxWq3kZQ$Rqt&;82l4-()O^ z|K7cAg`*zSB*SMW*H-`Fsx!HZP{d#$vn?>Dc_11xa5GgLcFx)7Jwjcw@|ZiNkPlpp zWHIu{lH&RE0$h1_@EjrN%Iv+aEqL_%$b~?d_&NzK<(4ovxl#&ckJq}1H z+Ci$w`oTm?X`=h3vDoG1nAuBU$ddi=>2~lTIWXU}{VDDtY*Nw4RV|8j$XvRdgPA$% zoQYW>TK?ihJ@D)Eol72};F8)b|(@X$k4_tZeltCIkdpiD~-^vJMlvK_@Qh zd|f?xX-X5}PunDv({_Bn%8_JcX-l`cvwwmN13CMmj$`ECQZ{C{Y8X4dRhvjQ*?u3r z6cdvgqgz$*wln!1BZaodYhlF-4BiT^&)~31=&a3vZT*HXvadNeD#XH~pPE>WCWoSx zNVyc_IFe%}$fG4XMDk=iF8s91LBQf;q4XS46-$|RGbLg_NXymKMM>k9eV$0Rgu|K%WY290K}SrArq)%9qTmWqNQ_q3>*8|S_DxZ@Ej*Zy z%B%-U*#^!=6;1>%EWSu`bsi5tOTApzO1+#mQztLl+X!B+A*&-!C#%cTma>%q$EoJa zU{WVV=c0G@*~+V7NZ&D*O;Ddxos`Sce{hr;!ia`uWMllaGNh%Ub^O6*9(J%YAF)Cb#PMjwS5z}>IQe_2_`I7wuS&B+! zTvAw0aqUs^xlJ@9%&+D@3a$vIX6`AuC{S-M`tZ$bsd|inKCja$*7LXTXJ0q==9iT#q;A+wsT8*YSyQ4&bMj70aG;%$Up8$kI~M(k7(N zxHgUdFgw%)C0 zO)K{wTfY@QIM>SU;Bz_ADN__`**xGlqEFN#{f0r9mP%x|APxh!ZFGl%s zpqe(p(uqSVMM);`XJ{yUC;Jt{v?-FVxtm%R8`rWC8G2Pc$C7LK@d++v0ptAFH!#lZ z@qB7ZBp#)U1!x_cyZK!-so$~ZP`N5=K!BT&OOIc&193!}tm^Z$&$c zcjLD*_zB*X_;aNQhS^L($rk*IX;!3cl8N8iD{`$bwavaD%wHb7+8a02GL14*S3oVn z-ZIF?mgX(z(bSUV|MG2(SCK$39$&8pJ(IDSETG{^S;RPO>h-pP_-j6;cjisXRfyI%n2Am=l32!#H%NTj zCOmKK_qRSQ3%i@&X5-kVNXGh3U2@X@SbT8?Rj{Y`U}5FEW87*N^b_g=cb{}tjdNCp zY<9Ytwr%(Xo;A=-Q?0qVyW|d=M+`f)1BEl|MvjQ2rk})Y$hH?6o=-SGPJX_CPMqQC zngV9zbLftoD9Rr1q*(8xZ7+djuk<65e4qw=-hKz=*zsi1l=wz?AO=$LE0USd-!Vn5 z`o#9YF;brG?;Q5SrS9-LIaD6GTS$NT{V1f4Iq&7R1pScYY?Pm=Nc@zSHAA~jY{DM9 zTkt-+;#W-N$IOBlraqA>d_R|OgOmQvAFy0!-_bu%=E=1w{fb!5mM@lISM84@BWRNV zON16b6wp~VX)K1YFba+qNsM;kWI7;sAIpEEbai(z_-v~sl4>6mU5J15UJ6*VC$F9! z!9bBqmWz02mZY~G9~Unh%T28`ENKu=clC}y#(l7N!Lq;E<`C}wee}3P;ULT0Wsk0lb-Qc?pFXWwoVW+ln;Qv2)Pn@ zxep%Dn!N$@dnsOvUOVxZA{D5bHpmJcLO)YBv?u`pFSJ$%DF9%&hw?ScMpvu~hBgZT zu!_jb0D#kYU|F4NRf7f)uiD7MKt#L{*ZZ-bWh)6~-$QCCL?%1pubtok0G?}dumCh{ z=T&%tq|Acfp0q+k^|`-%>Nfpt!q20sk)+7^9Ssmh$pjRD_;$4Ul`*wOAoS=pxt2eZ zZ`BQE;Q(|ceX`B-gCGlLSQSezG#ggBxS!IcY0tcuP(7M2f>v4)T+P?71Jo7&c;V&M zVb5Jn~5gE1WBEcE$%ixIKhshT5k5--H8qxNjJ-bPhO z2|z&IY$7+8(+vK(Q72KBzjV^uKJy`@@@AyG3fNW|Tzm5p-~|vBMtSD8f;iM*02&57 z3U-XnXE?<^89(b)Ujfi=jpNaxJT7M*AMOaDWRPA$EYuyfg7R8D%`~C}K=B?gd->og zqUmzvKswb6_feP~1;zVon+I}hSNf6I>IN4Uz%SqN^r8atj28uu1kyDvoI+^pW$*w2 z!0No_=~wlarxaTI&+f2n00;y|H{N{!Kp|f5KHH?!(SZQ42G#Dv5Pvw!5q)Oy`xMa+ z-=qDrEda!wXXTIruY%y^w~#Zyj~E5!U!|+)r@Eg2BtiF(uzIEk3qMFa7cq@NV*W2O zUg7xuk8uBsMTg5y_|iN7`Ne<`6tM?B48Uu8ji9J zdcTUW>SX==xPK!wfX`0C`(r;?0CDB%DGETKyNnppx4l0PU?H_```{pNvzJ^C=fC{@ zhwyyvs!JK;^PNWX=ciO}KzjiVfLQ-@9_^V2X?ih6h|uV^&<@*S*;DEZKzu-hE95p; z8j246CIiyV056}4f#=Hqq*L(G67RPOIW)jml|{ho_aT^=a?Wph_-zmn5OAqhUw%Kj zQT|wp2S?-MGhr=_@F5aTHxwFOmwP3*QnvgZ8R1Bnk?6iBTab$x`fr_HEk+StXgIcz z7zHqxZpXd^@c%-1c0~ZyxE>bZHId`IqB0WU{>*wvr|4;h*wlu``J&Z(_5lIV8mU0P zuIhz{eZ2Be3R->v@FJKlde+OJAx!Comk5dK*0vx0D_DeJox;l57l0nU1uOu-M+ne8 zGK4Z}3(;zAXG>}Aru_OvSkME@<#+4QrZbc$K9EVdtPjK2%WjSgA(xkcRd!+3h*5QP zsO0x9W`I=+lNsLzx?A}>7%x&gW*8LdM;KT0k5_Gs)o{;^_DljN#(Xee1D~XyZ@d&P zJ|rx9)Xn)hXlNGFt1Qe~RO?((c_c2<&%5$PgfbQNH%f-q7ETz}CFfpL_Z5hI-bx>E zl3^$E#EwY_5zR-u1>DP7?Xe6x~^wXhdh>t3_0 zAyZL7!#|LNI$v;gz<_M=eaeMRJ)r#2gT--+Db3%N@jwWpf-1DD5GCD$aQV@7M=R># zo&xl+rF?#P7J7*HFR-u?0Y>8IWTFIWkx8$U%7+bSo6P7(R`Ryt!XN*@BLy8 z8kGRkFdw$87rbgg8_G;eD5}>59tjOiCZ#^u@3W5_&hkGththvJGbng;aq3Hz&ULH0 zU*<1a$X&QoUJk$sZAoj{ThiegkjC4N8R6f#0~V2zvo^QXpEeT=kXufLoqGW77rA8# z6ISjzSqpL-jVGk*SNL}lWk;w#KMDbJgi@uc%04D1mWCYJi77KqBq`O`f$5WumK4cRGal1qE0FJ!aR-gcY&E)RZu2)aC81 z2TDo2r^DGB@2~6Ow{B`ZG#VOQ&hoKR7g*1V$=HjuE=GMv_OGaKQr+iLWFs_Y-WQJ& z)-L0eMwB%&=9)N`_x<>6aQbC2s@Y{WDDl%#Bhden9cv>z|xqfwh|4KppH*Yoa+j0%}nT%MLqxW9cWelT2W zv^n6{z7{_lmolAh*1nE>3?`!UJSV*h4>Vf7{&l`(cVq6f zL29n60}%O`9>?EVEuV2&T4xY zG@DLYd*pbq=pv2E)t;oiJ}TovRoIRwhN@wVQ?Wko{w=#j)pe*S(2=XV7_2^al0U;q z2O5E$=DRv06r(|1dgV-(XS)aBdV+MOFtDx)Pe7TW{hw<%d|L+8CI-*4M(nI)bV;j;J& zhG7Zw(hPh~8*N{8YG+ewH12Y5Ua7xpdygWGM9wA(?j&Jkv1rL;an9PzHS*YJ;2<|| zT&;J)uOI?nV67-M?*wf*$UQ#HmIcfZ)PVJm8ZCVis`T$K@roZCG!EC1*?B>uQM+k=O8>fFXBDHN zNV=;>p5C43R&FHLF?)RarKt{TScsR+szdR zzzauUc=3{WQQ&9NKW=Vev7gX5E0!6LKJ{;Q(YrWUUig2fY)U^;sJ-n>@1gXt6*>c^ zsRM628qDZbJvNd*x3}{Tx^uXIM)-jTfq0&{siQmdeEtKn_^yNJvKZ3vTWR1E&$&g# zO8GKSo>F}R~p*vR_-V62I5mZnjJwyp!V&MhGUb3 zhnV%w+9Fw!Uht+#(8CdaAn~H@mpW=5Ma8#7V}l%In3t^3WD#E|gM|BI6JHNQdwb1L zaPoQwz-ag4oU@8=3E409(ru_^ZFdWpZ=TMIb5;n99yH3UVdG_g3y-=tB+>50GLGT* zf7>aF8X8`F0U|TXq*nWxuQ7^*62FXY%gW-f(2ux+gQP)_X7{-qzoR9Zq8~?X_I+2t zrifb8Y|Vr51jdsZw#;$dzI(V#3-u%j_>=MO5i@UWfsyTwmWK{kH?*Q{=jL9mU#uJT z4C>X9u*W8OU~oaXvIj>ABNvbCn?E)}V==LEvn-&R#dOaV2ZPjB50$|5clH+q-9k6%!;J zG6bFfu%0EQshAf!=|nwsA8Uj)d+)r`So^NgA*A1Ug=BF&&0QP^SuC-JB^wv-e2!we(81Le z>Pc{JWX}SyOgJIS3rv8c6|)5`WFxkP-3LD4xq2JRQ_rum!rDb}@cDC1!BOHmpPEHY z_%|6deZy|5TXDex%>5e7b7x@tH{~dE51=TJZA_UYs*Bv;HCCR`=wypeG3w=Xm@0ABp5jbO;un55V6X{ZF zpB^S@;&MfePDH?bs*C~Xbu+k>-Ir zMBaawK3jq_Hch2eUyJ}lrOtzx1rxvBq)Ga2pdN z1ayZ^%R+)#;(8>}EOx)eW;A&ph;JfS)<4Qeee~|uBQrH~{FbyiTUV))aTHv9P)SU+ z$$!OHN7%^0RqF0-k{PSJmr3TZhvQ^n%hi2d>H&^al*7@AsMu%loAoG0V#@vtxa2Wmqrp7 zukRrZ5ii5G-XM?8X9BLKui-30qZEkwd_dOfDwDZffYpzKp9TuO7Te~W0Pgyd`G3J<0Nce%MOZJ) zjp^{Pw%_aG=T)74-6t;0)U}!^{qe%#kI6pJ@+1!IA8bOIBE4&K8QDvdUjSMG>&|FQ zf&oO+FfcG9n^1tCH?@H0r7Ym?rAqzt8G`c2TI1mVxg5uzk%W|P0*2LJ0!H5iejE*e zRk@~Mg8*QWNDu(5SECWPl%GN61)$eiBl=m-PxWM+XCu8ZdI%s(==ZT?fdppq1OPx5 zLIW4TYp@kr765~SoqY3{MKmWE$tY72)xz$HoCV;G%y%yUsZUCFugguDQ zdZ05 z9R_g8e4MujPv{olA(y6vWd;B+5gno1j@zsp$B*w2TYr6qPIv`dUIWTIY$D_Vy{;Nf z-f(nsg5Pw~o{^`sU83$%J2-vBeO3Uk@Iu63j|&~3SIOk`4uZ&eg~fURC-$&jm!s7{ zZ~-R3&T%>H0J;FxnVc9z_9rs;E})!{LhuE#x?JY7t&agoZvd;CHq*!g%Igy05RQPU zzPfhu4#_`Y1_ z;JP8e^!w_cKf8={JWxzZ?tj9c35`DR1z@%F6iI!x*6{hg;rN`)*zL8tNgkOwd;0>Q z8s?kxmuw_Tt@JaxiW7EB^M^I+^>#>rMQp8B7A#i_x92r;r!ottAB8@hhW;AcT?3yLZq2Qp!U-0c>)8dq$4_lOj{g zciOt2dDiDS*<4;o?U@kV%jk6W5AFrCn*9S(hHYJEhjTrs2n+b>RGSa`iq2{N*H1hvb?8VY5as#;( zu-w>muTM;Z*!oOOEVREHLLbRh1^~FkScZUD0IyGcmqMOz0j|V$1)#xhw`W`rEpVpj zw*-LL&Y&OOn(!IR0swZ|d1CoBOYc$jsp-lfA(Gdz071k60=Z~_$VcNtfOQ}AbG1wS z9(k|H4W*BOND&S`t{c5RMi_|n|eUcs0$f9~kKU-{`V8r!Q?L8Rq=wJWF&YvIgOyek6M6Uqr){o?W z?g5*LM2NUHg2by=62masnam$tUJ906NSdVQ3~pBABFV?9nla+hszvKpm};^m37T0q zgvC)sckNzbZ+INl=S-CFYE{g9NGMcG60sUh98@ANno=+tSIZA0FVHWvlZuiVeUOT# zo?WE!uS9MuQ;}$UtNqiauvxyOU^0T>#PSN%q^GHzR5Pq*RDpwR8LDVhh8?bzB4Sh~ zVVKghU0klTw-~ihkS6E0d5JE_0rK=YxB;~%@4MWAVgH4Ekw5S4V~-+8fkfN>JHX$! z$=|!-;fx`?`?8H5YX#+tK8qV$sHvin`7+Y^k{o_{-E(g zw+%$Lgb(=P*L@6PW^7`s%zRCkr9B#6Z2wemXIpe6Wlk|_F!#ZXRD2=tXULmGNhKv8 zAFj$WQz35&_0GcV!cT|GaTM%w1Bt73myH5{RXcjsYv37}qF!e0T$vfcF(A3j^fLg5=3Mv3Irlq9n3mon(yWnCBj_ z15gver9ZxqVUDRbzFAwcSK$a*O2wB{PP|8es(nK>`vLYf8ZmxPHpxJdSrZxx$*NNx zVpyO80o6n!iMLDK`_ip?C@ysbgvUp!8dN$0{aKEr*MTF!p{H)i$P`77vR6`09+kZ! zb&WFpTQ=3s$uTP%my(VHsDyPVj1=B*L}%ILLWsz(bSi(utWlApvudc|BULnEQ(HvfIDIvmiDA;dR5Q ze-&#(Ml;;O^&F{Xx5W)h#pwC1`eEuX?rlRoKKcWPg}j6CSK$?(6xF{jH4j*9l(mYp zvElZz8H!oNi%lP`*zNjfdh0F0Hl*=6dvPlu0JY$ue0HtrV|dL#SX&b!%R> zN0Y45MT{x&^ep=Kq#8yZ8Xa2mwf@lmXLW#Ruq|Blqh=rw=r0AQ2YL#! zv*RjaCxY>GbX@5u(;C6<30Nm^pg4fC+Lp=fRMTdS$8|K(JADoJ?IM#_&;H|SqGExr zI5_1mTdQW8CvLu1D|y1yvS3go*KOSm(UB!q)t36pFD9|xY3cD6O}1)sJe5aiMJP{W zeO4Z~$A*IH&3|k=-U#G>vi96Xgt0%o8s4${N-C)H^)Z3rOIz> zbgqXG$y2&(A4{t zr6AQoNY=Ek#k7~CXh?kIe7nt@q_fbUrLw7rc3*CY;d5`2+Ih@d-97MW_X`zA5pBhs zsO5~}4X*PHly`gJH7caE3ChDlSv$>;3$7fJ5fkFXX73C{qLZ%%HauJYbSPH}1f1Ev z0(Wm`u~ropG%ZLjw_Tx_sh59s;vk!TeQcGD|9r8rIp*J<5xm(>D+@(5_8o)EdWCB4 zdVGC-t|XOhKugW7Le(*^y7X%ou!Tb?0Ds&o=_$;IFz_+@Lu*z~037akWG>5U<^avM z&G~H9OU@fO^sx@y^Dvm?OJv0}j;A-firdQRr{W#x=Wm!bMPJZkG%E^jM`OE~Q)-Gr zM?~|hDiiMzRbbyl8`$GO^9&O-H~CRdaJzA9iavg`qhIvayad5=i8foqLWCotVGksG zg+6g{zkes)&y zJIm<9O=J%(Zc@f-mAqXBvoWM?!pXuM^I2G_fZofm$KGFMv-Vd_F}e%vxRPT!RA*Nf!v8LHL4i)3hN_RIZ*WARSDIGh zlLU>#ro1C{zv8JZ-Dd`-&kU8NOl&nwte>qN;34-6Ok0w}BSS+p@O^{QRnTY1$YGz5 zXv2o2!w18MYFBgWvfjq%&*Iv>_?`aoClBKJ(`KDBn?L|$GV2Uk%6|Eq-T%W0-v9C2 z=0CGEp1*HCPxbyb&6A>&H2nJYt8Og|$Z(2wnR=z3h|{e~!|v354U02;F&XM0gu2`b zcS_N*dHHLPyh-X!_qN4tXR}DCN)x|(>v~pp@PQHC6>uSESmKPnwmpITJs3KGpELI@ z$wk@P6gT}tk(5j*pp3n_))VXFi1R?YxHu8qLX1{@@9ZMcmCm&*z3!y%DO4Q+z3c~n z78&D*(>W}i&LG6T53Hd?4KlP~QSRO;S?C`kbAEKEmgy>UZs*Z2 z`?;F8Ogz7FjKhYuRR3fQ^%s;#_F;H64EZ) zE|mK4SQhZ3HaM6^1CXJx166yJ3Mw@@c#1{o%n`IbO;4^1==`-%-U3Z(UO{JS?j3m4 zPN`D12Yl$C#wC@$t@KFNym3RRznxlCO<`~hzlD*}U4-sq%S+#ZZrrB;AzQj*UT^T3iVYt@sg8xpO z_y|j}$giGeNWrZdq<`0A$1gNFOrhXa znNf~(B2tF>H8owZ>YT>+_3IKdKy1XDl)4+&`$vkHa!mA3kaa z&^g*XE|6>NV-WZ}_Lc37^NJZ_H!##p21MS8NntHfbUhperd34a!OWhHtJ7P08@~#i zklD5Eb;0oC_vB#2`gpRyO{5Y3DYB7o#&Q3R^<%c?Sg<@6%MurjNZ4Z$d(GD?Sw$%3 z&jyp5PjV<|?jyg7UuYgrZ{9X_{vc86ufXhnS|G z8Q+M5@_Xx&GXx_#atXAT(`RsQeB z#s5=+*guEG|B@v3k704v(Ud!4(fdvMHqdcPG!&Lrqw%}mUYHlP?A6j!tko*z&W>7_ z9nal&Fv~=i^tr-{akmlwCY&Sf(jm!R^?^uY2ohUC{pxUBcK3qdD{`c5LYr{|)Uvq(0{oumW zpV1@>6zK&;MZB*Wx4Pp7h^tQ@C4-7r_e+%)IXj_a zDISCJ@wHeq00q#~+%Db|!sIO!h4tzh_IMQd>|#t`%Mj0t zFgfdOER-t{JB}Xh^+2eMWFHpSd7js3x49t=XP}E z4RXBQZEfk=6?tE_7ojo8@|eNNxIf?Gx+08q@dHZ*DPrJ6o@e%w;Any`fX>iURPuc* z@W$5u48zg?oT2(|NH=*829Ga>c?K=X8LZh;Ld>;oCW~8Mo7!l~%yxCQv;`~;W;fVr zA?+rZ@^&a>mMP-#qPbR^dYg$P(Csuu7~3&>Lb(6l=c-d%KOmUZGIEojqiM+CD$WJl za@Lf$1-^S5u4n{mOcTZ%Rg$XdeQ9=GxoX-lxbg#ZJjRRf;j6aaFTO|E>g=&={lJwe zVRqcT%c2-w;JgVNCDJcT{l%Kf;JCS`;eg~>bxPJTJHdg4?_Z{nRHK?%pc$Rl@}k?3 zB}wgtAw7CRFl}Q9^!QF_3nz(`y6$?>V~PJoce8Hn zX1Huo55~p2)TQn4&P3WO`4hgn<92@{Dc%N{i~`xyRDpQm;Ur?W*&=aLI{(*%Y!!*T zO1Ud%VHdVpM!GZt5K zzYjq}zVu36z>YoT5G~{pOt076`A32jZez9EW{0I_+sCE?>E=%1k~pEA+*)1ASp>F- zs+NMnq>3i30|ZHD&YQA%(x|_>2@4~;YnWcevFOFnHsbG@@Pu+(dCwxw#+6W(4@kln zBAoOXh_2&o*-f6$j16795udss>ozHGVirRoaq!t<=O@0f^9hBkUK6;TFd6%5zK$Uj zafDWG*={1b-;i&s-yCfQlGW!E9&Xs#b#=p$!WYm%(e+F*tg?@q^N}+*BswX=Alp2? ztKs8;1-dcJ!!tDME2m6o@u6DJsh6jpQk5@i$Fn}{?!U_9Ya1}xa;Ry$!gp1F)75e< zmFu;OQiLbSP*zyMw(1HFUr*~+GV$i)AJ!W8>6_cfNBqdXvEDf~qexDq^617w3Q*PX z0Q2SwWb+v5Gm2orTk}b?@vgnfxRPPaU9-ibyiGfM=d3x`Uz90a*nEW-$M9!L_y>-D z#{3$F7lCq+Y3%dy_o46qC(!SoL*IV^{{E4-OWuDUm1sj8QCdIwuzxt8^ePqu!$d*& z{$T()q#niEBDz1tEEL&KTIA9Ex?N2?+n&AeOXQ^TOwoskc}ktr;f(IWbxPaZ=;yKh zkAud~pyt2Pe=D@$HK{}&%(*Z zNUO`iY(UFmpvyq3qp!z8t8d7lqo>2hqOZ@c1IZv}qGe*DW%>6_fshnp7FKq~e{2f; zXEXg5i~jKs;cQhcbe`_jv6rAm-yY^iM5qodSO=9nl{S0TiQ*K{W@#NUiKCz7r)3YRr34CJoZ?97zkdpO`z-h0icJn=9^ISryI)$Ub#xz@K@e*h!xD*W}W|%p+g6OEut| zQI4Hp)z;bld}Iib=qU6U_B7Txo_vyU9`N^Wuh%%)X`N+~&KuY%3g(6XTjKkF%{Yg2 z;D2AT{j*D+v(5iMWt_+P7$bS30sx{Vf6F*$r)M#`8$0mOH?xz(}qH<+q zm83QKig_6>nlc)%sk(}K4N2diz_b9v#{$cOX;%*3qN!;$)4ZxU$1!Psl+fw=11yIE zrH5bT${JP0A%XHo`?~5OQ}gat=uEG0W|WHrL60ah$Bn=fFa$$)+FoMp8-O38$ZjW2 zCUOj**M1rriB}0Q7@JtraY4hs1?`gLHd#D9faX5V8B_$0XjPft9_AYabF&vEHth`` zR$(dA;x#v#&gHK<-K45Shs}#N#rvfp)o56cOprdceC>dNU0?zZoSOCAz%G`X50tOg z$yywk?<3wdMV;P7ou0v3&9USd&sr}@1B+5PTxUPur}FGsH!R=|((#q~Od*FJbT(rRgm%xkHXGw#NZZ%PPVrX|4Ev-88eCr?8?3r3y}Z6zn3 zx_2knwNH&^$K0IXRNg8Fhy}9qpLX$w{UD4BF!1kV6TOctl$d^OWCU63Ed_n;;Hx6- zxeT{6pT&H-P6x7wg@~ytyX{n51j1qz-)^Ax-=EXa2j5suJkl4_5X6|25b@W}K%ZHX z4Qp22WVZ214>YshRHs>aTrCn6)ivznOk7Z_- zhVN=bthqAXYy;kECm5k!VvTXl01VuV&K7aX<7M`IiozP%ol7K8X5PTdb>CiH&8CH= zCt8`QqD#i}*v{o|F>oF$#+$gf4MKU%FrZQa6FAOVtcrktx;ZaTuh4Mah17>Y^_l@1&OpAx-S>23>-oiYY z6I5Hw)P@!TAyT9ClJ%5qGgroGo7A}lvJ9TmNx63%)vFot?=K#QH}mIMU+~|%QIEm5$!`R zmR&A=WzqLZ-NCFnuejb)&Ar-H&0RAloppkvsEIGh!mOZ3raCJhuRtN37Mfc=DP&T{ z{elFA9$NPcuVB zPT=FlaBfA|l@;*Q{f-8{isuQI*x=oH0{?RQ9R>K}QOc_J;JVgzagmho?ILIqWy5#cb(HCFK)9@1YQ38C&m#jH8P@ThF5<--49DT9PrQjGn9k3r|>Y=h_Qw07gIm-h=RZekuX zI#IE;*bqHYdd=~9ch`aOYySk1qP59gUPoJ#EXwZ&b7Q zIocbly{*}Xm1E=lHOovmHl8{yBDFp(DB>dJG(+Jy%(PvV(ev2A9anjoT=8C^*5S*s zziO?4^9rKPea?5And(8GZ(X!0MZDa6Cu)l~ZXXS8xU?5;Xo?T;uKyQtUjZCfl58v6 zl4Y?iW{sGc*|M0K*kS1THs z-TO++nYy!)o1$5NN?SKN9O+kprMlIxUG6 zuo9hPnNWIue8f!J?J1MSMO7kNXd^v^FFB_kH=<@3wqwsC6E&v@Q5#@?D>EC*a_XkQ zo^Eg!*mSXa@QeL%JfiEtPeGY=sYYA2Rz*W5$|ZwXU!f+ir0NW$pd$6<*UBYHrXz0r zN_nPU*YEeR6aQ1cjF&DQM}eDxfH$zaa!un=SsalyNeHf#23t$@%E`)+`^(uThLNG` zM$Omh$O~IN3fm#&{-Ed3kZr|$zWMU2x5N}W_RJ7qWA9ov56?8lywqf6#5GKTF>jbT zb-EhsAh{(Q+p~_$e5;YWk{!m!4IMkEg(MX4GRP)7Hcc;mpgyR9`?FHZ$e#|W44 zl$YnCk}#omX2z%G2lTRZGG$29CrM7WYKpGsPg@#!O4X92WtX7hMHv(5|jKT;8`GAi=jFL=5l)p5?_skC_a;>zJm7cSIOw1*DGoAvz~ zP(>R!r0zs$mJj;A)?}bD0OQk;6tlz?4tTNm#BD~p)x+}I7^Rw@Uu;H7DMJg^Y-lSh zidsB6N1h=g-N~icoy}t6qXxsH)dvLz zeN;rllc6%SoM*NE(){3k1dNge%DH?0dbf6p>4(ewc8r`ONH_cBHMV2X@216oKaNv{X#!L z=TLJouG2Fw7FSC5@3%kU;XVo2*#W{^Fvb~YXQfo8*4C(1S zcok+J|H149Te6@=KmNrZ`^QvOtIE&)t6&4w65?0$E*gUg8Ia8arqf7gR|I*93fTRX z`=3)6=SIlix&hlTL502jdu#9Y)v|0{u*ykV%C1$D15mf#jVmY5$(c@`|E<3*YC5AL zNU;8t)ZeT6A6jL=F7#h_U|Vr(|K>0Gd-+m7)!&rpI^VN+@X%vT`r9k__rp%f1WO^ z(P5*jdA37>>9B4UnEXQ(=+5)qd&obALgy2(l=d&xf5}`>F$TH*rQ}zp?WCy?0_+s? z;J;orsLNbItAMgg#La&U5GeOYY9(}nfCO{4I9L1x z)>`uSNde}Ky(#~T_vRe-x4B~#pLPBBNwsx$fP3=C-T&%$Tjn3V{81aKPB-`Gh-^Uo zMfg7&fjMgmgQINsR8-gt7Z}J6?j)W|xG6ie?B9&Bqr}C|nlc=71o#ajh5|pwbu_RRa)!??@tKo z7sgiGdNTAcF}7de!+!_|{5xXZ-}htx3#s^DFt*^t5$!)Pw&=uo4qE)r{SfJJ^-@T- zMwZYr9t}swD3M(yG=<{y*4dvCwL(HQ2Y}f`?Y@( zoN~|C1)9{x|Ga+!Qt&_2?|*|k{!77M`zQYb^7sQC`!p^37{i!-G!*`H9OZRgcCb-P zbFwV&cih0ZPi-q5aab7<>d1j--NvAh(7dN_ErTF@i&wtJgb$>>=^FEJufC~)!3mo* zi|ZmLb&ZM(ihkXZqoL8TojiYxhmY_4NXb*RJpl8~v`=a6l5o-e9;>>+oeBnh!2rkQ zJ<{r^wlfHwiil*yGk?ri+a?z-d)zH#H0oVa=3>sw2v$K7f^l-O((uH{YP(nz^xaD~ zBPmU$sgsCJh;Grgg@97_b7euWf9|q&p3nG*t}eq`^HG)N*g}uB`apu385VNd=V@dJ znpgjMYauU6Hhm<{dYD)1fHg^h@gffjDFb!->fEEXP4_g^TTE2H8H7NCkIINusrE%q z{>JfZ%SF>tFqL-wnC|lgwbnxl=D~AxB!>My3Uif@;Odi)V=sOL0_>1GGovK2X{xsE z*$N~&Cz*2zYyFt!Zo3qkz@pmR{;s?-Z+?~jk6v^P0I}wVH_bpz&pv+miy((ukVp2jkRs09@rR zeJk8Q5EW2B6N&=ErPcsM_x@LSE8}-|CVggB18PQgJrJU*ZvetunHWJIx-4`CObmLu z47yCeM{NA%g#DM0jdYBlu#K$$4R198kre(1qJlGz)x|v2q4Z=bkYlm=;*}$9SoNwM z#u7YLAR&F+g?>AraxVa(K^KN69=)y8J%^nTe^;oCb>gIX-QK`jma@Y;FwIS^I5?ML zOuc4qP&jRoFbj6%;|Ru0yrn%2-aKQ^B2<^d=MjD;b2sS|78^R9agj)hrt9=_G9L2& zXh@0QacqC0r@JeGmUa7FF8|Xm2Gy}Rfpjc4QMFr388lc zXD4>nzG%~1y0Wr8ioJYS_Hf1RU3Ueb(ZmAbF!esJ*kt3TMX1e$`t@v{LTv1!h#f_Y zx^d!eGmJ4a%A&z6uo|Mj@p&zQWice$1=rC``8yJC<b)0L?eDo=aMEE?B0G`9tZ*gX#Y#lv9N z@AD&>ORZ%+X}7Q}&06!sn`uh6jytoRHw$6v{-@SyysB_n&j|@;bkC9&pzXhDO7y1+ zOK-S9EZ&Q6A!)a5I~#q;CmB1)@Hx_4%@)$(t|TgEXl0Ngbon9k9i+-nL=^tesaW1Wxu>dKO65MCmHQ zG>%OzJfG9ldQ~IStWyxo?7&4eZOWEUBIFi^_pPNdi>eM6SHu)j3n6IWWPm}^o;~j} zJLwvl(h~)SHkpn7VcP%NW{8K?_gDl`-t+4IDf9pb`ez`|&NC0R30a9+1u_NC9Sgg__v0|Zuy3>+olAf(=v-`HM}rp5URjDXks?TltT z_4+%pk|(4zCMsE`OVu5^vzfQuHihlCZM!)%(?1w{Gp#QGQE%U8y1z@HW9=RhOH0G@ zIG-i4;A)54sWw$mJnwyw=5?Iod#@>XTP!J_M5Wro=ymO(t!3LA`-JI8^K#=M?&f6K zLc6(Ir8gXl!-*xwDc?OYc&yBVvO58n6u`UYen`y>3H3C3Z0vc$+RkP(7E{moiqktjuSO0 z@xEJ|tL3pdEKD>C3eCTBxynV7Uhk{}WAip{RoVpDNBY)4;cRs3Y_*+kby;=ZM= z^{^0w64ch|2E0B1%pTcIwr{mvWNdm`uAblJqnQI=fk%LYx2@-eg+$=d4FsUmc4E<2 zr{Aeyo$;PdqXA0nVamM-yIx;bZ>e&_7}^C?HUGMz~Ma|8u9SdUpE=&9Hu{CwgH4bYF&5V zTO=!WjH%{}IN(IEK=560I&a~o-|O_cIn5MJ2x_Z)JdZj+ z&2$y8-XWayq^I9I$K$=ym+Z$^w#Rll+7RBM%4$Ci!wJEEYXtz;g%w}vtXcuAyl*$x z2okUj&$}a05wOK@X}wu5#_J#AC=r?g-DJI2cv-d{lGRUrRpjp5qi;8Ue74G0B0kPEIZ%lwVSl# zbwyd5g_~nlhuPWjqr1cdd-YM@hK0n2yIFOL`GEOaIl>P;^bwjz6$c;YNk3-JZNy34 z>$}!QG}}A1R8!B^R#h8$7PMD8d8(Untf!i&N2^)~uOsS`7N)7MNSj%5m+#+rz$0pI z%r;o-Hdt&wCk|vnls6AFXh_@XIkC(xB%fKFPuNddjT%Z{Sel=Uu@>a|!?aRp#42ME zWF$+S`QfF>JE`Bm6NOfWoG@+EqEUA|%uHv61d93r%`K~{o12+cr9Mu=A5>zV#juHF z-d+-FJ9n_$|5Oe6UgI)ZIb&4exwvaY96Ee})^XGPyb;06Ln$l4qFqe`2ufS=bX3Yo zHm_J&bpw9A3XMX&H+emagkyG+`h?i857_j4%AuU3jl`uUN$UjOrg?STz64C?_6yjJ zSIFP)q@4mAn_Qcx z*b=AQnmWySsv3o5h7!ewqkVDHd9s|1bo0Z{8#$ZN@XPCI^TEKR2;?k|hB@8)vCEkz zbj`q|;|X?KXSz;Tx+JJHU+ak}QQnY06Fr@P|Qw4==JN&IUE3H`6rjzzz*%FDIAx z%yzylqF3R8fvw8rk#3L<^_qSklrORjR(x^>cXN@>F*_U`XjD0~FY_f9B&a;0giA>V z`Q2=3@fpP`XRftTNY?v{qV8>{XGZb6wOt)GwVRd$`>?AXE$qwj27B3byAg_LySileO;x zvn$LiXj#)oKNcUW)*7|nBnBFzF&QX5^{xC2Q$C7AS;#AqU!Dsr30#SUE;S4&QJ}kz zeiY2Qsj5nbrptapgV=aWKl{WRk+I4NS4*M=M8Rg#MY7I`Rc$_&s*$e8+xCC zypV4x5#YzydssRuUv_CgA#piSGIvl?WHgvv{yk`cGWAV2Jdvw){c5>A9r~gi8neSq z0C)p$%}AEM11X0W^5{sUyrR#ZdiqW+HY6*26WDsvL(JFP%e>#FcR?2rAElU|Sr*FS zIF|u1W#y8b#?WjgVRe5X5F~xZOjo@AUYUl}J~}xUkW~tg*#Pq9SQtx1(muV0YqgHy zlv^&U&uz1#Ov;fjlSz}XIZ6NdvgWMeY&d0o5_9#WIwYg^9vbSEq`sKK)FZ>4`8h9#_Mm5IAM{$h10SuwW>wUR&U6AG9yTI1#F*u0fC`oOAzMc`k zbCJ3t>xoL1wDX{67I%dgx!Wld-0865d4&Yn|oBnyQ^XzAz zhrT_U!ie?iG|v8Ej4k;DT?0$Xb~9V=1vJJ2POWyJGnV2=)p>H{HKAm)!Vp~nsCK|f z2&18maH6U(UH9!pgM2cRZSrTWY%}s9hI-nVom={0V*po_RAsVJ7Qn^cRTzk0su2nZ zFPA)6Mm=n9 zFz#Va=!Sa*EvJIlK+r;DgW{66@Jpx-81H1Nh zv&X=b$>3HE){C!K&`*cGh*fQ?+*I3lnqmmzLo)+cih`79;bOAKxmZ6#AN6};$|aiz zMVhdgQwTmRoLY-YTg{V z4E+$5V&g)Nj>Wji%SED}ZN$*!$@*F?3mvhqu6b>3?BGOV^JIpJWk&WEQ^bFgth&bl zEE1l>$4$Zu?GU^*suNz%rIB@Oxdw+R9nwwRGA`QzN5h^0S4WLEY`ScLZY;chlz+Xs zhlRb_Z+SYYe7bbtTfZrV1oSp{7bt#^?JRJ%*OYAQ0e7&%f|{OXrK_Uom=s)xZVdd% zk~Kar>3N~8Ipy*?Q`^cjg*}JEO4pQA%*j!k&|HvURs$^P`LUDS;?;CgUf5r8etN1g z*H8pFJ?()}ws)M+p2tvEa6XIRqzV>qo(SW3a-%gl`HOEW*z4K^?gfZEv-SJ=48H?& z(R*_lHEP8DFKITcpQCg@ zeK}LN8n=;x3Mw65RdB%CGO2%w_7ACRd%tq*AM*eEWm{%C_Xg6hcHXRoVFPaOyRx!2 zr-u>2!@ok{z^e-x>b{#AIe~Wt!?LOU!qdy0XY>BSaodEgJMLvIibxQ$W#ZLTLQ4v*BBVtmLspa_M$o9T z*2G8bcc>UaHz=YYGn5FxFQC_JByi5`vVY6V!Dy@@C$-a`hm5 zKZG!H#Zc=gutkyn3WlZQp+y8SA_P7&?urgs=!VVY4ngr@;u`vk;NexaO6=5y`nnZ? z?}Ps&f4EhZz*LEiwJiVCgKFv(!v5E%Wkr6scm_(9%HCaA5vjV$lDXa?Heee+JH6Ip z=e9Kse=ax$&VJcouTrSg{0of$$)jp_OM0eYn_QdDM||^_uXIV9_hdR6=uih&_`-v| z2ZMqbfn%{?p1~UYKKu-ZO7t7vevTiLWh54`KKbl%iGrotWIvh0@pDp)W~}w#c=urB z3*)$`sczeUiRvtqDHC(Vv90pg?`~4r^TD>pRu4HRSeL-E6{Z=oOq8D5y{U}GI3OfO z@7qBd!%^}r(4g;ayJd(XZJ6O>#3kvPh8a$HbjKYynlIOGz&#^KeYYAm?c0!y?YWv_ zzq`!N%gP5u#n~(-E+)1XQgQ}7eh$QR=;#QXahm&y zrPgqTgR9%=W*l>XguQo|w)HROhNJPg3E7#p64&54J?1Kf}Vcfy~!b>k&Sjs*`*UQX`nY@`c^`G?yG65&HKlr@d{#+{*f zllj=@QEcqZt{?zTlOeZZ_GDvHTDhPdU_^nVL+sI z?fF&MYi{VHC;k4tS!Mp4oegucb-ZeR%r_csx0C4SrUoG`jav^vHy7rI{rAE&U#>+@ z{0y#_HHY%F%sbld2Y>9_b~_U6*zZQ*?TI2TOrgzgSXp?z4;NjkkNE`bZCE_sKb*K7 zS)Jqb;&wz|zvtzsPWuFWd_LRw1dQ$oMbOr}E(r1|&RX9P_jiA2tgI;2=hteG*08;0 z9J|)K(&GGb&40LFQGG4mTz_@2{}_ulx7F2Zkmmmxj;TldkQF;59>K(17&cBjYNcpr zeVz_xW3>@r9{Ufg{jo)hQ8Ox8u z#`$P7dv#ceNXoQ^CaUJTWflq6_bfB!Ov{Go)Ql4BFH)b-+Vhzv#a3<6{fH+GSX4{~ zjIGtEe{PMjFeufInmDS#4d`+rPNm*SnyhH&E)(^=h0q60Ql|pqdbVhFgSVjd{HBfZ z4G`05!(9+|{IMO(G=!Ah`j(gdMN*S!c$^<6%{r1LgY($036J|j56o(zVweU!Z}(8o z;uC+hK~Y{G^>_}4?Dssd;xJW=*!KP`3Mln*vE2X0!$B5%AdO(rx^~TlhJy@Md)&u_ zJ^b?dn9h1&3{SM^XCh2wPeLjAH=;3U?@nKWlo?P&atO9+sdp#7*EGcD@FS$XTO52` z6~2(Z#~s!pFgwJ|Ye))KGr=1OOjQz}-3WC8esdK5*7+MQ7RfRd7MJv^t$o5sNOWx2%bnh3}Xb z*vwa12KcT!f#yh1EO~^pDN#n^O`A9x8+;u5sS>ddE1#-+yEeX-LN#;}35@CdWaC*H zie0fjFCbzzeISAR%R!NJ8(Yg}Su&1n|e?qcMoI$K6 zmuHB;n%do7a%ON`^VWhR?1TIgKZGxL+&q&L%B6zi*T9d={sR>9-k$xvG=C7ZAjfFJrWmH1v}ByTUTjxSXmeCmvaHp5g|xbqt;? zH|OhP)$^2C(AoejSiLSqUQs2IF8S0Vd7i(15<^l9BsW5IsYmpS!)Io(v+U<#c&9Gp#K=e1Bs6@j$chXhuImu2(6BPmM!dkMho+ggr*X1%;@cO*Rd0{nZEithZb=@FXY}M_0f$ zT6<@iyC?v1`t&Y)M=K#d5R#mmthsxc98>j7uAfMhj2nYm^a>|)5KGh=bxK=q*JTgA zPncHt61ra4r|gYX+*X_-weAI}iMTn$S*#zn2AS0a*N*&LN>JX}lRzsJfoTU>to6&_ z4YFcjnPav$vgc>dPCLYUl-ySmj8q?7R+IM>ooP0~5DE}AOCcq3pTE?UW$Lr}U`Ubl zHhsnBsjbnnQ9x=9$-rra>uryg$9oldSkCFklELR}& zrP{HWG=^vlsf5@CS0{6lJ+p;PHPdsqIILEDJc39!6C`&nf0T2BL^qey=cw32{1T2x zf;6?rF~<(kov2T#Y=rptdP92hGX0gjNLjFY{&SFXe%KWW6hb_X72`YbpA_Ro$9mBL z6Uc0EKhu!*=$emZ(<2YSTjzd=B4$Q{x}S}3aXh-TpCct;o+Om_u~iTmi+qEH9|oWy|1x(6-q2m|s7C@o)OS;$Hl9-Aw zK0&H3VHCb=N;D5JahApr+Gp8_GzGk|%!%YLSQTaV?)eX13D)Ys@35~;dk#VkWl?;y zZzD?Un!N(M%-sJ?=TDN!uaQ*bvSr8v89;si!T|aYNhbf!0Q&nt{x1fQITFYKVu`f* zh8P>|#HJ%;9h@+=l&;6gTjb+woC+o@E?ye^5qs51%F^yshf_vG^{B55d?c!x>gxhj zpYdmjupC(WczPX0EWQzn0rcMnI6TSoryDNy{7`dh?$1wKlOAFN@Vx91mE2!9?gtM6 z9v2(yPo8$F`BpS!qwk-M9(C#10F(L7o-%EPyEp>^&OL(g@*93)?AGpI;G7_=Vzq%$R zCD>d)3E6)47<=p4))W$gg21UyUmS%HEDYpBBS3gju(K?5D*k~?n@o?`U`dwDx#fjO z#}f_?c#O&2wE^trh7%f!g7FGfRCm#1)RwAf4|TKgn^Tb-!t{T=&LjfsZJGACg&$?} zt(WrLhUeRkn8}Ugf?Hr;@7tY34nx^nAb=E?KrKt-#~pe%Xv1fZc4V|1&1uVb=MDsK zpZHooMwdm_O_ms~GhKwpgcgE$?0|8c%d$|K3ftP`c9+$Wa$&{JjC~ zrux(BM7&o!%Y_3Ycoi6p-;*Pkx!4RJF^3{QGP6S*NeySSrlO)^rv3aKqJyZqistuE z@`2}7+2yRTB$MfN#tgQ6e0D!z0=;IPmAMgxG^hs7F3%9@YR^VmzxTBwo4irnqic56U})d9HFeHnyFwlpVbGzJoq2u^hAS)WE(LrIy9r(E z=hB<}{HH3`c(P9G)ejE$e%7?;PIi-t_V`WEmNH$^ueP*CdFcb@2zCnUDUV-Om63!o zT41RyG?c~D@Ea-D?0+=wD_{*KBse$~^KBfUjT*mWgFx+m9f)*&r_6|c-!JZFl+60F z=K=B4tm#RFP@wJOrjG|J{2R9@%O0GOBFeh1rivwv`ZK9|@wjcbl9FMwjAg#X;Rp;> z7^*6?7&T7*LiN@jL_XsohJ++$It1+ncc(mlT2xwj$;=#NEX#Xqd|@YAt>6^!!`?{h zG=%=)0S%*)5?fc)B#g-wV>P%2PnDN%v7Zb5V77qkPM%b}cPC`iGi#Y`f-znBqgo~s zi3Pxy*Nnsc53KuR>khX|?FY+W`ChoB2m?+ZF7FZMYw+P3o>b?r`_-NZi76-cfM`afqs^`In} z5A_N==9;i-F}5Zu;1Pc7= zov>9ANYdoFd**=&tat2#=n~e>Xo!lr2L3!g)E+ssd$ICC>v+%<&WCPGZ9d2s;Spo| z;A!_Z_XRk*{ZYrR^O~ySj!`ZFV`ZsEQwlNxLxBiv03F9r_|@8#z?wrdeUd?mR4;GY z_gikl6*6W<-MG0u8nQDN>Iy#^JB;v=P#4MGtl_G}3d5`h+fRRM`ebfF7PRq}a4G4S zd1LAZm+*cTNGU)vvA8RN=#|~4;kooYiR&&&vt@Gar7PF z%#z3~svJpoz>Lb}Z&E|D%4Ix?fRReK14IIo^QZo^2e`HT=3kZ&=l#t*>3Xs_$Pn^4 zsoBKs^mL)Yk3A69c2#-^z=#U<8+%!gD0#8@W&OB$Y1Gtc1V@JE0UxHRl@Z#zg@pxs z$~d?`2w(I1yTY;w!w4f%98D1q<|LHPLdDQgg1rhMbBKMwLP6f@w}5V;npSnSqbUfaDQnDp);ATp;lmg3mw;%U|+lO+J6|lI^!@b$N z55|<@rn?Vmv2#TMX>=uGUD@BAI3Ct6kR}F&T&1pISN-d9Gt?Ycb`azoI6U=!lW5W$ zks2)E8fNkoNea@N9a_5LZ($CPMirDH2m&uY8IjUB?LXG9!^s)K$B}V-uh8w^CgzUZ zCil_p%Hq?tnBQy+8pky!UXDnZ1qE3C*pvaR1k(>VR{Gw8g&$bbmIiz%`FZFivbs}C z2994bP#|oD_Jfo-_E-(Uir`d1$LHnoqtThmi$D$281N9%y5jz({6zP3LT4Jkq6ds4 zZ!qMec7X2zqdS%T-S%eYqzEI<4=~C&FU_S;u<4 z@pjbb)b`9soz&!nhT-+_L)7e|RB%D&OBBkEst#?}r%P>bA~TBs3fF*}W{l^LG2^XO~?oxdJz z?S)d* z8E(*2G|&nNLkCg09odKeu~SRKko@I%Byunwr*AE}~uR@+7={)O#lZe9cN{O$eb)FN8(cjbJeJ^|F_TB?NY?J#J45`RQ@p6BnD_DB9*loOIT{jX#W9cs#feW$((r%%e z$D%|~fId^vOgxOwlNgzW?23s-Q_iXKXL5FgqRFI6;@78rTfnainFx zmVY&Dog#t~2A52ai9SQWzF|_h!lEfS(y=AF#lQV6$}FEECMv_K_Tj@f6az#6QaDft zGyNTG5Xq44^?WWHP7ujb{zu8!OZ#8X6$fq8;z|pfamc}NF`$r13(BJmjYqoj+yr$2 zwGq!z16vD%T)*BD%Zm909*av04Hma(E2GO1n4<5m9n zTz?E$Ectr|yi&2gxb%ZYFkG_f%h05MZaqa?FX+s&p+LPdIQ^Vf41htZq+!@_ddhCe zVVeZWZcS=SSY?fV*$c z1!%!pN5XF~f3BB*vCg?IRMS&I(OggdIhyN#PRjmsz5Lf`u74+;|Gr-SH-z)E6tBNU zbA1b@C(n!CgU6D<(%6B(Y;}4~VB>)y!%#YMwMq2hMdg6tpI%Z z3W|kj*)ou%;wuT3=v^Y&oj=&(4o*P;b!Z6LHGLMv0Bg+W;M8dQd7||^AX{4I7=Gr# z6X|_$&AK~9e6aqglub5mst$9y|D3**ImN3a_OOAi7z>ivbqIJCIlF~Cl>nu?1E5-D z2Y*>bjm{`GDnB6H;|C&%Y9UzS=PvrZg+l1hW3tnYBQa-2^2;(@u?F*x4+=`e(${^& z{mbZmFLCBv@uEU`T_rz6BKFHaRT{by(wn>47OP^sLT+uz7VGG7IeYs#<7D}eH99?X z;b{C=FQ=yaBI;xY3kU>9Rjui|J3$m_OhW$nk*NWJ2Pw1`iiOfa`!gDfKRA}?G9)fu zw38>(qGYoN)BfGenN}*avq_eDBaX`!C&COk!E;}3xj!y6NsoDL(K@pPk!dpx@6UsY z!!Zc_YGXF+1I!3NN=Zrp`>dF7r2U=*f20QPOa3Ns=Dqd1FPkJe$(Y{)V5jk2-!m4K zjA)B%MyRdlujON}OxayOXVmmW^ugnh*wC*F0o|)r)Q$n(y>~O97?CFFr<&#JXiT04hH#Y(6DkHqZ8am9SkT(g2)^z zZVx}iaV=n~KVq4er{#Fu%VfI!WNCx#AvK=1S!Lmq9JQMx8x_~0#XH5;FHjD}0(;=hfX8%( z0M9!b7pXfalZl5r!l~1dwa&|9PcRAS-uhVuRnOyP6&*Xs9b}voWPuuF!P8c|e9f?D zx{oNbSVhtD_TaaeG&9-$z>lNmbri;Ii17?|Mf9z@S ziEdS2D64Qy_fN&X=^EIW)~Uz0cg>ZlU2B~je`6i&?0ip9$UW}z~WoTQu?nLMraf}jwDQlqUO6zbp*tV*`23YV%xwoAw zTF`N@z0DPGoln?`f|r($2%fdHUf{X4YO4e~K41NmU)hT3#B<+1bBc0G6ej>Z!~5dJ z$m@D#6;9eKTYd2=x3Io@@pXh7Uxf2pHIeVU62!xaS@`U&yiJy~iN}2mR2Q017X5iC z-+nB(k}74_+i~iZ=i?Wol@8;>WR{0w-m=X$r5Pov5S@C(cdrJAz7>FrPB++WyQm7b zTsR(A=Sia=mf)4p-=%LH%YCA8XF>khZ%1WCP3b)j9^ex}n&eA!gygLv*{ovqj63(D zZy^~b8<_kF?IZmU1oEQ~pYv-N!Z$()ZUCJb`mm|FX&yh#jc*UABlct2IO-K$Dm_y> z+)f)iSBwQtQX6P!L5psx{w@yiPVd-`1mDlDufDH!#>6)ZrHP7+&=}O)-}=f7+w=K$ zJHf_9EY0RPrqQrY<_iJek%~QLXjmo{GgA9e^%!f$s4{#6q*cy$Y#Ns1(sSAVFKqpXV`N*QR14Buh|%o1rKt?GyLGpg7TT7E%L7aTo9DP*o!ya|MO zXJ!G!Uf_cH;_%saOU(w?w1PNXpnQo#c$or6=tq5~3asqACfD1T_c5-ewz2=05 zsvbl74W<s z^Hqxpbig4DHtc2gSxM~|Au~qbm;_b|6MB03R+3 z?*nz>l{=<*24G&)c%+x>U|iblW&D;k61-UZwJDbA9lw)+pgy;`T=t6iMr zAyg%x>8{U73PNFVN%2$_2$Ty5(BFm#3PJ^N54G#et|@P{>lhXHK%rc3`&mSkhROcI#rj6 z0b+`#&-Ftyyt9-LY>DC<+6@!ou4z!i)VWw+8#L66d9B+T>2qDu?_+RWy)o-KTJ)c- zgeufVlIkP@g>scNPH80`H(IWvB=^n|Z(I3}oCP*?-QM$|J4J%Ed!6KNQJ~d$M3A*! z;ORl2jeG_-*zQaNbSSRy3)SJh8<)+9H1I=NR47D_g%1nGh|um0;tA*=$X}sc+f|~X z9}W-`JWsRU{^7L8U!F!^%x7xDK(3z`|J~){KLk$y7TW%Qa5VaTne@xa=wF`p2xqUM z2y(edi7v=qO^jlgMQ&B$S0`uEz0S~4NkdhkaA*{?d66Pb5#@;0Um*&psh6AfD{HAK z`XQ{jT?M_q!VRgJ2*Hc0@%amQ0)m0=J3(-EBb3hqQ3|bAhS&5a1twuWqN|%1H%yEd z>rQWPF83Wz=V=lfWtN<+ByyPS^tN{;BBFUr$gXFDBmC@kh-Qlerh4^ox791DCwk1) zSl^i=X8Fb69;KB9eCK0i_J%|NGa{;0E~9a(UYlom zT!fY}7Bw^+2X?}8!$vKmXgXeo4@WX7TWz+-;3z43vMaLy9hzpg)lPwgyl_!a9aXux z9oghsn;ME~!A^?vP9$n2YHBU1C8lYZ9Ec8*!v_b6D*Ziy&dSO)rP;M5r4{sKii--` z^VlaPrD7}&EF{MqC#gvFaC&lKYrqKA#sNbqb;svet4!e34WyISwE$pR-1B8%|LuP7 zF6qGQzFD5O?IzSNl_tuH>}>|d$lUBmo55pYbG`X!#Le?%=x_mtm6GT19>Kk>aX&Zf zaUmt@bnH^j<^u9Jo2ex?mD)zoe|UEm?By@GNz-Mk9js<% z=7vYqw;`k)kSsK5Zp$MXv*K^gpi?(DX@FUH9`#-ei?7GmCp3>*&pr1n+76o^EXuRm z&&D{F-<*#ky(n6`lcy_fZ`7*N8?0YH z#A|=-OO4XJ>$Wx?&!2p?{Ygd>K{a0g)ujMpgcq`^p^es2ams>{XXbIMLsj%{4 z+mE=JA`4FV9EXE#RQt#eP?6g(FjBZ~J2vU%b`PZ-=vjR^#GAf$Hw3siOLtkki&}Hh z6hB(_Qn7w}PNR97VkRZtd|C29Fn2jjZ+&}l;3j7E^6Z^Chtp1bx~V|ReXBOUyW8|M z8?2etvK!o53u4uHoT;OfZaprLlm`@XaoCPDxpTysEmpjZ2X>>4ehU$Hi(+q+>6KFN zD6!~Q<>&PUAK==w(@PYJjG)5n{NRf;3*}Uyrs`nDPo;$Un8aJq!B{FY)%zqLqVJ_h zr4)Bh_eskdC?I-Tj548NdpJ`YShI8!n4nn;V?a+hoyB2Arm?XrL5ZxzD7zUcxWT?s zOF0tN`f69$VZTXeRYry|-tkbj*nV`tY6zcnB|fMCEDkceaJ_Qsd>9C0eC)*ocJI(? zExyd!WqGZ_Amm4VvS_nE&qskEq<$K2T)g3aktmpWz1Y0mL&F^s!tAt{$n)7m82CQ< z{2g!*R$1Zp5azUb({VRK+y3U2>W-RDrpA{!PVEXY zC-g7`qe>9G1GuCYom7u0cxyJJwXJGA8mOtjLNA`(Aq_QynB!-9y64e~`-0ls^@>8< zCGkhN-tIoWe6_XU1 z$vs>HB=kmmTBy26@vXIm#kQ9uknh##F<}R`)|J(u0~-8Wck80Lv)x^S7TXOOlIQbG zyZ>9M&Gx4CZvEu*X=uGw>qAs&6kPs7VHO$o+v?_X-(-cC_(8x17pS?{^9T204kto1G=jheC+nf(zr=||eCn7FZC$hUajLc zR;NwWb!}90$Qw7>K?r#8qXXtDJSUGqE0ZhMo^{_Jrs?%c9)N@|q}-1YCmSzU%}K(& z*%L=eROlQPVg#fIILg~Hm$XB?Pj=?;*;Bkc^0ronCVBcB6QPWS2zP}G_oSSJ^VvgV zg2X6xD~tKFAqt>anCq))nC5sAwd}mbdGXl`nOYle(<~6xJ`-W>utO*oSdw5 zam>bKT!o-5@woK89o{CMtuad3HZo|=<^k9X6*vpmaLzAoAmL#@ajhr0@o+CEp=+kO zC8{Qg+x%!}ou7@vRI!(Mjh%SC8Hy!zoyoY$yo-t5O)?aNghQmth~c7LW?jk*Mj$~O zhZQt5!mQs37N}ud;!KMTx%1g@mY&SP;Pc9eQFYr}s^`9XJ=d?g`#IF}Lm7NNqE5{8 z2Y1Y=C_D*Lw()Ws5a2kWslmg`V5h~e>T=ZPTaHT}ktkS67Q&-i)70h=OG|x7J&v6c z6T8;TVP~lMd=i^&{|x}`RAn@ho!pb-2&9#-2&Z$1^SBW-~~z zlAVfdnPa$i@tKP!a}nH)O{AvznGUX88a>~#*p?99a8x+))UZMMWIdlQAx|@6zZcU@ul@!X zB*^QgQuUZ{+QBL${4{;a?YN4S8#iZxS2onXa5{mFyH)GMgAvJt3%}5KzE1&)BdX2o zm^ZV=PN2TM*gGAS2!5h=4}dQC>b-gp1&+>frcbzr8tK_Wq_X_TP$30 z{d1X$^?P9MJ>=6t)51%W(y-MY4$Q!O`uS+7%b+i)gk`wW#cW?>yw_-{&9zaf;e0Sb z#IyY#uT#BU@4Ef%0o3GtsM2XQr_ImhL4#*alZFBcB-3C%zmL$`>*M-+{Q?M=cMY8# z4!ksQ&~VSdps=zh*Dm$aT{JD-F&54{KimR@D|Xij;tKcL~zn z-Q6WfcXxM(bV#Ql-JMd>Al==9ba%XiUa#K!-+J$V>-)AG_t|Im?6ueQteGQIY175X zB+pD*rBgpPb7p|HLx#2bVRbvQgU@FQ^3@rb>4NLYUGJXy@T^CFT#2JYH4o`HyJ29) zvVf<8_3d6ZWz)UTs&+nON#h%>nb`a1l|XgIF~s zV_BZ=&;TFMtIvv+xFp=kT$JaSG;5|7BR7n|Q zmg_rLs+ANXL0NZsAiL{nE*%QzSB>EEUUQ2MnJFrdvJPhtb#JIfNO zXsNNb@3nD>LBdYQNtC-}AQqe8_LUn*_^j%k+0)Kx;s6rJ>6ernHo4UJQAE;~GcYAD zlrMArY$uaSD8w0zyk)zzV3kHk0s9#ZyEEUbT#Ud3t_oJ_cAFetrPQ*NsgFL?s^qUR zeSJ9CaX#vwA}*S?kjyTu!rH7YxH`MglNn-dEh({Zxj(5z367s!w6F+ow!p|Ucs%S! zHXh;>uPrgz-E}qdzybn#UW?KNXj}%^k&rZn%<&8;MBk<2+a?0QDQ~~!eq9nENiyzg zz}^IGQfV4TzzzUOHcJ+O>s2AZ$@=@Ih~A@uQh~SKPX$yoHCncw%Gz|>M7jX}07gh3 zUdp#^17+x1!QU&)lrJ}irhWhR>EAa01=&ArEco`7w?N0!y^rWiNF4z?8T5W)_4G9? z?nC#t%bxyO*<@{=`EBz=8U3;rThd)=07(4Mx2AzWK;@C&Z)Y8KO!xGp-UViV1O}A2 z23y+NYTe5BJ4nC>*f$ivVu@W2TZ-lQv{}M!jpq2b3s-nI6dr&7ky}mHH#D$M7A_FI z1wt>0lCmWGpT34AGYffoV%omDF|^;W|Mi2W(g5IYw4YZ2GYc4P?fOy`NZtZIT8wB3 z_W%=)cGDgW@OGZY3Q$$$xBKi-pwT{Et3vS&xI%0az#INF-gQr(e)}LvY$?K?uB zd;yrB@)WWMmwcw*UiZ0+z|%F}1%1y?0D!%lPho$33vGOz%DKQF=@;H&0bf4f=z$s?0mkzXy5+R$T$QKZBS1hxV;Xx;`(;>Vix=FYYsa;Ye``Mo zcnBzbY-uQDMu4k5 zkgQg$skUcQnq+3OC)!Dl862dlknKvD>bYexmxO(NenG=ZGs_bteo2*IP#UZ>P05<6 zr&mZ)fxom6vu%1{YEn?NZ!GD^#5^@gtQ%g@9j`A?T$S^oAct%!dQWtMGO5n);7)dX z_z_LqisD5Hz!d)S_)ANGr&cLckqRqbfKm*B|I;tg(a|9bu9;PSgH?{Dl~?^9*>YaX2>lnAbJdjr*v zWVxLswvT$VnHw=1*O-Wvd&C*FxNiH`x3Im|bL=8`_QrN@(;JVk@(v_go$o^JEVbxq z+&e`w#OqK#c`{%(0GehW#WiNJ$MfKHQj}pgB0-*j=oV2C6Ems*Y?M3dlTAm-q zW1rou?Vmob#lcZM%<*vA4a_06p7*1*oHV0+9d4R3rqTL-+^YHyH5>9K9!@Lj+S2gvtrh&eW{$}?0$7T+%I%aBaAk%pk+QCIRQc+{``e+)Y6->~?p{HGj~E@6ng>QLf@3s3gPe zZsCiUo8f-^H0+Y+=X11$dzlohRwvtnMOK1zOS6NBoW>8q_G${%1 zb4%l?(oRk+!g?Aoqt1HO1Xt`8j}unn(B?}qn8^+Cz5?|x3~Nru(XeO4NDi^Kim!{z z4)0gFGDrB9`(RF^eZ@YiNA6i7tb1{p?a}tb&itBu>sjR8)Fa)qe?`vU{?*`ji1mLK+ zPft+b6g@7#OctecJ+4u-xZh;nJ{T_o3Kj=+pY=VS?-kJSTs!qNUl=>ynl!DUpodTf zr_vta4?T3AD47lCCpF(~455&OFLTy--sfsr;JUJLHR)U@@LbO}j+oI*X*65u)WL>P zW_(m@zh6IkF0F~p_^Y&9j}lHLgjqMUT)gJ4 zk1pPq9CD`BKeC`%42vXwbX526(JFB}-iib`bP^8d#*RY=;I2m2tM~!0J7-JyWvVRp zYWA&kQ?Y9wjc;ws^*cRf51WC@Q-oJ#IB2WI?ILv+7+1?fB{XOLa0lG%e=LV;t$kEff8%Z9_+qksW zCqr{Pnfu!9d3TVa>3p(t-tIYE)^I*ER&n^yGm!Cdf4zYV`%QmW`bS5RiXy@au?n7( zx=Zj|_eruUxH3wVx)Tu*aJTqJp0~#{{-}{Vi9;YlXc?{>e z31k{9YaVr{Mz=hUuC=sYT0}aeRbPL+?}@i{<8j%)dbUOaBEn^}RmbDf_l#r+itA)V z$_++K872B<=p1a2uNR-$b=mqjQ~>uQf7i=~n&^kmbBMpPW}XC0(>e512=qO-JP z=wdPY^>6)hGGShJy$}u=Pij$A_R4}Uw>8N(99BPT&+P1$YDngb+{WxpS6(>b&Qbkr zfwreOJpz^@|I*BG%j(t3j4#N-XA3?V8!9HN3Z=zS zYueM@l3H4Z)?XjG%v`}MKbU!z(agWw-o-3obFnCOwC4A%u~XQ}y$NcJ3C^4tbII)G zZYRRePnJ9f71_E|tTeOM%PO<_y0D@?JQz2Gr!|*=zAXiob1NJV+DvF{c~-0s-V{^N zG9gQjQCM26p>likEJ%qe>~oPtX<@2@+DWkpc@aespgO2$;aYiN1cv8G5tVubNU+zcYP<(iG~p98aq1~A<%llwQLEEvPbIng`H=U9 zJiNL>Oo837YdDe{s25$~Sv?%Pcu~;{?J&QLOWRnc2jb2>g7;br7dC|B71sc=k=q*!ozoR{ zHc!s(8T_udFLX~MR)HGk6x_G@CdfQZdN9VMoP%Z4WV*a1(F-x%OB zQ{=kafi9#uoB1MZxfHR)Bm(W-lp^EXAC#}}fWln&rsBEOkpQ6)Q(wY!GuL)984m7n!sVJ>PRIl);mJ=M6NqUt26p@XPcS4_=<1i=l*g9A!2mZYd<3n=JNRT~i-m&gEq z4h55@G~Jq0MsvlAUOlp`c3EKH&}S|Ku)@ic#7Sz;U?y?i71^#<7R(LE+!$uJS0pWl zF|s^Jv%HjwkZxI$GyA|0Hu`$U7onoAly&6@Tc^0&`c}vxx57(-nBLZDNiOlO*arbY6218`TuI!p&i*n-T(SS+Fn{bw!z541aiM=3okP`Rw;rj2@@gyvGO%y_1%-%= zN;R$Flm=@>#oZi+vH2%1?HyQp9psG0wHa1UwpPi)Zwq2S!%Rx6fSf3>n_a2R}5v;BnB&H*9wSNy?nyz3(=cE#t0pkTN)c zg)d}tt`9zFF?ZCn#D*%|q*>HPm(Df)Qp$NcF{JRiY+DuW#Uyqt&UT3ms9P<8Qmc*x zy+X!ve!0}35(}QQ{g6vqTG~KLyi=)YZN}96>}+KPgOSj~qkSpAnkCs}Lv9t1hpWPq z)$;pjvn;Ax_|264{Uq&Zq3t)@#_dy4+tb;ZoMLLHCljhE0A*8JJMjAW`P1ot94j7H zQVrh>`<46LpZ930Q`(gFPCFfdI?_~?I{*brG~u|W<9f;SKY-DKyiVYFMk+P^WR?Mu zb}a`0JM)~P8O@w;$Njvxjrup>fI_xhfI_x3KML7`JQcEC+`yRwT-$d0`2_GpY?@F1 z`Qs?ZIz_|;3b`u61d{Y~D*9$89^tAf9e{c-PK11$hHpP;vor)e&h{B9z}w)hk+f|8 z9gOz$3WxG<5D$AxkFqd;mJ&XaKm)x`Y;^}%JU>R8<=~4BpOceaK#}04XEaiOumZi| za9er4+u?xg4K~VJe*Elz{{7fmO#1Cr{NXG273XhdY@cTA_b*obOrPGs3e`5%FTePh z5D&P>+bJ?54B#E{)`(FCSo?n(?96+|HJrL#FXkxULQ@5kGF56i;t)wRt*Kz^Wm}F{gWY6?%*WBiE9C< z2r0qTt%jvKKA4$|#6k?tY$XOG!~#pn&cZ~S*mpq_GTozhek>H^WYu>@j7B^+_wH-0 zy_=h!fRhCI;@H?sVpvV5>YT!0Mf%0T)WwvFQy~WiPu9gb2x=hz;hom@UD+^X+EfGc z!{g6P!Y5nU@qO>`2;e3CRW;84U&iq7`-`3o;=c%#|ETY(f`HStoT=2n$Vb%bF1dv*|ToCnSxp{;U#mHV9b z!jNwC#SyE1VS_+1#=z@@PwSBc%cDN>EEb&ASVpsZS;Y$i5lQjH1IOJk)armjtf1Z8 z4>j8;jUSIE-lkNCWnji7^zfZ^1s9cN4(vPMhveT0QSH&J>`TJfq4Q4-=H)eg1=uCo zFd_7bN@Q!4VQF*@xrjA`M$zK>&qviZjBMn`)tmTlm+^0^#!KU_CN%bWLaMmv;aM*g zwPEPBHb;?T-TO6FoMO=J-05M75asPy&w#`ejXdv_`AVD83TrcPx63~~UwUfCSF*2t zupXKd$VSd^@PNI2#NWd+B;=&UX?hI)ttiKLSJP7g;h*#a1Ox$S!1!Awj_)11o+|#D z{!@ppCkXhN2+uK!WMf{%Ip-_BRx%r>mSQoXnbN`QENcJETyS;45z{6MI(oNFm<+xu1mCqw+89Ao){!V>RFz@0rB$5F2`S(C&bnn-a%}$mpmVkxy&%e^ zm$8z?{6+DIFH1Zm%n)%mDzqs)mVSzGp`U+qe&y5^%6yY1PieK#ZfrqJ7i%&3CdxJe z*X}qYIP18I&)y=r%-{rS!)b}Irr){!XT?JD80_i}8@X_>xABuBrXVN&RyzG-ip7kG zOm0~ZP0T#8%bf6hTUNDGoQD*5LBUpLl9~s28F*tAAvbSXrenoeLuu9rzMd5h!yUW{ z#n0fR#%-t|z8W?}l&39jRA!U*L&3qcfMPEpp4)(DtJL5N4z4S7Osy98{*va> zoaK(tKF{;?J7EY1ALd`B25KFuJWVq#3L!qA$JF?ipSJoJ-_n7G<|gCQhR z$~jQFlz}cvxJvkSe9aB|-b!~|K|AN6v=zxw3RYFBb{$FXhTFwH_l=Zw z{S^+(uW)o6oWFx< z12Dt?D=_GP21BiDWo2(?Z>#egV*l3=iT(lNQz>P86MGXYOS_-=KHcF@boF!k{`}Yk z{{-F8#N5C{|Nj*<{I8(x^o$KGbp9ifPrFTjeK!cdK(aTnFfcc0||LMqQuG{;C?~EU_T#ifXc!9_=2X-aSOt zO#&ixbUIOf6I?DAxtAx{pp&wQizvHD3>1iAONx8}?~S$%q2wi(ni|EzJ&3Dhh{G5P zjB^d4snovaAX2JjwVH2|X~?-*u+!&F4qV^eS%ZX?kl%O{Tk|Wqhu2=La%A8U$_R!E zCW&Cy^~um|FIN=p6CogmbzG9DN(bm}MmP2qDzqhP_ejbof3OpJ2b>)rgXRf5oqNir zj4pGEh1ui2#pZ1!dJlm-`CNB7e!1}d?oK5zL6O;;=}!8b!-~@4q%Fpn>L{pqqj?UO zaqdJ-{EscLzT{z{Zjf;&5;sFw&*oR2VH(m!@#-NC3MxvX=)`RJmZ?Ax1oHt!)6v94 zt@Pr*=o9MuU<8jmI0t!N1MgEv3kRty8?L-YzR_<(@{U;F_0on<`b`XcvZy9QJfG}( zjOyzwRx+ZQFBcV)nIwuM_JJ`A&bi}eAjWhxxGbD`4uihat0g6|0QJ$fy z9e#b>?_lP!;__;^J3iEUyxYvnc)VY)+t_UlQ^v!rgRUAsaF%WwcL29s-aT7P*KzmK zzU1H=eqLvEZMm(w$i1TG#9H3ygVg|k&UvI>)a*<(+dVozlx4wIx(Th?UF63Vol39g z3`UqfatI~cee5aPEs|EdzG%|ql$w6r-c!}LcW}JseA2bne&x()5jV_}u~fRZ-nTZo zRdO!CT-)=Z=@KhKhT4ZRcil=jC-3T>!S3Dp${>Uw5qWU$7gF%w_RlKoE+DMH%FdN$?^t0M17nJiZ{SlIZcx%e))_%55pWMt)S z)tOEbLRP{BPnun@BbG**kI!xPK@m$Zbcg09Ts!dPA}7|*{_PI{N03uTE|fk%BW2^H>hkk zx+pPOI5{X9C$uHT z*nK@N*)MCuWXtFRnpwuy$m@^0GRRBTM>xlvE6-52^dKzVb66EH%gbis(3Yn?_e-GMHj+Gt@=vI zXjGFoo}=2*@J)JGQVjpF&0uai$@)l_m@F< zB&`v{W;A9ZU&}6ZXFQhoH5@ZaufFXV`>BQdcTQRXiYov;9bA94lBs`@RCKg#jC4%Y zf4P)@H|G9kIsd4L(G;mGX74r@V1ahAQ?jf!b`UDW25+2C{{wtaP`$YbY-2Z_k2SjK@=`neu;|+Z(o}f{s3HcXK zgNH$`pNh!SBuZOX2#f%V!kHOd2)bG*nougR5^ z5|k!u>bgMnQl{xmvV;l6J3nY-Qh@*myy45|7Wd4Leh5Z6y%@ScJmG3e3;y}cN)fQ@rK*Z#pnc~g-RgL7Z{V-i zkiqKlS3H@a+Qs5&xk-Jn;VKxzG+JX3pMMr2M04dvWRXunt3&FBuRjVHHcs$ZZE1@D zLgn>8+tASLZ!HI4JxycM=a=jYmdJ* zo~H2$Lj1BW*^~q;>{e(}2u_!9!Vnk_&4`&Yu<7L%YrnLS5rHvw_H_ymQ5Z3O?fVbi~H(b<8K5Q8Er0^ zMG=$X@M&(bUm{@=XlX;4S#5aJZjsYGMA_g5>GHpOmoo@a(caKcK}?qu@dzVhJhVe> zeuUvd6l#ULt55;#`o#rd$gRIWpII<5c62BtjZ-O?Lal?{2d_tiu+t_jTva)~(O{GX zB7Nqey;K3`>de?*(f(eWySHB)V#W0ZV`M>88w!Tz!v6fD-WHbUjkhEA`)dHlYL~lN zL>sH+Bi$D2srsxTKo9vh>i!4B{)s6*QTI46 zvQ)T#qc@Ij-=|$t3sMNOfmlUkxte|uR3%iF3n}`2&f?=4)BF>JBN+E*5*K?a34pKST@YMI&h6&=cP+j%(kJh)Rl->a?22zY7jX4!jKR3KN z^gn{cebN1vT>y-mG;uKSnX_SV)CbJ%3v{#fUXq$lS|o1yW^2)?o6Pz8k7;VqyyRJL zS2T{r5Cd?)$R{WifD@=A^T7@M(Lr75!7($L-@*9B(vBj{0jXhY4e{7j$BstHa-J(I zK*SNbXK%er)8K=ynU(F>&ki?f-ML})e@kAcvogN(DR8o$a~*O0%wF8C{bhV7jh84j z3T6}T4Jz53uA7fEOSMC(KX9^l$~#|hRe2~A9fE=LPa?}U`B`jmvpPFjB@LrI8;2j)PQ8jrX{(MDaUY;&%f*|8K zR?UzG0h{IbAfzH$BG;w9j zMu#OQYfJssH;(xjt0m|SJBaa4m2^nx3&^|!Zz*6F;8!;%Tkd(7wT|Asm^=2Kw~ti}}Fmxu|$~SUcOxdabj@g;ZI+=7_syxo3;4dbqP=Nn=hU zX%3YxT5LYt_@$%T6BV{#a4)Fdr`TDCmz(B{yz(n^Xai28s|D`5Sercjtkcn9s z^Dx~DT5tYK9@!J7emVAw5df;(`FE=PE9>9~yZC3ST(lggMW%j6K>ex|9xjR_yHG8O zIa_?88d=U{JollkYa(=fqx$^IO!T@Sg+iAC{xS4hk`-UH6`guPU#z|C!TvEVBNMmi znTM9sS?-4jH|BbGZLT2EBk`56QNP0=mfF-S<7smrShP}&S`6o#dz>xmcJND1LK!*V z7l?+CHHL53NTrTL>JpiH5NW&c)-VqetMJO1uv*8;tM^| zE7%q=$Y1DPfDh+BiSZ0ZZZvxkA5>B?i0}w?K4BLid6gK6kkt-MA$^ej!xg0J%>iC=wNJBw0fwA7?q$By z2M?xcP+V5r^!mH1FHgD%#K$r>l>L)nQ z;$tzi`|V;IBhVLkW4(5E0qlT@97Mgl0tj7g!#TtR`20KS=X}G{uj*d#d8H7-<3(U! zxT#6DO|D7X&6Ho1amtUJzMx9|ilVp6c9Ewru06C9fNm{~FwG)TZ0zj5bEOdX+(tr1U{Z}geqfE?+ZB9`8Z?-w(U&zGY z@BYu^*95T5k)`gR;c(ROw$ED2+Mmj|nS@|k?0t%OM{s0Kr8zHoB}*p$7HgZP(*ElL zT<=8L_7vW>j6DSzTPfDHn9vbnH0Ktsre>OilSqP$=`j9iAp)2J^3nP^3|$qx*gIqB z*xYPsB8bZeO|t$YQONeskeKhawYjkZ;OmLSWiYif@@5jdW0~l6Gf8cmtUpJMzh$Jc zJirhb3B<_VC}5xdti9|Vyw|VZ1{O_M*A{^PEHh2gyT{lBwc|b0=RJR7_<{AJa~j|h zMMgvHdHZJnUgK9Jj@KB<#3^+);tvSWA@s6D!$WvecOUn z1A4d@wDMwjIAn-vCZ;+Pg{00_cy&Egsr#{r_%18sC1W}zSJyR!FNGG_ zQ0&d#i`pEcb6X}h0i4-23f&||wfBq1te>*-%){AzAv7?QUhlbUBw1R(G=X}eJNZR< zgS*v~ayq1B9U?5udKT>{zIYXh6#FGMSU0L94u7Q)RJCVJWhEZE?BJsLv7#$5!{hxu zPhA6%I28|>rXPNu`M=4cV zD&kX_+F}7XY4c8A-|YCsEC^`bE*vo|nua~woEfGwAeR76-oxe#0#~vTJvfiI=^PT` z$QJTExs+J&or~9oEsII_h6Q!aa57xT;~Kyvk60*f;i^fud$cNW26sdcGr;cToMq>)77dcaROtX}z>+FCJ3>6B=t$okMOnXZ2>LpLis zwxurs(AP%sTq7)*n=nQRhp4>;p<}WSK`~G3 zzqX|Ar4lWB-kcsDPF!j|9G$kddR2xE55uejHvX!)#Wxw@p?90rIc<+lJ!4$xXfts} zE(SkVzr(*pb(ZGEUSpzMuOqRWSaL;QHl4BTSh~a+Fqw^CpS2jaolkcg>D*nVc`4hC zmYjb)G1LC&^zhvzQrCjw)(kkGE3UtyxFT)iXPcHa@lm)(xm6P`ROYTLHPlR1sMF$l)#>fc-Jf&_Q1=dSMgZq~I{wY~_nQv= zi?Rw?HK#a!0B6!6A03+|=ktP!WgBO&n`a@9)87HF_V z{bBoC`g0W|VI;Hh;$_oxe0M-wvj* z9CB@SHdC6(^vWKI@15fld8yPje*EgUGz#XdeszLhkImQGUJJ^bA6)(eNl2J z1)O_IoG|JOdpuxzZTUme^HEL1j>t`Kj_Kt>xnEt_!CqdZ^5sd>%JkP*|9JI~> zO(F7a`iOOcPud6LfrkB8vH(p~rnC;|!4cDGoG%%J@3XLkytkYYLJISYHkWPMqB_I8 zhXaQ0NkjN3MZbCAYz_5P zOy@+aXSyPS(rwuFix*J`^976!`KB$R%_IzW&-PY)Tc7wkJ4+k z)gTj%ds|IWl_lKZ5;4HoF^Qn0G{+jh9{x;v%ncFq5s5S{i0dF!uHbx#<`Cy@>Sp_- z1Ml(H_UMdiKLxH@;EUHC)>O;%J5p_O&5QIq=_o4jjOOK3 z9}4f)(@pow^41l@b9IBC#jHlUV}Atl!4NP8b{W|z9Mm0=ai>LZf%t0Bhg?7T;_7JQYO^P3>y&uk>xrLzS3RGyA?)By z4E`$%{!RD%BST|iU}OGE7W^+Y)Bn!>83;i@J!SG7b)w^RlYYax+)1R{@+S*^LYd!t zM|%-q`Z529V8>q=o_}z>{0)*d%7XxRC_F4GKQ=2NZkq^l-UBgDPdp)EjcHuAH7Q%z zzRjBBe$CWweolW}-AeyU>+K}v6|ks!o3;LcnzY1`>}wlLk`VJlPaY1AbQ|ftP?510 z=ya-~X$0WbW(&P{?EDSH4ECgUn}yi?QJSqe6X(a?;6B5V(S9$mUwyCvm1Ym9XGOzB=e?AwYO}i9X{y8uN|Mg%$KsP*|7AaKTtM6 z;}V5PP$P3!PkTd)cMy@fO1hkuCgv{|)n(=RPM;slk?o8LkusDkRIpL!jg2&06!b^0 zX5w}Nz1TQyQrmm`1h%Ma&MZIa_*(-sa)_n3&oe1Gmnodq__DL}A=VlvTIxE`XMMBE zFZeGY9jFJ^;7IkvI`bHb->}ERW2FvMz8=e(IOoyjpiFA^hg~8F6*?KzyN@dKPIuM} zJ7qG+DTJZRHL0%BC%Me|+CNnuaip4dY>1_vjCev*#& zIkfn*KTv5cZxzu7a6RFt&?b*9uRZsUVcd~Y5FIuv7Xa2ffRKlgoazQM!FJg=Xl0OI z=*ewal6Oin$5Iw_-nkhT1-c-IMQRsC_ng^YWRF2t>Me+lf;jvqgNinaA?+_c<9z~aN zCC2PDCX2ZyXU2DAvc-j5lZWET$7LSkve#T)kvqnsby%dP3qR;G;%LxmD5B899D|f8 zGCDlBr$wnD>}xDkOD+WTBv%X%G?~cpb^E0BCijC2%)sl>*KKBrA+yz-IVMS^S^;OppK7LL9{5@IA*u>7>%GTw3($?QZTz-Olis1h0xcW;C?$=oO z|BT@NU?0Cf@4p0afAX%U*=~*+#C!c8-By1w+5gGX|B+JS;Z5B{UBBH=3va)bGmgxZ zJQIM!651A8!CIMg^inn!p5+Loaldq$bOfuKOEA7cSgcQOu~&=Smj$u!@{0|ttH346 zG!14M{$wG_a|Yz2(|sJeQwXt?Z$2xrC!ZA`z-J{0@L91v`K;KmM85f~)U-2bwv=WS#D!3kb-c6IDJ8Zl&*O_=a-p(F5@u8 zn|mrL1t_0^SP?psC) z!_Ew<7RI-Xk|3Qh1DRP22VlBJ`ofZ#>sA9qXU`U(z929(Q7GTxV*%fU#7@Eub&mZ*$V0dnMgPw`b^Xd|<| z<)>xj^0I<8%tHW|)g^9EXn<9@wjJRsaU0MEDl}&^ z#$xq0xF)K@Z_!zkf~|VP@y`LJYZ1!4ApOkK@xdEo>PYwyQ9vdtDb=JQSw#5Zhwg8EI2PI?vsuwQW3a`|Xc{^J{v)IOdUh{- z(KPy2e2qeU4{M?Rvu7vIf%Qdq_<~)ND$QTKYB(Kquj05JlrLM{1=+NtM^=~l$nC$O ze5x7eM3=9jjpP1BQz;3k6e@0AqlP?AyHhc2-+P7A91P#AyV~c$d5`No(ZhP%8Ke@VBx1)Mw=hf99FUL-qadk3}IYdS&BR-;kTA+ z@2H+y-m3zqPp5G?q%68G49@aHt4ApObXs%LgI!J}hj7#~)|ep-`Ib1V+<~ws%#B6q zc(@+kPepn`@yxax!!FoKAx@lJeCrG+RT!lx;drSIACaq<9f|$u&t3E*%Kp_X0EC&%D?_YT>{e z)lZbrx&xZ#@WSYH#9`woqe(w;Gw~D$qmu_*2}Hw4agkJhgfj z`-@;G>j$7UI*6ENr0m2Vje{b&{npmL+wAjsWu6C{`}0Gu`_qb(L$rR845pnx2St|1 zu=uQx)mWMXU{yH{xmKo9SVx7kCT3E@(2@|Us4BUyl|8u=Gb7ISd8I!rm4~^0FsYWB zH=Qw*=y{ckC9T0AIcQ8_+Q%ywi|P4rw+i%#7CdFzzlRoNwvI>ab~He8EqcQuFq@c; zP)$__TPp19s9@$;%)6^1rNsa-gEI^TJgp%*%so~IQX)CrJGNUaeb~Y8dg1PhHjVn?o zdzwH@1m)|87+T*O{3(npU)X82`jHRO9ld)hc{lDWYmiyyiZksJVfZI3*L{*xZqdO= z^4;5#9*0|~gea@H=MYW3c+zN+FA7MUvm#K_3}o=yG16xt?fUP;!Q^RELpA9A&qz12 zn{z+Yp!K@ACK*J>z@;ONs2!yd5xvo&a_$2+iqra-=B~i71milZ?3<>#pG*ueYN~nt zkVP1D4pogKUa>BzNJk-zs>K)*`yy$vwO_mj4Zn!h$?F~YYSbeT^eQ65gM!YiYaBIf ze+>*)+;{#t{44ud^ut$Q>P6Z2h17Y;A!L$Y7d4N;!n82%#p^I*h39IpSRj|@C0Gr~ z3{j)1jXwTV2AL4Zr$mEP7{Z8w`vv9cD z{)n!;RiFNk9F>0!y8Tnw>wk-;P{?ZbPt=~&s^8T#zz634D^UOeyu`stHELf@ zf}0W9CkL`HWRtOt=b(LF5K-)6X!R0>cd?&iF5e3-h30XHk>EPDShWH+?qV!a99k&2 zgaoF3E$|W}JWKH?3Q6(L=ZZthuqi&srwiHGDnsvx<!NuFJ%#o_0+lUn0V4P z&dAY0j^HDk%6t;x2mlKm*pIfO|LkOBrqJm}daC8}ZX)WLBi$Jrq?pE@3-GG&jx=>5 zuMYT9>ul7Dw_Udxt)$-lIkO)AxXb4ok~e$P8l>>ppZM}$3|~S;GVzM?3j^XS7x$#n zkY%D!6g2ug`VUGPcMRRv_y|bJ$&@6N&|Di@lW~Oa#>npDvAdW)u;|~HLCO#S0xX!J z#3RalQ8BjCZ`g~1jl8cBS3YPIsw$%cG`zl@cXSY3 z_ag}I!U*qwM;Y!9p<(6y#ZLvuIyk7P=)O$8QY&*bf0lkxfZd#K@yBj}qwqlg!Ck~TEC94C)Sa&s zOe6r@z1h;GX5tb*$wZ})t#W2>gFlqD3P=}nmDKH3RASH~z)`q<+KSt{i}&zwe;t}P z==E@QPO{lbgR;pmK&#di*lbpdN7hq2?OQf&m^a}phZJt*sG zIV9V3V~RgbUmS0u5-|(N3^%Ef!U;*VYz#DfR#=ix3?d|ld33oLDRrd2KFgQ`fwy$?JJwxc%!`NQWdAP1n+C(zgLqfL<(>j^z`O>L&qO8g38skNBbeLyX zM48IC&2zUi21Wc5g;YE@k99aZpyR;yVyT&sg(1{F#IJpY`cNMHk-&5~Rgd&vRfA zP^e|xka7-KNwY{|bqHVfxex6`8I+Hv4dFlo`c@f`T4eSEA7iQx0NOhUUX+!Z!}q7U zC^bnWy+MVfJzn(IMB2+m=m);Um0)J&c+(a6u57wHPm0DaXGF7rt0~*}3YoHvcg^^9 zxEmGib3j$pLa@NMmlW}DBv$+>&}QC`tRudKdJ&#dvs+#CCih{BZFAB3jjzKJbcIQwg4gLz2A- z0m^t$xJOPGTzN^umIs}FOsabVL6j*N(`TLx&x5q8JM!wv{n>sffx*7IO(&T@DLO5L zN#U8+vLw?hLD5$jkuulQzGAG=g1Mw`;GIi4JK%j7_z}dcZ3RCOyicB`MetQ$2PJ>o zzdmq13{7Q#Tn-QWbrD&fOog-UdZCI%UvTbSW{a&nXV|b43e+=4pmN5__ZIxZDl zo4~?d2j+un3|kES6dO@|NI|CS@(dh9EFF zwa_GgGchPr#J4-8N;MZ130B3vb0xYWT>8~LAiXti{k_Hwmxv+GIij*jlDk#X(-CYW( zNC?u>Ad-r734)YJDivK3eV%2P-Fs(#XJ*dK$vGBY35vg@WOJRy zXof~~ETYd6QoNw_j-rV66^)9Ol{JIuJWpUS=qJ=D? z{4UP@C*IvXobF-&>eT4GQQ4$;+he6SwJzHrlBf!sK&vY4a3 zK!5FIQi6M7aKiYb!HZnXX8d%TPxezkB>NQ&>Fi zJ&u{hMV-SoWj~ESbSN?gxB6M$otYas=s6~$dVyb-?aOr^(t7|TpNGM+BGJz zOME}X%GHLAQJK~4ktB`Vb;?iJF|{tTm<`w*bl0%-(tUHzP}f?MtXGG*FKM^(SR1Oq z3rveHkI>OQ^!2?X7^Y+__1sdDT|A*(r$snDNI0VQv3yQYy70)`JD+3lUCznLawiLa z{pUJg+=&o-7b#&=`iF@m9%ceDx40+X&qiZ=c~qV_{qz?*S4N3CoYCkkp`)fet2QCa znOF+U9?#L_ldrXWPcb|gl^|}D@^<64Wz5!guZsU2yZ?ru?kD4G$3D#Un5+ZUzx0z z4v|yv`l>#9K1PS<8t|=1WK_aLvt`iNU!h+a7q8YcN&3RU;;6LPE=Xr>^Y&us&Rqrl zN2+SZD5X7a0tP*)&whOuu*D9($RWLzKj{e|Sq1j$#jiwunt0Vw;+ISG#AG9&WNq3E zjs@^yq-S7Rg_;j+s9yd%MdbF6gBcW79;av8BLy(a2efN;7(#)8d6fv5m}46ltGybK zNay8qw1GhEF$I=8%b*jXMbRrMvoIoA;VcUDN^k_TGCf>TiH#ACVq#NJMj}~QP>gJF z`Uk7$X|O_iJ8SXtPu|RRS%Zkc*QU+TYpFuqCz=EC0)BVd&CT05r>P>uxXqN~v+7^{ zo~XDVDm(swp-tCU=B#>@Ro}yvtb(d1PN{GLju=OVdMQEn&ITYjT->>Ha6t zxQ2<}gDg5%i4IRHbM6ltKO9r`P*K`I#YnL>czSZ6i%fc;d@KX^357*dt3zTGIZpF) z(ekKxiwR!C&f{*kd>d+*3~2Kj28#&Ct0pf3Cw#1=)R1Y5Q3=rfkz4$zSX#4lXt>9| zWZ~D42I<2=f#dd;^v1ucO%EK)5oAk0dA0JUpR#>9Cst*AfL=zZse0>O&CX&}9YcsS zmAAG-v9NZbiZ1(cRs{0oRTCZWMu&u@k2m{JSM+rA6K;`THTm`$TaHYx{iw)-j0Mi4 zv|k*$4S`l>wPPbFNsVI`-+zSHE2x_?2R-g`;}T@})kIEmWZrLnMzOl^L-$Z9MJNA#Ml4EnRQ6cB@dw(Ya;|o#HEBal_jgoO#kZvY{P>)#;OSvY-`=qytC9tx z`TZ>4KYQh5otJ$QHr!hGxZTn$R7jQkG?RlNx8i38w)8V0;wP?Z7!S!mrBx}?kaPK( zcAI^=X87^J>%R87uyB4hT8^*$L>KV72)S}zjKgb;+AWDK2j;Ag1zl8+E$;>}9xx#%JM&W+YTNnshgKi7aCAX)tO9N&+85gd3IMj2Llav~y+| z74W~$?F?$>iZf96BV_J8LBj2)*uj4TC#}x;s38NlHN0dc^rrf-8khabVr*9@Nm zD&v`Ij#rkR7ZF2IU#WsGQKss~G~y{Xb7t8x;a;Wqk-^|7vCN&!jUlE{S)rB7=R!Z> z{nj>$x@dae8I$2eJg*?ZPv1={9WoFTf_UOlvk#|q)PMZak3O!#{?Rbah=M^`X>i#& zrS-zW;D)w-N3o?Seb)5BXU`w9TqTrd%ON_;^LVuGrn2?PtW=L`4=ZK-bge{<7R9)s z{g%Fy?aBB`A74EjQ9NluG#}5@*B>k<|0Og}3zt@y8_CjAo{Fm`%;E7?xp(P70K% z9-a+!&yjxc!;+WCVDW1=)}!E|=RwuM6$hrIpXD=}Hua>>V&5^eqH*(=w(H|gC`Ixm z*H^oDKz}-(EDVnp0G?c|-^J$~dMA8mhsFtoK%!^7l;lN$2T z-;QI~=hx_DH;*-+R1l_F!Z&&IURA5&C?xL<==9stjI56Uju(7g-sCR-1epixIW`_> zp$rw26bu#C9UvGPz}F@RxYnc5ws2M7=o*0h)LkvK!Q{~offv;#eyf>swqUo?{@;ex z9oic-mrH3%zgFHU5rqqi2+|1t6gQ^+A~7mPdz+r|HhuDYMwT>2=G(ePhk*a-8M}ol zD$C1h4wZftO~D$&!FeHa1pcC2)a9#7d3yVsj0nP4N)4wB_N~ZiBdjbdJ#{hy79mrl z9)R6J$NgVLm6cL%6H$O{%$9c~gYw1#SmIN7;Ww^4mt+tpBo?nZCn|nkLsggJElcW@w&UCMQ>{vn!u7 zh8yqZ@abGc{=z|#Oeza!d{V3+JTSaqnkIBA{)B|XDW>6s^1_qYTM+MomffIA> z2YrYRAw?HW3*Vi!%MO!>|DsJ)qU%=L$KWkH$sYSdvD%NbLSN3|JE}_dax=MSXvN^A zGNb-%9Dgr$ou;oJV?>8gxoBFB~1>nQ9rsvPA~02_q=i8TnfR!g~5C2#hJZRllNR*7i;BoSX$=$mIr0i`(yKbExg9L zgGQ@KTy!k^5{$HGsyIvroV^955ihb9=5w=iEh}}RIDdEBSfpA_d=afp$ZgBWT(*t1 zF{#yxi5fvHm@m1s%@H=wg)9vpZZnO##r5;~9LIs|@%)#C%eQ-HXD}R+eP6{WexFGf z)zpzQ#be5ORGq8B(EU%4qe&b5MzU5n$pos0!Mw}&!|#U5n!0bLzMfh%(V_X@=IGYOmNny9(+V-#3dNLiUL{7uAZ`I~8YmEm_>4h8kvCsH)=US;Y6lLMg zf9EKUZ+V^D;rMKJItxh}<7Z}CTlc0=KrD{_{&N4%s|F0Wzh!YLKQ4@6;+WJ9jBXCE z|DHj8guPm(V?u^TIZ5?xl^{t}!qo~9*I1hZ&r!re=p`xq`ILJ&QFIr-q!dsVFa!qiBi)AwL|AlHgaP`5Bm3$ zQ7B#eU?~+nXCbBJRLRlBv5+_TA$zd=@rX^)S(>KbX7K~_3`FGtHsh7X))s?VF4OmK zWm}fNB$HmM4f3W-`Eh$PTN5{ImQoo2c&cf{LOo5K1H_4Pu;$(Yd*TLrT zB{HPz0~6`7UV$k|I!mnrvH~RP-pAZ;%e6Y!m8Lc3+NdAU2bs=h2E=@b8d~VmIQy~r zqoTX1*K9vt+s#~?dj7=+y2!ZdioR)yf{BAm<3o?=hT5!_aoz=J!#~E%HTu0tq3S8M zu`FH67b-jx71e%A%I>I5LQo|9hifS#rLKKuYYvYiGVldZlVH z)-;y;$xAW~-4(t0{pXi{!J#sP2Y&EoX3Md`!5g>vB_+p_Xqm&$rip$ONlUs?)vp5N)I$S6M zDrHl;GxJRb(+~?~LjiTesmpCk#M3YC%{t`t`4w~3YH+6rM7`)qp&aRyYci(zz>{RU z%&yV-@OqAUbr#lquKRNWWx}B^=1bMO^ke*8k7bign&;~|$NaebJBg-qR2G-|A5Te& zJ&;#lvIGXNTZ}AhW(#_!?(xhtEq2 zJf2bw8txq?`yoi*+xnr~ewyc>cy}?L%tea0rZ${adeS->9VIQI6b(c+h{zt#gHw&+ zwWLYh>F*oHqFW|L-_!@myf5P}VpH>;p5>oXy%`MuvQ%~Ihq+U9QQ=TL&w^^L*V24W zT8lA4#VmndVdRk*(fNiu%x2FqgGewt)E9~uUy6n1bvYR~I0<7Z)aQsLlnh_C#j6jR zbZURm9N&w7lpxtDxs2mb`@4J6mu$T7F89ZlMCrox_{C1x!f9RZjFuci-|8p>J^Sy4@p0;`jBZ{^aXXt5qsQ*Y4YXg zZXL7kATbtsGXLuOyW9?6+$b7RvksT<+@IyKnwSnyA&3*#wAL>0$i8}wI?c^P`POXR z+fkZg%Ym?qeC+*+sTxH|4m>1?8XL09DWXOJJ}&}py~Tgc)LIiAHve_T*s}+xA-#-2 zZHhDH{G6I@mdMobp{1V%KEZ(*mQUr<#MKdXJro?VsOd6sm*+p`@$DzX{GCSIru6>F zZ+U24*uO1jT3lKU`d{hq$GoHU<4D0^Q;BaQt zLe(vGHHRDg@m2>Nbh=4nO(Vkulr=hascUMtdQD-U$Z3zt$LFyHhYUE1+7qxa@K5iNwy=AaP8qM2w}9lhUNbdeOR;v9-IEn#~siU(9~) zu?&e|?o|yqKzZy!hRhmMAs3Q|o2I`Bo zPdkY`dxL9IkYAgdaO<6SjG1S==de|l{4?3P6Q$IAY4F!ArYT}V$^%uO+)A%5KIN06 z#GggR$j2u^3Wfx*U&i`Z*=f}&D9#pv_8FxF=mZoM-Q4)Ewa`f zt#aawngjV%R9RD^cf~(w4xs|2wBzNT)!k1P_mY0+0)HRLROKB;wd{-F^3AQlc*as7 zM}VhIlO=RatuIrkx-l<>O8;Qbx7pA8pJ6N|oJc`%ToA1>qK)@b)d<5HlKQOrODsn^ z=HR<$M>ORWGH*n_mHWn!G+K2|sV>xG#1iw@Em4!8_wivaZj}Q%u7~MfGqOY+om97^ z?Rcprd3&(_jjT9sa=iIvm&({u3+GpPCy5;TehXcS*dO{UTeL-5q|z437D-&9x-kN?{g#v`8;_3 z6bx2k`Wgd+(x6%D)thvB@RkS(s~Aj}`*BkrJQ(G*z6uO?yo4Aa;ctn(<*XJjRBiG(?|z%KO1Cs@aI!&3P!@|BftVy z*o_0Nbzn|uUhDLg!>)KUJHG%!gD~?ac&z{(I~B%#Kf(hJl@Qfm0_bY$F8#?eP3(9_ zp7+5gi|%T+;U)~&w!4SyPfE!`cqs}#DKv(0W8K72VTQqST)CNTKc3~+=_{EC zhxrA+^O+sQgxO9An1Cl~MSF!tSpiTf?|hFw3cJnu75L6ps?_FT=aIJAfL};LkYA2N zRMftZ!@rE!;5lLY=2(p{&MLNG{?7XdB8OQK(3xStezZG!iSTu~hzMu_bSYJc%`t@> zD+q15ZSe%%FSTa;1B2E4lcNvZUqUj%aKIfwsu=EJ4BR?FK0FT$h{AOJ)1$~rIGF`Y z9gs1=|CDlic3`(3xEKqP)WLV#Z#OEmO2JRy#hYMCi4Sx4OZRu>b`&^aE4A?UB#e@* zihQ5@1$mPdXfWMO*4#ftRv(IhjSRtRgpB}o0u&B}`2dO1tD=*wGMTk7h557hgO3qE zB{5&Qo7HN8g=zJ z+g}IRepJYL;3xgOZr%{0s<73YU}|;-oxD^!i=g?fUgoxC8ZcY(uwQFmfbYCQBBEfW zn-H|h!V+~3Rsod+y=odut*iaB^jFDNHL0(lf!q;NKT3KQ{J*3Y1&C_(vL#(5`{!Tq zP3eWxr4=G2x(lZ`Z1F?q_Q7}wXW%PI%=@@qLt#qx$(DA?fYNRd&sMP6(vH17w)VX3 z>xgbIOQ7tASBnqX(%hZ}D23?sc?ooT)N=LM>A`NCs<;OrX@{>p0lsJ2S@^@0u(bnL zbhq-0t>G=t{*_PXFpL$6 z38-uJ*j7~&Vxmh9xDQH~DmG#4Va9{D6W7RGk_*e7Z*ZGW!));U;3~R~ z3F8g^%pqon-+rG0b}kTUElLE^nW~CJ=5MDp>&+`P{s-bk**YAk64b3=db6Ufxll+gc zdwx3CA9vH|t-+}lLg&b4MTAVh;v3_o?3Qojp4E?NlKPFLk_y9^%wBhAV#k(+BuS)m zo#zkyaYx$-+rLbO?Qv4?)AG!s_V*cz?@LKWajB`{wLacBB(lij0RvovyOD?Df}UfC zBVP6~1h`yJYmGxnKkYV`z7`mnANCG!pc&Wm~KKP!5fL2)blpCg!f^P zV**wVTz^jxt@wQ4mt(LYKJX4;z6YGA!Gd9KF(9}TsefQlQeYKcA+<7fv^zC$mJDRS zb{$5eW(Rz|Mr~y_3m~}f@9edk;h~R;uE7JslpT1WA%r09Zeyf76<~FA5iEc)rr%&F zuta?*@c>MGy#OFT7Qp-<{E&=~SQJrAEO138B_=omi2^<$*jV9+M@-60C}kFA6f@Jx z3>Fg3fOa7KpA8~^Ca_l21TOd3G;X^phmBEBi5XozEnw91%*v=IR1S{_{uf*f;E8k$ zY}?PQtpM9%Mhyz=Jg9M`2U#Ex@%qoeHVI4)Sg`{U80{V-fwhg#t58)vMxE~s;>o?lc~N^a_s@SP)7|NCm*VFv1xb;Rt9g z{etRZDO@Y4Q-hA zn&(-I)}UuX?ks{0SF3&9vR?v-0pJTg4lwAUi-2~IcD)@ih835|x^C9n>SqI-z+-&g zt-%rMZvywbPc2k~0u>M*i{wx1G>%?NODhOVoC7Q7gc9aZg|? z68)5EsC!cLK?Al|dF_7Y^zIEv5E%NAUr=C1Z_t44xoYl)Y~B91ZQia{&dohe^!{u8 zTNvcgUj?EAK!Z z{d}Q+&*?N8r_Vca+Bj?I-?KS}#)cvuBChpwhjypBx%Y7#jj9@i>Uvs>z4&Et*CqN1 zM9|utycJorFuLv81S`z9IEx56SO8Wp^z%`m%smq~B(+|q4d!buf14zhfJC>g+#9X} z{Y(oece&&(xI?pg?37t!-COMe0{Yn%PzW=-LD)-D&Amba4FUaJ3MiDQlwG9pcaD=y zTbKZS?7Ya>m!vxTCQbWs6_h}0V8|B|6 zL-kJFH~isxQv~?HiY9=Cf}Och6MzJ9S^n{{N z6cZ&|pn%kaKPCvN2f#O5rx)Ex9UB5cPZSD;5(P{IueG|i2#$ZoxgbcuzBn{(=uQAO zTBqol{h$C$Dz*Rsk;R|BKRSZVSqau1!?s=SRat z4|+bet;j|FevnjyQFe#70NCxC$>__7rOw9aP~87uI(O#<7d)fr{QlnyF3`6C hLIppK&g#G05V)Ppu~wGRio!%;zEm*SEG9Tu{C}>>EO7t; literal 0 HcmV?d00001 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index a8f7c3c10ee1f..d0cff9650eedb 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -23,9 +23,13 @@ import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.SchemaCompatibilityException import org.apache.hudi.testutils.HoodieClientTestBase +import org.apache.spark.SparkException +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.junit.jupiter.api.{AfterEach, BeforeEach} +import org.junit.jupiter.api.Assertions.assertDoesNotThrow +import org.junit.jupiter.api.function.Executable import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, ValueSource} @@ -808,4 +812,87 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss readDf.show(false) readDf.foreach(_ => {}) } + + @ParameterizedTest + @ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ")) + def testNestedTypeVectorizedReadWithTypeChange(tableType: String): Unit = { + // test to change the value type of a MAP in a column of ARRAY< MAP > type + val tempRecordPath = basePath + "/record_tbl/" + val arrayMapData = Seq( + Row(1, 100, List(Map("2022-12-01" -> 120), Map("2022-12-02" -> 130)), "aaa") + ) + val arrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("salesMap", ArrayType( + new MapType(StringType, IntegerType, true))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayMapData), arrayMapSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, tableType.equals("COPY_ON_WRITE")) + + // read out the table, will not throw any exception + readTable(tempRecordPath) + + // change value type from integer to long + val newArrayMapData = Seq( + Row(2, 200, List(Map("2022-12-01" -> 220L), Map("2022-12-02" -> 230L)), "bbb") + ) + val newArrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("salesMap", ArrayType( + new MapType(StringType, LongType, true))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayMapData), newArrayMapSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, tableType.equals("COPY_ON_WRITE")) + + // after implicit type change, read the table with vectorized read enabled + if (HoodieSparkUtils.gteqSpark3_3) { + assertThrows(classOf[SparkException]) { + withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { + readTable(tempRecordPath) + } + } + } else { + withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { + readTable(tempRecordPath) + } + } + + withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { + readTable(tempRecordPath) + } + } + + + private def readTable(path: String): Unit = { + // read out the table + val readDf = spark.read.format("hudi").load(path) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + protected def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { + val conf = spark.sessionState.conf + val currentValues = pairs.unzip._1.map { k => + if (conf.contains(k)) { + Some(conf.getConfString(k)) + } else None + } + pairs.foreach { case (k, v) => conf.setConfString(k, v) } + try f finally { + pairs.unzip._1.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index 38221cc05c7ea..03673c7b47e20 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -17,8 +17,6 @@ package org.apache.hudi -import org.apache.avro.Schema -import org.apache.commons.io.FileUtils import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieSparkUtils.gteqSpark3_0 import org.apache.hudi.client.SparkRDDWriteClient @@ -33,6 +31,9 @@ import org.apache.hudi.functional.TestBootstrap import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.testutils.DataSourceTestUtils import org.apache.hudi.testutils.HoodieClientTestUtils.getSparkConfForTest + +import org.apache.avro.Schema +import org.apache.commons.io.FileUtils import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql._ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala index 6a9efb3371d89..70623814a1fbd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.functional.ColumnStatIndexTestBase.ColumnStatsTestCase import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions} +import org.apache.hudi.AvroConversionUtils import org.apache.spark.sql._ import org.apache.spark.sql.functions.typedLit import org.apache.spark.sql.types._ @@ -55,6 +56,8 @@ class ColumnStatIndexTestBase extends HoodieSparkClientTestBase { .add("c7", BinaryType) .add("c8", ByteType) + val sourceTableAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(sourceTableSchema, "reocrd", "") + @BeforeEach override def setUp() { initPath() diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index f500ea83120dc..d16509a7eb526 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -19,6 +19,7 @@ package org.apache.hudi.functional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils, QuickstartUtils, ScalaAssertionSupport} import org.apache.hudi.DataSourceWriteOptions.{INLINE_CLUSTERING_ENABLE, KEYGENERATOR_CLASS_NAME} import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.QuickstartUtils.{convertToStringList, getQuickstartWriteConfigs} @@ -30,7 +31,7 @@ import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineUtils} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} import org.apache.hudi.common.testutils.RawTripTestPayload.{deleteRecordsToStrings, recordsToStrings} import org.apache.hudi.common.util import org.apache.hudi.config.HoodieWriteConfig @@ -46,6 +47,11 @@ import org.apache.hudi.metrics.{Metrics, MetricsReporterType} import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.util.JFunction import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, QuickstartUtils, ScalaAssertionSupport} +import org.apache.hudi.table.HoodieSparkTable +import org.apache.hudi.testutils.HoodieSparkClientTestBase +import org.apache.hudi.util.JFunction +import org.apache.hadoop.fs.FileSystem +import org.apache.spark.sql.functions.{col, concat, lit, udf, when} import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.hudi.HoodieSparkSessionExtension @@ -57,7 +63,10 @@ import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, EnumSource, ValueSource} +import org.slf4j.LoggerFactory +import java.net.URI +import java.nio.file.Paths import java.sql.{Date, Timestamp} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.function.Consumer @@ -1678,7 +1687,81 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup }) } - def getWriterReaderOpts(recordType: HoodieRecordType, + @ParameterizedTest + @CsvSource(Array("true, 6", "false, 6")) + def testLogicalTypesReadRepair(vectorizedReadEnabled: Boolean, tableVersion: Int): Unit = { + if (HoodieSparkUtils.gteqSpark3_4) { + // Note: for spark 3.4 we should fall back to nonvectorized reader + // if that is not happening then this test will fail + val prevValue = spark.conf.get("spark.sql.parquet.enableVectorizedReader") + val prevTimezone = spark.conf.get("spark.sql.session.timeZone") + val propertyValue: String = System.getProperty("spark.testing") + try { + if (HoodieSparkUtils.isSpark3_3) { + System.setProperty("spark.testing", "true") + } + spark.conf.set("spark.sql.parquet.enableVectorizedReader", vectorizedReadEnabled.toString) + spark.conf.set("spark.sql.session.timeZone", "UTC") + spark.conf.set("spark.sql.parquet.inferTimestampNTZ.enabled", "true") + val tableName = "trips_logical_types_json_cow_read_v" + tableVersion + val dataPath = "file://" + basePath + "/" + tableName + val zipOutput = Paths.get(new URI(dataPath)) + HoodieTestUtils.extractZipToDirectory("/" + tableName + ".zip", zipOutput, getClass) + val tableBasePath = zipOutput.toString + + val df = spark.read.format("hudi").load(tableBasePath) + + val rows = df.collect() + assertEquals(20, rows.length) + for (row <- rows) { + val hash = row.get(6).asInstanceOf[String].hashCode() + if ((hash & 1) == 0) { + assertEquals("2020-01-01T00:00:00.001Z", row.get(14).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T12:00:00.000001Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:56.001", row.get(16).toString) + assertEquals("2017-07-07T07:07:07.000001", row.get(17).toString) + } else { + assertEquals("2019-12-31T23:59:59.999Z", row.get(14).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T11:59:59.999999Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:55.999", row.get(16).toString) + assertEquals("2017-07-07T07:07:06.999999", row.get(17).toString) + } + } + + assertEquals(10, df.filter("ts_millis > timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(10, df.filter("ts_millis < timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(0, df.filter("ts_millis > timestamp('2020-01-01 00:00:00.001Z')").count()) + assertEquals(0, df.filter("ts_millis < timestamp('2019-12-31 23:59:59.999Z')").count()) + + assertEquals(10, df.filter("ts_micros > timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(10, df.filter("ts_micros < timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(0, df.filter("ts_micros > timestamp('2020-06-01 12:00:00.000001Z')").count()) + assertEquals(0, df.filter("ts_micros < timestamp('2020-06-01 11:59:59.999999Z')").count()) + + assertEquals(10, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)").count()) + + assertEquals(10, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)").count()) + } finally { + spark.conf.set("spark.sql.parquet.enableVectorizedReader", prevValue) + spark.conf.set("spark.sql.session.timeZone", prevTimezone) + if (HoodieSparkUtils.isSpark3_3) { + if (propertyValue == null) { + System.clearProperty("spark.testing") + } else { + System.setProperty("spark.testing", propertyValue) + } + } + } + } + } + + def getWriterReaderOpts(recordType: HoodieRecordType = HoodieRecordType.AVRO, opt: Map[String, String] = commonOpts, enableFileIndex: Boolean = DataSourceReadOptions.ENABLE_HOODIE_FILE_INDEX.defaultValue()): (Map[String, String], Map[String, String]) = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala index ac83cf81918bb..7daa2dc69b9a6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala @@ -20,6 +20,7 @@ package org.apache.hudi.functional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hudi.{AvroConversionUtils, ColumnStatsIndexSupport, DataSourceWriteOptions} import org.apache.hudi.ColumnStatsIndexSupport.composeIndexSchema import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD} import org.apache.hudi.HoodieConversionUtils.toProperties @@ -29,7 +30,6 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.util.ParquetUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.functional.ColumnStatIndexTestBase.ColumnStatsTestCase -import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, Literal, Or} @@ -109,10 +109,11 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { HoodieTableConfig.POPULATE_META_FIELDS.key -> "true" ) ++ metadataOpts - val schema = StructType(StructField("c1", IntegerType, false) :: StructField("c2", StringType, true) :: Nil) + val structSchema = StructType(StructField("c1", IntegerType, false) :: StructField("c2", StringType, true) :: Nil) + val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(structSchema, "record", "") val inputDF = spark.createDataFrame( spark.sparkContext.parallelize(Seq(Row(1, "v1"), Row(2, "v2"), Row(3, null), Row(4, "v4"))), - schema) + structSchema) inputDF .sort("c1", "c2") @@ -129,7 +130,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { .fromProperties(toProperties(metadataOpts)) .build() - val columnStatsIndex = new ColumnStatsIndexSupport(spark, schema, metadataConfig, metaClient) + val columnStatsIndex = new ColumnStatsIndexSupport(spark, structSchema, metadataConfig, metaClient) columnStatsIndex.loadTransposed(Seq("c2"), false) { transposedDF => val result = transposedDF.select("valueCount", "c2_nullCount") .collect().head diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala index 9c4099035b12d..82f0342f729ec 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala @@ -19,6 +19,7 @@ package org.apache.hudi.functional import org.apache.hudi.DataSourceWriteOptions.{DELETE_OPERATION_OPT_VAL, PRECOMBINE_FIELD, RECORDKEY_FIELD} +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieFileIndex} import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.client.utils.MetadataConversionUtils @@ -32,9 +33,9 @@ import org.apache.hudi.functional.ColumnStatIndexTestBase.ColumnStatsTestCase import org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY import org.apache.hudi.metadata.HoodieMetadataFileSystemView import org.apache.hudi.util.JavaConversions -import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieFileIndex} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, Expression, GreaterThan, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, Expression, GreaterThan, Literal} +import org.apache.spark.sql.test.LastOptions.saveMode import org.apache.spark.sql.types.StringType import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.params.ParameterizedTest @@ -205,16 +206,24 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase { verifyFileIndexAndSQLQueries(commonOpts) } - private def setupTable(testCase: ColumnStatsTestCase, metadataOpts: Map[String, String], commonOpts: Map[String, String], shouldValidate: Boolean): Unit = { + private def setupTable(testCase: ColumnStatsTestCase, metadataOpts: Map[String, String], commonOpts: Map[String, String], + shouldValidate: Boolean, useShortSchema: Boolean = false, + validationSortColumns : Seq[String] = Seq("c1_maxValue", "c1_minValue", "c2_maxValue", + "c2_minValue", "c3_maxValue", "c3_minValue", "c5_maxValue", "c5_minValue")): Unit = { + val filePostfix = if (useShortSchema) { + "-short-schema" + } else { + "" + } doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts, dataSourcePath = "index/colstats/input-table-json", - expectedColStatsSourcePath = "index/colstats/column-stats-index-table.json", + expectedColStatsSourcePath = s"index/colstats/column-stats-index-table${filePostfix}.json", operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Overwrite) doWriteAndValidateColumnStats(testCase, metadataOpts, commonOpts, dataSourcePath = "index/colstats/another-input-table-json", - expectedColStatsSourcePath = "index/colstats/updated-column-stats-index-table.json", + expectedColStatsSourcePath = s"index/colstats/updated-column-stats-index-table${filePostfix}.json", operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index b1d3a17004bb1..8c121b1eafe6e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -18,6 +18,8 @@ package org.apache.hudi.functional import org.apache.hadoop.fs.Path + +import org.apache.hudi.{AvroConversionUtils, ColumnStatsIndexSupport, DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, HoodieSparkUtils, SparkDatasetMixin} import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.client.SparkRDDWriteClient @@ -29,6 +31,11 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.{recordToString, recordsToStrings} import org.apache.hudi.common.util +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, HoodieRecordPayload, HoodieTableType, OverwriteWithLatestAvroPayload} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} +import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings +import org.apache.hudi.common.util.Option import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieWriteConfig} import org.apache.hudi.functional.TestCOWDataSource.convertColumnsToNullable import org.apache.hudi.hadoop.config.HoodieRealtimeConfig @@ -37,6 +44,10 @@ import org.apache.hudi.table.action.compact.CompactionTriggerStrategy import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieSparkClientTestBase} import org.apache.hudi.util.JFunction import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, SparkDatasetMixin} +import org.apache.hudi.table.action.compact.CompactionTriggerStrategy +import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieSparkClientTestBase} +import org.apache.hudi.util.{JavaConversions, JFunction} +import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.hudi.HoodieSparkSessionExtension @@ -47,8 +58,12 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, EnumSource} import org.slf4j.LoggerFactory +import java.net.URI +import java.nio.file.Paths +import java.sql.Timestamp import java.util.function.Consumer import scala.collection.JavaConversions.mapAsJavaMap +import java.util.stream.Collectors import scala.collection.JavaConverters._ /** @@ -880,7 +895,11 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) def testReadPathsForOnlyLogFiles(recordType: HoodieRecordType): Unit = { - val (writeOpts, readOpts) = getWriterReaderOpts(recordType) + var (writeOpts, readOpts) = getWriterReaderOpts(recordType) + + writeOpts += ( + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true", + HoodieMetadataConfig.ENABLE.key -> "true") initMetaClient(HoodieTableType.MERGE_ON_READ) val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20) @@ -1079,7 +1098,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin @ParameterizedTest @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK")) - def testHoodieIsDeletedMOR(recordType: HoodieRecordType): Unit = { + def testHoodieIsDeletedMOR(recordType: HoodieRecordType): Unit = { val (writeOpts, readOpts) = getWriterReaderOpts(recordType) val numRecords = 100 @@ -1121,6 +1140,78 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin assertEquals(numRecords - numRecordsToDelete, snapshotDF2.count()) } + @ParameterizedTest + @CsvSource(Array("avro, 6", "parquet, 6")) + def testLogicalTypesReadRepair(logBlockFormat: String, tableVersion: Int): Unit = { + if (HoodieSparkUtils.gteqSpark3_4) { + val logBlockString = if (logBlockFormat == "avro") { + "" + } else { + "_parquet_log" + } + val prevTimezone = spark.conf.get("spark.sql.session.timeZone") + val propertyValue: String = System.getProperty("spark.testing") + try { + if (HoodieSparkUtils.isSpark3_3) { + System.setProperty("spark.testing", "true") + } + spark.conf.set("spark.sql.session.timeZone", "UTC") + val tableName = "trips_logical_types_json_mor_read_v" + tableVersion + logBlockString + val dataPath = "file://" + basePath + "/" + tableName + val zipOutput = Paths.get(new URI(dataPath)) + HoodieTestUtils.extractZipToDirectory("/" + tableName + ".zip", zipOutput, getClass) + val tableBasePath = zipOutput.toString + + val df = spark.read.format("hudi").load(tableBasePath) + + val rows = df.collect() + assertEquals(20, rows.length) + for (row <- rows) { + val hash = row.get(6).asInstanceOf[String].hashCode() + if ((hash & 1) == 0) { + assertEquals("2020-01-01T00:00:00.001Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T12:00:00.000001Z", row.get(16).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:56.001", row.get(17).toString) + assertEquals("2017-07-07T07:07:07.000001", row.get(18).toString) + } else { + assertEquals("2019-12-31T23:59:59.999Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T11:59:59.999999Z", row.get(16).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:55.999", row.get(17).toString) + assertEquals("2017-07-07T07:07:06.999999", row.get(18).toString) + } + } + assertEquals(10, df.filter("ts_millis > timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(10, df.filter("ts_millis < timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(0, df.filter("ts_millis > timestamp('2020-01-01 00:00:00.001Z')").count()) + assertEquals(0, df.filter("ts_millis < timestamp('2019-12-31 23:59:59.999Z')").count()) + + assertEquals(10, df.filter("ts_micros > timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(10, df.filter("ts_micros < timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(0, df.filter("ts_micros > timestamp('2020-06-01 12:00:00.000001Z')").count()) + assertEquals(0, df.filter("ts_micros < timestamp('2020-06-01 11:59:59.999999Z')").count()) + + assertEquals(10, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)").count()) + + assertEquals(10, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)").count()) + } finally { + spark.conf.set("spark.sql.session.timeZone", prevTimezone) + if (HoodieSparkUtils.isSpark3_3) { + if (propertyValue == null) { + System.clearProperty("spark.testing") + } else { + System.setProperty("spark.testing", propertyValue) + } + } + } + } + } + /** * This tests the case that query by with a specified partition condition on hudi table which is * different between the value of the partition field and the actual partition path, @@ -1327,6 +1418,69 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin roDf.where(col(recordKeyField) === 0).select(dataField).collect()(0).getLong(0)) } + @ParameterizedTest + @CsvSource(value = Array("true,6", "false,6")) + def testSnapshotQueryAfterInflightDeltaCommit(enableFileIndex: Boolean, tableVersion: Int): Unit = { + if (HoodieSparkUtils.gteqSpark3_4) { + val (tableName, tablePath) = ("hoodie_mor_snapshot_read_test_table", s"${basePath}_mor_test_table") + val orderingFields = "col3" + val recordKeyField = "key" + val dataField = "age" + + val options = Map[String, String]( + DataSourceWriteOptions.TABLE_TYPE.key -> HoodieTableType.MERGE_ON_READ.name, + DataSourceWriteOptions.OPERATION.key -> UPSERT_OPERATION_OPT_VAL, + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> orderingFields, + DataSourceWriteOptions.RECORDKEY_FIELD.key -> recordKeyField, + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "", + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator", + HoodieWriteConfig.TBL_NAME.key -> tableName, + "hoodie.insert.shuffle.parallelism" -> "1", + "hoodie.upsert.shuffle.parallelism" -> "1") + val pathForQuery = getPathForROQuery(tablePath, !enableFileIndex, 0) + + var (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.AVRO, options, enableFileIndex) + writeOpts = writeOpts ++ Map( + HoodieTableConfig.VERSION.key() -> tableVersion.toString) + + val firstDf = spark.range(0, 10).toDF(recordKeyField) + .withColumn(orderingFields, expr(recordKeyField)) + .withColumn(dataField, expr(recordKeyField + " + 1000")) + firstDf.write.format("hudi") + .options(writeOpts) + .mode(SaveMode.Overwrite) + .save(tablePath) + + val secondDf = spark.range(0, 10).toDF(recordKeyField) + .withColumn(orderingFields, expr(recordKeyField)) + .withColumn(dataField, expr(recordKeyField + " + 2000")) + secondDf.write.format("hudi") + .options(writeOpts) + .mode(SaveMode.Append).save(tablePath) + + // Snapshot query on MOR + val snapshotDf = spark.read.format("org.apache.hudi") + .options(readOpts) + .load(pathForQuery) + + // Delete last completed instant + metaClient = createMetaClient(spark, tablePath) + val files = storage.listDirectEntries(new StoragePath(s"$tablePath/.hoodie")).stream() + .filter(JavaConversions.getPredicate((f: StoragePathInfo) => f.getPath.getName.contains(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + && !f.getPath.getName.contains("inflight") + && !f.getPath.getName.contains("requested"))) + .collect(Collectors.toList[StoragePathInfo]).asScala + assertEquals(1, files.size) + storage.deleteFile(files.head.getPath) + + // verify snapshot query returns data written using firstDf + assertEquals(10, snapshotDf.count()) + assertEquals( + 1000L, + snapshotDf.where(col(recordKeyField) === 0).select(dataField).collect()(0).getLong(0)) + } + } + @ParameterizedTest @CsvSource(Array( "AVRO, AVRO, END_MAP", diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index ec275a1d3fdc2..d28ddf22c218c 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -23,6 +23,16 @@ import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark2HoodieFileScanRDD, Spark2RowSerDe} +import org.apache.hudi.client.utils.SparkRowSerDe +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.storage.{HoodieStorage, StoragePath} +import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark2HoodieFileScanRDD, Spark2RowSerDe} + +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.Path +import org.apache.parquet.schema.MessageType import org.apache.spark.sql._ import org.apache.spark.sql.avro._ import org.apache.spark.sql.catalyst.InternalRow @@ -74,6 +84,23 @@ class Spark2Adapter extends SparkAdapter { throw new UnsupportedOperationException("Catalog utilities are not supported in Spark 2.x"); } + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType.getClass.getSimpleName.startsWith("TimestampNTZType") + } + + override def getParquetReadSupport(messageScheme: org.apache.hudi.common.util.Option[MessageType]): + org.apache.parquet.hadoop.api.ReadSupport[_] = { + // ParquetReadSupport is package-private in Spark 2.4, so we use reflection to instantiate it + val clazz = Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport") + clazz.getDeclaredConstructor().newInstance().asInstanceOf[org.apache.parquet.hadoop.api.ReadSupport[_]] + } + + override def repairSchemaIfSpecified(shouldRepair: Boolean, + fileSchema: MessageType, + tableSchemaOpt: org.apache.hudi.common.util.Option[MessageType]): MessageType = { + fileSchema + } + override def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark2CatalystPlanUtils override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark2CatalystExpressionUtils @@ -144,7 +171,7 @@ class Spark2Adapter extends SparkAdapter { partitions.toSeq } - override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] = { Some(new Spark24LegacyHoodieParquetFileFormat(appendPartitionValues)) } @@ -207,4 +234,12 @@ class Spark2Adapter extends SparkAdapter { batch.setNumRows(numRows) batch } + + override def getReaderSchemas(storage: HoodieStorage, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): + org.apache.hudi.common.util.collection.Pair[StructType, StructType] = { + org.apache.hudi.common.util.collection.Pair.of( + HoodieInternalRowUtils.getCachedSchema(readerSchema), + HoodieInternalRowUtils.getCachedSchema(requestedSchema) + ) + } } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index b2a9a529511ec..6393ef524edbe 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -24,23 +24,25 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.util.JsonUtils import org.apache.hudi.spark3.internal.ReflectUtil import org.apache.hudi.{AvroConversionUtils, DefaultSource, HoodieSparkUtils, Spark3RowSerDe} +import org.apache.parquet.schema.MessageType import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters} import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.{HoodieSpark3CatalogUtils, SQLContext, SparkSession} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} +import org.apache.spark.sql.HoodieInternalRowUtils import org.apache.spark.storage.StorageLevel import java.time.ZoneId import java.util.TimeZone import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConverters.mapAsScalaMapConverter import scala.collection.convert.Wrappers.JConcurrentMapWrapper /** @@ -60,8 +62,10 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { val encoder = RowEncoder(schema).resolveAndBind() new Spark3RowSerDe(encoder) } +import java.time.ZoneId +import org.apache.spark.api.java - override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters +override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters override def getSparkParsePartitionUtil: SparkParsePartitionUtil = Spark3ParsePartitionUtil @@ -102,4 +106,22 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { override def makeColumnarBatch(vectors: Array[ColumnVector], numRows: Int): ColumnarBatch = { new ColumnarBatch(vectors, numRows) } + + override def repairSchemaIfSpecified(shouldRepair: Boolean, + fileSchema: MessageType, + tableSchemaOpt: org.apache.hudi.common.util.Option[MessageType]): MessageType = { + fileSchema + } + + override def getParquetReadSupport(messageSchema: org.apache.hudi.common.util.Option[MessageType]): ParquetReadSupport = { + new ParquetReadSupport() + } + + override def getReaderSchemas(storage: HoodieStorage, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): + org.apache.hudi.common.util.collection.Pair[StructType, StructType] = { + org.apache.hudi.common.util.collection.Pair.of( + HoodieInternalRowUtils.getCachedSchema(readerSchema), + HoodieInternalRowUtils.getCachedSchema(requestedSchema) + ) + } } diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala index 22a9f090fb33e..e593cc5e78681 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala @@ -60,6 +60,10 @@ class Spark3_0Adapter extends BaseSpark3Adapter { override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r) + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType.getClass.getSimpleName.startsWith("TimestampNTZType") + } + def createCatalystMetadataForMetaField: Metadata = // NOTE: Since [[METADATA_COL_ATTR_KEY]] flag is not available in Spark 2.x, // we simply produce an empty [[Metadata]] instance @@ -84,7 +88,7 @@ class Spark3_0Adapter extends BaseSpark3Adapter { override def createExtendedSparkParser(spark: SparkSession, delegate: ParserInterface): HoodieExtendedParserInterface = new HoodieSpark3_0ExtendedSqlParser(spark, delegate) - override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] = { Some(new Spark30LegacyHoodieParquetFileFormat(appendPartitionValues)) } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala index 8ca072333d0e3..bf5e9d219abed 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala @@ -61,6 +61,10 @@ class Spark3_1Adapter extends BaseSpark3Adapter { override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r) + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType.getClass.getSimpleName.startsWith("TimestampNTZType") + } + def createCatalystMetadataForMetaField: Metadata = // NOTE: Since [[METADATA_COL_ATTR_KEY]] flag is not available in Spark 2.x, // we simply produce an empty [[Metadata]] instance @@ -85,7 +89,7 @@ class Spark3_1Adapter extends BaseSpark3Adapter { override def createExtendedSparkParser(spark: SparkSession, delegate: ParserInterface): HoodieExtendedParserInterface = new HoodieSpark3_1ExtendedSqlParser(spark, delegate) - override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] = { Some(new Spark31LegacyHoodieParquetFileFormat(appendPartitionValues)) } diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala index 3a5812a5faa40..02237949a4866 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -60,6 +60,10 @@ class Spark3_2Adapter extends BaseSpark3Adapter { override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r) + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType.getClass.getSimpleName.startsWith("TimestampNTZType") + } + def createCatalystMetadataForMetaField: Metadata = new MetadataBuilder() .putBoolean(METADATA_COL_ATTR_KEY, value = true) @@ -84,7 +88,7 @@ class Spark3_2Adapter extends BaseSpark3Adapter { override def createExtendedSparkParser(spark: SparkSession, delegate: ParserInterface): HoodieExtendedParserInterface = new HoodieSpark3_2ExtendedSqlParser(spark, delegate) - override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] = { Some(new Spark32LegacyHoodieParquetFileFormat(appendPartitionValues)) } diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32LegacyHoodieParquetFileFormat.scala index c88c35b5eeb4e..dcaab6734e292 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32LegacyHoodieParquetFileFormat.scala @@ -340,9 +340,10 @@ class Spark32LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetReadSupport( + new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = false, datetimeRebaseSpec, int96RebaseSpec) } else { diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala index d64bc94301a12..181a9faf55102 100644 --- a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala @@ -138,7 +138,14 @@ case class HoodieSpark32PlusResolveReferences(spark: SparkSession) extends Rule[ lazy val analyzer = spark.sessionState.analyzer val targetTable = if (targetTableO.resolved) targetTableO else analyzer.execute(targetTableO) val sourceTable = if (sourceTableO.resolved) sourceTableO else analyzer.execute(sourceTableO) - val m = mO.asInstanceOf[MergeIntoTable].copy(targetTable = targetTable, sourceTable = sourceTable) + val mergeIntoTable = mO.asInstanceOf[MergeIntoTable] + // Use positional parameters to avoid NoSuchMethodError when method signature changes between Spark versions + val m = mergeIntoTable.copy( + targetTable = targetTable, + sourceTable = sourceTable, + mergeCondition = mergeIntoTable.mergeCondition, + matchedActions = mergeIntoTable.matchedActions, + notMatchedActions = mergeIntoTable.notMatchedActions) // END: custom Hudi change EliminateSubqueryAliases(targetTable) match { case r: NamedRelation if r.skipSchemaResolution => diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index e3d2cc9cd185e..11cc753d5e401 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_3ExtendedSqlParser} -import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} +import org.apache.spark.sql.types.{DataType, DataTypes, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarBatchRow import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ @@ -85,7 +85,7 @@ class Spark3_3Adapter extends BaseSpark3Adapter { override def createExtendedSparkParser(spark: SparkSession, delegate: ParserInterface): HoodieExtendedParserInterface = new HoodieSpark3_3ExtendedSqlParser(spark, delegate) - override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] = { Some(new Spark33LegacyHoodieParquetFileFormat(appendPartitionValues)) } @@ -124,4 +124,8 @@ class Spark3_3Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType.getClass.getSimpleName.startsWith("TimestampNTZType") + } } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala index de6cbff90ca54..add53e12e2493 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala @@ -342,9 +342,10 @@ class Spark33LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetReadSupport( + new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = true, datetimeRebaseSpec, int96RebaseSpec) } else { diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index 0ae5ef3dbf34a..8a406f431d086 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -17,8 +17,15 @@ package org.apache.spark.sql.adapter +import org.apache.hudi.avro.AvroSchemaUtils +import org.apache.hudi.{Spark34HoodieFileScanRDD, SparkAdapterSupport$} +import org.apache.hudi.io.storage.HoodieSparkParquetReader +import org.apache.hudi.storage.HoodieStorage + import org.apache.avro.Schema -import org.apache.hudi.Spark34HoodieFileScanRDD +import org.apache.hadoop.conf.Configuration +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter +import org.apache.parquet.schema.{MessageType, SchemaRepair} import org.apache.spark.sql.avro._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases @@ -27,21 +34,26 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY +import org.apache.spark.sql.catalyst.util.{METADATA_COL_ATTR_KEY, RebaseDateTime} import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{HoodieParquetReadSupport, ParquetFileFormat, ParquetReadSupport, ParquetToSparkSchemaConverter, Spark34LegacyHoodieParquetFileFormat, SparkBasicSchemaEvolution} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_4ExtendedSqlParser} -import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} +import org.apache.spark.sql.types.{DataType, DataTypes, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarBatchRow import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ +import java.time.ZoneId +import scala.Option + /** - * Implementation of [[SparkAdapter]] for Spark 3.4.x branch + * Implementation of [[org.apache.spark.sql.hudi.SparkAdapter]] for Spark 3.4.x branch */ class Spark3_4Adapter extends BaseSpark3Adapter { @@ -66,6 +78,31 @@ class Spark3_4Adapter extends BaseSpark3Adapter { .putBoolean(METADATA_COL_ATTR_KEY, value = true) .build() + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType == DataTypes.TimestampNTZType + } + + override def getParquetReadSupport(messageSchema: org.apache.hudi.common.util.Option[MessageType]): ParquetReadSupport = { + new HoodieParquetReadSupport( + Option.empty[ZoneId], + enableVectorizedReader = true, + enableTimestampFieldRepair = true, + RebaseDateTime.RebaseSpec(LegacyBehaviorPolicy.withName("CORRECTED")), + RebaseDateTime.RebaseSpec(LegacyBehaviorPolicy.withName("LEGACY")), + messageSchema + ) + } + + override def repairSchemaIfSpecified(shouldRepair: Boolean, + fileSchema: MessageType, + tableSchemaOpt: org.apache.hudi.common.util.Option[MessageType]): MessageType = { + if (shouldRepair) { + SchemaRepair.repairLogicalTypes(fileSchema, tableSchemaOpt) + } else { + fileSchema + } + } + override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark34CatalogUtils override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark34CatalystExpressionUtils @@ -85,8 +122,8 @@ class Spark3_4Adapter extends BaseSpark3Adapter { override def createExtendedSparkParser(spark: SparkSession, delegate: ParserInterface): HoodieExtendedParserInterface = new HoodieSpark3_4ExtendedSqlParser(spark, delegate) - override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { - Some(new Spark34LegacyHoodieParquetFileFormat(appendPartitionValues)) + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema): Option[ParquetFileFormat] = { + Some(new Spark34LegacyHoodieParquetFileFormat(appendPartitionValues, tableAvroSchema)) } override def createHoodieFileScanRDD(sparkSession: SparkSession, @@ -124,4 +161,19 @@ class Spark3_4Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + override def getReaderSchemas(conf: Configuration, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): + org.apache.hudi.common.util.collection.Pair[StructType, StructType] = { + val nonNullRequestedSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema) + val cachedRequestedSchema = HoodieInternalRowUtils.getCachedSchema(nonNullRequestedSchema) + val requestedSchemaInMessageType = org.apache.hudi.common.util.Option.of(getAvroSchemaConverter(conf).convert(nonNullRequestedSchema)) + val enableTimestampFieldRepair = conf.getBoolean(HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, true) + val repairedRequestedSchema = repairSchemaIfSpecified(enableTimestampFieldRepair, fileSchema, requestedSchemaInMessageType) + val repairedRequestedStructType = new ParquetToSparkSchemaConverter(conf).convert(repairedRequestedSchema) + val evolution = new SparkBasicSchemaEvolution(repairedRequestedStructType, cachedRequestedSchema, SQLConf.get.sessionLocalTimeZone) + val readerStructType = evolution.getRequestSchema + org.apache.hudi.common.util.collection.Pair.of( + readerStructType, readerStructType + ) + } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index 6de8ded06ec00..2b2ffd9f37a33 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -21,7 +21,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} -import org.apache.hudi.HoodieSparkUtils +import org.apache.avro.Schema +import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.util.InternalSchemaCache @@ -30,22 +31,27 @@ import org.apache.hudi.common.util.collection.Pair import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} +import org.apache.hudi.io.storage.HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR +import org.apache.hudi.SparkAdapterSupport.sparkAdapter +import org.apache.hudi.common.table.ParquetTableSchemaResolver import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS +import org.apache.parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} import org.apache.parquet.hadoop.{ParquetInputFormat, ParquetRecordReader} +import org.apache.parquet.schema.{AvroSchemaRepair, MessageType, SchemaRepair} import org.apache.spark.TaskContext import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.WholeStageCodegenExec -import org.apache.spark.sql.execution.datasources.parquet.Spark34LegacyHoodieParquetFileFormat._ +import org.apache.spark.sql.execution.datasources.parquet.Spark34LegacyHoodieParquetFileFormat.{pruneInternalSchema, rebuildFilterFromParquet, repairFooterSchema} import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType} +import org.apache.spark.sql.types.{AtomicType, DataType, StructType} import org.apache.spark.util.SerializableConfiguration /** * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior @@ -57,7 +63,23 @@ import org.apache.spark.util.SerializableConfiguration *

  • Schema on-read
  • * */ -class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat { +class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean, + avroTableSchema: Schema) extends ParquetFileFormat { + private lazy val tableSchemaAsMessageType: HOption[MessageType] = { + if (avroTableSchema == null) { + HOption.empty() + } else { + HOption.ofNullable( + ParquetTableSchemaResolver.convertAvroSchemaToParquet(avroTableSchema, new Configuration()) + ) + } + } + private lazy val hasTimestampMillisFieldInTableSchema = if (avroTableSchema == null) { + true + } else { + AvroSchemaRepair.hasTimestampMillisField(avroTableSchema) + } + private lazy val supportBatchWithTableSchema = !hasTimestampMillisFieldInTableSchema override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { val conf = sparkSession.sessionState.conf @@ -73,6 +95,14 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu supportBatch(sparkSession, schema) } + /** + * Returns whether the reader can return the rows as batch or not. + */ + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && supportBatchWithTableSchema + } + override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -113,8 +143,10 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean ) - hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) - hadoopConf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, sparkSession.sessionState.conf.legacyParquetNanosAsLong) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, + sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) + hadoopConf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + sparkSession.sessionState.conf.legacyParquetNanosAsLong) val internalSchemaStr = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) // For Spark DataSource v1, there's no Physical Plan projection/schema pruning w/in Spark itself, // therefore it's safe to do schema projection here @@ -123,6 +155,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu pruneInternalSchema(internalSchemaStr, requiredSchema) hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, prunedInternalSchemaStr) } + hadoopConf.set(ENABLE_LOGICAL_TIMESTAMP_REPAIR, hasTimestampMillisFieldInTableSchema.toString) val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -133,9 +166,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val enableVectorizedReader: Boolean = - sqlConf.parquetVectorizedReaderEnabled && - resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) + val enableVectorizedReader: Boolean = supportsColumnar(sparkSession, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize @@ -155,22 +186,18 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val returningBatch = sparkSession.sessionState.conf.parquetVectorizedReaderEnabled && supportsColumnar(sparkSession, resultSchema).toString.equals("true") - (file: PartitionedFile) => { assert(!shouldAppendPartitionValues || file.partitionValues.numFields == partitionSchema.size) val filePath = file.filePath.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) - val sharedConf = broadcastedHadoopConf.value.value // Fetch internal schema val internalSchemaStr = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) // Internal schema has to be pruned at this point val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent - val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) val fileSchema = if (shouldUseInternalSchema) { val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; @@ -180,12 +207,18 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu null } - lazy val footerFileMetaData = - ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val originalFooter = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS) + val fileFooter = if (hasTimestampMillisFieldInTableSchema) { + repairFooterSchema(originalFooter, tableSchemaAsMessageType); + } else { + originalFooter + } + lazy val footerFileMetaData: FileMetaData = fileFooter.getFileMetaData + // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) { + val parquetFilters = { // NOTE: Below code could only be compiled against >= Spark 3.2.1, // and unfortunately won't compile against Spark 3.2.0 // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 @@ -200,19 +233,6 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu pushDownInFilterThreshold, isCaseSensitive, datetimeRebaseSpec) - } else { - // Spark 3.2.0 - val datetimeRebaseMode = - Spark32PlusDataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - createParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseMode) } filters.map(rebuildFilterFromParquet(_, fileSchema, querySchemaOption.orElse(null))) // Collects all converted Parquet filter predicates. Notice that not all predicates can be @@ -246,9 +266,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) - hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json) - SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) } else { val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) @@ -284,7 +302,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu enableOffHeapColumnVector && taskContext.isDefined, capacity, typeChangeInfos) - } else if (HoodieSparkUtils.gteqSpark3_2_1) { + } else { // NOTE: Below code could only be compiled against >= Spark 3.2.1, // and unfortunately won't compile against Spark 3.2.0 // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 @@ -300,18 +318,6 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu int96RebaseSpec.timeZone, enableOffHeapColumnVector && taskContext.isDefined, capacity) - } else { - // Spark 3.2.0 - val datetimeRebaseMode = - Spark32PlusDataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - val int96RebaseMode = - Spark32PlusDataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - createVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - int96RebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) } // SPARK-37089: We cannot register a task completion listener to close this iterator here @@ -324,7 +330,6 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val iter = new RecordReaderIterator(vectorizedReader) try { vectorizedReader.initialize(split, hadoopAttemptContext) - // NOTE: We're making appending of the partitioned values to the rows read from the // data file configurable if (shouldAppendPartitionValues) { @@ -333,7 +338,6 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } else { vectorizedReader.initBatch(StructType(Nil), InternalRow.empty) } - if (returningBatch) { vectorizedReader.enableReturningBatches() } @@ -349,7 +353,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } } else { logDebug(s"Falling back to parquet-mr") - val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) { + val readSupport = { // ParquetRecordReader returns InternalRow // NOTE: Below code could only be compiled against >= Spark 3.2.1, // and unfortunately won't compile against Spark 3.2.0 @@ -358,21 +362,13 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetReadSupport( + new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = true, datetimeRebaseSpec, - int96RebaseSpec) - } else { - val datetimeRebaseMode = - Spark32PlusDataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - val int96RebaseMode = - Spark32PlusDataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - createParquetReadSupport( - convertTz, - /* enableVectorizedReader = */ false, - datetimeRebaseMode, - int96RebaseMode) + int96RebaseSpec, + tableSchemaAsMessageType) } val reader = if (pushed.isDefined && enableRecordFilter) { @@ -389,21 +385,8 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val unsafeProjection = if (typeChangeInfos.isEmpty) { GenerateUnsafeProjection.generate(fullSchema, fullSchema) } else { - // find type changed. - val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => - if (typeChangeInfos.containsKey(i)) { - StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) - } else f - }).toAttributes ++ partitionSchema.toAttributes - val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => - if (typeChangeInfos.containsKey(i)) { - val srcType = typeChangeInfos.get(i).getRight - val dstType = typeChangeInfos.get(i).getLeft - val needTimeZone = Cast.needsTimeZone(srcType, dstType) - Cast(attr, dstType, if (needTimeZone) timeZoneId else None) - } else attr - } - GenerateUnsafeProjection.generate(castSchema, newFullSchema) + HoodieLegacyParquetFileFormatHelper.generateUnsafeProjection( + fullSchema, timeZoneId, typeChangeInfos, requiredSchema, partitionSchema, sparkAdapter.getSchemaUtils) } // NOTE: We're making appending of the partitioned values to the rows read from the @@ -428,43 +411,6 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } object Spark34LegacyHoodieParquetFileFormat { - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createParquetFilters(args: Any*): ParquetFilters = { - // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[ParquetFilters] - } - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createParquetReadSupport(args: Any*): ParquetReadSupport = { - // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[ParquetReadSupport] - } - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createVectorizedParquetRecordReader(args: Any*): VectorizedParquetRecordReader = { - // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[VectorizedParquetRecordReader].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[VectorizedParquetRecordReader] - } - def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = { val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) if (querySchemaOption.isPresent && requiredSchema.nonEmpty) { @@ -531,4 +477,20 @@ object Spark34LegacyHoodieParquetFileFormat { } } } + + // Helper to repair the schema if needed + private def repairFooterSchema(original: ParquetMetadata, + tableSchemaOpt: HOption[org.apache.parquet.schema.MessageType]): ParquetMetadata = { + val repairedSchema = SchemaRepair.repairLogicalTypes(original.getFileMetaData.getSchema, tableSchemaOpt) + val oldMeta = original.getFileMetaData + new ParquetMetadata( + new FileMetaData( + repairedSchema, + oldMeta.getKeyValueMetaData, + oldMeta.getCreatedBy, + oldMeta.getFileDecryptor + ), + original.getBlocks + ) + } } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java index ad1918eabf8b2..e22fefa436803 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java @@ -29,10 +29,10 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.IHMSHandler; import org.apache.hadoop.hive.metastore.RetryingHMSHandler; +import java.lang.reflect.Constructor; import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; import org.apache.hadoop.hive.metastore.TUGIBasedProcessor; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.thrift.TUGIContainingTransport; import org.apache.hive.service.server.HiveServer2; import org.apache.thrift.TProcessor; import org.apache.thrift.protocol.TBinaryProtocol; @@ -41,7 +41,6 @@ import org.apache.thrift.transport.TFramedTransport; import org.apache.thrift.transport.TServerSocket; import org.apache.thrift.transport.TServerTransport; -import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; import org.apache.thrift.transport.TTransportFactory; @@ -51,7 +50,6 @@ import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; -import java.net.SocketException; import java.nio.file.Files; import java.util.HashMap; import java.util.Map; @@ -149,7 +147,15 @@ public HiveConf configureHive(Configuration hadoopConf, String localHiveLocation hadoopConf.set("datanucleus.schema.autoCreateTables", "true"); hadoopConf.set("datanucleus.autoCreateSchema", "true"); hadoopConf.set("datanucleus.fixedDatastore", "false"); + // Additional DataNucleus properties for Hive 3.x compatibility + hadoopConf.set("datanucleus.schema.autoCreateAll", "true"); + hadoopConf.set("datanucleus.validateTables", "false"); + hadoopConf.set("datanucleus.validateConstraints", "false"); HiveConf conf = new HiveConf(hadoopConf, HiveConf.class); + // Also set in HiveConf for Hive 3.x + conf.set("datanucleus.schema.autoCreateAll", "true"); + conf.set("datanucleus.validateTables", "false"); + conf.set("datanucleus.validateConstraints", "false"); conf.setBoolVar(ConfVars.HIVE_IN_TEST, true); conf.setBoolVar(ConfVars.METASTORE_SCHEMA_VERIFICATION, false); final int hs2ThriftPort = hadoopConf.getInt(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, HS2_THRIFT_PORT); @@ -244,7 +250,18 @@ private ChainedTTransportFactory(TTransportFactory parentTransFactory, TTranspor @Override public TTransport getTransport(TTransport trans) { - return childTransFactory.getTransport(parentTransFactory.getTransport(trans)); + try { + TTransport parentTransport = parentTransFactory.getTransport(trans); + return childTransFactory.getTransport(parentTransport); + } catch (Exception e) { + // In Hive 2.x, TTransportFactory.getTransport() doesn't throw checked exceptions, + // but the underlying methods might throw TTransportException or other exceptions. + // Wrap any exception in RuntimeException for compatibility with Hive 2.x signature. + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } + throw new RuntimeException("Failed to get transport", e); + } } } @@ -257,17 +274,6 @@ public TServerSocketKeepAlive(int port) throws TTransportException { public TServerSocketKeepAlive(InetSocketAddress address) throws TTransportException { super(address, 0); } - - @Override - protected TSocket acceptImpl() throws TTransportException { - TSocket ts = super.acceptImpl(); - try { - ts.getSocket().setKeepAlive(true); - } catch (SocketException e) { - throw new TTransportException(e); - } - return ts; - } } private TServer startMetaStore(HiveConf conf) throws IOException { @@ -288,13 +294,20 @@ private TServer startMetaStore(HiveConf conf) throws IOException { TProcessor processor; TTransportFactory transFactory; - HiveMetaStore.HMSHandler baseHandler = new HiveMetaStore.HMSHandler("new db based metaserver", conf, false); + // Use reflection to handle different HMSHandler constructor signatures between Hive 2.x and 3.x + // Hive 2.x: HMSHandler(String name, HiveConf conf, boolean allowEmbedded) + // Hive 3.x: HMSHandler(String name, HiveConf conf) + HiveMetaStore.HMSHandler baseHandler = createHMSHandler(conf); IHMSHandler handler = RetryingHMSHandler.getProxy(conf, baseHandler, true); if (conf.getBoolVar(ConfVars.METASTORE_EXECUTE_SET_UGI)) { + // Use reflection to handle different TUGIContainingTransport classes between Hive 2.x and 3.x + // Hive 2.x uses: org.apache.hadoop.hive.thrift.TUGIContainingTransport + // Hive 3.x uses: org.apache.hadoop.hive.metastore.security.TUGIContainingTransport + TTransportFactory tugiFactory = createTUGIContainingTransportFactory(); transFactory = useFramedTransport - ? new ChainedTTransportFactory(new TFramedTransport.Factory(), new TUGIContainingTransport.Factory()) - : new TUGIContainingTransport.Factory(); + ? new ChainedTTransportFactory(new TFramedTransport.Factory(), tugiFactory) + : tugiFactory; processor = new TUGIBasedProcessor<>(handler); LOG.info("Starting DB backed MetaStore Server with SetUGI enabled"); @@ -315,4 +328,85 @@ private TServer startMetaStore(HiveConf conf) throws IOException { throw new IOException(x); } } + + /** + * Creates an HMSHandler instance using reflection to support both Hive 2.x and 3.x. + * Hive 3.1.3 uses: HMSHandler(String name, Configuration conf) or HMSHandler(String name, Configuration conf, boolean) + * Hive 2.x uses: HMSHandler(String name, HiveConf conf, boolean allowEmbedded) + * Some versions may use: HMSHandler(String name, HiveConf conf) + */ + private HiveMetaStore.HMSHandler createHMSHandler(HiveConf conf) throws IOException { + String handlerName = "new db based metaserver"; + Class hmsHandlerClass = HiveMetaStore.HMSHandler.class; + + // Try Hive 3.x constructor with Configuration (2 parameters: String, Configuration) + try { + Constructor constructor = hmsHandlerClass.getConstructor(String.class, Configuration.class); + return (HiveMetaStore.HMSHandler) constructor.newInstance(handlerName, conf); + } catch (NoSuchMethodException e) { + // Continue to next option + } catch (Exception e) { + throw new IOException("Failed to create HMSHandler using (String, Configuration) constructor", e); + } + + // Try Hive 3.x constructor with Configuration (3 parameters: String, Configuration, boolean) + try { + Constructor constructor = hmsHandlerClass.getConstructor(String.class, Configuration.class, boolean.class); + return (HiveMetaStore.HMSHandler) constructor.newInstance(handlerName, conf, false); + } catch (NoSuchMethodException e) { + // Continue to next option + } catch (Exception e) { + throw new IOException("Failed to create HMSHandler using (String, Configuration, boolean) constructor", e); + } + + // Try Hive 3.x constructor with HiveConf (2 parameters: String, HiveConf) + try { + Constructor constructor = hmsHandlerClass.getConstructor(String.class, HiveConf.class); + return (HiveMetaStore.HMSHandler) constructor.newInstance(handlerName, conf); + } catch (NoSuchMethodException e) { + // Continue to next option + } catch (Exception e) { + throw new IOException("Failed to create HMSHandler using (String, HiveConf) constructor", e); + } + + // Try Hive 2.x constructor (3 parameters: String, HiveConf, boolean) + try { + Constructor constructor = hmsHandlerClass.getConstructor(String.class, HiveConf.class, boolean.class); + return (HiveMetaStore.HMSHandler) constructor.newInstance(handlerName, conf, false); + } catch (NoSuchMethodException e) { + throw new IOException("Failed to create HMSHandler. No compatible constructor found. " + + "Available constructors: " + java.util.Arrays.toString(hmsHandlerClass.getConstructors()), e); + } catch (Exception e) { + throw new IOException("Failed to create HMSHandler using (String, HiveConf, boolean) constructor", e); + } + } + + /** + * Creates a TUGIContainingTransport.Factory instance using reflection to support both Hive 2.x and 3.x. + * Hive 2.x uses: org.apache.hadoop.hive.thrift.TUGIContainingTransport + * Hive 3.x uses: org.apache.hadoop.hive.metastore.security.TUGIContainingTransport + */ + private TTransportFactory createTUGIContainingTransportFactory() throws IOException { + // Try Hive 3.x first (metastore.security package) + try { + Class factoryClass = Class.forName("org.apache.hadoop.hive.metastore.security.TUGIContainingTransport$Factory"); + Constructor factoryConstructor = factoryClass.getConstructor(); + return (TTransportFactory) factoryConstructor.newInstance(); + } catch (ClassNotFoundException e) { + // Hive 3.x class not found, try Hive 2.x + } catch (Exception e) { + throw new IOException("Failed to create TUGIContainingTransport.Factory using Hive 3.x class", e); + } + + // Try Hive 2.x (thrift package) + try { + Class factoryClass = Class.forName("org.apache.hadoop.hive.thrift.TUGIContainingTransport$Factory"); + Constructor factoryConstructor = factoryClass.getConstructor(); + return (TTransportFactory) factoryConstructor.newInstance(); + } catch (ClassNotFoundException e) { + throw new IOException("Failed to create TUGIContainingTransport.Factory. Neither Hive 2.x nor 3.x class found", e); + } catch (Exception e) { + throw new IOException("Failed to create TUGIContainingTransport.Factory using Hive 2.x class", e); + } + } } diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml index f771f29154ebf..b462b5d18b278 100644 --- a/hudi-timeline-service/pom.xml +++ b/hudi-timeline-service/pom.xml @@ -88,10 +88,38 @@ kryo-shaded + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-http + + + org.eclipse.jetty + jetty-io + org.eclipse.jetty jetty-util - ${jetty.version} + + + org.eclipse.jetty + jetty-webapp + + + org.eclipse.jetty + jetty-xml + + + org.eclipse.jetty + jetty-security @@ -118,6 +146,13 @@ io.javalin javalin ${javalin.version} + + + + org.eclipse.jetty + * + + @@ -139,6 +174,15 @@ javax.servlet * + + + org.eclipse.jetty + * + + + org.eclipse.jetty.aggregate + * + diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index c61cf64968c2e..bc2aaf9bcb7a0 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -186,6 +186,14 @@ org.apache.kafka kafka-clients ${kafka.version} + + + + + io.netty + * + + @@ -202,6 +210,12 @@ com.google.protobuf protobuf-java + + + + io.netty + * + @@ -368,6 +382,11 @@ kafka-schema-registry-client ${confluent.version} + + io.confluent + kafka-protobuf-serializer + ${confluent.version} + @@ -487,5 +506,13 @@ + + + + org.apache.thrift + libthrift + ${thrift.version} + test + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/ChainedSchemaPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/ChainedSchemaPostProcessor.java index 0295e80bed8be..b5cb01d72a69a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/ChainedSchemaPostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/postprocessor/ChainedSchemaPostProcessor.java @@ -48,6 +48,6 @@ public Schema processSchema(Schema schema) { for (SchemaPostProcessor processor : processors) { targetSchema = processor.processSchema(targetSchema); } - return targetSchema; + return schema; } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java new file mode 100644 index 0000000000000..9cd17c451f4d1 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java @@ -0,0 +1,330 @@ +/* + * 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.utilities; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +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.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.hadoop.fs.HadoopFSUtils; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.testutils.HoodieSparkClientTestBase; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.testutils.RawTripTestPayload.recordToString; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestHoodieMetadataTableValidator extends HoodieSparkClientTestBase { + + @Test + public void testMetadataTableValidation() { + Map writeOptions = new HashMap<>(); + writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table"); + writeOptions.put("hoodie.table.name", "test_table"); + writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ"); + writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key"); + writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp"); + writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path"); + + Dataset inserts = makeInsertDf("000", 5).cache(); + inserts.write().format("hudi").options(writeOptions) + .option(DataSourceWriteOptions.OPERATION().key(), WriteOperationType.BULK_INSERT.value()) + .option(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true") + .option(HoodieMetadataConfig.RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP.key(), "1") + .option(HoodieMetadataConfig.RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP.key(), "1") + .mode(SaveMode.Overwrite) + .save(basePath); + Dataset updates = makeUpdateDf("001", 5).cache(); + updates.write().format("hudi").options(writeOptions) + .option(DataSourceWriteOptions.OPERATION().key(), WriteOperationType.UPSERT.value()) + .option(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true") + .option(HoodieMetadataConfig.RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP.key(), "1") + .option(HoodieMetadataConfig.RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP.key(), "1") + .mode(SaveMode.Append) + .save(basePath); + + // validate MDT + HoodieMetadataTableValidator.Config config = new HoodieMetadataTableValidator.Config(); + config.basePath = basePath; + config.validateLatestFileSlices = true; + config.validateAllFileGroups = true; + HoodieMetadataTableValidator validator = new HoodieMetadataTableValidator(jsc, config); + assertTrue(validator.run()); + assertFalse(validator.hasValidationFailure()); + assertTrue(validator.getThrowables().isEmpty()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testAdditionalPartitionsinMDT(boolean testFailureCase) throws InterruptedException { + Map writeOptions = new HashMap<>(); + writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table"); + writeOptions.put("hoodie.table.name", "test_table"); + writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ"); + writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key"); + writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp"); + writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path"); + + // constructor of HoodieMetadataValidator instantiates HoodieTableMetaClient. hence creating an actual table. but rest of tests is mocked. + Dataset inserts = makeInsertDf("000", 5).cache(); + inserts.write().format("hudi").options(writeOptions) + .option(DataSourceWriteOptions.OPERATION().key(), WriteOperationType.BULK_INSERT.value()) + .mode(SaveMode.Overwrite) + .save(basePath); + + HoodieMetadataTableValidator.Config config = new HoodieMetadataTableValidator.Config(); + config.basePath = basePath; + config.validateLatestFileSlices = true; + config.validateAllFileGroups = true; + MockHoodieMetadataTableValidator validator = new MockHoodieMetadataTableValidator(jsc, config); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); + + String partition1 = "PARTITION1"; + String partition2 = "PARTITION2"; + String partition3 = "PARTITION3"; + + // mock list of partitions to return from MDT to have 1 additional partition compared to FS based listing. + List mdtPartitions = Arrays.asList(partition1, partition2, partition3); + validator.setMetadataPartitionsToReturn(mdtPartitions); + List fsPartitions = Arrays.asList(partition1, partition2); + validator.setFsPartitionsToReturn(fsPartitions); + + // mock completed timeline. + HoodieTimeline commitsTimeline = mock(HoodieTimeline.class); + HoodieTimeline completedTimeline = mock(HoodieTimeline.class); + when(metaClient.getCommitsTimeline()).thenReturn(commitsTimeline); + when(commitsTimeline.filterCompletedInstants()).thenReturn(completedTimeline); + + StoragePath baseStoragePath = new StoragePath(basePath); + + if (testFailureCase) { + // 3rd partition which is additional in MDT should have creation time before last instant in timeline. + + String partition3CreationTime = HoodieActiveTimeline.createNewInstantTime(); + Thread.sleep(100); + String lastIntantCreationTime = HoodieActiveTimeline.createNewInstantTime(); + + HoodieInstant lastInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, lastIntantCreationTime); + when(completedTimeline.lastInstant()).thenReturn(Option.of(lastInstant)); + validator.setPartitionCreationTime(Option.of(partition3CreationTime)); + // validate that exception is thrown since MDT has one additional partition. + assertThrows(HoodieValidationException.class, () -> { + validator.validatePartitions(engineContext, baseStoragePath, metaClient); + }); + } else { + // 3rd partition creation time is > last completed instant + HoodieInstant lastInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, HoodieActiveTimeline.createNewInstantTime()); + when(completedTimeline.lastInstant()).thenReturn(Option.of(lastInstant)); + Thread.sleep(100); + validator.setPartitionCreationTime(Option.of(HoodieActiveTimeline.createNewInstantTime())); + + // validate that all 3 partitions are returned + assertEquals(mdtPartitions, validator.validatePartitions(engineContext, baseStoragePath, metaClient)); + } + } + + class MockHoodieMetadataTableValidator extends HoodieMetadataTableValidator { + + private List metadataPartitionsToReturn; + private List fsPartitionsToReturn; + private Option partitionCreationTime; + + public MockHoodieMetadataTableValidator(JavaSparkContext jsc, Config cfg) { + super(jsc, cfg); + } + + void setMetadataPartitionsToReturn(List metadataPartitionsToReturn) { + this.metadataPartitionsToReturn = metadataPartitionsToReturn; + } + + void setFsPartitionsToReturn(List fsPartitionsToReturn) { + this.fsPartitionsToReturn = fsPartitionsToReturn; + } + + void setPartitionCreationTime(Option partitionCreationTime) { + this.partitionCreationTime = partitionCreationTime; + } + + @Override + List getPartitionsFromFileSystem(HoodieEngineContext engineContext, StoragePath basePath, HoodieStorage storage, HoodieTimeline completedTimeline) { + return fsPartitionsToReturn; + } + + @Override + List getPartitionsFromMDT(HoodieEngineContext engineContext, StoragePath basePath, HoodieStorage storage) { + return metadataPartitionsToReturn; + } + + @Override + Option getPartitionCreationInstant(HoodieStorage storage, StoragePath basePath, String partition) { + return this.partitionCreationTime; + } + } + + @Test + public void testRliValidationFalsePositiveCase() throws IOException { + Map writeOptions = new HashMap<>(); + writeOptions.put(DataSourceWriteOptions.TABLE_NAME().key(), "test_table"); + writeOptions.put("hoodie.table.name", "test_table"); + writeOptions.put(DataSourceWriteOptions.TABLE_TYPE().key(), "MERGE_ON_READ"); + writeOptions.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key"); + writeOptions.put(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp"); + writeOptions.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition_path"); + + Dataset inserts = makeInsertDf("000", 5).cache(); + inserts.write().format("hudi").options(writeOptions) + .option(DataSourceWriteOptions.OPERATION().key(), WriteOperationType.BULK_INSERT.value()) + .option(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true") + .option(HoodieMetadataConfig.RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP.key(), "1") + .option(HoodieMetadataConfig.RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP.key(), "1") + .mode(SaveMode.Overwrite) + .save(basePath); + Dataset updates = makeUpdateDf("001", 5).cache(); + updates.write().format("hudi").options(writeOptions) + .option(DataSourceWriteOptions.OPERATION().key(), WriteOperationType.UPSERT.value()) + .option(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true") + .option(HoodieMetadataConfig.RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP.key(), "1") + .option(HoodieMetadataConfig.RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP.key(), "1") + .mode(SaveMode.Append) + .save(basePath); + + Dataset inserts2 = makeInsertDf("002", 5).cache(); + inserts2.write().format("hudi").options(writeOptions) + .option(DataSourceWriteOptions.OPERATION().key(), WriteOperationType.BULK_INSERT.value()) + .option(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true") + .option(HoodieMetadataConfig.RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP.key(), "1") + .option(HoodieMetadataConfig.RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP.key(), "1") + .mode(SaveMode.Append) + .save(basePath); + + // validate MDT + HoodieMetadataTableValidator.Config config = new HoodieMetadataTableValidator.Config(); + config.basePath = "file://" + basePath; + config.validateLatestFileSlices = true; + config.validateAllFileGroups = true; + + // lets ensure we have a pending commit when FS based polling is done. and the commit completes when MDT is polled. + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration())).build(); + // moving out the completed commit meta file to a temp location + HoodieInstant lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get(); + String latestCompletedCommitMetaFile = basePath + "/.hoodie/" + lastInstant.getFileName(); + String tempDir = getTempLocation(); + String destFilePath = tempDir + "/" + lastInstant.getFileName(); + new File(latestCompletedCommitMetaFile).renameTo(new File(destFilePath)); + + MockHoodieMetadataTableValidatorForRli validator = new MockHoodieMetadataTableValidatorForRli(jsc, config); + validator.setOriginalFilePath(latestCompletedCommitMetaFile); + validator.setDestFilePath(destFilePath); + assertTrue(validator.run()); + assertFalse(validator.hasValidationFailure()); + assertTrue(validator.getThrowables().isEmpty()); + } + + /** + * Class to assist with testing a false positive case with RLI validation. + */ + static class MockHoodieMetadataTableValidatorForRli extends HoodieMetadataTableValidator { + + private String destFilePath; + private String originalFilePath; + + public MockHoodieMetadataTableValidatorForRli(JavaSparkContext jsc, Config cfg) { + super(jsc, cfg); + } + + @Override + JavaPairRDD> getRecordLocationsFromRLI(HoodieSparkEngineContext sparkEngineContext, + String basePath, + String latestCompletedCommit) { + // move the completed file back to ".hoodie" to simuate the false positive case. + new File(destFilePath).renameTo(new File(originalFilePath)); + return super.getRecordLocationsFromRLI(sparkEngineContext, basePath, latestCompletedCommit); + } + + public void setDestFilePath(String destFilePath) { + this.destFilePath = destFilePath; + } + + public void setOriginalFilePath(String originalFilePath) { + this.originalFilePath = originalFilePath; + } + } + + private String getTempLocation() { + try { + String folderName = "temp_location"; + java.nio.file.Path tempPath = tempDir.resolve(folderName); + java.nio.file.Files.createDirectories(tempPath); + return tempPath.toAbsolutePath().toString(); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + protected Dataset makeInsertDf(String instantTime, Integer n) { + List records = dataGen.generateInserts(instantTime, n).stream() + .map(r -> recordToString(r).get()).collect(Collectors.toList()); + JavaRDD rdd = jsc.parallelize(records); + return sparkSession.read().json(rdd); + } + + protected Dataset makeUpdateDf(String instantTime, Integer n) { + try { + List records = dataGen.generateUpdates(instantTime, n).stream() + .map(r -> recordToString(r).get()).collect(Collectors.toList()); + JavaRDD rdd = jsc.parallelize(records); + return sparkSession.read().json(rdd); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java index 87f875642be33..343ac46a6e8a9 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java @@ -161,6 +161,7 @@ protected static void prepareInitialConfigs(FileSystem dfs, String dfsBasePath, UtilitiesTestBase.Helpers.copyToDFS("streamer-config/target.avsc", dfs, dfsBasePath + "/target.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/target-flattened.avsc", dfs, dfsBasePath + "/target-flattened.avsc"); + UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source-timestamp-millis.avsc", dfs, dfsBasePath + "/source-timestamp-millis.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source_short_trip_uber.avsc", dfs, dfsBasePath + "/source_short_trip_uber.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source_uber.avsc", dfs, dfsBasePath + "/source_uber.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/target_short_trip_uber.avsc", dfs, dfsBasePath + "/target_short_trip_uber.avsc"); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index 60ed1b6732a58..2b258bcb54256 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -22,6 +22,7 @@ import org.apache.hudi.DataSourceReadOptions; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.HoodieSparkRecordMerger; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.HoodieSparkUtils$; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; @@ -31,7 +32,10 @@ import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -45,10 +49,12 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -91,6 +97,7 @@ import org.apache.hudi.utilities.streamer.NoNewDataTerminationStrategy; import org.apache.hudi.utilities.testutils.JdbcTestUtils; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; +import org.apache.hudi.utilities.testutils.sources.AbstractBaseTestSource; import org.apache.hudi.utilities.testutils.sources.DistributedTestDataSource; import org.apache.hudi.utilities.transform.SqlQueryBasedTransformer; import org.apache.hudi.utilities.transform.Transformer; @@ -131,6 +138,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.URI; +import java.nio.file.Paths; import java.sql.Connection; import java.sql.DriverManager; import java.time.Instant; @@ -637,6 +646,506 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); } + @Test + public void testTimestampMillis() throws Exception { + String tableBasePath = basePath + "/testTimestampMillis"; + defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); + // Insert data produced with Schema A, pass Schema A + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.configs.add("hoodie.streamer.schemaprovider.source.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add("hoodie.streamer.schemaprovider.target.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertRecordCount(1000, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00000", tableBasePath, 1); + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); + assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); + assertEquals(1000, sparkSession.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts > '1980-01-01'").count()); + + cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.configs.add("hoodie.streamer.schemaprovider.source.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add("hoodie.streamer.schemaprovider.target.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertRecordCount(1450, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00001", tableBasePath, 2); + tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + tableSchema = tableSchemaResolver.getTableAvroSchema(false); + assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); + sqlContext.clearCache(); + assertEquals(1450, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts > '1980-01-01'").count()); + assertEquals(1450, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts < '2080-01-01'").count()); + assertEquals(0, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts < '1980-01-01'").count()); + } + + @Test + public void testLogicalTypes() throws Exception { + try { + String tableBasePath = basePath + "/testTimestampMillis"; + defaultSchemaProviderClassName = TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.class.getName(); + + if (HoodieSparkUtils.isSpark3_3()) { + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.sourceSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.targetSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + AbstractBaseTestSource.schemaStr = HoodieTestDataGenerator.TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + AbstractBaseTestSource.avroSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + } else { + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.sourceSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.targetSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + AbstractBaseTestSource.schemaStr = HoodieTestDataGenerator.TRIP_LOGICAL_TYPES_SCHEMA; + AbstractBaseTestSource.avroSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + } + + // Insert data produced with Schema A, pass Schema A + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertRecordCount(1000, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00000", tableBasePath, 1); + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); + Map hudiOpts = new HashMap<>(); + hudiOpts.put("hoodie.datasource.write.recordkey.field", "id"); + logicalAssertions(tableSchema, tableBasePath, hudiOpts, HoodieTableVersion.current().versionCode()); + + cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertRecordCount(1450, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00001", tableBasePath, 2); + tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + tableSchema = tableSchemaResolver.getTableAvroSchema(false); + logicalAssertions(tableSchema, tableBasePath, hudiOpts, HoodieTableVersion.current().versionCode()); + } finally { + defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); + AbstractBaseTestSource.schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; + AbstractBaseTestSource.avroSchema = HoodieTestDataGenerator.AVRO_SCHEMA; + } + } + + private void logicalAssertions(Schema tableSchema, String tableBasePath, Map hudiOpts, int tableVersion) { + assertEquals("timestamp-micros", tableSchema.getField("ts_micros").schema().getLogicalType().getName()); + assertEquals("date", tableSchema.getField("event_date").schema().getLogicalType().getName()); + + sqlContext.clearCache(); + Dataset df = sqlContext.read() + .options(hudiOpts) + .format("org.apache.hudi") + .load(tableBasePath); + + long totalCount = df.count(); + long expectedHalf = totalCount / 2; + long tolerance = totalCount / 20; + if (totalCount < 100) { + tolerance = totalCount / 4; + } + + assertHalfSplit(df, "ts_micros > timestamp('2020-06-01 12:00:00Z')", expectedHalf, tolerance, "ts_micros > threshold"); + assertHalfSplit(df, "ts_micros < timestamp('2020-06-01 12:00:00Z')", expectedHalf, tolerance, "ts_micros < threshold"); + assertBoundaryCounts(df, "ts_micros > timestamp('2020-06-01 12:00:00.000001Z')", "ts_micros <= timestamp('2020-06-01 12:00:00.000001Z')", totalCount); + assertBoundaryCounts(df, "ts_micros < timestamp('2020-06-01 11:59:59.999999Z')", "ts_micros >= timestamp('2020-06-01 11:59:59.999999Z')", totalCount); + + if (!HoodieSparkUtils.isSpark3_3()) { + assertHalfSplit(df, "local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_millis > threshold"); + assertHalfSplit(df, "local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_millis < threshold"); + assertBoundaryCounts(df, "local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)", "local_ts_millis <= CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)", totalCount); + assertBoundaryCounts(df, "local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)", "local_ts_millis >= CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)", totalCount); + + assertHalfSplit(df, "local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_micros > threshold"); + assertHalfSplit(df, "local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_micros < threshold"); + assertBoundaryCounts(df, "local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)", "local_ts_micros <= CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)", totalCount); + assertBoundaryCounts(df, "local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)", "local_ts_micros >= CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)", totalCount); + } + + assertHalfSplit(df, "event_date > date('2000-01-01')", expectedHalf, tolerance, "event_date > threshold"); + assertHalfSplit(df, "event_date < date('2000-01-01')", expectedHalf, tolerance, "event_date < threshold"); + assertBoundaryCounts(df, "event_date > date('2000-01-02')", "event_date <= date('2000-01-02')", totalCount); + assertBoundaryCounts(df, "event_date < date('1999-12-31')", "event_date >= date('1999-12-31')", totalCount); + } + + private void assertHalfSplit(Dataset df, String filterExpr, long expectedHalf, long tolerance, String msg) { + long count = df.filter(filterExpr).count(); + assertTrue(Math.abs(count - expectedHalf) <= tolerance, msg + " (got=" + count + ", expected=" + expectedHalf + ")"); + } + + private void assertBoundaryCounts(Dataset df, String exprZero, String exprTotal, long totalCount) { + assertEquals(0, df.filter(exprZero).count(), exprZero); + assertEquals(totalCount, df.filter(exprTotal).count(), exprTotal); + } + + @ParameterizedTest + @CsvSource(value = {"SIX,AVRO,CLUSTER", "CURRENT,AVRO,NONE", "CURRENT,AVRO,CLUSTER", "CURRENT,SPARK,NONE", "CURRENT,SPARK,CLUSTER"}) + void testCOWLogicalRepair(String tableVersion, String recordType, String operation) throws Exception { + timestampNTZCompatibility(() -> { + try { + String dirName = "trips_logical_types_json_cow_write"; + String dataPath = basePath + "/" + dirName; + java.nio.file.Path zipOutput = Paths.get(new URI(dataPath)); + HoodieTestUtils.extractZipToDirectory("logical-repair/" + dirName + ".zip", zipOutput, getClass()); + String tableBasePath = zipOutput.toString(); + + TypedProperties properties = new TypedProperties(); + String schemaPath = getClass().getClassLoader().getResource("logical-repair/schema.avsc").toURI().toString(); + properties.setProperty("hoodie.streamer.schemaprovider.source.schema.file", schemaPath); + properties.setProperty("hoodie.streamer.schemaprovider.target.schema.file", schemaPath); + String inputDataPath = getClass().getClassLoader().getResource("logical-repair/cow_write_updates/2").toURI().toString(); + properties.setProperty("hoodie.streamer.source.dfs.root", inputDataPath); + properties.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + properties.setProperty("hoodie.datasource.write.precombine.field", "timestamp"); + properties.setProperty("hoodie.datasource.write.partitionpath.field", "partition_path"); + properties.setProperty("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator"); + properties.setProperty("hoodie.cleaner.policy", "KEEP_LATEST_COMMITS"); + properties.setProperty("hoodie.compact.inline", "false"); + properties.setProperty("hoodie.metatata.enable", "true"); + properties.setProperty("hoodie.parquet.small.file.limit", "-1"); + properties.setProperty("hoodie.cleaner.commits.retained", "10"); + Option propt = Option.of(properties); + + new HoodieStreamer(prepCfgForCowLogicalRepair(tableBasePath, "456"), jsc, propt).sync(); + inputDataPath = getClass().getClassLoader().getResource("logical-repair/cow_write_updates/3").toURI().toString(); + propt.get().setProperty("hoodie.streamer.source.dfs.root", inputDataPath); + if ("CLUSTER".equals(operation)) { + propt.get().setProperty("hoodie.clustering.inline", "true"); + propt.get().setProperty("hoodie.clustering.inline.max.commits", "1"); + propt.get().setProperty("hoodie.clustering.plan.strategy.single.group.clustering.enabled", "true"); + propt.get().setProperty("hoodie.clustering.plan.strategy.sort.columns", "ts_millis,_row_key"); + } + new HoodieStreamer(prepCfgForCowLogicalRepair(tableBasePath, "789"), jsc, propt).sync(); + String prevTimezone = sparkSession.conf().get("spark.sql.session.timeZone"); + try { + sparkSession.conf().set("spark.sql.session.timeZone", "UTC"); + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "false"); + Dataset df = sparkSession.read().format("hudi").load(tableBasePath); + assertDataframe(df, 16, 16); + + if ("CLUSTER".equals(operation)) { + // after we cluster, the raw parquet should be correct + + // Validate raw parquet files + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(storage.getConf()) + .setBasePath(tableBasePath) + .build(); + + HoodieTimeline completedCommitsTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + Option latestInstant = completedCommitsTimeline.lastInstant(); + assertTrue(latestInstant.isPresent(), "No completed commits found"); + + List baseFilePaths = collectLatestBaseFilePaths(metaClient); + + assertEquals(4, baseFilePaths.size()); + + // Read raw parquet files + Dataset rawParquetDf = sparkSession.read().parquet(baseFilePaths.toArray(new String[0])); + assertDataframe(rawParquetDf, 15, 15); + } + } finally { + sparkSession.conf().set("spark.sql.session.timeZone", prevTimezone); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + @ParameterizedTest + @CsvSource(value = { + "SIX,AVRO,CLUSTER,AVRO", + "CURRENT,AVRO,NONE,AVRO", + "CURRENT,AVRO,CLUSTER,AVRO", + "CURRENT,AVRO,COMPACT,AVRO", + "CURRENT,AVRO,NONE,PARQUET", + "CURRENT,AVRO,CLUSTER,PARQUET", + "CURRENT,AVRO,COMPACT,PARQUET", + "CURRENT,SPARK,NONE,PARQUET", + "CURRENT,SPARK,CLUSTER,PARQUET", + "CURRENT,SPARK,COMPACT,PARQUET"}) + void testMORLogicalRepair(String tableVersion, String recordType, String operation, String logBlockType) throws Exception { + timestampNTZCompatibility(() -> { + try { + String tableSuffix; + String logFormatValue; + if ("AVRO".equals(logBlockType)) { + logFormatValue = "avro"; + tableSuffix = "avro_log"; + } else { + logFormatValue = "parquet"; + tableSuffix = "parquet_log"; + } + + String dirName = "trips_logical_types_json_mor_write_" + tableSuffix; + String dataPath = basePath + "/" + dirName; + java.nio.file.Path zipOutput = Paths.get(new URI(dataPath)); + HoodieTestUtils.extractZipToDirectory("logical-repair/" + dirName + ".zip", zipOutput, getClass()); + String tableBasePath = zipOutput.toString(); + + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(storage.getConf()) + .setBasePath(tableBasePath) + .build(); + + // validate no compaction and clustering instants present in the timeline + HoodieTimeline completedTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); + assertFalse(completedTimeline.getInstants().stream().anyMatch(i -> i.getAction().equals(HoodieTimeline.COMPACTION_ACTION))); + assertFalse(completedTimeline.getInstants().stream().anyMatch(i -> i.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION))); + + TypedProperties properties = new TypedProperties(); + String schemaPath = getClass().getClassLoader().getResource("logical-repair/schema.avsc").toURI().toString(); + properties.setProperty("hoodie.streamer.schemaprovider.source.schema.file", schemaPath); + properties.setProperty("hoodie.streamer.schemaprovider.target.schema.file", schemaPath); + String inputDataPath = getClass().getClassLoader().getResource("logical-repair/mor_write_updates/5").toURI().toString(); + properties.setProperty("hoodie.streamer.source.dfs.root", inputDataPath); + String mergerClass = getMergerClassForRecordType(recordType); + String tableVersionString = getTableVersionCode(tableVersion); + + properties.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + properties.setProperty("hoodie.datasource.write.precombine.field", "timestamp"); + properties.setProperty("hoodie.datasource.write.partitionpath.field", "partition_path"); + properties.setProperty("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator"); + properties.setProperty("hoodie.cleaner.policy", "KEEP_LATEST_COMMITS"); + properties.setProperty("hoodie.metatata.enable", "true"); + properties.setProperty("hoodie.parquet.small.file.limit", "-1"); + properties.setProperty("hoodie.cleaner.commits.retained", "10"); + properties.setProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), logFormatValue); + + boolean disableCompaction; + if ("COMPACT".equals(operation)) { + properties.setProperty("hoodie.compact.inline", "true"); + properties.setProperty("hoodie.compact.inline.max.delta.commits", "1"); + disableCompaction = false; + // validate that there are no completed compaction (commit) instants in timeline. + } else { + properties.setProperty("hoodie.compact.inline", "false"); + disableCompaction = true; + } + + if ("CLUSTER".equals(operation)) { + properties.setProperty("hoodie.clustering.inline", "true"); + properties.setProperty("hoodie.clustering.inline.max.commits", "1"); + properties.setProperty("hoodie.clustering.plan.strategy.single.group.clustering.enabled", "true"); + properties.setProperty("hoodie.clustering.plan.strategy.sort.columns", "ts_millis,_row_key"); + } + + Option propt = Option.of(properties); + + new HoodieStreamer(prepCfgForMorLogicalRepair(tableBasePath, dirName, "123", disableCompaction), jsc, propt).sync(); + + String prevTimezone = sparkSession.conf().get("spark.sql.session.timeZone"); + try { + if (!HoodieSparkUtils.gteqSpark3_5()) { + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "false"); + } + sparkSession.conf().set("spark.sql.session.timeZone", "UTC"); + Dataset df = sparkSession.read().format("hudi").load(tableBasePath); + + assertDataframe(df, 12, 14); + + metaClient = HoodieTableMetaClient.builder() + .setConf(storage.getConf()) + .setBasePath(tableBasePath) + .build(); + + if ("CLUSTER".equals(operation)) { + // after we cluster, the raw parquet should be correct + + // Validate raw parquet files + HoodieTimeline completedCommitsTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + Option latestInstant = completedCommitsTimeline.lastInstant(); + assertTrue(latestInstant.isPresent(), "No completed commits found"); + + List baseFilePaths = collectLatestBaseFilePaths(metaClient); + assertEquals(3, baseFilePaths.size()); + + // Read raw parquet files + Dataset rawParquetDf = sparkSession.read().parquet(baseFilePaths.toArray(new String[0])); + assertDataframe(rawParquetDf, 12, 14); + } else if ("COMPACT".equals(operation)) { + // after compaction some files should be ok + // Validate raw parquet files + HoodieTimeline completedCommitsTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + Option latestInstant = completedCommitsTimeline.lastInstant(); + assertTrue(latestInstant.isPresent(), "No completed commits found"); + + List baseFilePaths = collectLatestBaseFilePaths(metaClient); + assertEquals(7, baseFilePaths.size()); + + // Read raw parquet files + Dataset rawParquetDf = sparkSession.read().parquet(baseFilePaths.stream() + // only read the compacted ones, the others are still incorrect + .filter(path -> path.contains(latestInstant.get().getTimestamp())) + .toArray(String[]::new)); + assertDataframe(rawParquetDf, 8, 8); + } + } finally { + sparkSession.conf().set("spark.sql.session.timeZone", prevTimezone); + if (!HoodieSparkUtils.gteqSpark3_5()) { + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "true"); + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + public static void assertDataframe(Dataset df, int above, int below) { + List rows = df.collectAsList(); + assertEquals(above + below, rows.size()); + + for (Row row : rows) { + String val = row.getString(6); + int hash = val.hashCode(); + if ((hash & 1) == 0) { + assertEquals("2020-01-01T00:00:00.001Z", row.getTimestamp(15).toInstant().toString()); + assertEquals("2020-06-01T12:00:00.000001Z", row.getTimestamp(16).toInstant().toString()); + assertEquals("2015-05-20T12:34:56.001", row.get(17).toString()); + assertEquals("2017-07-07T07:07:07.000001", row.get(18).toString()); + } else { + assertEquals("2019-12-31T23:59:59.999Z", row.getTimestamp(15).toInstant().toString()); + assertEquals("2020-06-01T11:59:59.999999Z", row.getTimestamp(16).toInstant().toString()); + assertEquals("2015-05-20T12:34:55.999", row.get(17).toString()); + assertEquals("2017-07-07T07:07:06.999999", row.get(18).toString()); + } + } + + assertEquals(above, df.filter("ts_millis > timestamp('2020-01-01 00:00:00Z')").count()); + assertEquals(below, df.filter("ts_millis < timestamp('2020-01-01 00:00:00Z')").count()); + assertEquals(0, df.filter("ts_millis > timestamp('2020-01-01 00:00:00.001Z')").count()); + assertEquals(0, df.filter("ts_millis < timestamp('2019-12-31 23:59:59.999Z')").count()); + + assertEquals(above, df.filter("ts_micros > timestamp('2020-06-01 12:00:00Z')").count()); + assertEquals(below, df.filter("ts_micros < timestamp('2020-06-01 12:00:00Z')").count()); + assertEquals(0, df.filter("ts_micros > timestamp('2020-06-01 12:00:00.000001Z')").count()); + assertEquals(0, df.filter("ts_micros < timestamp('2020-06-01 11:59:59.999999Z')").count()); + + assertEquals(above, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()); + assertEquals(below, df.filter("local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)").count()); + + assertEquals(above, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()); + assertEquals(below, df.filter("local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)").count()); + } + + private List collectLatestBaseFilePaths(HoodieTableMetaClient metaClient) { + List baseFilePaths = new ArrayList<>(); + HoodieTableFileSystemView fsView = null; + try { + fsView = FileSystemViewManager.createInMemoryFileSystemView( + new HoodieLocalEngineContext(metaClient.getStorageConf()), + metaClient, HoodieMetadataConfig.newBuilder().enable(false).build()); + fsView.loadAllPartitions(); + final HoodieTableFileSystemView fileSystemView = fsView; + fsView.getPartitionNames().forEach(partitionName -> + fileSystemView.getLatestFileSlices(partitionName).forEach(fileSlice -> { + assertTrue(fileSlice.getLogFiles().collect(Collectors.toList()).isEmpty(), "File slice should not have log files"); + Option latestBaseFile = fileSlice.getBaseFile(); + assertTrue(latestBaseFile.isPresent(), "Base file should be present"); + baseFilePaths.add(latestBaseFile.get().getPath()); + })); + } finally { + if (fsView != null) { + fsView.close(); + } + } + return baseFilePaths; + } + + private String getMergerClassForRecordType(String recordType) { + switch (recordType) { + case "AVRO": + return HoodieAvroRecordMerger.class.getName(); + case "SPARK": + return HoodieSparkRecordMerger.class.getName(); + default: + throw new IllegalArgumentException("Invalid record type: " + recordType); + } + } + + private String getTableVersionCode(String tableVersion) { + switch (tableVersion) { + case "SIX": + return String.valueOf(HoodieTableVersion.SIX.versionCode()); + case "CURRENT": + return String.valueOf(HoodieTableVersion.current().versionCode()); + default: + throw new IllegalArgumentException("Invalid table version: " + tableVersion); + } + } + + private HoodieStreamer.Config prepCfgForCowLogicalRepair(String tableBasePath, + String ignoreCheckpoint) throws Exception { + HoodieStreamer.Config cfg = new HoodieStreamer.Config(); + cfg.targetBasePath = tableBasePath; + cfg.tableType = "COPY_ON_WRITE"; + cfg.targetTableName = "trips_logical_types_json_cow_write"; + cfg.sourceClassName = "org.apache.hudi.utilities.sources.JsonDFSSource"; + cfg.schemaProviderClassName = "org.apache.hudi.utilities.schema.FilebasedSchemaProvider"; + cfg.sourceOrderingField = "timestamp"; + cfg.ignoreCheckpoint = ignoreCheckpoint; + cfg.operation = WriteOperationType.UPSERT; + cfg.forceDisableCompaction = true; + return cfg; + } + + private HoodieStreamer.Config prepCfgForMorLogicalRepair(String tableBasePath, + String tableName, + String ignoreCheckpoint, + boolean disableCompaction) throws Exception { + HoodieStreamer.Config cfg = new HoodieStreamer.Config(); + cfg.targetBasePath = tableBasePath; + cfg.tableType = "MERGE_ON_READ"; + cfg.targetTableName = tableName; + cfg.sourceClassName = "org.apache.hudi.utilities.sources.JsonDFSSource"; + cfg.schemaProviderClassName = "org.apache.hudi.utilities.schema.FilebasedSchemaProvider"; + cfg.sourceOrderingField = "timestamp"; + cfg.ignoreCheckpoint = ignoreCheckpoint; + cfg.operation = WriteOperationType.UPSERT; + cfg.forceDisableCompaction = disableCompaction; + return cfg; + } + + private static Stream continuousModeArgs() { + return Stream.of( + Arguments.of("AVRO", "CURRENT"), + Arguments.of("SPARK", "CURRENT"), + Arguments.of("AVRO", "EIGHT"), + Arguments.of("SPARK", "EIGHT"), + Arguments.of("AVRO", "SIX") + ); + } + + private static Stream continuousModeMorArgs() { + return Stream.of( + Arguments.of("AVRO", "CURRENT"), + Arguments.of("AVRO", "EIGHT"), + Arguments.of("AVRO", "SIX") + ); + } + @Timeout(600) @ParameterizedTest @EnumSource(value = HoodieRecordType.class, names = {"AVRO", "SPARK"}) @@ -2302,7 +2811,9 @@ public void testCsvDFSSourceNoHeaderWithoutSchemaProviderAndWithTransformer() th testCsvDFSSource(false, '\t', false, Collections.singletonList(TripsWithDistanceTransformer.class.getName())); }, "Should error out when doing the transformation."); LOG.debug("Expected error during transformation", e); - assertTrue(e.getMessage().contains("cannot resolve 'begin_lat' given input columns:")); + // first version for Spark >= 3.3, the second one is for Spark < 3.3 + assertTrue(e.getMessage().contains("A column or function parameter with name `begin_lat` cannot be resolved. Did you mean one of the following?") + || e.getMessage().contains("cannot resolve 'begin_lat' given input columns:")); } @Test @@ -2890,7 +3401,7 @@ public static class TestSpecificPartitionTransformer implements Transformer { @Override public Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset rowDataset, TypedProperties properties) { - Dataset toReturn = rowDataset.filter("partition_path == '" + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + "'"); + Dataset toReturn = rowDataset.filter("partition_path == '" + org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + "'"); return toReturn; } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java index de21b33fff4e6..be0b2f78924b1 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionQuick.java @@ -59,25 +59,13 @@ public void teardown() throws Exception { protected static Stream testArgs() { Stream.Builder b = Stream.builder(); - //only testing row-writer enabled for now - for (Boolean rowWriterEnable : new Boolean[] {true}) { - for (Boolean nullForDeletedCols : new Boolean[] {false, true}) { - for (Boolean useKafkaSource : new Boolean[] {false, true}) { - for (Boolean addFilegroups : new Boolean[] {false, true}) { - for (Boolean multiLogFiles : new Boolean[] {false, true}) { - for (Boolean shouldCluster : new Boolean[] {false, true}) { - for (String tableType : new String[] {"COPY_ON_WRITE", "MERGE_ON_READ"}) { - if (!multiLogFiles || tableType.equals("MERGE_ON_READ")) { - b.add(Arguments.of(tableType, shouldCluster, false, rowWriterEnable, addFilegroups, multiLogFiles, useKafkaSource, nullForDeletedCols)); - } - } - } - b.add(Arguments.of("MERGE_ON_READ", false, true, rowWriterEnable, addFilegroups, multiLogFiles, useKafkaSource, nullForDeletedCols)); - } - } - } - } - } + b.add(Arguments.of("COPY_ON_WRITE", true, false, true, false, false, true, false, true)); + b.add(Arguments.of("COPY_ON_WRITE", true, false, true, false, false, true, true, false)); + b.add(Arguments.of("COPY_ON_WRITE", true, false, false, false, false, true, true, true)); + b.add(Arguments.of("MERGE_ON_READ", true, false, false, true, true, true, true, true)); + b.add(Arguments.of("MERGE_ON_READ", false, true, true, true, true, true, true, false)); + b.add(Arguments.of("MERGE_ON_READ", false, true, true, true, true, true, true, false)); + b.add(Arguments.of("MERGE_ON_READ", false, false, true, true, true, false, true, false)); return b.build(); } @@ -119,13 +107,14 @@ protected static Stream testParamsWithSchemaTransformer() { @ParameterizedTest @MethodSource("testArgs") public void testBase(String tableType, - Boolean shouldCluster, - Boolean shouldCompact, - Boolean rowWriterEnable, - Boolean addFilegroups, - Boolean multiLogFiles, - Boolean useKafkaSource, - Boolean allowNullForDeletedCols) throws Exception { + Boolean shouldCluster, + Boolean shouldCompact, + Boolean rowWriterEnable, + Boolean addFilegroups, + Boolean multiLogFiles, + Boolean useKafkaSource, + Boolean allowNullForDeletedCols, + Boolean useVectorization) throws Exception { this.tableType = tableType; this.shouldCluster = shouldCluster; this.shouldCompact = shouldCompact; @@ -141,6 +130,9 @@ public void testBase(String tableType, PARQUET_SOURCE_ROOT = basePath + "parquetFilesDfs" + ++testNum; tableBasePath = basePath + "test_parquet_table" + testNum; this.deltaStreamer = new HoodieDeltaStreamer(getDeltaStreamerConfig(allowNullForDeletedCols), jsc); + if (!useVectorization) { + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "false"); + } //first write String datapath = String.class.getResource("/data/schema-evolution/startTestEverything.json").getPath(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index 56d435ddf0f17..8611eb36c7416 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -29,6 +29,7 @@ import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.AvroSource; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Row; @@ -49,6 +50,9 @@ public abstract class AbstractBaseTestSource extends AvroSource { + public static String schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; + public static Schema avroSchema = HoodieTestDataGenerator.AVRO_SCHEMA; + private static final Logger LOG = LoggerFactory.getLogger(AbstractBaseTestSource.class); public static final int DEFAULT_PARTITION_NUM = 0; @@ -112,8 +116,8 @@ protected static Stream fetchNextBatch(TypedProperties props, int HoodieTestDataGenerator dataGenerator = dataGeneratorMap.get(partition); // generate `sourceLimit` number of upserts each time. - int numExistingKeys = dataGenerator.getNumExistingKeys(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); - LOG.info("NumExistingKeys=" + numExistingKeys); + int numExistingKeys = dataGenerator.getNumExistingKeys(schemaStr); + LOG.info("NumExistingKeys={}", numExistingKeys); int numUpdates = Math.min(numExistingKeys, sourceLimit / 2); int numInserts = sourceLimit - numUpdates; @@ -140,15 +144,15 @@ protected static Stream fetchNextBatch(TypedProperties props, int LOG.info("After adjustments => NumInserts=" + numInserts + ", NumUpdates=" + (numUpdates - 50) + ", NumDeletes=50, maxUniqueRecords=" + maxUniqueKeys); // if we generate update followed by deletes -> some keys in update batch might be picked up for deletes. Hence generating delete batch followed by updates - deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50).map(AbstractBaseTestSource::toGenericRecord); - updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50, schemaStr).map(AbstractBaseTestSource::toGenericRecord); + updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, schemaStr) .map(AbstractBaseTestSource::toGenericRecord); } else { - LOG.info("After adjustments => NumInserts=" + numInserts + ", NumUpdates=" + numUpdates + ", maxUniqueRecords=" + maxUniqueKeys); - updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + LOG.info("After adjustments => NumInserts={}, NumUpdates={}, maxUniqueRecords={}", numInserts, numUpdates, maxUniqueKeys); + updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates, schemaStr) .map(AbstractBaseTestSource::toGenericRecord); } - Stream insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + Stream insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, schemaStr, true) .map(AbstractBaseTestSource::toGenericRecord); if (Boolean.valueOf(props.getOrDefault("hoodie.test.source.generate.inserts", "false").toString())) { return insertStream; @@ -159,7 +163,7 @@ protected static Stream fetchNextBatch(TypedProperties props, int private static GenericRecord toGenericRecord(HoodieRecord hoodieRecord) { try { RawTripTestPayload payload = (RawTripTestPayload) hoodieRecord.getData(); - return (GenericRecord) payload.getRecordToInsert(HoodieTestDataGenerator.AVRO_SCHEMA); + return (GenericRecord) payload.getRecordToInsert(avroSchema); } catch (IOException e) { return null; } diff --git a/hudi-utilities/src/test/resources/logical-repair/README.md b/hudi-utilities/src/test/resources/logical-repair/README.md new file mode 100644 index 0000000000000..3aea5f2ae6906 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/README.md @@ -0,0 +1,88 @@ + + + +Test assets + +trips_logical_types_json_cow_write.zip: + +this table was created with two deltastreamer writes: + +write 0 with 0.15.0: +inserts to partition 1, partition 2, partition 3 + +write 1 with 0.15.0: +inserts to partition 3 + +this gives us a table with 3 partitions, partition 1 and 2 have 1 file each and partition 3 has 2. + +Then we provide updates in cow_write_updates: + +write 2 done in the test: +inserts to partition 3, partition 4 + +write 3 done in the test: +updates to partition 3 + +this gives us a final table: + +partition 1: +1 base file written with 0.15.0 +partition 2: +1 base file written with 0.15.0 +1 base file written with 1.1 +partition 3: +1 base file written with 1.1 that contains some 0.15.0 written records +1 base file written with 0.15.0 +1 base file written with 1.1 +partition 4: +1 base file written with 1.1 + + +trips_logical_types_json_mor_write_avro_log.zip/trips_logical_types_json_mor_write_parquet_log.zip +the two tables were created with the same steps, but the avro table uses avro log files and the parquet table uses parquet files + +write 0 with 0.15.0: +inserts to partition 1, 2, 3 + +write 1 with 0.15.0: +inserts to partition 3 + +write 2 with 0.15.0: +updates to 1 file in partition 3 and 1 file in partition 2 + +write 3 with 0.15.0: +inserts to partition 3 + +write 4 with 0.15.0 +inserts to partition 3 and updates to 1 file in partition 3 + +write 5 done in the tests: +updates to 2 files in partition 3 and inserts to partition 3 + +The final table will be + +partition 1: +fg1: base file with 0.15.0 +partition 2: +fg1: base file with 0.15.0 + log file with 0.15.0 +partition 3: +fg1: base file with 0.15.0 + log file with 0.15.0 + log file with 1.1 +fg2: base file with 0.15.0 + log file with 1.1 +fg3: base file with 1.1 +fg4: base file with 0.15 + log file with 0.15 +fg5: base file with 0.15 diff --git a/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json new file mode 100644 index 0000000000000..dd6e2a05616be --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json @@ -0,0 +1,6 @@ +{"timestamp": 1761335069636, "_row_key": "3f3ef947-c3e9-48a5-b08f-6cabbc6d6533", "partition_path": "2016/03/15", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.3017074196681322, "begin_lon": 0.5626109152945691, "end_lat": 0.5649785382157525, "end_lon": 0.6160079798524531, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2015/03/16"} +{"timestamp": 1761335069636, "_row_key": "d8725d53-826a-45d8-9b70-b812d06d9dd0", "partition_path": "2016/03/15", "trip_type": "UBERX", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.27641854803317645, "begin_lon": 0.31700440770954075, "end_lat": 0.16654733508021524, "end_lon": 0.3555821110759497, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761335069636, "_row_key": "1d4f0480-5300-4f68-8ebb-1ff70ff5c6ea", "partition_path": "2016/03/15", "trip_type": "UBERX", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.9074176919785227, "begin_lon": 0.4117236492462387, "end_lat": 0.6994811148788228, "end_lon": 0.3772709703853857, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/16"} +{"timestamp": 1761335069636, "_row_key": "a6816ca4-60c8-4bab-a77a-31fa2c000987", "partition_path": "2015/03/18", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.4444402031840541, "begin_lon": 0.33692826304653933, "end_lat": 0.3043284603831268, "end_lon": 0.11042503421042937, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761335069636, "_row_key": "b1555002-2fe1-4687-bb60-406a52f16bb5", "partition_path": "2015/03/18", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.43246488469100974, "begin_lon": 0.3648005645136184, "end_lat": 0.3781839595846225, "end_lon": 0.4638740649211893, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2015/03/16"} +{"timestamp": 1761335069636, "_row_key": "615c45b5-57ad-489d-aad8-6fe563b513f7", "partition_path": "2015/03/18", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.75916203985879, "begin_lon": 0.49855855157343465, "end_lat": 0.7432577319020379, "end_lon": 0.33072999799294, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/16"} diff --git a/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json new file mode 100644 index 0000000000000..105ce19370e16 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json @@ -0,0 +1,3 @@ +{"timestamp": 1761335069637, "_row_key": "fe818873-3af0-4fcd-90ea-f5ad0e7565fb", "partition_path": "2016/03/15", "trip_type": "BLACK", "rider": "rider-003", "driver": "driver-003", "begin_lat": 0.9122211080491403, "begin_lon": 0.23232697706220873, "end_lat": 0.8967870566670471, "end_lon": 0.05065495500664263, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2016/03/15"} +{"timestamp": 1761335069637, "_row_key": "0af34a1a-c231-4c6b-8a5c-50d8e36a0ff1", "partition_path": "2016/03/15", "trip_type": "BLACK", "rider": "rider-003", "driver": "driver-003", "begin_lat": 0.26483577112225265, "begin_lon": 0.26862954952340434, "end_lat": 0.2727902211619275, "end_lon": 0.9138712331657564, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2016/03/15"} +{"timestamp": 1761335069637, "_row_key": "f448d495-69cf-4d28-afa8-3af2459636ee", "partition_path": "2016/03/15", "trip_type": "UBERX", "rider": "rider-003", "driver": "driver-003", "begin_lat": 0.8873308149149347, "begin_lon": 0.358940823441969, "end_lat": 0.2646927323955117, "end_lon": 0.4091537968746116, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2016/03/15"} diff --git a/hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json b/hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json new file mode 100644 index 0000000000000..48109422242fe --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json @@ -0,0 +1,3 @@ +{"timestamp": 1761341703299, "_row_key": "092ed4ad-0e67-4df7-a051-c66ff30f08a7", "partition_path": "2015/03/17", "trip_type": "UBERX", "rider": "rider-005", "driver": "driver-005", "begin_lat": 0.20841305367042184, "begin_lon": 0.41269017191959156, "end_lat": 0.6266431410358951, "end_lon": 0.4514006891788409, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761341703299, "_row_key": "5affef5c-f36f-4374-80f3-e7c6d3c38d25", "partition_path": "2015/03/17", "trip_type": "BLACK", "rider": "rider-005", "driver": "driver-005", "begin_lat": 0.3287214805934826, "begin_lon": 0.4292459075453131, "end_lat": 0.8027879467022967, "end_lon": 0.07863000273562926, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761341703299, "_row_key": "5b22bcfd-e2a2-4cb9-b6c5-f643223c48c8", "partition_path": "2015/03/17", "trip_type": "UBERX", "rider": "rider-005", "driver": "driver-005", "begin_lat": 0.2218333443775823, "begin_lon": 0.441127835026775, "end_lat": 0.39946947397642374, "end_lon": 0.5064153585372088, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} diff --git a/hudi-utilities/src/test/resources/logical-repair/schema.avsc b/hudi-utilities/src/test/resources/logical-repair/schema.avsc new file mode 100644 index 0000000000000..6392fa92d7826 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/schema.avsc @@ -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. + */ +{ + "type" : "record", + "name" : "triprec", + "fields" : [ { + "name" : "timestamp", + "type" : "long" + }, { + "name" : "_row_key", + "type" : "string" + }, { + "name" : "partition_path", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "trip_type", + "type" : { + "type" : "enum", + "name" : "TripType", + "symbols" : [ "UNKNOWN", "UBERX", "BLACK" ], + "default" : "UNKNOWN" + } + }, { + "name" : "rider", + "type" : "string" + }, { + "name" : "driver", + "type" : "string" + }, { + "name" : "begin_lat", + "type" : "double" + }, { + "name" : "begin_lon", + "type" : "double" + }, { + "name" : "end_lat", + "type" : "double" + }, { + "name" : "end_lon", + "type" : "double" + }, { + "name" : "ts_millis", + "type" : { + "type" : "long", + "logicalType" : "timestamp-millis" + } + }, { + "name" : "ts_micros", + "type" : { + "type" : "long", + "logicalType" : "timestamp-micros" + } + }, { + "name" : "local_ts_millis", + "type" : { + "type" : "long", + "logicalType" : "local-timestamp-millis" + } + }, { + "name" : "local_ts_micros", + "type" : { + "type" : "long", + "logicalType" : "local-timestamp-micros" + } + }, { + "name" : "event_date", + "type" : { + "type" : "int", + "logicalType" : "date" + } + }, { + "name" : "dec_fixed_small", + "type" : { + "type" : "fixed", + "name" : "decFixedSmall", + "size" : 3, + "logicalType" : "decimal", + "precision" : 5, + "scale" : 2 + } + }, { + "name" : "dec_fixed_large", + "type" : { + "type" : "fixed", + "name" : "decFixedLarge", + "size" : 8, + "logicalType" : "decimal", + "precision" : 18, + "scale" : 9 + } + }, { + "name" : "_hoodie_is_deleted", + "type" : "boolean", + "default" : false + } ] +} diff --git a/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_cow_write.zip b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_cow_write.zip new file mode 100644 index 0000000000000000000000000000000000000000..901120035f1e11a78c0dbd4bb34654cba79bb02b GIT binary patch literal 61182 zcmd3NbzD`;xA;L?B$SjEq`SMMC8fK&yF(gjq`Ra`T3SlFySq!e;dhQ+z21AD_r34? zygz;ev(N0=vscZU*=wztkro4ocnkm~6g_3tZ-qZDkN|7|JuMv*jl3Kz06g#J)(GnM z*2vxo4gdym1_l5?fZ9p__H~#auhTO8FCoDF0)dYCKLCQ@Ue@#kNE|4W{}l)|wUMQz zp0U2Bm9~wYv7ND{g{HZ_owlB~oi??ujqXoYLG1pGRp4VFyhi|}h2%mI+bF<0m~T=m zOkO}z0N!Zr zIX!bZJ)N|bJuL8^mVW*9z+itrC2V(}L>1%_Jl|9o@NcfGH5$$$QOtx-)W4Vx{aass zCmI#l)hrVY3{=km0IX^#fb3t1re`!@(x#)=rea{AW1(WC*QTdpqoZe{V%66-&^6Fy zVxXtjrJ~iOqoSv$qG!Z z&qqc|1 zzHdiR0_dW3nS8nw4yZ0)^gjvYqSlikZ@?6Qw`}# zvS7y9h0zzfyT9;`byY+R`jA?fn^0-q`u)?jWZ?p=Duo<7<7L< zg`{DBaMEU>&edQLak27rJgKu^Wv)_#5c~dv$%gI`ZR$vnDV%I8#^%v%-~EhB{jf!x zn^4mD4s+dZzebk)vGZNk0aR(frX_~q(b)C=8E(y+=IQ;D z={W<_*Aem@8YX1)hzaLARgMpgtXuo`Eje7Sjp6f^y)F_~Cm(_Y2UG1g!#wvEj{?`r zMC&cf(YWO2S+B+^##xaZveak-vEGzN`-wLkYkbvQ|J;b~}trmqage3^0bviQC#WQmD) z8+B`wHu^gTl9uDb6eW{^SWL!G->J?M6_WcGB{Cch3m(QD;$F8S1~gnZeOl{U*d!jj z8=!LaD06XN%9W)^z1^ykqB%z>W|_Oydv{QlMtRS3D0qRbQfIzC3XK4pJ80qR8av_` zZ(P<*v1n$ERjgp!IBa088scDUS1k5^ara_Eg$k{7c#R&53N0&pAx6Q}6s?vpI8Ma; zE-tF!mY%j?VKSvbYjk36O-dovJ=?=fJL99%II_>|EKD)$izX<$VWxD~`#GvC%cw%# zO$ujNQgMk;Uh11v7MGf-Qk+=dh%#2M?Hqm9nQ?d!kM`)?+vkHJ+qrPnnQDvs$Y-0Zn{^U zHVkF(Xt2}J8Xra1LEU6O(7Vj4k0Lf41>=obCfRi4U3t2Y81wU73j4v&<1P+ekeF`; zSSrp8!TT(xSa$WnyC<{S#@6893W21=!_G1FX$ZO@({s&2rf!au znK*t#MEQbb9|Ev66uLz5oDXrw7YfM>?vVQ)SA(1<9k83#!bfo}n}{^>L7DC7u$1OB z)Z6xUR}_QW#--&rl=o!=;1eMBN)yu3#2vcuacNpNu1^IO_Ski@z zzSL>9yG`;b`GtJR0r6X!8ui@98d8<9`^!QlfU&GA?B__d&+Y?s==<*5epDRx7e)$b z9x7RCxAHWD%5EBaN%!e@@|BmT`Hd)^3Mxm3oUVuWh?7+DH5-Z3xiGyf{nP1c_qI~= zUzf8MgFS`~XEg;J?khG<_I+%pg}nxWwVmG*-*p}lbX3^^u86oM=0}@T}ATUy+Ch3 zOCd*##$oQ`%%^&$gc8}H6N-O8>Cg#@AKJr91DOB$UaAMh6w!#?2j_01P zu+}PcP1_g6@D}!xKxQ*{lWd9g_VglzmrTS2^Y@t8cg)atR8gCpWnbGvUL!s(un+U# zrlv_knrH9uc3ON#2c(vRqFA<*WHszP=Yx}Bjan(AjPrfVNe=cX(~=;+6gdzh%|G4kFe zMHSW|>ksmg_s+$fuK0zA*X08)ax>;t>Y5W$;|?eBQ;wlDeFETk6h_5D&0>UujvsK0 z$nkZRo-<3Ls(8tD+-Dr7YaYGCHFg)Lu-WJKRI zD^tg|Pr7a1Qz#5=V&0iU9=i$SA;ik4WS1GU1@McqSz_&OwM9zV4B*MxuO(0A60hIz z#Q;gA>%Q`EcZqY8UAeoq6UGY@et*)a?>4fLH04TC&^!Je4hD+CN*%2F<3I<9_m7oW zT80C_A7$4mjlSWX)DX#x_cU3P3*)xCjcE!HEsI~XtayFTOe~Cdeet?Osb^iov?%_A zhl~vJ$AOit46M?K)y)Z;GDZ#CI)`$dY3ebWm{C@l>$7gAHNvEkP2KsAQ=4g%Ta(5> zm=<*x-Oe~gF^ladGF#OhA&q?3bbmTP#Ya*{_s`Xv^J8aT3X`VYcEMF7VkEyFP>rD^$ zvX{)U*FD3jmFtnvu+_Km+E4KsK-s?{$=+_e*rkqWxA>H+qr=gg`mEj2jw^d>s>Po8 z*e}k>`~9m-tGrx+{rVJ*sMBOIVZxb6Bf*}ac>IS^?1@?HI0q)KxT(H@9MUWEU4j~xeCGnu!9pf0 zzvRz}Xnt$(U1Vzz!-J42>na*-`3jQMB$1n{(cmFWb9X3O1f@=I;e}DtYcmL6-MseO z$g*`Ds%_$2wvx69AXn5Vm!W;4i3VFK8krdTF}Z~J+SrG=4jSo6ZnfbutbZHTu!D#s z+N|BW!4%2!*EtWnSD~9s61M#eu>N$*BDSHb_Xcp;iGE^kUWroyD`|9d$Az-QLR6+^ zX2~W=GpzlFY4dYaN|PfPObq8WJc;Qqu3wuZk768aYMFj>FM$pfFfBpE=Qw;X{`G3* zmWYy@YU`cV#ZHM2*b=Rn0s5r?00yhA%);;;4v=K)SUMUM9AJp$$vO0zSNk&TyzgzSd<_P6 z>A{5sb}={zLPR(#UC!f!AgG9h;~g_L06yY+$1-B`3pfDpc~9`b>vQcYP%~h{ke-u% z44C$=x1rVALLk-3005%P0zglOEHRvL0qMTM-oifFb~?kRU^ehs4c~~Ip{qm%0G@~M z>VStLJOwodB=|sIfdN>pKRp7&PqLCgybF5lCn-CQ006w@{IYuRtqrhT^u?$jU{&-< z9~xLw>&pPaipDt)3ou#PL!pO|2Y~_)&UFFZ(|DA9NFj=@g{WEtKc)N^__tR3xGFgW&1ma$3n7)qHjZ5x#j`a=odO*pXoBLAl_ns ziDUujabB->Atl&`zDv>oTWjbuwgJlvo&W%@;d2$ODnFTkB%*gD6`dbO`1r$eGz8$l z=DTL{GA|K819;Wdq`%#tS>3P#n_8V>AcLp?rLi_-poWnycT)m*OLW}~fmQh6OobpV z=AGz03_yLp0|EF1S;B8(a;r>pY)n4Kl0xwgtB6vi-(K1kJ$hUm;UMlpyWzW&< z2Y<_|Vw4v!0}UX7f_`!ruB}pz3!eD>#kGihlkYt}9hCI?ZAhm9FbMbd$@^yhb};96 zo!|8ElqGct8lcC7s}6Pz=Y#m@YD=lwtwahm_c#a3S_1&5u$zC3-?!XLSH!^Xg=pJ< z#I;0E3;2+!z6S>P9Rr}teeT#7n@Ny15Ux#Ilh>vS;#EoD8oaPuN$-bse`_?U;zf-^ zYb+wE#%?3ik7Jlvln#Z%3cKe?^W16u=GDaRGw#kkxm+Ge)0(Q5EHnG>2aPIJ=7k&^ zr%fwgbC)(x%r8}4?LxLW{^8Aji52OPAsfMj2_WxdGLQ|eU3oJ2o_9!b+2Z5PExbql zyvfm+=zSe~;kaP|!}xTzO4Q^*e$8YO=lf1W?uxyyTQ+>G_dMRNx@ENW)+JXDV1I_& zK#{AK9B5Tz;OycA4*)3rHF8D$e-*m=84UbQ_)3iWwSktFmJW=4i)^Tr z&gmmpqo%m%`B=d6w1(p`a`0Y?Gepu%=K-Qj%d%aD4pAVrW*rf%K&R3CyYc$-Q3o>5 zuGel3+s!LY`1)^b(AzE6Cx*S=?K&{c!`~(j`4=e=d}Y$8h$yQO5F`kpdU<4P^N2?A zWyoOVCvModru_Fg^R(yLc3zE-@puNH%2dxOs9UlP!@Jt@yKWo(n>UtiXO9TqI7VK1 z&KyE{we9uSQtx_t@bK&qxd_QTHGi>F4Y-&pp1db*k;;76=& zM<>;5U)T)`{Q{c$Q$4W@r-zV*6O>uPHGJGX{==g;7_+)x8UShrDCX`;{K21cAQpQI zrR5zx@*nHK%{3JX7u@S=gd{~?71|MJMT0xe$3+vIAdXiB)FH& zU?e!{&7I?Noo3sy_jd`2TY*%yQr0nuItnnY0PvL8vF&?5D42P0gv#;I8Btz7$u!i5 z4$kVuF>G`So%>jw;t z>9o}^g$<^`Wtq*FQa4(H2zh(iL!Or9o$7G`w+!F!qJY>qu_k z?aUFWMukTYaGVEn;(|)KDc6O~-CVO_5nT027)~GGwao*v`o>YzAFJAcJ;3I9_=3|3@FQSUSsArJ4}Ud_gyFYQmLZXG=7+ta!U|uWvelz z@ot3`{UQ#dVrn1j*i}KUc-DH(0P0^BSpKp+!t#HyJivQcTXFzHb-GU@42>Cn>~P|?xrGcnTvpXuu`16LY=%M8GO*8lBF z1Du_jehT1g0;BJc4qj!HWJWoYd1S0){0GsfF)dL%>^ixcaoq4iBld)Z4`re`5+jc$ zbkLtF&ySx$;;P4-1rpTZY6fHln)S9)FC8PH@;NA@a+V{K)&=Epo)3uQMj0f^kft!- z-I4b9-8J3ZJzRJI{}wg|QqGDk^FpJA(S#?8gQEp|_%nEJc=bsw4NOLJoYWgiGC1jY z*m$OSJbSSC(GT1-&0mrR`x1;2N{A7lwlttd_S6nPpAM>;e%OtjpfWEqH6f+XC%4;x zD?ld~C!B6F9?c0GkU-6tU1`3P(dWM0Y>jl(%L=d=j`JG`2{HaqTvAM}VrSDJa#JC7 z+2xG7>AoD~JfFYRydbwS)>qVb0B4clakkFzip8?=cmw|G^Pn$bGfa3;LwbHS^YrkL z5}p_%jd|cWN#e7zbN?@TMP!j?MN&*J+27-w4MQH~Tsyg+lsi!-gmmh*?D{KZQZL}{ zio9ISmn&dikWRvgQCu4+ZC*LHmC|U`);e#j(d2k&L7Y45Zc2LpaDNab*|177ihJO; zARp`Eywr86%oaM)^+re2y>h!| zR*FrxGq-m^+GEi4+CuYc$Fy|OY575-{%Z?Qs&F>~>?;!|aiK^KmE40=$75J5 z6XyG`BL~gw`zvG=B<|~-;ask&JIju4R@YQmmX{rVlT>cEs;cK)b%reqXLxt>sJZ4G zFOH5XGJ_S1n$C_(IZv06A5MB)C2#wX!*AQ_bb_rtg!$c+ZCxAKVvWp1I(D#R@|9fG z)rE_yspmfyF_Vhb&;)nP?&#RJq~%o8&C2&r2rO3bZ+o>(%6xnr-dcpP;NB91GwNLG zY&ThZSdhqLEMqi#fR+2?pnW}(=n6Nl@0@p?3Vko(uG@-~@t9l*To9-w_<@ZEX z>q&0(*tM2c7e#Khj)RR14xJRe7kw^k7dR)>v$Kvjg{h6#iJPg{*UdtkmTq4+qG+!6 z#tzdK=c`>>#_BW?)iFy@ak1p0WT9?*Ti9D+P}76| zE^2W$DVPaJy^`j^bko*tuIJt7>_U%=$zZ95tBhPp&r$wejhp68vs5&8^OHFqfs>@Z zhOXLOZ@etWXh-)?J3FoxizjO*{c(!Ek{$=6n-YV$@=x>0up!#Q?HjnFuntlmB*~KT=hZbUo2sY<3^KHlFlXyj$H)>1cZxSI4Suq^|jJi0c*u&*QMe z*V^nLemUuO6MW<0F`mPsezbQj%df7=5RMk!e365Af2(TQN4>=Z-_-L3kHdcMa+L5) zch&Q#noE_U#x(!Hj=F_GzH!_3Vz)}hI&Dw>p# zF<$}_8p_}Sv)SbC4|OBv-QrSGlf1LMic&7xq?*j8&E@x})eC*n%Vhk1e7XruaI^bnEEg|5UFgyK~Og~JR^v3S ze9pam)OJCP*P*fjSp9f}Ht-7=B`%)I(i1j;=yZaQ<`^AWKQC#Mpf68Yjd%8Vx5lDd zd&!v1Cv&sVvo+&{VD}#r8hqLjZmDByTt}i-e9Uoma`L4zc^k*%s>w=v!3HLd#E**S ztYA4dAnYUmJ~tA=YgL40lMf%~>o*T44@8<;5Kt)vUr0h`xZ1Mnt3ELYV*$?fZjs_Nr7 za;DRVcFpOL1@plLJCkBs+ zvqx~=?azac_g2U|P_aq7*4j3rvNK(lO+AUG;f@{lSV&NP?JjkOI;dHCFVu|(YYZ-Xmh zKR4>!m>lPzC2myfZ;d|3QtYgLPl5eB=<(vYrNm1(|J=kf^E1arhP;n<;%M7?iqeF6 z5hPHZ^;1}Ajt@4W9yd`KzTV`^WpCmb=KIaxffF`^fPwlqL zMQ$9gU#O-)NwI#TImugBnnQv-R@gm&Lp6vC_G(xWfq8f#rTbK`Gb3HaViMkeS&p6E z`AoB^>CE0#v8s^_Hz+lBd$&jwLadriy~5eGppckF)+2SRvxMhtuP9hWRaJ4K)J#3b zL}PJ!rmiTL$<+SfFwD#^XRpw1*MDLmEcC4SGo6WiKC+fuO19*WC>&r3Mnx4EyaPtY z(EIJVfWKA{&K97c?9cS|Suv&kBfw{)UtjzA6j(_2<>dVt&|YmVkRkc`G|TA}4nW() zlr8yIH+WR+EqYeP80^odpGhrVLId8eMpAr>*bzs=e{3_(1O%Ag>iThT#gm%# z0{5|1mji`C69s#j-U5`+_vgR{t;@Ci&X4X_Z~$65Hm)y#UE+3>4~)OQg*7wJNB*PP z&mZE55x1M5LpP8tG%Dr4!(pX$g!nw%JE=zBV)s)_OP@9wKvB=nNBY0Oc?;Xh?*_&i z`0j|)7_S5J8{8$MRa06ZK-QijGGNzt!b>TF7}je2$_Nc$m2@%y5(U5}zX%4#lxbgx zha6G?3#{Z$J?P(F|7_haJn*gb{bF?XSrM=#a+JaRqw=k8QvKkvm8~Oi72sRKzL@*b z-Ft>WCH+?Umaqrf45%L|9FTx8D&ckeJ&^eSNLIce854(n*O?!+Kp}O4>w;vVrA8qJ zQi(s(XX`%2OHe(qo1_gv2;RbW`>XF;t ze?v)Xu>ohh>Apn#uwf=~#z|tofnm1B(LQ&BC^1nMM*^q6Uu<_KE}J8kO0|v=RTE8f zBH1F4Bxx2a32SwK`&@26lWnjs73*>1DNjb9J!%HX7ryiSGm;D1Zu;AelfX#C|F;__ zZCIgbT!1QN_xJEKkm&!bc*g%U{A{&@P7d?5p8lbCu$%+^K>cgJTvJyDwvDcai?>hT z7_!A1CjC%RD3|=lG81SlzH08x3s-&7M|vGQB%Y4RY|o0_@ri(;=io^!>(N3QTW0U6 zXq}@NlBBQQ=yIvgK9elY+X%&C9^gaqxW_ZVCOlAjn-Vwf0b`>n-6#_9woj(7CuL2( z;5cq~cHK7jmxc4%tra!m@t$^YP?_d!UrfR$FHg0X(lgGHd~ocZ_nDt+2fxj&e7l9^ zLSrD?7V1}w@TxM%6)j)j{MBYCPo8${qMvcXmdsv83sI=Q;tkTk@s2b<&fhU<*x&jq7_XlT0lQ?+X)>JJdoe?D;)Wwze_116F z)~aW_Ee&BO;?{Kd!B_RSbW$=@%O)DvmgZ!=nDAoc>Gtp;;dX>3A_{A?8*A9hOnfTt zb)s>Qr#-&3O{pT*p~|cvigG=y1Ao)zHHHpiyEhd#ew`3 zZ~LhKx@DYpLSzIR2mzEyet|&!&t1jF76xX6SuCVxw)|&wT(V-U$d>9zTUU3%{f5)p!{0~ zp967V=Okkz0ht5k-!jO^%tl8`{SOAqfxtoeZwP*6a9crn?oXyLfCvS1Hg7?zy`W6< z3sZlQ=y!yF=jx|qOMys1`8SCFI*Z3Yq47@!extO3X2$bh5PqM*sjX}*t@MGL?e%TH z2`v{07L@-6I#g!X1d9&I{X!Ar7<*Y=w;o|D<7H0ts}y2{4ELwH#tw?17F+K>dTMsZ z5t%;Q3XU)sNKRm5SCo?E6`=EDEjBz`sY_I!3|6mdNSI_fVN!Nact}v=+SrRVY}Zj4 z@9eX)t>KUpXE*f|c&1$%(V(@M$%i=X(u(TKTWkr-x`gwd8rIo6LO4IA6>rLuW3^L! zfUw4e2L8QJ<}=3Ep)b50bmu6p@ryHhZCMY&TZjiE8=RlpDqaOqwrjkwUCIi1PAwsZ zB@Nc8he;UiLJt6s%z0D3Pk_UJ!hK8hQX=leq*|j?p~BW9k86mD-ba?VJiv`{!ZZ-C zYQr0uah?2&@z(~kbhD(w7+xu5)P->RQh!*htY_1HA6wCul7er})COU+=!} z(e%VPCbjNb-_|L5#b;S_(L8-l;_7kPH0xGy^0CHeDsl}K(*n|?a_>!WW}j%pO(|k? zDs#P5tzXSS%E!5a4-%JyG#|2MF|?+I3#MN4S;}HRkqdPWx@+h%;4*DBA~I6zAVY0>pQpxp+b3_4NqV_^P8hrgdS z_(O+(9Ns7Zs#eKA)$7NQ$t7=kgO3#^IIs7#))s{;17H}-fBf`)I!*^Rf3;wi;=H+- z!k?u4eTqZ)l@vRDbE`imy5F3Q`F`F41c!UAXz&4OHdF##4E$f<{5seCDzBVh)0H+-7>)N90rvEgEGqN6XzNee z%v(<}hZrT0c^;#;hx@>|af%XqJQKzcd1Cjh-DBy@aSVGGUAU-&W?+=p3s1;{Rj5si z_3^$X2K=h^Gbr$iNIcV_GjiYecLwc=CoayArIYl%@eaghB?j;8H+Ghcq6{t>tnya) z81Ca`i!z9--q&OTvJu-W$Kp9!$6W=3vh>)+*^yXYi`u6bzUVzHz^rzx=IEK`t1;M(OU%F4L z9?90LvnLD5H`xuAUznw46FGlSow(vqOuZx^f}bz9m8q z)SZWtt<|t%1&dw5riF1SDe+0oEr5%r8&DpGunY^Uv3#I>-g2uQH>C;9qbndvDD5C( z$?%{$bIQGuH|l!fXNH7;V~_k9;gO#34B3VFmgI@*fS2}Ti0nOhQgV7V^=$+RXyXfg z4!QUP%d_2dv07S&p|_dTj=b?LZ1L+A-qXQ0Y~_sLm3hmeC)kDS-QKrRS1PsiJa=#> z%azOAY*yhKA3i)mqCm2@w4dR=^NYs*7)h4ayVGclwt|@i1x=U#Ylr;u)_sfvntk1Jjb_qVM2T6SxHlI7|r~B`GF7j{T{~Vxy|A@o? zGO(z%?VY}l&urj)@SE}Z{pj1@!C?Fy47HA>rJb#vjrJcS{y&EJ@?Ri=j!S(%i)Q;L zy&w#K`$Ybp?HB(B-N4vP-&pUz2mR;KFiuDeZ*Snoyai~p{BElHb@Kf-RsCk--B(h# zTwz6VPt)`|7Jt1Gfq2b2Um|C$STGZIo*@@oW|=2XGQgfQOTE$@ZvN;cV?uc%qOros zTi%OQS2vr|LCCUuT1;ouH;Jayy&P9@E0`8c!&NNoc*PPEYj2Qv!jWBWQzG0NzklN?q_hOygCA~;T~2ucZrMOl9Oz-^2YuXajLrJ>>G2!y zH*Z&lnxuU*QS^@zoFkeQ(-Cu}g)R_fOMF?eOP8)*p+Qbc`u5`#^vEw?FZGN#T(Tey zwfi{S%60_$^S>c+mY+yJeVP!P2&4G*sERC6sDkpf3sM|#!fG$f$SX<0bx1Q~>D#Tr zIj>vu>{^GH&Yz-ljJllZPdHGCl_W4Be3Bt7Si*4Doc)0wrf)Nf8 z#1)~(W%#H0;jJ37g(n5rJN7NTy@sS`I5bNV!f?m*$#R7Y# zO?)Nu){9-{)~!M&GV*}wC|G6c%>{!we;uNS7OAf@f)g7dHNa`}i-UY6#lhwPaM~<% zr%ijlrbkrCH)R*s&keOO(d=2d@??yRH|O%ixC@ff`5ol2I&D;;-&?Sclu}^)-ID9} zL!(Oep>Wg?rL#+CNk?J(0Yte1bNkP3c9+nT$&F!{xyEMuYI2wtNFH6795zQ3d+Q3` zL299cRa2OYaM9S8UYhquYC)lF-a=xOz>TPF<4QtrUzcMQgx+v?aP8`r^dm5)=Sks6 zTUa3&>^k@F3$lfqCxEG-V4~0jsvt106}IEWV;xO@7xOe_PJg0wXl6L`X|ZL>@zhz_u(=|L4IzN!wXQ@hB_IzL6oy-J#i1LuD_5 z1H{K|2XE6#ENLtc8W58Pe?M&?{W@*f>Kf^rYySrm(ErHK_;s52(|h{qXZ*&4`eCj8 zp7f``kgjc`Yh>)8ulFC&{?mX37Jdcw<1FxZP$265^}F$R`|wxq=O?Hiw(GwcbiZem z_ZLS00h0Ew3Ft!V>6_VU|J|+ohWR(5{u49ES^c$71^*56Ukvf@X6t{6`e#r7H?HiT zWPr^7VR*wz(B}x01^&lmeAl&K$@qsu4cg5K%76Q^{?NC5%F=C3{>ZSS+lmViMSp%0 zuh;~h3Il!IRbUL7=iey$)tvu5LEn1hH^l7ANr-{dAvInoQo34bLrUS!mPku0IjC|5 ziu6e`F=~n%ikWVEvb>u;l%r`hws5xG=wCbPs*8G=uP@^5JFI)+>zz&@bi!CGHW6lj zD<=YTv596}_GZ6@ao>WkcdL!RF46}2IPnl7947c921YgnI^Cl-vIm< z{!(WBFw~5pr`QpaR`l(9Mq8x=iRXG@Mqi^L^@FEO9W{ie@vpTO#Fn6Trv!u-T8wnudSXY;yYC;!uPdPkS(n9M|+w4}Yn+Qz%VeY{@trBo39~M`V9J+uip(M6B!V$`aYDE3lkxl!zp$&xoO@%;#lz-9Zon{-S z5K0||yU)Dp`D|=A(CifLwnjWH9|J7p{GU`zHgSk(s#j0t>j>0|4cJXW-ul!Vd;OR^z`!xZs<|4&^I-BPuCG zGGko4EaH}e;hy!!JKVf5F!_+I#Nn!dQHzkTWD!n7#go%7Z9e?_%isJ z5KN2uD^{?^3hh4!GpH9b=+G*zfNMsel_KVU>xF;qgn#RRd1Wz6EEc4uO+}=-N=Q9r zp`FUF{iXP?QlQOh*#v}m_hry_)n)K&->dUKG&1lX$d*@H7o`W%RCh1;eO+Z~Ti#ox z_`=R$k!&YZm~n6R0a-DOV^&*u@2P>d2LUqQgC{JZwi(iDdfLKn+n`4CXg{REoJ8;T zUBS@g{X$Y>Tj|~PAYU_!V7`nXim)FAx!8M|nx`kSDLEsNkjWH^`IOn}L*+g@UlT+G zFb)+vOAYB9Gu7mhut_OGOi68Lj(pLQIei5!@An?c4DRPJ%rR9`L=4j&ah|!aLm%Zs zY2=n%yjs)<5=r`$&n({5!2}o#5teM<)h}6V)`LO`wC_1Us^yynOuQN7oqnA+I8f;>ql9Z2|KSHnx!W!7XssX#_(4nelpz-rzq zu7S=N8>ameoZ-At9D5Y0yD5%hT{0V-ID1$)p~iUT8ZLwf2|>%mj`Ge;p$i$DU|(Nd zoX9D<6Xy7J+ou(NcCz{LNsnVywfMOuA*d2XBHPg>+?|V9?MQHqn4Q$m7TF2MsrujF zd0r-89P`CDKlY@Kn=cS*Zc=^oGN!PW%NI6>WX&j%BQZ-}QRvON)(h$r9C>-q=WqPI zG71*S(jteFNfJJWN|^JYJQ?vy+=5S6Uh;4ceup5k?%Mvf>mo48$UtN>|?D_ zqxe*_(RB3uOf6uJeI>y-NW^clW;`%atC*w*A{ZdPU=V$kLIUH}QhhgX=m1;xZg1+*@_96+B?#dj+=yLj>LgbEQ~m_ z_B{=s#_HKJxj9bSXq`EX`>!}JOUe9a!A@V@_BioKCrdn&d|5L5NyHW$HP|(zh8zoD zrBa(yG_b;}+p8m?W0m~9E+sLi*xNDTu90hMN&ENNK_wsJ^c)2Dk^90#yjzCZjZ;FH zhcVIJ(mku4ClZb2)9J|Q8HFOKbLbOYQj0IUF3n=Ql9{oz&L|wWDxfd@8vRE$EE^P4 zJKrJ?-VQ?bF+EeUj-#=9=Fs*k{_?{Lqn?|;8@%TuFI{3B)jja6aIk1i%6f%}YL*DS zh0}&-!A?GAww~!~9Fs%_+3Ix?M1ojxG)p=bn+zJu_4YI9u;vXONk$9Lc3^`JCM@ZP z2aXt0ysjKlJ73nTPBw-*G>xj}=o@8jg@KPQHkxB!OqZr?avL9voNaQwn6V000W)WQ z_a14mhJK~{;U!|qJCU3zXQTCI^!ul(PHL(LkZo-QAr1VDEZ~;eL=3{EH?&%O&pHqg zRB;HoKQH!nZN--PAG?xKT@JrRDXk;MEji66at?3~y)RbH738sch%qftFcu>krPP0! zhCb|Hs5-1mf(_pW{yxa0pen!IpowloA1+TnBjR0UBHXGQEEIJcb!h7Icf%^2z07Eu zw;udQRn-k9(Uko*TI-qWjh_`t{KoBO`@~JX-MMztbOhm@Jsxzrz}ZrLI1;$WmA6)l z_!ew)cFvJhEwnJumg^}kznGQz5-u9T5Y4KbP<03~F zZHHmtpf9j{mG$pV<}Uq(ot^lePB(47X}X?si43Tfiql&s!rh3=uA{nZ&J?U}o7~Ng z*MbIbMSVj?dA&)Q4e(SBfs}!StXWxae$p6KGm88?5urF?rz?g5(E4fK{sqhA^xWVWUa)Y-(2a^^ zQ7?_>SoA(WCszY-6P_>Scz1OX)OdF*hvVo*5of|t+p&9cyRD|IM_Iuz2jIJhv`Fu#Z)|wes&lnc>m=I$;Z1_$L3~5elwkOSK_)1h!g4K za>VZ{m)MfUL(RAB1s1a!DY3&0)iq4owoh0Q;+D_b*brfgoFOgR*@a6wxs6SU=%*%0 zL%=uOGspPC-6@W3}PWepp#7mh=o z3OLruldzNP0?<^CE$t6`kV0d%dTjkmZddath}@$`(W6g<#O%^dKk~z+?pDhzf4H09gV7+0fD-+g{xt&d ztCjsvf_{1{-)z<2Q1hoO{vOW2`#0N|`af9z0(lPay6gd>zyqlX1pkfE{VPlVaKOGX z@*AgNUU|_HlMTt^P*eNJ#_<`_$cILah4&E!BHK&^AKZ;Hd-gED>Q4$)qqhp)9h12z zWp1|sgX|Y=ai+F-i)3zA9!m~O6UoUZ*(!pKhbIj4CN|87d^KE){p}}N;s-VnILk14bG(eYN4Sof`+r*8AU1vzavD-*}T^hQgV`5l*=egzWFbcHa z&pk^f_cMiRn$)G+{m5#$ykQAQCWcw<83hZA`krZJzcA0{ATPLo{2oadgZvt2v=-G{ zghX7Cxr&3dYtk#eiz?k%1b&@LXoN84;Aw?$o$#E1y{=k4#|z5c=VH(4$a}2veV>LE ze5$VN3NfKK5Raf8FKbdgd@`!ew^@tgZ<7HVkWvt@vq54SO5sCId&%oTL%OD=F6CiP69Ta0 zQF8Y9 zR3-zJr$*vI5TVK@hvQ(R4k?ioEdU(+>@*XI|@{*;JshjRfvmWgvPBD1O@<#{d|tUwEX z&&Cs9oMDp9AltppTr7x~TC=rduFtRmxR6ju@hs8lj{AM(i!o%B`O?J+f&a(aI{;^z zX79sshaID1+fF*RI_cQ9ZQHgxwrzKej&0kW<7Q4;Gm&Fp}qEe*k2)K5%T%S5B@&9AK_0*Dq13Zk)J zvI*1n$Y71Gxd-{(QO?mY7{Z?&ySa6tBI|ZCSU83vqHPDpM!QtY%aUVru%V5j1gd&i z!n=b;gCjYBv{s)Sini6PIm#Al%uFu? zaq5vY>vrVx^okOIF;Gu-LF=ul%<`8^42s!~Z%REwkcqYrdP4DFx1Bzo&R#c3xZxCl zIrk@LFAjSlWRn+uU`7t{BO-fC(O5DP-uL#5Xs)=I~6|9h0SZ_>uU4*t3Giq(Py#6Iyys0-+S#$FH zA*^Y)gSqT)CVctl$mHebB{*Em9kzP`n}?Df-)bpd-I5J_dTL<~{B{640AHevG&28*pwh zahcs+l1D$|pXXpC)`tW>aWWuK{)`ZwXx_etDN| zkVNpi39rSy`dJw~LRVeIg#X$Y?Qz)3(TbV_h^eBpOsQbNelUJIg}_%moR3s%wI+`J zI7X`J&gBZAHeE^D0Qhv1rGVF{Ng1PE17msu?o_1|7SM}O%b15`sBXe`$dVIq2K=sm z4%h%p15C=(^D?^0kizf-#~!(%@eRea!TjlL*|B)JpUq-zIiy8BnfjZl8TWGVV?THd0=PjWFz z>pQ^QSDO+nK@I*##hp?0o8nga`kUh3`Kh?+|4`hrzbo#ZKPzsfuMFe=q_}G=-c&1W zuNC(SGfg47aj%33rQ=Ez&77&Xk}yp5__fRV2&-1&XRPHgu`7cfE;Ov4UEpPUGL#sR zQpjYL^>VYK)va#VQL%2HELUg2)kult?b$+66R|5Usiy|i5+=;zZ#<%E;GuFMk6L{O z9UH+qRVF$sD>ZgQ-iGUgSta-^4FjudM8F*F$I`^}V*Et~O$dePLx>q0OMVNO1#Uvb z=~7cCNF>okE`TM_wRyV85OsGt`9CzNDgq)ObdO|VV&n?2ySu5Sb1Onu4ktV<*c)Ir zz9o~vXn%k3uzsGhD^4csq0LIs7Ge2_gPFNCc9hb?BwVB*eb(|U(vX5VN-(kFId#xs zUYAP6>Icew1OeXP0#N= zQ5&^mM{8JxWyHB!JJRE`QY&fEipWxS`$c?WKD;fFew%WJj$GV`_OLfM#h&arrRJ0a z!a0#T{`fn+UNrJYeXTyp;2K^=_3Eh7#$;RGk+kS(pifhxy~o*PhSG)FD|7um{Eo+6 zV=fh;uckSTNejxRZ#d1lD|T?Z70;PR_AP}!$}CZlthL1~BDH-6itHIBvA+Bur>FsO znuyw+#6p($1QiV;fJT0aIOvAt%|*YA6LU0f*b{Dijyh31+ z-Aq@m4g)dlLM$T`7DBH^uI-mY{?4YRFG!IK~1W5g$D1apXA2S+eKIRnGH7*D5Vm}G)oyz+yzkjk5$XH1EfnAF?T2OI!s zDdaj~UjE>L@;fRf8%2un7eX&v6~PbUe$0#-t+~dHU(v{+2jgOA(jfX{kJZY&D7>i% zHh4G1tY?};LATPvl_<;&XY43yw0YAa zG8ynyO4_c&Lltop6K4Qqgyo8(w))9_Wx;5-#T}iQBZahi&-cK@60u+X)^xW?Jmtym z#aSwdc)7YPo;X!Nly#f|dt%=+CJ>BrosQWC{o%~d^E(Q*X@%?RIqPSkF}%+W0G3Vl z29l4QXcM@>I)N1!{_riIaGbyC_0$Ahrq5H~FL>!yG?QZ!-xc9G_a9)rxHedzzVyDJ z4#^$lkAgom6TJ7iNc1+tzFzirX~g%PD7$I)$=UD77&3wHa0v7-UDaD-2&`OG)vh&NQK& zarB*Xw^Qs!}GFIh>%h`d^ zE>TC~;>|gZma0RUQIqH6S&dhvAzL+*|Dj6G6^R6t+s@ z5pRTuN@prDFC6+D#?>rGS91$QEiY*w5{wr5uY0GuhtyGUY6G_kPb+#%6jF{tsk z=)H83DsesM@5miu8nGNs9_3K-RnWy2RP)_~OGa3lIO_B`cC3cO>~FpyHwZwC zR~}$G`5fOkS;q~C9KrIE2M(?6Ila^4Q*7i2maCORr){}Sfg{rcnh=1+~zYWnxa~@Ng*a(aaN&hv<;V<7q{yS~p7t((ZqWyZrf5s91 zTa1?a&j_Jki1^v`-#`;Tqga>imjVAXXyV^2$3MpRS3#_Q22I@TSDiqD`s+9J`3ChK z%kf|J+11x3ygJ4en zXXR-Tje;?QEirY9QX9s_k&8j*h+FX>c^*^=9_lSE_BHBaSwoD4{1{=en%@Z=LFc}A zHf)`5N@N`H0EC2JA|T-Pu+&&;Hwa8h`D=Ypo5P-d985OsZX1Y2WyBY|?Y4%cPr9E- z3b&Zz@BK2x+D8kQ*gAr4@-}Mb>4AOcl@=2O$i)bDFt%!v{33(aY;y^2$+NCNjnJV$ z4VlU6TP1XI1~-J0A(!(p!VN_%X>1D;)yaw%E}90*lb4p>mS3xuiD|GhHQtb~dQhd3 z)A+r#MF+6GlWYV1Gn>j-P@?=RT=jbf>}2Ge**vI*T1hXp7Z==qOSJY6NbH|C=wG9O z{(6J{5r+H&H1Uh=^MCJ9{J%Pr{|p5Cg^Yjf{-06FKccMfAO7n=>%T=M|AB1#ZIAu} z67mnpKkrWLK>Vts6Moyw7j4gkRW>%sii1FD}h`JTp#mWr8*N%P;Q)#&G6 z{Wgz3yHNG--8e}w{2IG}-ml4nkiw^Qiy)nVj*5^APD(@>y_yP6O3A_qp~r+CmHlnO z55G-Ooh(}pxi2rsF9DnoS11Y}pOh3F!UqgyJqE*ZrDQmL;l+V_hvkgH@VzL;-W&Jc zNGn7C=C>zah7Ydqxi#A}eV#t?0`Oeyb}IWL!LP#tP3%i(7<~{YGGx8GBQjnv>U?5+ zeEa}^@Tp5)O%xDjx&h&%7@lCj%ybK`9GL>7Vn|y+d=GptVSUYAmdQ7N|H7QF-_C~k zL!~qh*_LcDsXQCrS@2y}0C6NWeHye!p`51GqeZH!#2Za7KB)*uDIV%N&p@C7^ryL1 zBmKs4&?(2>yy~Ip1FlNR!Rz_q`jI{T1tt| zk%9cMUX7oVc8_1!{87eX-(+xap6QQ=p@#-msssVuCs%`5B&HrU-y+#*(vR6b0tGLO!sv`o2kCK`Et+WqFeF9(lUIa$B>V zoDXhhuPVOyRleYW0Cd)7`6Tk|N^HLg$qS|uj2($UytrW*)s41!mhDF^1tZ=+mAThe zzwOyND6u`CdcU7c7xy>+)=r9zC>Hz^zy8yKZEpY>4x{PW?%~me1$P|F7XTVK)i;Q{ ziZ=+LgD||pu%0Zn=3YFRp_-?fC+=0(u3#~xJu!Q5pgh+G3Opx&$Zt^(6?RNcfG_-f zBD+%ROBMEjlb_)BRc>Ted+O>E5CsD&O~iRvv>#3i$~hpJriW&`CT9mjNWe^h0i1KF zQb9N$@R^(MB{j2Lfh8zqA|^^hk;x55i_)eY@qKJgg|8!4Rs%*xAF+ zSea#KKWO1~pEHLB^A&H+j}Stycun1r`&>KEwKl-P0~66CC{+S!@uAyE45lOl(!lk zRp*E#5ecFfIbP5@UKZO@0WZg#@G96;PYosI~ILS%S- z{&+C=^TYUy$|H-55IIhnQx;*<54!h?GXxq6A|12iQ2z4ZW*NTyKOzZ>X&RO5Wh^(T zw)JttzSWioWPH`!Rc)%*3vC05w(oPMtLy{P$4)pYk&RlsWF8`J=+Dm&dTAzT#9mRo z4iGwOagq)gfQ&Yh!$W{urRWq7HvDi|yT&lNf)}F#zMO?5ad8W0K=p-jA*(*Y{ApE% zD*GAl!xVPZl{oTCeIFO8^=kEwhimAFRL3VH~I&+On z-$}X+dlMw$5kq!hCnvWq{r&f1X%@Z)HyLblgM>q%!!vLMGK4UGO|ca^tGeUBpkhj-*?S<7V<0=MS!#gM(1A!IXxZA1zY~ePO#zbrXkoSshrMV5Qa_{!&baF-aaGXoYW~fP=Fn4twq}$lxZ# ztAi+YwA`>O!W!K>$kli8IWJO{tS5;At?Gwmr%ncR>32XDF!ts7G@^KnM9jvC&nRDK z13TUnNX|tO7&YX*i)AX|SoeS??dvqB?CsA7H0RP4#FlDch-3227&FpK#jlF_2I=Dy z+I^J^QxZv~Hd}nK*J3+{<3c$KWhdaFy|pMqJdx64iDmOT2K5j>_K!E&N{Y56vOU*J z^W_{Gp$--h<3 z$!5sMxbb$@Rvzhp8eWJr7uh-ds(Nd?hwwxG#|WPpf}3GOnV(vy80x0xp|&p1Zf!v^ za;_mg3o?)1v0kR~6oeAp#3vKzjHDYjxiH}Fa-parFd>A^JLUofg7d`EE4MYHX+7%7 zXVXb)5?2`d9yOc_nC!KFC=`rhB9?Er@D>rdDJd%Z6Nf^UX?AVK6KZfOT83Q-r_Ic$ z>LM1w#x{#V865yYJN<*tc6{nJr-R^#NbA1+k=1Ol_L8M)tWu8dxH%r_rWPwUn*nd) zi2Xs{sw?&K*C4KD2$kq&fv`Jm0f72nJqO}8RFzgij&Kj_72h9B5+s7eWx zhhm3%!=#Owuwk2w4Z)6J;M$68t;5XDaoMs=F(2ib<6Dr>w_wtBI19&Whq~=(e z?N~+VS9%s3^Op_LSVKbNE3d6qxf#!4`PN^tiAE@yIV#K`$wWYNYR&~0=V#}e*U5-% zDdMe6`e3a3&Psz~l^EKWh!ha_$F<^?#dMHcZHM#6*TW3Rk5kf&?iq60hYQko6L9)f z|G3`xvG9~reL&T}iKDW(geB2ph{{8uUteyTUEL*$0c`LBMld2P)KO$$tmCgqv}R&x zD~}#Vag&#m?O#n-X?%$*LC0lVsnJ`PlnTR%KY8f54^4C$v9!PCEk=07u5b4&GGW}7 zRkfkvN|aP*aFHLI(pRk513g~o=x+dVCvj!NnJ*{S*0`FKy{E2558NMTe0U;^{<3Ua z@+5ue5MOzkotCfGT+S}TkQ+~obfX1$AuWGBiq;3UFFYv#gI^;dK9mwL1_g%^6Bgzq z^`U>1g8$SaGsAzVSQ4K~t4IwFDQiBpQv3^#Y9|i>krr?xfbiYMc~^=D$qNuIZ%WdX zH(m-J>u9AF<|+-V$P`#06f{IN-4zs6^&M?L7WYBm-4<2W+lG8sAS@1-+x_`Q&)+{h zbl~+qz5Ygn{s$vqK>u*w?%j|{WBdh2;qS)hKU;wRZ2Y~Wy3H&Nsp;P>Lk1Qq2Daa9 zz@O-b|6~KIK*%8u&VKX>`piz=sNLZeGV~F0lXZw(TEIb&_ka*NRg#p4y^9RcVI+nxr8O@IW?ry|2=I26H&@C&M>$&RYG z7NteF5S$LyhVVupB?ejpe?^rZxM-!!{xkbI!T9(pEbUTduM_aL8A9z!2|N)E+3`{44f2K<1di^*90FQng@2>rrFxJr~L-|Xf;TOo7q zLS^S*-S(YBNJrBoM{;v|+%kUN6Zf)-ug&66@r%X+U?41@YfFQG-j)p7j(?`WMMvd> zARR%X9sh{Ng&pW7fD2!bu{oiD7Ej?f(uoD69~@N9o2^HEe+e)cUMp)=u2ph~j@6Xq z4&yW}v_QsX2ngos^W>B}KgT0Nc8ZxzKnxmdA090}=nfNAyE`Vm82i*sLWv@CTtbO# zqIrmfpRU{dU4$gB0^suMV@0|R(D8e6OFl90I%Gc^t**DhLq`GDLO2vRAQ_c&3ZHNj zSFdQP9jXFW7yu4bU^7xl^13=kV0*|YK%m{&&#*x(tcIh4YzW7QvjAkKl(~LyEYJyx zP-Q1XDztXIOqP8Rm$oB0U~cXh@eu7h0zn~Pv*f^~Z9;l7Ku|#Oh@&_qBtgXx4$pLazltTT zq5&Fx#}gb>;nDDNEKPoXP(+HQS2raY;MFMhcP&`728!+E?KGF;ge|s2f;|;$i45|E z;O*|_1?O@nE*dHvPrpSRVtp7)&+gZx+T!Bcd>fUCkd%pg?NAwVt}om_&k%ARy#S@a zlkyXut=9SxHA`;zO`O04ZxSLYJgSQr^z#Mq0UOF3D9q+kjvUz{nWjqqt}pxq62;do zL{@iofi>nTp40Dm_4ps|?<+C!9#8v0E>RbL!0w--SjX6|1BUS@M4<_?+EH?$U55#fNBG=0o$AAC&f%-367C6M972;N;KOAs=XAr@_YHeG_d=|JIGx=rwYVW6PPaN(Ckfosh;VfT#@$9y#1MRvt*T)eu2Ku}3NhDyU`28d2+B z5N-b^HIu7@bHZ%A)DN()}^cLn1wD^*@~p+#dDObqJj z?=>0J#1*619h;?{J9R$mw0e8~(w96Nu%)T7nkKC=I?_1VqS?U>3%a+=FD1AWxD3lm zikSjExA3V#{l-D(8aT_rI{QvO-m|3L**y@noZe2rl&0?Q3Oa>i(awMAFf zrqWXL6bW{vlbGse)VM@P!$eRl_8NV1jJa*)0{@GH?npi-L9vDbXJPc})yu_G;Wh99 zx4D2jLD_KqbjZt_8gtLLi3*!KLf6OQ6hz@R$=8;%Sw0GN#IljR_MQ{j#)Lie%QYOx zx18A})}4`r3BFV)!;D>2WWip4#)_d7bgjjAY*4o6_rfz4DV2ESSop6U>7K z^I8S5v|tuB|H8Bo1(KwNbYb~%doXOu_K1zWP7|B2cSNF4-$7NAaob;C=14OOB~$k0CWZU?nj_mmcX3p=Ui6}=D)1ZwrowDk z_QoUKM0YCgJu7ED-lq3h4?(qN=E!^MSbgUvmA=6nVCIo?PkP$T`>lIh-3;?y3Ruwb z`I?a&t)&oP!-~>Oy0yI7qm>%p1_7F5KJ190sVelXtCg%S-iy%wUYbZa?&_+^MK03S z;-LwzP0b5V-Jm!Xb#s@wD> z7s}Q7O^V}_DsRXyjOGggq6p)L43EskIcwM!Atq^7E}TfI8kKf$VzuSWjfn*Y%Zp1m z(9S<#37^J<6=h9F52G=Ca6Q++9K;ntRyNvH3X;*0eJE~{fPY%g+|94?0Ci)9R!YJ|;4ynBCqC;DPyfRQ84R{(l9wgkRomIO*PK7@0}!);adSh(vS>^`H@yT6!* z%dpBAE*f4lC45$>qF(dOa-ZiUdf(4YynQ?ta(+Xh!m-yIlbIWP%zGS61M{oVukO%JEc$@qTm zXOGj=VeZX%*`8~tkQ;;5phdcUsVQj-WjYUv+@tl^xz*QMRr<@D2YH-T^G}P{t5vmo zogW~YL1{dGT!6E^u=>i=5srN;r`y-DOAiot&(L*Sb_X84UrH^BO}9AwqK(D~Lv_W9 zti!4BcCQ?fPkHx}@UpSg@U6kdKgJ!ATB(BaJgq)A2G#Adm-a2Dgug&znd!-*P1>P* ztKc}#y2!91v!b7qxPuJVExssOO{X06>T-yW*F zSk)23&N&>Grnc3-iEV5VC;>qv#}7_Sk3*$0?XSoAA-nD^ z&t-dOo*C@rm8XyHz9m6D4j}C@9!{p?#su}>ub!C8{a~G*b!d4Sv{sJxi7a5;btg$PeXqI11wHJy?6P@qW%XM5X80J>W zASF760?-*^BvDz^G=T43Uwtbj-v-iL?FDlxQ1)_e7G39-mECREpX>nz7fak^OM5r8 zZO5A!pRY~PxosA)nPzFYB7KQqg_`B4#_Bw4ZOsgtmo}kodtYYk)JWQ{dx~J&-`2w+ zJ@jL%-F(GmG$IS9<`P@YC0nCy1S9RYgV2xG*Y_scxFBxxc%iakGRw72m_piZwYu%< zU%E4KEXuAXU6EB@ay`xfm-OV$8}mC|%i1HX`tI>Q6M)X~M>OTl259P9e7s#TO)7CO zaw$I!H)86|Mj6-(9pEu;}3Cmv~#(arOHJ9M9ZLZ zgv#(w9BnG$C;oZHJBhc?4y9r0Xo_4RU4=y<52H)59E!9syCx?JbCL&HC9rK!v~mo( z^F||bw%UyhC@Qrtb9hJ$(QBSb6HM(;&Z$iObAX6G>=wEvn$Q2jz<P zywTb60)l<&{z3-dnQ2w<=WSIO9=ghIs|a}oU$S3v}BpaLvfutmkYo>_oUSTcv@T%ogPD) z7vM?4AzG7GDA1?=;Nob5!pc^0NV>svT=+mfZebq;((qJh{CcFpaYBS3@W{Qv@qh-I z_|;e?{JcH_A;Di*=et{_Ve%D5b;Awy^y&sa~X`Mm4?V#heU1lp#|*wSe1rFuR>0S`=v|Q z8VG)vY`TXSl$j0_kV~j5TLeVWXPJ9$r3(Nv7~m*AmbF<8q8&Vkvf$#$SIXv@P(*)% z^fV+04rs&i7N&<59!Q*hG*Jj2PRv2rx|o~L&zcmPmk>^Kmc_%`t;0g0cLFpPj+VR& z4{06`xrZj^Jt}KS!f1_M4 zjm)=apkkZq9y$z+IOtp>XTo2=vRB~Yo7qm*z$+XqA+>(Vpq(#bCF^DdOm8#*5dx4* zp>MW-kY7Z#&(06rNe(g~U))q-p>u8W`Sh>d%}!s&GL?Y3lG_#$1H`n9DStZna*+M) zc*rfJ{Dkig{w~}QOCmfxLeRT|pYA8CXB*)c*&9{E!}im`NAeNG4^gKd8@xvejs=V= z19T_CMwLJ!OSng*^?s8FBQHAi_vSdH009#G#_E0PQh-DD$ZI)x9gdc5g`I4LE5o#t zZ-h65IQhKq4dz3DFF=ZQ%cUlG_esF6wg(I*6D=Mj=HdQkt3Q9C=__O$MIKDwEPl!} z04+Z~`qWf%C(o&$h{m7=;E#JaF`gg0Ue05kR{WTdQCTIYZ8F#E*;*-~Kc zbY;H{9yfSq{V%9P67@`Ntr?e}Reo#g|0R)dLFNWK>X zt~@aNF8rdDwp7KUdkQTZI-rR#;gFa*uTp?BRfdOq!*~B4_}#ykd>!(4?fB{6cW?ae z-)pi*-2y)g%=QMT-4c8sjRq;TVInnK7fE&so^~y2ZH*Gq#dAM?@9jj??J=(XTO4kobM|= zI$PuBl;2C}r?*xbT}oTq34nGbVS2MO#lngnMi2;xv4qM7_0@pc1xmaex$^r7zNx44 zirv_OG)>9gduQay_(|YtdC1|G$9*}$^XuFcDZ7Lh}UdT@i~Y!AOdnVcHBt@P& zsWU{+iCOF>Grkeac*MkolFtJ*@&O8?$sPzr_!7>!7ewaKuO_zj#imIdoa#~80Rqpf z|L6%^6U8r2gpw1eGfeC=#QX_vhpQh-|5V2Z`r0zeQZuT-;i1K$?-uoWpf4UkJu}Hi zBe$1n5ZLv;CAIvs0d#1RxdIQhcJ|3JeKWSgnRb(6O)X)*Xn|dh{b@yJmtz^J8daJ) z1$nRi$r!|Rx@{PJ%&|fo;_TBIIfE10$5P};(yVJH%{`70S8|GBnVz;nysz+#By>9y zq%~BT(-CT#88yzj^1<58T_)1`q9?ax{S}RKDTl@Lo-1w zD~ql4Dg{&se>Vt~a|Q!@Z0T-R5(yDHwowpIs@EZrqj;Hxpc+veKn zc&8?0J`QyZCu9%kH0;+1@}K&{NDiX8GsQSh$&XSy_sx)P-0RaH11Q=(Jbm(Z#x88O z+}1y`7rPH>Nt7Re36xdVY>f)C(5bUfed%CIo+@zZEk<*j&rkWJgBBI%aP1OZun^$; z!KUBVQQbIk8%!LQD!#Qf(e`we1_sM%xzd0%!>xr}nf&2tDO8h5uthg8UnL#-q*gd{x9}MG#EM#v9c>Gqa=N+Z34#mImvB zgo33Q45p(+_54%K)$`?Z`7KnUQ5F*kz9uPbvz4pXJnx!-BPIILmf8jP#ru7eHP$^% z_p>XB1a5gCxYWF6;82-RRE??7Q9~R_|Kv7u*w{PV$#zKks5LJ>5Sq z0-UKYvQDB^GIlQP%$zMh9EDyy`#9H+viCb03qLkD(}ZEFZ#6!x2^xA@Bv)G(WKxIiMQh)i}J>QhOL zygW|+4w`)Wh|XINeAD-O%{q^S(M2kTbS2ouwW`M46B|}jU6uaqUSn*SKne>ISBwhG zDkA}Tp624jI_Ej8v09bQJpJ|oJdg4PG>)gZj;c}j`rG;V zau*AQ#oJ^-1|4@ii;?7!WGN1AQRvUv9%kA70?c#7FD8dS*gnVu*E1%`qtJwkLc8am zB)RQ;d@3;mOxr{^Gd6bZVcgpeoiddKtCeVOJgpwxPL=1nLg;rN9UQ+W^avnbVs~v@ zMIJOJ@MX>^s#?Qf%~dgfW!>+djRb9paO;rYQBTDzqZ}YU!CSL6sDqD8A8Fpb%;|Zq z!zLrQ5I(JFZ=`V0keEs$Dd|=cWl|Us$6yremzUCvejWcbK@h)^V+zT5`cOOneBSuA z$Ifs**l4_T?zBPXtsurkjSlZm6$7Z#v!-7P6{7yEjg}PRQ{%<;%Q? zdaS}}lIe`poS&cHXd4rWc30owmqnpn<)ak81Vcu|2DEw1QqAoq21kKoL)(CvF*f7R z)+s|3v8T@pi743?wTn}p{`S@SZL5KzP=QP(iC>slF>1M~FH!;xoy)IIzuoBCTe7Y$ zY_^l0pnkj-IVrbTWfCqEMZT-^DVuiUR;6}Kl<>WMB^*_mxQ11a9=m@G^-j0|-MYF} zc{Ot;I9rNNzL9<1VN=aBb53rLHd!7w&SUQI7b42V$*6fR+oF4xUbNFtTtM|{@9pW( zl|3wHO70la9XoyP`_l1QF0;FuZ7P`2j z#?9?m=^M3L!NKhUCkZxMwK5%|j8x`Qv`RhmDB*+^ugWU|@bL8A#V6(fN$sb6&)zxO zS|nzah)jK#Ci@OZx%P1a-(+bM4rwB*;+vlFv&@&=?2TH*m&DERd!Xwiu^mH}#gFr# zM~9fLy78J$77No}4!n(L=+w96+!g+=Lb&rO?MH4tDpgG znaGzOtpzL5RVPpk+;6S}K_<`(`5HMoL*Uvm@;C(?XQy zc6H{$(Ea{P!AEJD`9t35M5*OAW7B+3r=?53R^d_CY3LzfsM~bSK;F=;Vj<7;jP$x8 zE{EY{%Dp2R3RDt$rZwkJtp-=soO2jK4j*kVW`2Bm+a}^TOd<(-#T{E;ux0XHeAtUY z{E{YgWbR*g)qawsVaG#psN*rbD7_y+(_`DJTQTF=TG4plt*h-J-Syq^C5(Zu{e^SR z@MH`B{+R4(o(2PGvdKR*N>WOnOHoVH z(#z8_R?~ywi-RgA!hsS16UdftKQAwj#6*oO%c%m(X&H5bvWybdeIx9dje`2ETEagd z1FP5e$9nI5wtIg>|Dxa2{|jc|4>0gAa09CY@C zeE)TC|09lr?mepW{`=eR`iY4C4=4`VTX6deIRh^M5?lj;Zd_a=FGy&ZZwP5TBVoC4 z&>Vm2+s}y8XLY;c|GzfOA0ekdCi53zr{5>jsF|W>B<;*%3y1BlufrJ?3^^s}^~KCa z?7QXVSdX!Us{|-bx=Q1#>#D|k$=uA6L@2Cux;KM8)B@3`Y(^Y<_8CZq>dUcv>birH z4emFiu`eZJ6jmE-Rpnj34f7v7g9%$gg5Po6!hf*@|INAh`x^W)$X~3(?@WW%GI*bV z26hK1Oeu@M=PZhQub=T}w!y!rCHW@={4Z*eoG5%T1)@W0eWFN61Fg&Ei&TN)gW4>= z{^}x9GtQhuQ*9|o@p?xrtyosYsmK9pu=_%eE)(lEQK@^q%M)RHn= zZ&}v1INNNy&t#t0V93SQ~s-E`Wuz#pO)ztd)_}*T^xuPDd6k{d06d(i8OEGA{d|dF}b<` z>eO)+*j{Ljp&;U$D??6simp0U>$oe|?#^*@tp12Kq3C2vu)Cn>SA>ETeb|LX+3^b3 zmllh&{H2a}9D?Mn7BO;IF)qDB9V%8q1#YrXkpfb@QA-9uU?rML8BrgzHBRA-Sa0d!a9r&XbRKWs(VvA)vl9%0B>igHSwWCoIxj%rT5&zUPnM)&OW% ziG8^~9~4%7(!xXqkzbS@vb-OzhRInReJEJqQKplkVyF!Y&uJstAT5=_Q5%t^>JIkf z6efnRNFdhq$_k#gLW$iS9)`o<*Whj{8em>%d9(0|4&Fi?Z!E?eswCW|;aM)|*-Acp zo&~(T$0=3A9L(RMry(w_dH54g=x2ri3nj(X-)s2%vRcRA7UN&4b^IbB!0)FM2wb$k zwCGOnm^gqcnbu1Im;m5UqJQcd+HLYrzq4rm%xwI-?*CKqnLloXpE~3}=>6YiQb!Jv zh?*lU{)060Q_j0`_~Qrk{feTerDOhElIWiY|NmsF{PDlnu_lx#tKDOVx@!rUPLz4N$9rydp!L6p1{Ns@RWLMcC z(4m+i003at|55q<_fBgY=+Lv$>FQIlurbk6F*392Qn6|4(NgKtv+5f#=`zvju^RkF znfY1yo%z40{Qe)5?a2z1o9xa?=S3`YS+qFlwjWZ7mKd@UVr;FkYA9zVw*0XJJ&i+sT*#2-gI&-jm}C|!i=gs~b`sI4 z{4Xl#BB@Jf6nr*gv3n1hwi)kSnq@-;O+BReO#$og&~(Cbkljhh&L39}v)i-%^Jp^y7&`*}mh}(Y8w+ z_zlpe+1J(8VH}!Xfv#K}j>~6tbC19$k^}WYb!}4kUFgVBJNUNoA=*~r2?6F@dT=p$ z9<~jCcTvCe45#ilTBBXtyvoF=cQ~+mxfoj@kbK%|{A62DAq&o({IK@P6WsqIS4J&k zi73q4Vo;}6iG0{j|6>EM;;BQkojllJmTf8)wXd9UIc>`k;}-NsN9CrT{Ry@&`a?ntuaPRqPnP2y6Lz{?H~y$`7LE5TiYUwWj|TC1&yBtnF!E0%pFCVG zU!PNFQPv)p&$~(_M=sPK?!OSM$cE&%>KJ{#9)yC`WO~}INwG+_tGt*j*b6}}ai}m~ zWz6EWp3yHe7|oJROs_T>jPOKh=N58VZ87iKc*9ItyJqemQTETA_Oi z1wJbd!@$OI>y|Q@!LB};S9x9_gLLL1>%+fTEuOK$^FXn@!7asn!99$Zi6Q9=!iH$^J2*D z>C8;mgt^?=T&T1tY1e8lnNp}Ax1^MQ7&)hVb6S2-o{uD%GG^(R5knoNL9#rRJyz7P zxKemV)jhU{+cD!kjZ@k-Tau47mh-_N1GC9OdY$8sT+^Aprl#)YK8z0ymPBzn#zqi@ zK;k?%%MWEav%|>MSFs!)bf8brz142x2AAFv5Qi@D^6up&jlvag?$FXOTCN|{Xe^zKwoox%otK_8DHnHFvOBhfbEsJ48jR{sJss|nJ@2h}XNHzSgqY3I z8?MS@#qOxf+m3y*aMoI!_sf`uub6R2Dh{FCwi>7#DBt3@&KV*z_u(s|6mJj{j@{0( zOz$MFSKC}|XEJ6YAq-X|PpSnUVlAy#tJYJ#xH6Y7KknAGth(%78{&*LNj#mWJiWc- zEP%G8pek@*96U)ucSveXY09V9Zl7_l(!VaJG8HxDM{@(YHe zm}ZlxTk!XHFzPm++Sl+}*pIwNVLzdlSYV&sH4=7=Y0(O}#T9(sJQ zw~XAJ>O%cU6NKD{o4Cf%(bgC=?)`w{Dq(3V7VDInZ7cYrkc;I-=MHLZ#c--HU)}u< znL*gR3Us!8KTG{mPRVa{&9A>z-tEFU5NE}>_gj~R3*6O!tBXv-Pz^J zl38ACJImFrRQ-InGuKP>C5QO27BhyLW6SBY0QK#8IWo?{%lHT1U~F_O38#@yu&Yn;(QIpXX7=j{43zo-USZ>gPS)-e1#T`!v1o ziVthPz5q7OW6W8WQfNE~&?fC?{pj+WdbS+Rq7jEZYcNl8UR@Q&b{$1|?Tf`pyPDio zeyglYFD2*JKU`wgc31+gFnbF$AO1d!atk*ySrXuOnOTH0+y}W=YBxgWOphjUJ+ZIV zty}K?*yc}=;WATynRkZutg}X~df%|*S!Z^3U7^`f#-Y09WOtWoewzPW&yumKz0`3x ztEs|Q=UB>0W@7$w#-fn^dYhU^`9NuD+)ish&e?&5qWfj05b0?}@zH1L!jcQz9TEha zvv!HSl^5<7mAP!-gvzUehVxYajp!j&5{+49!lO`qkssg#%SvG@4b9hS?}-8}0BQb6 zI4>r0ueEu}X1ljxntBVxl^mtG7Ac&Td!i+S(KcMxJZ_6OBvLMyefSV=lHz;EjHa{B za;CaO$`+4Zqm9FUgOyJ$&xV<5bw@u!fe{}io(gy#e(<|ec_t}g%fBi+zBJVptqTbr z^1T6z50xSM#$sNSC1!&-C-zU~VT~id>P!xJ+AkHpRF%so(btKRjc6z^ z@j_QH>^VFXne69SE{$il@RUJ#qAw7|a-KbO)NM(@j%u(UQm#}ZKk%6zl$kCmc3@iT z^ez^_ZF65Tg1nue2jV_o^NbyNXX!|4ry49;Q+Y_mYTAUj@M6luff_{ublMI&7hz+_T+@H^wzGGNW9JY%O{sHI;b<3nkR47t z-{08yVG*fV{r_5f3V^7Vt``-MH0TBa0b%JBK`=lX1*AdJpc|#JrI8Q?kyN@{y1P51 zJER-_dlyAkAK?4HH}9>x_j2#)ITL5*4AT}^xd*lpuEsyn@WNNQeL{!ij-?+?SRayDHlup3D&6PQ ziA+&cuXHn-8MJSs+qKV%Io5$w(d-wJ{r@#F!JoKCHLmn zVgfD6Invuym(avy7hGq1{1ESd-$q^5cz&giuzW|`^+X>jfr1aUb9AEF5Pr?B^v7&D zI*-}<$(+2z6jGce1%r0Hi(k47uh4PFg#-}v6(FN2#Y^W$d6p}OS*B`gFQ&ZZO-4yW zEo9)l)KYCWY|_TP8ls7vXfG~|FoPkH_zVwwvUQ;cUyJX}b{KwyK6lzJVr->_pc(nZ zPaZgv%RCF#dc?e9UM|a}gI@ye+|KdjE9kpv)zh$!^-`1YyzFjyWozZHFx2t-bxp?1 zlHt1L>x>z5J>pGDul4u<{G^Q+%5%mRS&YJ@Ms8X!Dax*?$alp#3J8;jI$A`uaa*3& zY`tkGjL>UCF`$`uw<0Rm#C%Ij;w6PHiCd5?M?R>(dO?D?xm|{QKsPf()7eeO2;~}vsuA5~!pyoGjNiUq zPQa%0N=VAMF;iQ2?(Nb?v2vkQ%Elr_v{2cd&wZ3e$oW`=9Z3b|A#S0Tw^wVNuVZ;C zX1Lkb9h(y=a=8Xx%n>d3knOWO1~e8BD6gQ~V;WDbYl|-{=yM&c59EC?rl+K?@tv;Y;() zVgeHoz(M22kGx4BXN!lmflPUjCbS@ADkKgR#n*4n+|YD`Jq!FpZm;qh6(Yhy-3d%w z?3eSeQVJlB=t6AZo8(yYJ(o+^={pgKDf?nkAOd&$3C=-~WV5I5C;m7*zDYTo_&I{3 zR&kOFgzlNm=sqCeM}vIxTM~$;U}hlpg!2^o=5=qQV2zLyDA)8bA>5CaDqznFj1DC7 zfDgMjJ3$(RCw5lMA@f7D7a&VmoQeC;1Vm#*EFn>E%z_{ra^7Uys1O1d<Y zd)#*+cQbj1BXHJ$01{5jSqp=JPPJ0vv{ORV<^|>6#e_gS%edJ+cFR3itv0Dw){ zX8-{c^Qz4P6MK%ar6D*FfrVvxy%My3X$cPCPJyoj>>WsjWY36eY6*oZ6+$Z}#4*#j zR4zTV8Q>Z$O9}5`Du!JMTD~}eo(Ik*+C9pUPaih4kv$0qZIO`7fP=+h#oZN4=EqNn zHm9Kuz$ZSr$}aIoobSxSuR7A&|2* z&;W;=1U%mahx|bC49#q~a1WB8feg_migzl642DxV$}$7C2K>>yzk&%uUfbvcUYzb9 z0lvr$&B=kZkL9W^z^(m4@TM9Y>)B^vfEYlI_664m!8gD%;iz}ciJ=JX-QJ}mWj`Q3 zpt$4GfZ!nH81t{lApvh@h=2=6Z2916!0i3wnY4#I0XiWYWc6yNAXKOZs13-(3xN1; z>8Jp+^$_OoP0NG)zypx^kU3!aCGZD^6?Ozj)YF{jz=fmr_czK*mxe5pxV?dV9FD^S zg^qyI8jN92($@LH2n8;(2g+C>C|o=+cFA+Kd$6DzRJ@;o{2hRCkmi^@e1Yi4sUQnp z{wU-pO+Ho)4ix(b~Lm$0q36fieoHTueCWOrB2>r=A;M6?pQfN z;G_&rj6opVaTfu+6#`)Ogh0m8ERi7LS1tJ)u#|wB=(yv*^-{sRl$m*R>`s`#4}_P#B5v(kVEiV-V20C@QU&=@O9DyulnRTS70r0@1zUSTY%M@ zADqk!2fAY|mojvcy#Q4RC`|>syngI*>cV&YJ%*BzR@s62j`{hD=WKH9)3S$jAU1*Q zie%Ll`yxtT5CP;BT6-ZC0vid$ak?lAto`AyZ&;~5jL&H)`nlv?fovtVx_)DSUtU+tpk`U!7GGZaNpswnol$H= zD!5UxMHd6N37REgz;pkW8Px%idwtE%ZRFx^vsy(Dsc-O@0`%k*czQL z%(Ai~x=&N0X=cl6KD#ua2Ry#W1n%z!I)u100@)fYXIBfqiH1 z0$2Bd4>%)waH4H5ozG!W$wTJY{w6BPnas)}ZES%8L@AkBM?~xDYik)$Dr+Rt5a%%j zJ1T(xdMb*1&&2*vjnwBNjJ#EM@~Rm-%B;5R z8>3LzKK80*uR7n6tjE5qbM8!JmHIB~j;K`th`+H@yjI{WFeSyV+DR7|$?kGsR!L@(q<@O7OZj6I$} zc4t+avWezEoQ@2}s`9<}NK&DjcYsZ+HmC+we~_(iG2ba^S8}o*x$(k7;X=*P6bBpc zp}xu9C>+8aih9B6aFRu5jX?))KJ~c{dv0RYZ54;=+6vnc#1}nlH4d95_E?$ z`sH3l zUvZw6%9Qp=6h5QMztxgKSZ)s~==Hp?li3Qu8Kkij+ZH7B zl++IwY6#D)t^$c2C|rWF^HDzv?=t0BoK}^xswPm>&J*qOS?g4c`B=QR$p`f0wG*NS zlm*Y)t()sUPY+clP41bk^ylP!P^%zX;TqJqUlH*NX5ZlloV$vou| z(|h#E@5%ZixvSHz8K|0w77+$UGDgZfjeqjLw90pP?rC|D{^L-!$((4@v&sXdzTZmT zmrq!fxnpm-wcpn$`OMHR<-S*`VdI$zxW)j;^J!-U}!KmKN?MmnLs z?qtWXS%^y(`;tg;$Dm31oD5-bbczH@S!>!iP3uKo){keO=p({P>D%AUEVVf_ z=?&LHrlzE?YPr=iMmU?}s?2(x9;LrAD*RFH`M@{CU2am^gq+@%rd-mnUMxyed^P_3 z-OwbY4v7EZ%_kO1b_#&h}s%IFzlZ%*u+w#bIkK5>tz;GmB-k?d>C#-Zqr6Kp6THgFUq#Kf zy)1L#kM9^B%K03u7+y!|y*Pi9m3sa}(XJ~V31TDJ@XF4~#$c<5m5$@j!V-vXvCvG` zP+{V4p4U=>zG_o?K*0E&iQ}OjC-pSU-T9mQbRDBu%n#0~eIpSRo8-HVJzq*Y_)m-bDP^kw?Trw_PFFdBmu zV?v>Qh^y4nX4cl#)#ZV*v1n>R*_qjNpc-5}Tu>f%HVqvub{%alW_DO)gM|&s0w3Af zi;)5ARZJ{&%=ca8Au-FJu97=Vr_c}=gUzxMI0D#qha_*x%T`W?agNJ(G_Zum&&q6L z%EOF3_ip}!e0ka?TSS?aF%HxdZ@1s9)u=^qca7(9@va$EyHSST!EJkZZ9#pbova5r zRHV?NJMioFV#ArLm;`-(@@f+0`JM9=ix|1Aid?6A+E*sC zACFD0Q%jx~^FBjfoxx^7QW7}__&a<3!NoW9I*G5DvTPDJ z9>3Kk>nBT&4D%RV4t1dwy$~j)8wh7znucSP#FXSweKiTd5JSS*A&&Ff_ zuE!*8Bdy}mm}RQ-c8L}xflKw*Pr*06R3D-GwtTPOgl=Ztn~saQzN@Ef?%e!tGu}Un z#B%JphH3L$&n~t;?vy>8yHlt*=KZubGJEk{nB<*$GE(<#t{VyhOBs}k3v=3xMuI4x zyX$k3CLHcJc3pSnGh3L*k2D81V%u2{FP#~%lEZF;D$ zJ}J$(PS`j%Hlg^PvzTgs`!h~+7HLK28oXSklL^I}inv-O27|?`LgSiwqINyq^gfA^ zxoxXNEy znKwXNedD!!S%x}I$OkTtvXX&s@&IQLLskS-=lu#zxWo9X(s7gwFAsc7uu8`nB8JBz0u8W)P=UY4wGmy5RS-$?? z(SP%L|e1=G6&2 z#Tz#j<2{xU;%8ra$lso&df1gJ;?&^x)&AQV5AD(6z+LUSK%Tb&JZ6f$BSN>K?pXJ0 z&YImwBX?DkBY%eRmXHu#?nN|V+9L5PPjU2TsVbZ|g#0SdON%!I8F2OJOD~M1tZNEJ zt+?L1ruE%O&ce4+pM!8vyy>)-6m8I1k{8{r4GrI|wXWxVA_`xRCRbUb;ThDFOSGnt z{i=vhUp73*%w5x!`F040`NEvL%L3Dr2kZF3!*5kcjb!EOZg z&OqO7O+K$tF?U94O^f@L+anW2!wm8&K3tDdhAe3Iqp;z|fGfHlbvi5W%nhT;9++gu zNVF-5D63Jozl$~WP^4+k47&WbQ@c%3PcPa6H>O#eBQqBFzHbY(BqiOzppFeQAXTf> z$YFvg_5NyPQ^vC%YU9hZo~ z;dk#X3UHzoBX^VVEtSO|JAa<2<#HKpjV`+MwDC%xG+&fBRopbBXiE1;RXPeU12HC5BWvG_T;RG+;U<{^F71 zSgHc_a!gbgU5-SuHp@G6(XzO8FV>NIXM~LxkJ7ZLd*sbzb%m9lfnKql_u#TK?;XaA?+M~c zF;{CWwZXeGsyY|7J;FO@m#tU#Vd2iV$9m?aI2k0|v~?)La_;nS zFdONyu_Vg4>bI*tx;%Q)s;=YmM8nRiuG*$hXOqs;JhkC$Yvbkw%PV|~@kTF4#l6}d z%9!yo;Z5DA-L-HjhjOuFry@(LcLLGlDI(GK9mVA;)S%fK;~VRtc`;?Q^jvf0Sf5q+ zl|OKq2=#_f^j5#E$ttSve(6zzfyZ$t5I<+Us?KaWhAnc^vJD^o(SXz3SEi4o3A&i5 z*3>+o*zjF%Gx?6V^X9I0b#AVz?-t9`f3~_|GP5qjcXK|J*5^bYB1!4ZTul4j`7vTW z-T<1un8_~jR75P7*)7d2MdazO6B;f430!?<8GM;DrIIs#pQGovB2!w#|a&4y4vMn*)rU<)M~it9-#TqVCx>E!{Sc# zMzMqZ8&wtlsTI9aqHBdEl5QBIOhg>TXH=GHY^Y`s^%7LrRCYh+3i_vL==UmZ3f4}} z+4Ig)+EBf8u*zTY*x*y1F3x*vxpKZH?@as7YSwb_rBe5xhLs+~k%X7duY(AQ#I+>{ z=q;FHvldToV&3l^UyYp0AR2X5m!o#j^cbEm8cPq|-C9KHbuBZ$UqM@ovv%RxaDjVuZ&T%RJo#yZg0qMpqZ& z?Y^}!y&t`j+MRdTcDO=^Yc@mJbmO_@Y)_Z*3<;$geoC6XO-p3+ZutyX-O#-2E%AYnm7G)g1;9T9fy(S0D!$t9>Yvhto=WaKCF!jJGSWS`YCVc0++te`V zb*+;|r}_y>zl*=&-K&9k43pApaEw4mY+Jqd(En=yH0pq_4L__Y3ottcz7lBO4}RUItZ-^ZfH=`mK-f?YTEM4 zX}v3hSw-=e;ufSl!kq7`e85w;zMZ6GT=ztRMkqa?vX@`Ucq2R4ePS`8@d0JR3#;nk zf*hyC6~4`3-)f-_i)yA?oHks03#IZ=e@b_kt_@k|1!2V(4ZFQVXP+sb8JR|x#4!_f ze=sGbhh{y7sGaczuY>}Jr%S{AV!rx96=p_$R%iF zY`!&H2$d#jwyN5G7g@z7-laY6{IcX_PNcs5sOI zuL5d-S=o6}9{rq;+NTz!5MpRVH-IY7zlPvL_J8#uLF}}`eyO|bwE9J8AqJ3Q!V!A#5 z1*Rx;!yrw`%(=5Y&B2={lC;DW`JbrlZ=#hh)~nQ35mU!O2eg*F@35qXoT}cq(<^s3 z@CnhSha!2Sa%tX8ZS{PMlS+uW`cxqb=`rl9BeYQlG>vI)i*osFnK2z5ZcMU+ zqqenPw$veR#kE-cJ+6xqb37@svi`EC^2g)T8I};QhBn#8-5KpZ&;8z2reI~DP)?D8 z7AGsk1bxJTTPo#tTn%Z(q>Y$}Jm00sw64B8O;W8ZNw-6EFdPEBkoaXAM16HLL<5Xm zcpBckt@Td1oPsikb4^iS8j;A4`hMs6D=*&26hOmx(Z)L(pNU!vMr6sXa|(4!4zAHk z(ya!mUc!CRZri>?FWYecQf)6!m96p#WU`N&s1s%Ic%YS>`6zMc zXQ7dtm?FOdL2>guWQPXIZ-(=vTTW1?A+K>;BT~Gyb)l6$ugPR8;Q%ACen*NrQ5D6` zFvoks{( ztr$0S)caW1Fd3~x(DF~buYEuA2~ zG*LuQ3)OqLy6%$^CyEJm(~+0+6S2=&pwRKAvI(5YOHW2Aq;k1rQ_a+pGv$w-lWn~1 zt-7kJl&{^{Sv@IZ7^M2TPNmA{VnPx_c7c1HfczWxt*|DlMB7}%^PL!EG4C4c78Fp5 zc(b)4#(N4asJ*vl7=wvvyz)mBUSGE4EtfEzR=G^nSa$<0C+?N#NWM@)6Q z<^w`W`nT(AuE}gtq>#GV&f$a$v-R2(#lPU>{7QP_ zdow7A4`OFr8?tF77aP-f$CcC>^S{yXDp%pErrm1zXd+S2eGz%7)}A_uf9+x1)cx8SZCJGKqPMMa2<49BQ9% z^6b;wz6eVRbO3x5vDai`Uw|C!U=c7iVI(Ms@_sR78!@Z6+~NjC@$9Y%pAlGy7?`&S zT@wMq4d~?Au=o{({3$d3DbFtmncpdE>-w7vmjvI(=>?vRN1DP?0-j?%DFsApfcQ^U zL`lIW`0Eb(sy#^ZxI6vYIqFtSLK!G2a2fcij)*oh3wzsDtHZqXJs8AUhp%R3f zU@0365A}b3xWr{I0>11RTUd{(_*57h4&<$ivkE65+Mi?z0$=npI3ip>`INmZjva`R zaZwKt1UTn_p{oXmwZ?NTFo^?z1cWNq^kA|DbSIc82qOc;P;$%nT-y)m7!ZLw zEZ<>z1khu(y#ti4%{uG;6AQtoNHcSC?_V{@f6HJ9Z4Nvh^x^J2SH0MVzRXZqLMQ6R|(?GGc^ zFMk45VAhZVuvkI0G~19Pp1%Pw-+x0w+!A6ZluxFp^(Yyp!X}ilUs66_N-YZNj)j;4 zhDu;VQy++lgH|4B4K6|ez}@S#^nZ{JWrAHnJp0yeGI4yrK?7b2oEH~2T_*t`U_+mvn1KmRo@kAj6f|ZW?rBS$$$`S$EifiZzXb=I`L}V~dEP0Of>E7NUNI0Y9 zM*zPzo#gNdfvX{ch@Q@wazwyl05&>UloxXAdSDR@KKaOZY<%qtv$qjF-&lXA1nxPq z7Y+a+NdZ*NOaQ9mjqg}^5bewH4#34N19Vv+lx!T$oJb~F7WIXT7zkDZTMaWi5FWZs zvHp}KH)+|cB1l!>TaJJfs$1CpekUN30kMytP}s-=RAQ`Tb<*oy_5&oX+Ng@)h*i+r z3vsL|AcZy?&r+DEL}HLDvPRdA;@yNm&X-V<-yz8Ng9r?=PU7RiD2C9uI6)8~1(p35 z0B$|+j0Zw0)OyH~fLN@?r+pI#9&I|+4k*`gx|n;e0{tX!;CW%-`Ebsa)(QbvJXCEe zP6KWJ4V_#3hSPv}MT)LF1N3J%a9tbiXMt%Q2=SL9Y5;%=1O|o3_i*1MH@=Sn3HS1G z5V~H4d>UAWqEU}dp38FOvg5SzWP*Ey-ClUPg40zIh!!TFv8sXyJRx$_BjV-;b+lLR zl;xu;B~{M(e@zbS>>fN_Vdx-WssdI23{CLX6Th2L9{X0WM>42VpLHrWjidFlOY6gA zLY2ENDWY#SQWv`ngI5%;;HC_xk4L+C3*TRO$9LJkQ>!T}f-tOfBA!!G&sR$7!x(yv zk$<;&trH8{us}@vH`cN*T!c$yELy5FYQgMfz(ZHkNMq8nt91PMbj>W%w3&%&?q-$s zr)M+Vbt$CIp&B{^^!I3Sp5^VH3-lO9^KVr(+;Wf>LpVWta?i^C-2Q?8gTJ$o8A0$T z5ZwiLVIV?Ig3;UGju1Ht)M>~;P*;C~f^7`>_t4PbpgA3Y2HUIhcL3qHi2NiEFes-o zc!0-l7&!{xVUhAZe1L%h_&9+tm}J3SE}Tq9?dClm6&T;sY^;c3hqteI%;nUKo}V4 z{zsG8lL1IrVI;Eh!~JN|LBb-ywi~prA2Rr_DBFuI9Yx(g+djOBeIhtO9oJvifWu8} z1RmgjU}B(n)limr+<=8cv#H=Nq=Vv>1DfQ0@p3Wgaxio9XfpEv@j4D|C_9I?HdK?9 zRTs*_s>8v_0{l%!lM~Q8pmgAW?*C2if36ZaX!}N&;0XKQZQqm|iRlT@&oPIs?Kmg{ zDqz{%0U0pw(69YF80`+~r!23jIMhN5w_j~pQ?il>X>h#NCncHqPfs2&D4-K5Y zO%Fh0JbXj%KIzA}^WTp^yab1A?f|ksasAi`xqmbotoZ;pD<8oe7aL5`! z&yL^+(!CaegZUSSe;kSi6W~AW>pa;0f;{aT8t?;NVN|TYz#mi?+|E^ag&oSyo;8Br zC)g+FDFc8D{7}Qm@YMl)a0=TyU+&;mS-?|+=^c(x*YDK`a~~DhOThvb%kJBTxsHMH zF_mdg8z4;n;n@0Jz;UDk9b$4gZ73grd6>9hN;#&={$7hP;3!W8*3a%!zIQq)AdHBk zh4DK`d^nJszk`H3nDpO6r-Fkn!Sx%#2cW~BN&4?GQNUq}ISLb)T;riOc$(f(XnyrU zt{~z&I|DVY5O6N=dPd^@#n3@LAGA^!mSZgRccq+#V`h-|7nWZqjUGk&UKInO9`2?X zt{i}R81cVOZ`^lN$2j8OSvd#CO3(qM2acrtJ6;aDCwQ>q2f+SJWV5H|4h%Ehp&p8!_&uf|U#W+<+tHiVAiT6kc&G?7Q_zOz*{RQv? ze(bS$Bqe5A4|EJPj_I84H430-TF6^?@y}fp$$Li#`p7D?&+p$jCeW@AN*%uQh^cu5 z5sXJmecwOsd+L8F_6u-^=X(?>@MDc*@PKaepnBn3g!!ZJ9F+SXZt~Y)3MTkt`i7w3 zNnc)Y1qPkKU<3kzpJdY0I!f?=^a=jIBL3Mc2E`HXG<+1&qm})$iv5jm`@KcRgVVFB z_CIhA%a^%Vsg7xA!Cv0<({Vp^fFtnpvzNEWVf^91|7OPh&@S$;9%i-UOv?*k!pI+( zUl-hG43zFAokJP>`*ff%`5r?CXy5Qhv>u%FrZ+m2jK4`TdEmfSU{rrhrvTJ6{LwK7 zDIszApD5YS>Up%A`=uKA+_?uk|B>#8Q*y+fRX_lLp$hoaw+98F=5aXMf2E84>VJ$r zz{&xiqV^y;H$4w0=kG0apO#}p1NuMs1hfZfS^R5S;Qi8%V}AqNw@n4`=;FH20B*)VGeON5Y^$W`Jq+l*PIU6uKg^C`|S-ls3h=*MK{4*oq(nf)xbZ4f)7P1 z`~>xv%QJp&;o)QF?+-xxxq9Rn#evd1Ja+yK9^l`!OX^N14Pg8Nla7I4Fbd-%$Nm zLfH3{#|Q#6G5D>>2dSe?f)~VpwMTECxG-&vXpyLIZ>S{ZxqEeWK-X$Ip1D`kLm3 z=BAp$kj>(Sjd%rp^69bvQdbX}W{iA}jsq3=;gJ6qp=p?Owe{$9nDnUW8G*XgOgcK6 z)LLw8bkwX2Y;<%?TC8k(Oxo0R>d-ec(?1+6bU=C_BON`BiMc)vy*eE_2L6iizWg8BeGrsIO8qSHF#wQ_xWeU_Jr^MWAK>~cOL{TE-6 zkhttI-NdvC1C~@N5%4*Yq|V*u_}N+w!e~1eKhrz>4Z!W}pkL1`jQ&?e zB25U-o;)Jb=x_can%gJvx);yS&!^vz;PaIi;Z7vaHeZ(p=F7LCmIEy`WhJ|WRy&fq zC>!!C8kbE>p==%qyvBtJ7_k3jiJ0K>=|v=>x{BLt_qEoyuhN^i#` z9o&-8FdSKh^0+95m&C|-_I+&k&c0@q3g(B+OSs2b@Zx16z}ajROX#wZZDJNYXEr+n zc3Dd7m-f3cROMshQbD44>!)lLjLyjksQA8EQKYX|_Dz4V2yhuvGH8$DvF&sJq;^;) zsL+L-U-zyLbG#}AZ@$4-!m>TBZrKk<6ZKj9zyptU z)WM`iR;P1&83iVU;NdDy8#QCpNx8vm9*P9>lzC3YTO|iKn^l=;x7@rC5n$(nOI{g4 zzT0%7d81OJ%jNl}io276yUmzc9ycz9TL^3I1yRzC#%qUTL{%jf_VAhNPkL@J|L7+V zV|UzBGvhDNLD~Kp0HE;yt)I~U?6++Hp6?A-j`|mwkm_|rXLqkKW zsCtB~4Wf{~=TPn3s0`+f+h$W%FU*~eWj?;rwp@I6*rL^w!j4<0){5C?igvqHku?^r zdhJ`o>=Y7s>&#HTr~Dma+H-co^yZqDE-!+j>UCV`vqt2k7c@4)Ex2uJ<$Mfie2cxg z62#w8zog9D|FHPH3gczck&B`==4FvHLW~JfuEfQ<;_zKa>c-gTa#%nLvzM9cW7{&K zT*D3JVXY8+)P|M99{hrYhQ2xFQOq^E$gJB7tJa1rL}@?E-OB7_u`cj7Ps$ws2B~o0 z0`Pn(A*n&gWv**Q@*nzaraBfqx3iJhTXep&l=G!3M)3v67j{FS$CB~u1d z=Xk0lg^`zOwnT98=$cZenf;mb$+LcJEU^Oio)D@Efh!4oKgiYfh|}|V=hkAr$W{=k zp9|G)kBIX|cIh1(Od3o&Hzr9!@=q9!+sOCy-ecDUUxU%8Hj)b1m4 zP1V8ob7{@J5kJ-O?^hqA^$S>8EK_tKBU`en0r-0&gW>JsoXE}|qV=sUMRq1DW&EKl-3&p3z9J}~-sUrIITrM()%bE3&k=`7 zXO}2|SI0?VV*C7qJV!R4gvodyKh8QuvAq|&^^+QfvDpqYl^vcv<-qU@|K?rsI^WIE zU|*a7glzRAnQR}7(|r>xjVmuS742&rl}?oc_R%jizKKBItGm_EC_<%;fdZQ6{pwLS zPTL2@8g_+^1+Pk!0+rGIdwNF-_*07g;nkg3!*TBV&na@>pDiAY>kqHvL@cSOc$m{#Td?4Qx=PE%xcRsQT))j7z2zu1xM7(VST5hM z0H_P`oLv3YH(Q88_()ymi5@_jr4-#nen3@)D zB-`&%RX%l_&rGlCtGZWDk5+PZ#2!85%fT>}XQk6iXk}_-gFEbK#xYI?*H81bMc+Zo?9jbvkjI&i^>o zr&s^F)>lKp?&{0F8jc%M3>btS!{0k;`W3!dzNcaEW3=+VWWb9o$m4}0mW<9%n?S-> z_4xrS_f5AEgpLB6;e4@E%;`>!gf(9`t=sG7bv+3$CZ3_Hdm8wXc0L{MKeCR^>ZZF@ zFDjhF|60Wk2|cy!ysfScYp;qCGJiB&iH+^quCMoUbJ+@)+&PDe;yTUtA~h;&;S6uiYEZ+e;yV8k3SFp>!A4a_u*kcB>eXQaep4!KJvW2 z=@@9LB;dC z#Vn^l+)bC))->l93Bv{~MtJr|mrOAk8|mW#O9oc+h)mH03?+kv*~?Q4!G+6|`}@lU z$o;*Ox`x zY2No-;wl*k*tC2BKxFB*FDkDceHD{zif9BMkz&u$XITt*O^Tl}kKHDK8ys!l<0){v z?QW$MzDmbCP~*G^A|<6YNUHqwX0LHW>GGbh)n19bVBip|L#HyUNBgS?R`7Tp&_G0X zoFuq?USlLSH<%<)$HYP~LX2yAtpw+(lx(fdiAR`?j#`g;Mb@hehnGTM2`Mj5W% z4gsEc+oNt(g1+j_7HjjXlKG3gT$95MP4&i-3sl^T?TUK2BEy4|!)O!8E>6;QhbS^- zbfQDQQm^3Xop~X7gZ^Yr_Uu@eeFGQ6_PGEmwV~Tu4>My*qw<4tBk2YS#_qB_qUm^9lcDY0%*v(RtHJx>7_ zWVlS-fT9~O7T{J%_c;}+Y20?l^gkt~vb*Td`_KxbhT!D8twzIjlV}$RS^>8>GoJ)6 zC{P=U;D<`DmMGc23T)09?cVh&Z(f_&dhO^?_sQIrD6et`OTVaG73rCZ*a?+L>yME7 ztF@O2MMwD6c?bH&kSj12FV4LJ;h3Z_S$oeG8W|F;N1Cym?2pMFW1UG-2Na^0GhC0W z_omBhFE{v0=YC+IuJc!3r8O_CfEE~p)tPqJBjDYXk8e&SbLSZCrsY-94XRkqybABn zS6`+ySj4MWDjrX{L`$3J)a)h#^$*#ZJ~~;nrU~NmsGl!pP}1h8-e0V#Pvf)IT@8a| zC-GuIvoH=EgA28-=UV3*Ja?_2naq8-#>Ukw5K3cp`FU=53b%}H>*2s+%?*zjWMf{E zfZ27-aigWAGIPQDqH^E-3I63=<6)#SehK)h&sV-So~6NUEo;^FmVY((a&EcK@Q_#D z;3nbFxkA6i5A-Q1OKi~8rGypa0amZHpObQN>BY`ZHGhTll<`p6S+)9Te{&ra8vIFy z%&1oGmBTd_SYu<4A3GOWkLE{q(qcX5;JiaieFm*acPN%^04t;A$oeOno)EZGCZ`-5 zt5WInG`G~~WW2DzdA&H5+%6|_G?A5|HVrN9f|?3)^O3d6{o+ON+A%49?Aac7 ztsNwvc(tNlSN-&g*k_;C)x;c+8xk*n4^DO2m_saeMC{l-aeGaLf?azX9hl1b5>BTUtV|<8k_U)UO6V9?soYxJ=^jUkdb18K)Fa9@|LY zQC>`$Z-%_&n9v0W-mMyjCXFAp3tuRR$&HUEPz{%Z((KJzCT`|mBz(FJoR!bJs?(57 zrPE)82@Me|$;b)BAYf5>^gy(Fc*pUyJBn?=X| zLV6YlO??7QT@D^jBTofeLv}-zm4xIdD>y4O9VAQAqG~gH(0@2<8tFu=HsEED@h)r@ z+ZcD9ddojGx=z85nu#o+ZrNj_7ScREpD1mcVqwgk6%YKxb?l?i6iE=}&afZSTPxD^xJWBkH33;6hi`ZoeyvsAJO!p4y;XC(|z3-MmgBNB8xS?nUOqSE5p-x&DX} z%19&|gt5l2MAjq76``EWAuoh4pjVbI%Iz+qgBoKK|~ zJ?#t#q4G@MITjmH0KqrXhCi_FRJLJff|D&jfAMxFLsv`ZBVx>Xa7k_+(XQ$D?@``4 zszPiTrtTb@$Cf6z3*`uo)1TiK@5%jejU&yfz!R8c>CSUC=aVn?$5sE{M(v%`>TUG_ z*D29uVO4}7YY|VGghH6|>q*c-PVZaBt4XBsd-Ap83fDae67r>B%VmDWH(hWxav|a1 zMsf*L*`9QG>+|}`GGi`^ZQJttN*i~%*{n{1OlEdQZA|Cy-x4r#CNWOzm0LpOB1{fg z%NY|4a-Vl=m42|5PUak#7hLkCg)?Dq?2pg1jo8cf=g#+`; z>$O5tD-=k_331eVvb!=QoU`j4GwZBN>dE#pQ7?s4FYQPzZrIb>8Vi^R)k_l7_a%FK zx=9jdNvi$>Dn08Ig~gl=O(6sCBv9Ya$lO(^N+=|+5Mox#}z>XzRB9ErsjZ@Uw z0_aSnM;yAqZA*$?CKu(Vi3Sbsq7r|>7JNw{73#=LAJf%UkV6JBRt_A+ljkx-KQ%g4 zBPOLI;d@hEZ-vChdj(oxaiFG6fdk{H#*z;IQ&xJ6IF&6kWo0+4+@g56g!XM)MT`lX#u<@%$PW8+u2G)&`5ZDSeiP~3J%zdRcSdW909JX0n+ zV9CBuxHr#zu6Y^+fYH&*f%z2)u$tSGfp^DPod^zUPJ?2pKS`%rgdHP!xpG`*ByWH( z(GyPUjV+MOGLRw6qbiFYSd{n((DZ!8{y77$55dF}pC^NBuz+v(RRAdV4*)!QEMN1B zjmCS6m#=@_j=Y`}g2t`^29zHIhR}WahCJZqQAk*r0D!QkTy%ggmAY3Jbae8M0A2Pf zkq>;n$tS*i5Ym0M6AuvpAc`XP#WR-10|34w)v^QJ`i*>K7*0Hi`2AJs zi-#xo)R<)KHyQde*=?&#P=?%PhX5a>P0dG7>uY$s7?qvGr`c8CJ1+Y%{>=NCAhG(9 zRhvd4Rdd+4k_O}L9t-3_gXV1=OPK2*^2W^V?7 zR7e2AL#78JG(8%54!y3Wb;!Vnk=s*oNPG-_1bDBsW$x7%`M?G6wERuIQ+#?f^tOxF z@Tl$WLf3Cuo>Cm>3Oh}g`Bhy!2Dm@{^zmmKcrU(34Fh=c@^b794wbu=(4a3TkQo4o z3$;UJ+m1EA6yCOe&PoTBz`fzyA_8D~IaYuipu<(L@E#WX2VrNCI5ITK{tQ)niVd~| z?);T|L+ZZ3#wsT^U@eEq{*DS-%nqT(`t7qo1Vz7D5&+<(yESOqOBaza%KpbYsFXt< zcs2mV(8zCqFzO$}P+8r1TlLV z50UZMorf^@r)|BO001jDXDb$Kro%(V5N!P*dLq_p#5FP&SHOF$)oKN_@9wj`YahC3sfG=plVFX3eF|9c`lGY`=H>o0 zjO&ufl9&?!sP;|(0RWnN$k4oNPitJEDc%Q+LHjG9Iz;=A?)Wxc{)(Djm&<)_Il?2$b=RQmuQl>-$YF52-&fGvW8o#wFKB$gLj?+W;THbI0C zntu}Kee|myT5bq!&w0>I>)S>DM>-JUK8hreoK*x+ZWjgloA?3|z|yZ10@C$@t&2V47DY-sQn#$!MUK`u{XS0#S3 zF7Gxus|vPkv{RejpEMJX>LiP!dX=kaJ`|giBl;?^w)s1~Jgz8K_ta7ZQPMZa<*ymB zVZ0YQZH%Ysg1cLqTJEIbIeJ#rOg;A?Usn5$VG)x;{wJ?qD(2Y!Pw4{s6LI-cQ3Iy7 zvdX!(G~MgNDYA}RYYe=s$2^{=lin>Y@%%UU@DH%pFGDWQ|71r6{<5P=K%pwfe}}97 z!Q{cB@V9nUAnkwMj*0{oT4I_s^`QkDM;RtKb|XJui~)YH?YW@gj|QZs2YXj5xy zYSL5dGPAM(=~;lRnp!MSgXAy6Bk->V3|4v|kb#Njw*liPR`*~)7XSCcL$Nv=72PDj zkzjnZ1aeu^molvU;`Og1h>;-A@Q6&=8W}7KtLGjz5pmzOM@0lj#>xOsU) zX{rSHx%LV=y=dh^)~lBSTTUdfBv7c9c)hh1LE~?o^hrVm^_6BK7zA%_leqN`?hY>R z7o8fN4ndm(b53%z@$nXl_Ja23I~ImjR2YuCJkh=-A^wG`xcR{oEMMDqGV8NlG?w-| zzqm5pwfMF)S6Ps8a?I5+so;xnyi?(Bv=Xb)5WO+iF00PW&aP-^`W7&>;zEJlBbuIT zy+2bvc#)U%V&;|8LFS3ck*`n-?&$ExxUJyS_T@Su_B1@(C^&cle|D)WU_&!D%L{BnAI?0eiB?dxPRq5@EGs1$R#2XNo^4zp$hTv>tMoRhSR>6q zK956gab9K-wuEWvb(0X(j^Ttd+&04$bEbv)5{TZvjM7j#$A0JZ^$TKvjH|w zn!c&UBz4`x8Upjvr(DYBv)7$NS?V)K=i}ZR zD`)W=9Wx3b(AyD-$#IPA!<_wqHm&0L&uOXE#o3v~*=!TjIEXVc-!o~XM#sV@7jl3d zH?#|1ab7pZdlIugDs|j1$EKR9v#o5n+#6JJyR4u6m@3~;YtcW{p^BvzuL`ri%`5>4 zHw~oVQAa}xZa|8D?~$s_h@e$yvUq$tViGkg-0DW&ZC1Jt?~;%teNl0;t?zbbO$(mI zBVOhZuGlv!oU^*8T&!9S?{ozY5!9c?ZyOIxTaGWAi@F|l61c4RU?0?<=}+R>EoAi4 zzM7W9xjF9xE2HKvFT=Zrr-SsqapMrIOYq}k^ z9+Py_ba-EO>aQEM4&=Td*zfEIpOlo3uDko2z{m#YQ&crqx0Tz&r(9<%Gv=5f_HB&an-e_X4-%Iv zG50njiP{H)d{9;_E78&Ccf+dcHT%iC?D)0dkWE+I18@r}uKAVg3517s&Y_zC-}MYN zHAy}0;sOpXW;c5hY_Uu&&Vw<=bvy0Gp0V18%lP8dcY}5^!wsS`uj^LjHm&N0ceARE zd!WLiVr4%AoWCE)!q!n9_dGJZRvz}{k+qz^yEJp#9*;I<{7?!xX$SSw6v4?x$8avX zNK%ygI-ghX<=S{&e*`U72z}tGuRb|8bQ08<>`35=H(=WykiQ3A)D0D?Lw0t!h>k|M zKHCdijEoy? z#C42fGh2J+5`2`k;qBIWY36vGJy^hX86ZCpk1b>-!X?cL+2^^slCG=+a;5XUc8m&1 z;jt+^*7w~#U0t-fya;A2pfUu{_D)7&BdK1L5;s4MzS9xZi?&|aCzc^oiQ^&)di^HL z8$T!iDP%go7F<72`=M7qQ4Y(DvF$*$1*K5X0<8-Dpf;8c9pd^TjUJ*OI# zMZhwZGB=g7g(UCWQi>HNhd)|O6BR&;j;EUa3^&3H&#U{HKY!|8l{N?$7*t=ylEqDO{ee?fOAQ1UejP0$F6&&0_1cYINCbUF;7dUjee$3K_=s1&wkUS zVba=R)2%TNI?Jk9n{d`ieKk4?$2d-J-`&x66t*4bHGEY2a{AqeoBckU;{iVLlEQQ# z3+shT4(%p0KnzI$`%ejo<4#o0`_{yy&=b zLTjE%Z$Fn(5oJKe;rxg4J+m#py0 z#IU_kNSw?ZZxemIT6h#!zlzD}vp9jfxP5~SS<=v4%QNc8!PN}RdwI04rt^o^lhvoK zanFeHcI9=`i;>D^MZSy2$Z|4KlTEV^BrVKwnsqC}IZb_LsNN&Xt}13Cn|B0o4LlCK zNsUX_JOwt0wCyue*!3eIRj^0SZZcLs7Iy4Ryk0J;4(#!QfU%w? z%Ha~DM*&5){dXdq#dr`z`dfVOBBzw-#gDM;m5GReTp=&Z{96YjX~L%Q&QMq!_930O zbFxyFrbSF4=3MTh(T^p~oOi)TeS7f)T&~Xsdrpa)uO{wT{kv83EXo9raXO3iXI}zkp%Qm(7-PR>` zd}sz->)JrSZu13rycODf&?sh65DiHH~vxjwdqvF)y!<1w5OSjWcos>`Hzx;`>0 z`(iGjpEquTe*R_-%Zgt#b(?X%$b|Oo)RwTPRKC;J{N}{@^~{oJMZx3w3K3gM3P&%k z{94;1Yl`fux?&U%uWaXNJmCbxZ1>Sj_G$wdMRPqqTb!?o@cCM zFL9{pW!%K#?qVCch#8!57L;TAnyt*?O5khiqj1=og0TX!SZpsykT8VK_w$E_G&h+8 z_v)j=leRvm3U}uDWi-2Sb!+h0i`gx8AqDDy0%Vmzg;b0-uPJ6tNgFqxl+2nr4#Ek5 z%C4`8Z=alw^nUFVHLScda;1(7Pp?gztAxSTR%NTr+5y)*iEu;5_|e<@b~x&(5G>rt zC^5XJoSLoBM+r_cw4g%jL=C-8E-^%u$tckxhZM9tL;UU{U1N{=&aGt)8yDM5sd|3} z_3;VhN*{_%lWlPnlAB!McTcPF?QF{45dn9J_2ZASHW!&vxFffz-xTylpV_q@^R4$` z9`cm#bLGx(3uDzmT!mvy|2wd?bLhZ~DxawHG`O2_NWVG&aWrTX%A!k^8xanJ|>5 z=6keA0T%9@^U%3aM+`U1L!byT_wlb8{nC;l?ZX>yn_O&Jw>UYl0q&N|=<-mAgNO|i zz*+_3x1dIS$np;g$jEvqR}jY~<0F9k-Dc#6pKSz=%JnBY~Fp=S!Mgm1Or| zm)s5OMP&Bd#eT-SBX-*UguP1Su%JTYkeXfn#Hr8~#eXaDBK&am5#Y7dT+qWrR6|Qq zZ4WjRA%^RhD&5KZ#2>kT*uw^`?6QzY(a^FSB>qE^Kd&T| zpbJm({v^&v?&Q7IzknXP>#CNYJe!Edp|60Zy)e)%M#nMs%z#QS!qpp?^eaC)?;oN8 z+`F3&>3(xnVS5lfmeV=tt3TK_ z-2Uy%nCc{2uCW-b^!<2;!hL%JnpgOQc-2#5bV3Wf+-cr!L#E zL~(kxYWuoi))-gI4ttiy(rhnaw$-Id<6^vReVXTG@(iOtc;+xQshVE&GXy#Rcww`` z<*W!Ew_p*t%tceZM9p66N58uJ?v$%)4P0+eiu!f&38h?i-8;e0{WSh_*K4s!LnYFi z!#VY#yn9i+%%qwtV7(1$n0<2{PA9lYk>__t$7|DVKWJxeik@`PB-aa>d+aGC8iq(Y zvz=cXJD&`aWXpsZZR8=c+`m7wy|v?a-f@(TafTmO-7{Q=!4sgPGU&EJ?N}&LUnUK5 zIR~SWYV~gwqM>;eS7y4}3uGs}t)NK?5jLAiK$q^OKP61qA-6g!&oqm6FP@xWj!)~| zwW?3~*n8!mwhem0=f-hLgrz#LsW_tRn2Fgsi&`Mrnm8MX3>qGtKWn1Aj~9MghWlRO z>mV9g@rj|mJb^T z*5PNIt)mZ}QnL!))$$FZFh9Nae=Rk5!1dznK9ZY2A#As@e}l+$^%!+@8MGLv^;or_ z5E&gS1GOd{P>Y&{PLq)h$jHFRq(={JV!zzkSpNwkV`65d`)7#k0Uvxg{>5YePlzmm zi8}Q?bts~Ap42;U&m@}@gZQPYFrO2>j4h-dBZlrd!A_Dm7C~|ZuD9?}=gl>KRK3kG z$#BiO^ZSw({HnZioIM{EZN}kZyfFzn$IrKh)8N0*4>&fnj= zEiRfe#4*+m7=?_thxjMCI_9y+VZ-{m6>;N$EHW#B(-9_$$}wiDMT^bmGv>|B&Fe6s zU|TAqK)B+J;*jy-2-{-*;+yqj7-YNzWVhK!JX2$i2vbZwX4u?h>BetmL6C-pg$0ue zkYv%`4i8xR&i;hDSmcd^gvbJZUO-g)mf?Y6hk0S!b!6(n?k9eq7^L_1TYduw*e0d(El zscm%Ce_rRfNzD_>D!Qwc+j;rkMH6*Q-DqC=nj zLhZ&8JY>C6e4+e0A~fIaE~eppCx4SA=~jO7;5K}bM!|XUV`oY=Fxj1V&u^-mk!7W& zQK4};&w2%UGG$@hWu~vabz$Mg^c}qOH;`s;IX3nERpW&7>YaZZ?`p2Cf{Dwu5&vbd z>}8^>&`NW8w~lC$<04!op2W7r6o@DWQ2?!b3yY5HWxjO#btwT_srhVVXoLQ_A-XK* zasDI=Rklp~O5f*$I)5ADQEyY_YF%;o&48)wQDtVe`{BC==i80BdwlFdH+%a%)Y^lq zX-n_bdgp_nFm>?NbffFaa@OF&`S5JYRlCb#oxv7nbLNHkT!{@sYj?)WfxLNVwSwB` z#z)1q1J^eb0)a1kHV3*a4&gQnZ&J|lm5+3ex>MXX&QhGsgrh*-*c+4A+d9ZLU^q zbFn)bIrtq-bn>p>$;h*Dmi1`8XEICQHNr)p!Jarik>aZqH%J^_dwotkrF5|Vow*{0 z%2~WHK^*ZVZKIvyyeP8QcDB|*NwsyqORr@i=VsO=RzYuQucNE}dRz@;;W#%*r`h&f?ll@nI$e2{E4*%_f*t$u$A z64-TKe~BzVF59CFTBIM2$87v`Eu^-;eYU_eA!oHlgU}F3_@!_VDF3>qp|5-@PdD%_q_GqymE8b7IBc431c>lwbuHH7v@kc5i_ z)?y>=&aZ6+h-o~MWh;%0y-1#@_azt2EprO{C0Q$^hm0%{vkYh#o$+b5DvG!~Rr=E0 zDc#;&ch{+@a|(kBW@IWagDUY#lxGd9Y*~4x?T2)`SJ7~APV*-)qZ(BU(ECw|kO*Wv2#MQ1#%W|L1?p5xGS$qFa^$8rkyV9rNuu2oKO&%T(8*|tm`=&!o{RLK(v_Bl z6dp3xBkQpV29$G3j9R#uj&YoG`gldLoHvXb_KG)Xb#P!abC#3s#GdS(C8i`%m~2Da zROmRhvZ8!r-kOr!ToZ7T30y#8B;LO~{NsZs!E2eRC`vD~=CSr-ocqr5XZP~B%=h%D zi%wdjox>4|sIyL5r#4fNy`gN|q=qJJ^!n1}llUKOvKMsHO8bW^?d{07T3|B?l`~}K zX9PcjXQZs1NwOsti3<~j=q$a+^{$LWTcgS)wRapi&1Jf*#&e@PvBXWhvv1|-hnblr z3x04h`(}lXJUbi8?R~rHOZsk67QH-c<7iCs)%6UfbU5i@1~I03?OSGF-{6s|&<4HA zs?qaLmIN-MMkt{;8J6burrm}d4wE!&rqEj>6!E8+b}QwSAYlm{z*nLwJ6&ipI7~DZ zhcOw?c>w}q!vanUW?xcwBtvc+n|7w!{C?jMv3|`W^*8+)h06HBa)E|I9Rlr7u_yUr z;(mNILT$Z4deK z&rPQh85enSq(d9G6P`CX7Z>@R$JTkwZ+h)9bIzga;MUnE z@aZw>l3$iRIw}jz`qpmKz0(xq=n@uKSSRdFQhP(jOWP??ZZ?RQ&brLhTKelo`-Qu* znD4AJMES*G?So}^-;Qr}aTJKsyr@q*JQQH-Bsb%Lu zbAa{cn<=wP_i5Cy`GS>FP+|j0%w!BMJ|$)0RB74wd79dF$$;R|n6#0Ybmfsd;+U+= z6NU6@5I>$$Zw|t(wPfgzkDAqU1~y-T__ZuM__qEt$`sfNb+Loe9+AZz$b0r;a4aAZ z*{SOEmUZnLJG%PDo0IEiIvOuo#rxvV38*`^*x@X|m?z_^0kWTuQd(wKsVsX+wX zo}_TaIdrX@lLUhm6OLB&|8wQLwz1F#Frn31B~LD_{F- zs_S0g_P+eLj|ep6o3Zv^o_;664n!4kcvO?C3WbU4On=z@oH;vXoR8D|OyJHVy6`Y@ zZd%y>YV;fX`IYncxvBJYcVM%-0~*xg5GaM5BgrN!+^IHf0cP^ z*FhNgi^boaIpJ}0pxS?+qmxbCe|W$A3j6C>V&gwlf>Wf&KWX%@8KCzv-98V|MD`2-=RmwcY#{tWM-ABOb5^2VRj>OLkWN-Hu}Q%@J0LMX#MnurHpj z`#61Sfq*|}>Ra3Mg8eE?N;0a6lSIGIGVG$6_7BH2=v1(Uqs&I$#ZBl;O%KbJ*xZ0$ z3TJVY8b@lfL?X=Iv>0sv`vm66v2^8kfeJI^PPC8jC3^U=RSko#tuJ`$ujXfZlcXFQorUSc zfCx*CjlFBT+k(_F4C*Nytx2ZMSuS>RqiXrqa}2M~cp1wtjpCAqP4NX< z5Mvz*CG7I~L!4?UKrf`UNo*C|qg{(GY#dyt*>;#0EG~Z31_~E2h)yLWYc>=tcR5Op z;l483LYF#SZjJ^#R}Et?gpY~WD5MM(v^BO?9p%HBZ9JeRrLk3w51u}I+S>dD=u+1v z65wbVu>j(fcAs`pY1jA57a7RvM>i83PQUUjZ#c|!6gV6?)Eg~WvVFjK+2#C`hduExCG z^8^j**vro3k4Q|PV4gISSWWjWjm;#oo6P!Y$X1>@tYqV-4!2PZ6B-A{k>J&QZSE=l zh=>LCU*`LWfNKG@a0j81ld!B>AOFxbN7l3_vxOBmLsosnj(_ZDI-CrwqyGAYzW7yJ zMa#yb`);#yD%2&AMOC>rRa-odQ7gFd$!Oa(3;KuBPtW9-r#jvsa0COo-mC_{cs1}s zm^oD*?v0ZOqa|LLDf44Z)8m7tT~|yYJ6($c~M`A#7-e^KjkF&@@koC zAd=)}{IJ`?`lcqGT$83kQ_c5Ciw|xBkzc~SF{FyAcUmN(h5X9^@;O}mQ1>|7h8v>F zDiR#UJyDoE1=46c`?^;ZMGMJ==e~TqVTGxdL2-Mkmcq66Ev(q*6W2&Kej$oPjE*?S z&LN?zpXNKdOhz;*wkUnakvgit8nVaF0@1>r4gdmDd*2Id{b>4{85rMDhWI*VZ==nO zyZ!7qfa-l~6CwEyIpMY^3gH*dCP99&V=Ez#_@`Brz=N1%T+z`(_65)$-y!5%5g;UT zrUhvQ5{)CG zA>|Y6ljeC&9Xm$l0uz~8yE5nsQ5&FfAp_yFmB-$}p(ML>=jo{Lxqvl_SFJ0Ydx63uX!!wlSudp|e5eQ^q{=+l$Pidb*VekK8Ym2SjgM%<~ zdiM2~Lv+Sk!#jb&)lM1Q&>2ud5h0f%YiiEQa7C43M0ltfe($9HA^1>v;$4jG3hn?0 zM7Oc*qP_AmxW(|piF(*^&}>KimR47lA+bJbkZHs15Sp_QqUHv9gf;Kk%HR(E)Y>UA zvh`3qRt64rd*;PT+xX$*DWlmebEE^ap9nvdY~57 zXXe=km5zi6)@C2!cXqPUvm!#49b<7qbQgQ==;*vjF42W+-RakfGifh^=Z zJfJl**x{5{uXw_=Qyy*ZTUV_imD{!-Z@bXTKQ82CF1l$yqwqTVf?QUSb-p@$KgMJ} zp|^?SeDXFSQA|S1^)MBHZ&Y30;UbDmQw=w<7fp9zW{5}I(RWCI7#E}U-3lY&Y7bL| z(GV|mtnH~*R*3u7sR%QuYxusFlgaMp7!LiwOv%JgIJh;4B4TA&PqK)UrrN7R$ohEO z6l9eYN>j0SiYF*_M|@_oqYu-%~ZCt5Z+M)Q`oyAA!$(7mQ1EjW``XZjG0F zO$UC<$`ddxj_^orNlX@)gS#gR?-u=a%`0&fy1#qN&SgiAJ8y`Wt=zr&r_vcsh zZ~k`kqr{~oQY={%1EV#M9QHQ_avA+%-(DT64Q6M=#N$SxAJ2B!!{?l2qdOp1V;P$g{`e*bzSQD}=~D*!n{ z+hPIo(5SgKQc=TAP?ofJN*>J!RM6mO9KQP%g9#KPa7T|itzd|>8hQ<3gwKKBvL6ZC z|B?L&qpZ%DxMWjG6KNH5QNy_PTlOPUQ1+u8&p)ysK@LCIjpMvG4818TP?YxCy*C=x z`O%b_{~Mu%tDeZugRMi&)2XP4yOAlUW(w^R%8nc#N8zT>p_frK#?jXqsIE+lVR(ZgZ)8hFDpNBRWQr@v47Njgc$!yjnQD%A{$n`m5%vl z8c04dR|6Bp%jI~v-PW!xr8#5+#ClIDykxt));A!Z5Ih{LUK)+xaH$}Yk2g1itNhH= zBp>K@uq&Q@t8oL$yK~ud@{zM}?rbb+uIB96<>o4M>`FQGQjQ=~n&6mtfG647IuJ5D zA=iD8_0K#)A9x?u9pv}2@eh{K|4%I9E4qN$!#^0_cYugCDzaaL2Yd(9)6)|Sm1-64 z3o6CYT}aU7r9mT=+e+px=45S{QZxNH#`>PGb*(l&aE~2f} zJ*_LvH7g4Nf*-9J(2W`0<5T)h`ufC!p)mycGx6Tmzj1q_ny@udC}~$niwwj89?_y# z&$C9qn~U|w?|x+tmww9f2!3u*2wJytys zpY>)tB6`;LHno$h{HRPe4*ljJ1^j(CB^)DHLs_jVb^QbSUaAQr+%#tpmY_5 z&v>-A3vgQUZgjeiz>)Zb9e;J*N-#Zn*&OP6aZP@AR0B|}blia)Ajy#Db~>d_a$ODt z%-T&s=W?34+LMVWI!K*=qIbQs(XZIU!8Z$WYph;~8AM(kABd67n$sT@G{zZtKb|B7 z_;km%z5yADn`hqBl0*SJr^Y~*x?evR>v*xKVJ%>%rgSs-lF_|tDK3C~j@@EvFKM@S zzdU=**`9mS&2ek3VsFy&Mf>ZmU8RaV=k~IlBy*>MI$m1U+w4*0f|{e|U4q1e>$?n| z%8bEWHr}Bp<^|@j$#PPMR>wDe`*q4VH?d0P z$4mlFOlQNfVydFNsM%mkIUM$&aqPtATv_UOrzK}namoBFBu{;<7Mid;+q-UZwJ(dRzYPs+tTdP?->Y9aZUZD<@3sQwILw8U8eOiY*6Os*nyx!p)NfjjaR#alC&s%RWfD<% zti^a#{ZF=7Z|shmtXGoH(QwxAl(V!m3l&BhtWM@b2|c9-kY{3b7kn1bwORQrL^yUD zm~(RojDlYE@ZVi$j@C26LvvTqqDvk_b2^*Owh}rtuPl*TepX|1-*D}bN7gGo8OseX zh0~=co{g??pi=>bxzj$T!8V@@@txx+(U+I|bnYXWc+z4|;4ZE!XfTeRE9tZYEbPLi z(T)x=I@Asw1K!OLxLu~N=^cM`nODx7wGcWBrMU#gpK`_V@RY4=UdYR11lgRLHlr2pGM)=3~0tmaH_bU0l8FG;`3H;N;BVxhWvHU9J8I;4gIA zFNlX1x&)426>=OE>wB>*@!sx|Cb=)MmV|ZC;Gr9yq$}Ik=Ng}_Vq(~k=&7Bw0XJN0 z6Q5KaEU5DwsTl0W=}usT9FNBNFyC2>N5pRPTJ`yJ7S{9N7;X?We+pwVhVmiM*=hk^EZw^=Cm9JJ&(?siq0CfCY)_4?8mA4$dcW{ZIk%o0A3mv^;J~&=-&E!2u=bSxFzLsY+UoGNclG5Nk4?sYNj)?Xg<$Dgq;d(MjC&Q(m#TL8y z+UB^!>RwOOyXT>O0(%7SO=lvrv29$0)Gx!Yqx8{@=u6f1Q#c;b03)%K7`UCIu$|-* zaW)wx^>uRW$!etQ@wkgUhXzLKtJgY7oR)~wtMy|`R03}BaHP4pvu{a{4x?>O(*2@w z=OT6kPHYe+aBG>QIoCgEYy`En`_FoYIG7L?FIT;Bkjs#!3=Ji(@yah{ezdnfhqbk# z$Ur=)uCF7aA4;?1e_~7SHH_Ah3FRy}IAc+mcFDQupV-DngdY$|t5Bw~cP&kc;S;Jn zsGufS{8^*WuZL(QY)gcg_c?cYim6v2B_`DXZtTMFOf`qK9{0&wdbJJz3l3=eG7}z7 zQQ}5w*v$&O)AUs@!G&$laQ_p7DdD{PSEek(@|l8Wu^6K!@$*y0Zurq_w$1mD!cA!A zK%Q?hPe#q0AE9;MkH5^oCf4l4cbR&B7AbTZ1%nPy3CBAmRrI8Gz4;`#Py#K=fa9b? z)nhgf5Fr|fhFUT8vuu|oY+|-%RIMf{B}PULnL2G`qdxREd~qOJeKm`ZJr$~lXHi+t z&G4djhn2FlkhzlH@4>mr=g=sD#OJWx03@rA(w~RiRoP|?c5%Z@_Zvls2&fRdr;(1} zGWa|=F%e;~xOQ}iUFu1!#0}p}o)Q&Ez?sIQVfyGVpL&OtDw?%X`Mc0gvXEEcb(D*^ zknc2Dwc8Ykpg}DSC-afd?&sXx*U5l(yl3Vh3wrR$<0t3QNHCM`5)6-5 z2SA~drkp{d9{hz%$6lI;>U}HCx$N~jbb6OQEi$xlt3^|Ej-?qU?~tPe8q`r`3~==M z)jzj$GY7VmS~>UMN*9=v8iSJ6B~(Z;USD%B>+3&O;^lnSQ_z@7f|cW?WC|0rxmJ1N z!sHNcoZCb~1DIhCOZ=<^a2vU)dqosY8yx9}aV*o1$0an@MzbqAacplYY!W2Pb081M zVR&gU7Asp_Ejf0W(+frDz~dl=LM5Dm%6)K~!$>>BA|t1fgEiX2Y@;c=7nPVeZ|FYA zZ*D&Zh?Z|3QEe7!PH`Z@1`IR4vDM4yK3#z^qUD^3|L{PF%m!_EW*u$j9TPj(+GU^; zPND)r1G&~%g?oJd2nms$@DbdKA@Wb{Q*a79_F>J^-g4Ea zmy%0;ep_Q-C*qx%;4O#U_;KNdP1SIP6kajc`A`B}M>Fdx29mvkt=XHr=>~W0sdTi# zNjL9nPx}uyz8jG8xY!1rtI?ZySP^O!dTt_D!MWPXT^Jz&cnp0BA4M?Voh9y1YxIWb}|mXFfyhe+;j#5YMPB~O@* z!S?eqntI8K<-6qHjlEX4Vh&KRG}7&iPf6~UPrIry)@r5%e8`$AB2CQcDu`6hJd8_# zANW$0R$uQb*cx#9@nF0hxU(aU17Pg$x{dFZG3eqFDJwMI6K^O^#)I%627vwPL8VT3 zD*)sfE?AQC&C4J3RnurC*yTSW0NE5IX$u8de{}?p2mv619Hn&@@KnI>f$}`KnCk*yJ-;Y+1`s&L5$0Huut$W{`z`{2g&f)-DzO!vg~UtGW1V z^G!WQf}q}f*>JzjBSvH#fn&5VXAJU#CmQe!<5Bo|eVz zi?2iZ<3|j?i|+F#(bnKBP;5arHz>Vyh)l$F{498UY|j&vLEDi*mZbG{jT@oCzuw50 z*?x_&_qfvdI!U@Pmsi+q4At!HlshRC<(B*ELZF~3ES1H23 zO}8mbYpgl9Ea@s-|LeP;*TvG%ziubl(-PUL~@s>@c!Jfgx6y#ZH!dD?WS-@uUxNjCx4N)i$L%_nF$9>>O&L zaj!CGh3)9{uacZXE#<5da5~PUpupH{J7|KHTuDhOHMx7j?3Bwc%6)=!X6-H+6U2x%*2hZg+R}XSH)TyIs3Lcn(^z_S~h+wgFCBHS|IxFSOe{GAwJcj7;*W7!#9h z30CSSyH9qNN71?V$)iu&QcE0A)bZzvc&1A@8c5VibIb&jwQ}|=nWv(Rj72iK^KuX+ zoa9cthM6v%oSmFXrV>e;yW{WnMwqpmD?1r|=3me|7+bt`k;4H71G8&NawBx(3RSNY zOQSy70yyQ6b(|kmA|V}*u7vif+wV?9!SKG{snTv$7?kW-v~L5YJBl|FttdM_(M8Rc zF=iS1)K@|3q%55+AZ>l+Tl8#fd@{;3TRGmu9D%WKLyQQ!qJlJzQ7Yf2g!JROv1y-P zeE8I|uMMCqZSf4~I10=SRk@DSrk%acE$`e@F15c%y1RbIOmJ~U;Bwhw&2w^OKX>U< zux+bupkB*a?zqA_;eKmdYkqrzEwAFRLAi&|c0HSA0kquNIa60L zDCB5(?zWo8SXWu;;2VFJPJQu4`AD<5wUdcjRc?2i##AgrNkW_@A zd+Z=yd+6ZPN2%-Ce%%RDV+>vg#lDo&|Gi+e2)Wdq;mT_4+nLjw`2&+hW=! z>4nU|s4ryF;zLtLT_nSO!TJ6jMdkE3ESYgQIPqv*yBz{D(^xm0rAJIAlE^z)h(Z_z z>FW>i-znmqZwtcMjaOf_ttijhA93F~*rVXyOY}l}9(FZRXDL77%A9m}zA7%MauS8Z<1o|{dOP0kez2MuU$O?u zJ6PztIux3C(BpUHOv{Ev+eFyF;&xi*U3Gqf!>W4jThop+u(i!$f4QnDIdDp_hvi6C zTV=TfGk08Z2}!-K+xoToS;bmV zgo%-fVx+o0uXWp{p+76;zFNK-PvT5$mlmqCYcH`z!WfoGg?2$ZkyBlKMG%`B%L2Pti_11rUd9__W5;t=dc;4Eb?R3!^>#|{M12X| z3~bgLIqi9Xn9hcK&n^f$`llR~r7s{T``K^8=eHyU#7Xx64c1pFoq_lC3KKBqW0Kr< zRi_XKT=WNsF>OOS+^pLA3GG(pdCOgP7dGS@H-wq4Pip6$z~5Sl-pO|(OEz8EOx)~ZEaw;IS8K*9am}851Ev7zeyOYq(#bz3MkgkPcYw>{A78J zgMX)|so=Ja-0za*vRygTsM~qb&izT@d0S$=T|;rB^TduMuamT7*<8;SkG<2yq`j-- zM5=vVL6IMtsLy2S#4fF!I2SwRX)ce8WZ3{KDCi<>)5=G6?IJU(Z+%;eRS zaV9d#9FEgUA}8-r##Sh&zEsiGEg%%Du^?wLGqMa19^@`<-0q&4i4L{rRAXeXX1jDy{6N&t7Yh&aSkkekDq;cN_cmj=3Cm;$VbfyC9O3pE;f+mqP3tqUQ|*8&Se zXg*Tf7s+C0D+^<2g5_v}LnPWHtl>S#uSAKJ$phrY9_w2}YMyC0QE8mieBRyXKVAqs zuGsO3lL}qO?^aiwJ}j}K#;6Fi%W+=b+*xRgYj2P5YRe?mCU#m_8soNj#Sdl+HtW==R4jlZ6l$L?&JY-hut zQZCv`+0q=gE9OL^Vy}aujv;orRy5VRzU)nni_-?5Qx2Lrv*tbRQVXCeEy96j6ZYkqELZJw%Xg$l5NFM`3bp+SsB?Y z;W2+^JoxL#N~>+FyeGA6@X?e0q4MY>uy7lZI|l78iT-1^gcxHxF=r8T-LeOZK5yTb z*E&6yi*-{DI$p||zRM3VvZLCkDrE{a$78XDTdfifsCMC$yGqWP zzP%k-h;^__T7DHJfX>Oq!M!DvX_eq~k>haEZ96vPnfRn90Onk~cv@Wm&(X1yD>0}W z%3*oEC0hkk;2s%Fr`FY-qU@E7tjx8F-5kuK^o_yPC*Y~)`F&eJW=QT$a`la(Oh$Yq-t#|J4$7U7K z(e17=LUD9cW)6pC57L)}G!GkO5o|=M0? z;T^?nN7t69RX>l*$7qf04Gnly5GdFxCX=kqJ95cD%~R$k>oTP<5X)ZuoW)sjhf_-` zJ%nBFGCg5s4s9jwd4C%j#}cNtRDOt~*o7#d@nCelRpClk&$=B&e;i3*j6<*YO7ux~ z{YBgKrp5{%kba3Z*Y_Cx<%RD;L;gaA4KR_|ZXB~?_;O^klkCOG))qXY=laq4$cr)} zkjFGb>q(|+iFHa|?}L8D1> zSyp|Q)zg#`ml$cy*|9(`Ce|B?N=>#c0o#q_KaF zkZj6f#~~~R%n`JRMM{1#6f0LQlr#<*qe){%?>@aPXAFmW{lzt341QkY>aej_4Z1wi z%Bz-SU(<^$wU?S(HJdGXgjvoT4ruj|7uOXnY>#Cr)oO<5KEZB9C0=w)*d;br=Uy%9 zys>w2%W8azw=*?fq`754Ht}Z0z$Cr~hxayI8opqAdwXWGuDFzP#O;NU3*5XDRDn!l*%#=F{B9;qQ*i-(?ZHkqi;=Q%y%LxlQ0N*|JOuY`_AH z-@5`4@}ee}t3TqF!T;Txp&fW^WjXFaU`*d@ z5FhyRLz4Z`9F<4kM+u1T5$3PLSNDN)GKAWf>ke!r9w*Wt4?$Rb*eLuuGz$txe7D)j zo{Vp@n!}@`In8~QLnKqsv-8k;+UaN>_i4d@{JF5bbA&~NAib7Dgjj8^mHqm%L!xPp z&HbjQIk`WV#^U|d+YKm=dLPh4<~3Hy@xC>~#&(`W`R${}Tvzjc3O0n=hdOcuh|iI8 zE+9GQdue*w2jA-1eYnI8T0{t1Q1s|8#RHRom-mPKkG>4@1pjTRE3yHv+Fy#AP2I?S z2qdI)v^39ZzM|MfF5eH?zHa!?P$}Bu3V6K{g=Y_*pZSAqG_fs z7X<=IF8=wqM)|_-z6$vY8aJSgfjBq4pfUgB#YFSj;$LR_m*zm-c$$D#^w+N=Cw{;L zNE*cS1n>3ViU%YIcUGWzA1EP9k+ZT{ku$}153+6zEplPOUqz3p3IUF(aFOHk3cXVC zE)%PlHTgPQ$Fd_866K`IN|%=m{xUht@do{`l+P+M-uh>(o+4^W=j0bt15ig3O1Zhq zw3%>Uo8Tz93bHYM>On4$U``81%pM~@30Zx>kUN4i%{YZPb$Xi7F*{BNn4TVud|g4N z3}}tgQFk_1c2+K7-R?`1P^;dnl6|pXdk0%^IT=3h)ekRyhy2qR4$h>LD!m^LQ{Z%ZQh%3%zdIqnr37cJXoRz}VVWF&XczUH(0DD14d zsyUx+q#;Y#Xs%DC&dt-=acmB>73KIm}My!3zlWs6>o)4v&T zKY<%I+i^RxC(I)$+e-L$B~-O+%GewFGrZ}&wGu-_So5Ox9EPW^P{BE5cc zR00FwiWSc`kV%dF;y5fXA$e4oHEofWULs@sIxc!L#Ow_Kpi(B4V|>8B842e7Pcp`UU;i3ojwqTP5(gKlk8 zT%a1v1J!z?pCRQob3ePWRqS1MUld^p)jlI0lVtGefw_muvYz@_`3KH)l3y z-s^aEyI8tYd|IY~{b+=jgMf&WFN>YTB=2+VH6|TlmD!$cCgr?u`ZsWkS9#Y54WYi^*tliv`N z;2)pBq7#bvP`k!870lcRN(*=`5Y(vN`w_*jI4Rvi-(y|nq|ju2^Dt@Kqtfuu9H5ov zn5_yk+cFt?r!LsUl1IYH|rGUCNtN{=}ey+ZJ!ePud*RWDIR{!6MTI0t_z zw}S*OGtXZ2=UbWlIn^TfSIJ*|y{)IqR=MMF1#88MNWB$GS?LA8YbmQrS43;WrR8Pl|Vh$NALSb#%N%E@x;Ui;p(sS2z+VcA8y>=@^pgn z>K}0HC$@uIaH|-E4+UX6JBT=f?;rf$!mmEl-$ML#bWLnDwaraU4Q;+Euzu70j=Z?} z5(`TGnmiVVPzaBS= zz+yDVPx56PI6O6BD25|Kmlj(-TcquJJo79?#Q4y#q@8M&yE8$Z?bZipef}j{hd?)* ziZou?jKxA26SVU#xhJ=1T^rfEDwf_>!Fd|54{h*(;(oxeS5{_@g`^ujcQDBzuKDif zKnpC8a9>ZFW?Y)p%4DKaO2=N9KVnz$5OM8lgtSg2fJ!@Oh>mDRdO1FC^L6{eNoym7 zAKC8lk~*BoW87yo{h}X?WYXRd)jdDI9>=I|p83!i{`QL{vrfDCNZ-QCiU}v{Nn>N- z+jxaJej*a)XRXVH4nw3Yd4&G=jXGwE-4~QU)XFY)35G|`$D7X7!g^N((% zb3Los40NQtz(IXWbd1g^Vm4qkH9E$ZbHG`%<4VL<&wVk#+y=BeJ%H{Z1Ct=^N<;1nw;tQ2WDi(aG?1E_GiSVA7Jc5$BbN9S` zIFIVuhhwjj$EHdt_`)|spO(ls(X!ghSDuYe?Na{4c5nnpuGvkHCkWe{LFsyagYAD@ z#`#xl2dk@L|IVm)6jaSq*&e$lsXscveWESw!6Q*gDLef!ry|wW;Q8`o^k`0Vo@LF> ztBXBtP7=?@8H#;=WvPzu`OXrZU8*KJNfh0&g6je(`k13v60Sup5i8@?r*RTO$H|Nj zv7k#~;mDguAMC{!y;mRh*X+8Co(ADMo18xQKCg& z+qUH_HI?;oz^V7e_26%lRZgqA8`!~s$@~kc{tW|Nnd%bevYaGz#h*# zLYfkKx*?qu1D`#bGt4QBy!Z;PnD96|w8cz1_b3N<)3&*%N1x&Zk9J;M2>y`alT7a0 z`13Gbq#Cnl&+L7i1^reh6X;_a-e@>gigv=99pcyngYY|?-6en?`mMJ?9;@#Ks}Cd) z+}f}XFbxB)p)wx|29op!9C!eDdH;LYT{LQ&rv0G_*Dlr~QA`tvSnx-$f2 z*u2u>73$!Y;?d>t-|y|DYh@RB@y_yC*S#ZRgpI-E#(xO;ko-34wr%g6v1Q2qSpT{w z39|k+>2gGcwc<$?`{x|5laLiFDHqw36=5392d zyC2ggJIWS2C`X~Kk-WypTpG=YgBp|~QmkUQV?V4|?V5I&8e}ZU77<3D@nxJJ8GeDLa*TydsnsBp zhrb7@p_q8zH$0(x>vLxDCI@8|N}ZYagn0*N1*@ope-(iA6wpk|COeE`;*?bxYc6hD zCn+$VTkrm|0xkso1k;22A%k-oV)k14E_!1=J8G|o9=rR!s4-~63LE51ZyVPs#IcSW zyvD>6`iWHprq4x^cyG)sko2}3`*vQjCrvXzmQip~sG*eOntYM>=cQv^jJPA@5mwq9 zAwHQ(jSPC(u@zKZ$+0Ej{xlsz%YX$rezo2Z12qoI+xw}M{whkkmPI1U1 zY2KC>UKvtYj#o>Ha-ZK{dAAgC&S!1`dg3|vT2~pt`28()H<+6b*gmMG038az6?}W~ zcWzGeuk8HCY5X@V{d=&5?p2B)Kre|WPZ}9Di0rr*sCx2Nvamgh* zl=?Gy0rrU&JK($BM-*>KKfF;cgK29nCr4Rmi{cb4dVkkYT=|$n@2usXq4oYs{l($P zh(Y25#VF2=;*RMG(W{Y(hwzbA{iOs`(w%l(nIry*J{4LA`id-+ zc(v9|liVM3GROhbec3B!&2byvkr1tp$9$Br&qi$~((G4^|4^9?*x)X!%ELJ)lqM$N zA-~CJM{-}w{=$tMar*f4^Bh6J$t)sjX@><*2anAr+bsPPhC)}(jGR}%tZSvq0!vjhlJsML$d*<+h?Qz}xW~jB4 zxz_sB#?;i(trz>YRWV}#0{P**r!D&)YKf5D5_s6i>!x+K^SX6K1ga*n>S|OON4h)? zw}?OC63lm+rei9;M+`Ck0t>&w<=+|44`BEKpx~>u!SyexrM?5G7lr7;$TZ5s7wI%9 zOZvX@3IWa3KL-K^O0AVcR6-DPfa`Au@_VTKF_7Ot=J$ad33x1u3#O)#PpM#h*-k7r zMqzP19T>~|IhZ5Rst9JRU4xY>;L7y7RrwZZ^ec*3>4L7)wYJgK`7s(e5T1bRZRjow;5880=>;8guz#7}w*Xff3oCOAUC{OIy4GJOasmP-aQ#cu zw$kF}sb47!;Ykp>!51M?q_8I^MFd?L*>cl?%#)A~#@anB+2?1*{9aL6)r9+Xj!TYv z2UV5RpHJ?qF3-Q?8WxZ2AEuJfvy8(VnL0AL_(b)(#^v&8|NH^ZG(ADOVkDKXNSP~p zA}794el_1>@&{p_qTHev@M_8}By7>f%8%%pj7V&I+M)Sp-!Y&zYMwc@OU4EXX2?E$ zLXytcWlj5-1G!twlDte}olIRNl-WUz*20MY-&J=D8-@Qjo?-46-UGw?rZ7S5}|Emk-3#29;N$(D0DmJ z^M)?v;O*A(=aS<`B#v?UCEE)GKj9VjKf>!D@x%X!89&gG0R*ey`WK-3JG_EPqk&*0 zq$mhpBSBQ)cWjcLm64AAPw4n3Hu)Q>xFaWSe4qYt-4*EWP-xx&IBp~Aht%(DhhJ_y z?c)JxG<<15&p)4w7f>zlI8`}#923jxh`sv3miqPkWDItze1;n>cK(3J@ZmG$#+@7u zKoeV>zS>v0OU)B_+wZY4>AJavg=^KMBd&@{!&MiSkHeIijsS}%io7^hNfnFtkC|9TifX&~TRdP1oX*4$7Gg1OH zl%EZB5eFZTYCfA>kJhG|Dpf6Hve; z=06d(zwg=K3EL@%@__6A1Qc)}s|>8L-7!c*?Fll5{7!cJ8?*RIbkpRhYtEPr~ z7yX}83Z}n~yQon~GZTm!) z%L3~xo;o}Din@7H)S(L&gg5pQINqtkQeVAzMfZtWpJ#U@0O^$WP95V7}m|knP`>X2?wWqpWXwNfh+ip3I-zFOP+!qWyW;tq1 zFFwmA%%26&>I7S?XC*+`p`dUMOH{87m&MIlAt5jdfAMO(N9-ulLkUdTAna=}@wM8% zebsmhh_wFX`S3|uwf8&X{${GAwHdX@+~oA9gDEW$tI%UT5zrUpM<1;W-HtI=y`}nY z&YQR$r2;GoxyXB6#}to_*F<1HQ%G}RuGphtqWWriLk6t#APwn!e8#oU)6c{z?#%{A z(@GbxNEcFn;~pZ?J`ti|+Us+68}M|?YZ}Ifrd%3D=>v9p;ILp^bwzVsm9#~JDu!~`pEu6%#udN}4gu-9Cuf)$}Cwpz8{#Lcpeq?73=D?KPnJxnEl`IN& z?Q~$si2Mh!R2j+ukup_DkCzGv`QzbQ@l8niVyR2~#d-Eryk1_BJhtfMRd1LWnCI>M z0CMD_D(zm)n|bRH)&hZz&kR^4>MIbLUTcr&UnzPNDWsD*oiC^6yV8Zgdp+NIw!(U? zJLN02vwA&Z@;NUiGYVU&tz&35+~ToZHZsj>HpYTLOCoW^dwJ0a+NMDPBeuZTM=QGT zgP#^A&B*A)Zv)*~o4>Rj1nR4dq39~09%2&ijw~GX+PiH*I9?-zB1v^dEMqaz#;30T zMXinW%NdTwM%UEhr*gaM@9X~aEC?o<8&H?kwEw=pe+2UX%FzF`zhK%4uK&8fO3%LS zFN%KJzwPhHANwnAs4j*2xGh_6>Se{j=n+8et^j~zb9MH@1<-@vo9<9)P5=ER_o;FEVq)j=pUiF4*E1^5|GhzCGN;_=9<6v>bDuCwTrjCEE z0-cKbV?50eg9BFR9kT3+3QsaalS>ocOhPKTY=3;73jIaiWlNtG zeTv2TY|fO9!SZsEU7lG0TkhR8^!qCt2d<;~ISxMjJMZ#9;wh82%Ol~o6}&(BCm8LX zDOjU{^M!-!?}V^_!Mh*ltl!|?_j49pQ-$A#oZ`&S&cVqmf*Y27z|sP^{s(svAy0;b4+(hX=tYWP=(|z}N!XnkbpsqQy`APkHsF#HcMo@o?KmF{Oo{f%5 z6Di{>+6QsaRQ=Z$ksbt$uddroFwRE|FrW^!IC``;VP*}{lu&OpAkUWdTbG1~Dc*M^ zJBoXeh`ji)aGS3q@KSP6SITo3<^@OtYtpn9O=iFmjC%{8p2iX6TUVD;JLURCFHgSEdB?V^ z@yiPHo9vie2AkVp?}8#t(&Q&GIM%K1O@)Mjh9Zsu9GxVew9o>r{`9xGZ&^`b7L~x- z*C9bojPX%N#VXE(qs2v7IM*I)hp~ZATxXnPIfDkgYzeNitdWn!=L-^DjeF&ob{%l> z?)Rf>?AaZZ+n4r)T;0)T!RZ5mY-M*5y6Ed>I2dX78HeYpW|+!4nXPdNcBk$r#fJKg zF!gdVW}OpG=H2FYc2KKpZUh7vKV^uYx{kctD;qk@%HYdbZXwtdrkpO z#=LC*`py9ah7!~#k2-+5jGOLG$WY>>7HAD-RX;9A`iRZH_)r-|trx1ur$~Gh@Cx>J+Hy2EdMr%zR)1&L#{?Qz zZW#$8H98xVV|ddsIiVetS^-XvZzPBpQZ)C|DCN$4*@s!>2^mBVME~p7q)ujLcE+e5%Mv!jnnj5$U$;o|FD-%-+t?V|7@C_|{~Yf>=t+J*?Wh0Nx}KqluA$C#GGFRg zIuAKLZnN;8{eyX6tSccD*hT@azwQ5TOz_9z{)Q2L?_Z|%qrSB$;{8|U>DtOrD{pFs zhi*<<9ZVQE?|pR5wk`NM7O>jqKY-Rh97g^OTEEfwfIIr8nFOCarkmnPZ9u2dOHer2 z@1%|2A3VM;{BI5&J2E|HEp#Y#N9a=h0wkVXqr{OS!ujtUvix%V+Na7MNtm!Bc@f># z7sKBRFr@vIgYObMm7p5>s@7C7s_(@YF}=K)V*Rk!pF}>Sn*tR9arusOCte|d9fz>{ z>ZcasY1r*jDSQAk0zRZe)fa+-hLGX!m~!-?Fsq&vaSfzJNzF**WLedJFte6ia85I5 zS>w9rFCX_x(0|bwrm>kcg3tD)yY-L_i61}RK|=edP*8Rm1m+T@E%}Fc41-TnOe#Vk^c)VbeW*PJ>M98vvVXb5F-G{LLdN=MXsW zdPg|~d4(^EQAvy-4y2-hiAmC&-ar40*)Qsr97pFjlG;w6TcTu^}^30!{*(0@nW?+Eo9>-u+q zzQ2!O(rV5imqp{oZ$l6D!S6H&*8O~-4zC^eWrbj>eCa%^($Deym6D+RN{X$u4RlR4 z{~w9^qhDImw9+;(wA0o3k4=Fs5C27P_HU+#ANcxelgauRA*Tlf zxxRrX@5db}e|fRT9sll|f({z!I=42UKSv4(l<0g7KGetdE+e-yi!l$yj(`+&^ns3z zd%-kU-lDWj&6z|&{-l@|onnIr9iEa=q837sPikInN=KtCR?|I!NydDqUGi|IVT3yS z*${1%g1>rV=A#o;^LZ!D<+GbGeUSh5wofwe_QESSGd}s;H?Pj(U$cM|bd(PDVV1q+ z%$io{lv8aLNKq_#evKX;M6-pR1k$gVWYS(8(sc1{K+;n)KZ>%rQ7d4I2_ky|zbN-C z8_9;bZNpWnb+x~{wT`!-Xyox7nLILIx|ZLx;2|Zxxv#7U>{!z@z4R^4E!32(FZz}X zksUROYGrJ7n270HRemKDTsh(CKRvN$Uxi)+m&kzxK=W2d`}^&r5-EXi>VJE{69dzsn<=_Bk(x;jI_25&Nl z2WjJj*sh+F4?gY84DCJZO~|cI-G17ausib_Ri(o4U z>E}Ll8SO3uIhfua%9r_`kukI~hZJ8q2vj&Hcr3ka_zW7h|(cuA`+{~)~k55 zbDl&CK1d^h4$@BeR`)0q1-#bDV!HbHn_!E?`tQAL=^rZ}Jd&}+XjgfdIMxhEUD}D` zrIhfG`RIRRYoQG~NPE@^Iz?a14NqPP2>ha;(?b!^OG~gCKm!`Nb+Xg-db|?7eL&T% z^;TsALmb}lRF^SQ^NjVfd?Mr10)oKc=RUkQA#CuBE{T@bQU$Mkefhi(CF3v!@+B~q zlX~U5eJ&2vW6nC5&mthth5|t@4Q-|Grz$y&6L0xN?m_7uuGKl1FY@!FnON{W8f$$j zGLhW|xwjw&aE12qWIeutXcW3GE+rUXfEL__!N^gbU%ELy)m&9s{>3B%CSy8`hL_(@ zWB9)ipFjK>ezE5Lpbg+t>OY)RztaXDz%=EWCDF!D8UaS#QXHn3|8*YvJMRAI^EdeW zy-$h*NLC~<`GoK1A#1e3te^dZ*~hOm3K}#|20D6{Z-fOgy4hojXuK01G)%X6`b+onU*|gcI zb#&=Kcc$sm(^G4*(Q8q&vd}ZLXftTD(b0jje*SUi`=2hf`oj(3*E`?A-1#584nP-L zxk$?RUPsK9YZsQnetT#X!Jzn3xjT#SK3pJWTt=5TjyReyJi>jifPl*| z)iA+vMHQh=?6!lQprA`B4O|;+-8VuuFAhB6&YLVPYz5@c$PA~;UW>vfzqWRtLk?Y@ zwH~;bn_yq#VN5rQladKcZF?QjhATz=Fmq0cP#|GFJC`vAA;YS?=Mz9>WV=8AQ3cRN zFo7Zz4Z2vkEEofR#qg-z3w7s%1;6&WxlglESsY>qi8oWr6t@U@o8Lk~VQ#K-RWgnG zAVDF!0t)qY1g>T5bCo)jXAS&p%3@#I4K3shB9K@}o@szCB7?m^b8}s?Xso0e#M+-Q z^cHMRM`OhVR+|V<9qcN;6f(o2H|)PKmT3VhyHD-c8ek3Nw69d-wO}oPD573!-{(qM zg?y*L3E6_evTAWBJ(T5a01Hsgi0X~G`!X-h#ksogIJODt?Q8P8#lboP7^GU~8**o4x2qfEIcfx# z8guEf!yBI@WjEs<%LaO*g+1dWXNN7Zkr>yl4xmlgT|CZse!dFp!jjpLoMYV|Hu;FnjuA}x?76(G4i7;6^wL+-}ORBW7wIvKy_8{<}IH|IogIc-H390T1i zT+}LEeOTJ147##o-j}zxY`JgRts`x7zz++VBU8m`(YM?=u~w_Ay4J{#hvnIn*eCf7 z%LyFQU6FbrhL{tKE`!lplMbi)@)iZP1(s^l2i|GXm16ZnROV>?@A4~5Dkz!s3r53h zJ}pDXcuL{9t#rY&XpV6^jE%ahA7w_fz_mOJg;rMDdRWfJQS$(<#rX4loNoGb?eb3X zO++13IKmE=r;43v-)W^)iXu-A#$6a1a1K7?;+BVGLg7R0K^iCi0o;W7WmdndK`-_Z zrpig_L<9TzP6VX{rDDZu(`lD`-AZqxxcmBnZx7G4UjH#QC8o>`mHBqcrG7`y-StY2 z)yS-5LVPhe7}Cg>*KAkYRLFI`$M!k*kzd zA|VpVN@n)n^H&m*O=R_>6d`+4sSu)&nJpu$vLpXL)lpFG!I#`cnhPMC)e4{A&(+ zOpx5zKN=_4A?aybWN%Y6Uvon5%Yaw196c?@UP3dSjl@p0*gpC7YiDxK#a^xaRykcC z+oYChD&rT|e;B-Rx%4$$Ol5pR*4s*HVJ6~W;df^85&wj@EQK>#m0K$L)viqB_YA#p zExKOXqIP$&^@P^zTsk3ZYN2n2V-qbD^l68*>^aymmi-^qVlaD^)2B^TzFieRY5RuG z!%-;2henrXtjd1ASc1z-jOvDRhF;tKu&0Z)JEgw*gq$$)M7NJ!{#i?`L$lA=TrfAu zrN-4|I-n;a=V!70cz*$XnqZRc)1Uq5e+2YI15M_ZABpw)-#KJ)+_K>1X}q53tP9Nxx4n zHmDboc9W0l+MSLRik`BYew1O<+I|}+uRo=D{<(3rsX%DqiHB1QmmKNSu9LTnsw#BP zKiACDEB*cjHEMOoTsm^Zah~cddjbWje>D3{PJ35(?#-gff_)=%>5KUbErumkDxsX` zn=BW_&PYxv7`q$9OnYCA-?M0LI&l#*{^~*0R2R2Mll_ky{Jl1)=M+4CKP>gde}+83?uB`Me{WU1^bpQ?(xTUN!)rytS7Fa1;e7j?0ASjn!}(x2cCZZ+FOEs8|9qsURJhpt zx@)&{@#Mwa`Bx7Cu!jH^RBXJ%Q5k$ z?CoJt@zfPmnn{v8vRL;-zV$+h`C#srGX^jCdUT3tTlL=9-pjl?9&{p#s=umHUn@pt z$iE`mR#d2Yp(-vTbi3t%X)v8{77_X0#{3_Bx8u5}PK7Wo2K4X$=30i?)RoO?p}v)Z z#?rf_G1^vDs+|7>Q{HQC5qgD)xRdXjX}38z1=cYb8GOsas5nW>KiR=xE1zIq{UK7A z>zIYUQGBE_SFgD`GskV;53j^4PG;=V@ZqHO7s}#&BTnJvJ2f{S$kXY)>oJ-pv&>V< z)Wn>xnmi#`shEd4a$k(4W;%UoT?J2{kusNx_x1{(Oouen+4qlF=dFb?_C@ubTQLhM zp?5oOxDQPC#JeRp8%?O#sq((k)eEW_o8}9PrWSg|K$g8-;Ke?xg86sZ!CqpePx@#M zP%to72W{k#@k=^$n1QK0iB}^y$bl)R(8>jEA6YGpE-A5cE378p?8h;L`8Y<#C-JT6 zLHB0+B>xOUl_SNv31`1aaRi>NrQQ`r?J8uXE#ifFoU0n8aUs$w*~dWofv)x4&qa{| z#8cG!LQsC>V^P;+GRJn944xgW^tyTQW~pTUgEQT;hm|r8oRu*gn>2SY_o8i1s(cLNbPV}7g9mQn%O_B%FjFhgARs>vZ^@}yUuPx__c~;c) zDvDFS`;Ale11Et8Q9adr_nERNe<<(_y*<<|pH&#!`X#-5qk6)Jc`;_coat^eZpjTN zWXMwA%8Q_5lP&J_E?P%_7RVBuyKGXN=^b0hH<&t}9kZ z^;rGbNxzTzkj|&w%-wDS5vHak2Z)^?eP|0g2obq@$R*3<6wm(ttBmOWYP}Xm^Ioc$ z@2wAw19YX6Q>z)hO$ov{5l)_>)yn(B$zd=4+Z(m4j^+-rB8?!&wv3YzG?4 zO!pUXR6O#tDm_qVs(4+ovZ3&G`-=0o_BEFq`S;9{FYupU@@wOxak)S$DB${A_oiWuzFNV=75d z=ATOUNHdo_?rD*)FCI3y(Q?6M)-WRyTrjbn)sJEw2xScp;pk zpNUcVKut#!nz`B9=!1Nqjtb|O#}b@B($2<{?Qj}RP)XzNjVw7ipgo`WR|4fW)=20t69J(aMbd8(&<)8j^Sqq#48v1 zW_lgQ0|%aG(>(H1^v-`B#pAP0T0}SM@N5ddzf6Nw13{Mbq!2> zq)1#Cb~%xHY~Bi+@@)0w_@@o@#11XOHcupN!IPUdE#!U2B}3=GIl3r!wF^8Re4E+% z0;aQ8d@crwap1=0owj$tXUxC_!FQ$L&)6`#(v7vuHWYyHttZl;2mmN2iToYz?e?_n5F>i}CnI_Ig- zW2JYAoBU&C4KW6EkVMQQJIp@lg~P2%MF5DWuRa3A3jMpV5n%4#Z71wCFumWtOnb zCs2h-d)0c1IRIg?e4^*?CBI1DYui9J!9!24%RY3mv)oLP2&nBJxJWc0ls<9mu8|Pg zg_&6gj~cLmTe*{!5{TokuI$i)0$-3_3K|7=l$JvE=k$Z-NLlWFjVft&(K-sguT`>h z*aPGTD2LT?AK4Qdqva`&{&+HW+wd3gmq4w?*FaX_!Zwr36}x!b%WdjGje&Ea5BX!Y z2aPrikYWNhd2~nlfS$Nfdj!fM^f_n@F2N+IavdPySKk&2F~nZM-lU=%fM&0x!2vxxrVaX~e%>1Z?hPY5 z0iAB^e%yD?`h2ha^>yqI;XUK+vH_Fe-M3>LN~-u+XbySp9}tqC(G|%Dx&8h^RecsztNo zII3@Znc!gWH3Hn#YA|8YIP4PhU+2 zi}YPu8=@66bX<9~&H#Tdshib~Eq0!0saIVT^3*uby3Z8b+hD*$fXQ8A#Ya2I%nCax_n zeE-t`L|X#os`c;7FSixmu8*^&03?Wx^MSwJ?U6wQaFg;; zx=nOt?GS#X#Yqe>9(;cq{BRHdp-K8mGQh3A(-Ul116q*}NorZaO6R~FEy|?U(1;fR zCZr0#Q%exNo7^DGt!hIIPN{IE5&f%<3Tc!l%K&C~%udpKzVO%p`t7|x*69Aas^@T8 z4qR-rvl=#kyA`jr+3BB>wcfrjV#`-9D9~Sh9;{*Z0q= zsQmnG;zIj&R9k!v-g9c&eSFWDbw1;9Ifnvq{T=)E9phtt zm744@&dMXL$}tlNa;p~1D$VihvMAL`xS7@TYEfM=OB|(Q(Py&n&Zluq3i0Uy zgZEuEYW-q^5}C+3sQgXzqjYDXc@x{{PV}Nf)jbrTC6uq+n=tv)ZMNze37wU7{&cg3 z>?lPake2gBGD7ItN!43BQ-$;vg;6Ie)iO4pzJF-Xi4Nl~De zjBn*#x)N6j%AQagX;+$(8+F4XMq|E}oHqtiwzA4=UZ*_GZ!uhY<=sDK1NRxm?F5X{ z)XohQP*bp*#>FSyN#&`1eX4w9fwb;yc~mP)nSKymdu0mWQ_}Y)F}6DH1N9m9k$O5T zDRJVe*Gebld7>%hd9sEh81nBWxU*i4dD*ZAJ^r+5Ph}w#LaY-9&j;&5>wzL?0ZM_2s%_s(Yi6CWy+Ec5jPQ z`^}cG6DrO85wzAEpT3~vD&Kw5yJ&W+#wE`q%^iK==`Q^ybq6$N!ot4B&IkCh?+?dYg1vZoqPrD)qeN7?w)Y*JS9 z?bEVTF=$Ru+sbMbqp`IoL3}Q@p6L9L5v%aTOT7j*bLMKBxff?*W>w3t>Cg?MdHRp; zemT=GZTx)1+ghA%;;VVeIB&+%^oT=Un9!#HCe(U&~4w%e$X6jCHIuwKc&i-3nU(qBZht zE}|va4_B~PRMlK8m*$W>FUc<1D-Uk4ksFsq%W-jE;^NKV=E~vDR4}vN0>0(t60->!t z1n41`k(TIzb5Hj-NPz?wmDW>Wt$@JcL35mVcc?7%k~u&H)(emIpm0uvY{e6_g9-wG z{wgio3WECb8x)#XM@yGWn^zYsCd{wH!K&$h(ihVb?5jcDvwf;*}H4_?2{>|MonzgGR0tp-heR*hCXyMnx0CIfI<%79&jgtPkq`W?pZ4~<5U z9M)`W62rwBwA~C0vE>WdE-S$Mr)6UNxNZa5S+=UC_#SBOc1gY$A@DD>OK4I4LbH$> zH@N!Go&_!M!Y%89#QR%S$*`7Pb+H}lit#Q_ZJZJR0Jc|Lg)}&^li*z@%(8!c)E=RV z;eQ&Ckl({?#10vOcXqDW0_Jdb55T7zFm`|IL}{$q2*eN6zaWp{h7*erMiBeEhjJ6{ z{2;3#kKTrp6B9v_`&(vP_UW3Y!$%x)&VpkH1|T?Y{40iJx6V1v>kUyjD98}V^OoU; zP&$o-`om@i8=-z}p@EN3;OWaCuQg48@6M2i&cb;OokjBct4GkTM-FGSm50p2$%RQE z$^G3UXF+zWB&Xr#dpLGb!$Teh3TNjlg=F`qCme8b^Bsl$9^*nxI)05^SeFtQ*dePS zk6ncGDv&|&`ddcB5+h(RXs(Ss0|HL0OdUb&Pwhphf5#GAb4ICC#&2bUeM4{?K%hL_;1@gYa${gJCj_aCm(~VW@65>Z$NE)&-Ax>9Z zpO+1j2Jc-~Y@WxOl>ijvZ)90CI8?kfo;VC@Wv!a^J&3mvvO*KwgE#H*Q1H4qtL%sd zHx~YhYAdTS!ByMgfX9QYwleKvDfbbJxk^2z-|sJx)qdbQ(mUZX;X1Cg=Wn%tzyMrj z2Ob=FWpy7o@WZu$*VcAMI_nm%xqS61_ZBR_^A z!~DBSI4uKX9lUB?AN;e3m6d(qpgUdGPpi? zm2qoTs8u|Q=m7uYKuErP=L$Wmle&$^T&iXOh9-eA3Njh8mH}LkAehW`O}5;pu5UZs zUMB(V0xhOX~l$LRwfn0qf8A3v#zu8gErRfYmQ(zw?pk1 znP3bDeFFnUbh<10zLlZ9W=FVsym2lY6>7|40eGXrwOyI7xZK%ewOzBL$7_pBG=^*Y z6b4E_+okC)0VSbZU1VZ09FIK^uOtF*<;-LWk3b_{nLG@~%Z1^=jJiq(i8LyKxdPNX zA`^*mptwoU{vf=PaL~0oN30{)>=<#5gytv61XsAC3&D6ygke{P&yr!+ywhDhT6Zi* zD>Q`qhQRQa(^(0(_}@Food~IoFsZQAR>EReCbq(jb08E(c72a5hr#lAMPMerGPxCw zd=G{U171JX6;yZ4dk*|l2xOuwJpML>;njtKt_*r?*b$frKn_DD0K)ZCh`>Xwl@3VQ zd(bQgnfwRGD1>9ItIx8}2b|!Chb2gG_Sp4v@!CRrhk|oxSlz1fe9&qObCQ~(gKP1w{}Qj{j_Fph9`#H vtpz7$l8Q9M-(t2Lkbl5#?bU_HnMij3$Lazx2^8yR!9Q;p!M&$h(7XQu*!7ZZ literal 0 HcmV?d00001 diff --git a/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_parquet_log.zip b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_parquet_log.zip new file mode 100644 index 0000000000000000000000000000000000000000..c3d02f98eaa124bc51346b358e98396245850468 GIT binary patch literal 59817 zcmd43WmFx(vM9U>4grEg&;)l0?u1}LgZsuMxVyVM1b4UKuEE{i-QD$VlH>QB`gV8=2@`JLi z13Ul>;tC7^fOzdE{U3+J{B=0}r~eWHl;=Mq0RImV7+C%Qq55yo5Ms-Ccnd=AHBw-`Z=}AG`!A$|nnCd10N!4B;s2fN zSER}1+{&b1nE}d*|2m%@o4zg+i#8LDjy5AB4HLT-3yl^dD>Drny$<^)MtVj@`cLdM z^gz%ZGsA0Up=Sj(voX+`m>bYC0O@HMKG86+{SCQS2E69~fZRQy;%TBKVffF6$;Y~P zZiLX2d@K^$4>7vgM^Ljl&O-8!aY_}|*@JZi=8?RFRu73m-5g=>xYiUq>9F7ZXu6U_ zezA~vG6H)g5Di2Q%x|&4_quqi04iyMGReOX_#gTCzwuI*R)m_ekGgMgM7CFgMpRss zT69weQIh)GsTA!udPaGArb+s0X5H)PyphPv~83O<8VB2Ev z;$%kzyjh~ycYaa75(doQ!uWTjnKHmwJV8S0fB*m}{R`6n<)SeBW`6P?|aBy0BOaqLo6;_b6!$Aq?eUUiGMm}aoXe3Oy-A@;!S z&WyTBdZ0qruqh3smB;d$gNd>?{Yd|K5HjLb_e93%Ax z_(^za)5#l9jaiy=kT%uIwzNip_2-m?b{=ZyP(K{<)n}#DwqKMsazuI&tQX_D-)&6S zi3G3oK8hTp$m_Zd^Rw!t!pe-`FHxThV9dC8A>0fYLq<~dT7f;ws{EYOo0&!l8^C+{ z!6tpniVnQ;+>P48$(CNU*2A1FC2mKZyM64k)>wn$xuXbOZp3!9qZ7i=p+ibRLUgW4 zeIaZ(pCBISO2TS0!zkZIX*bYNu65=Lc(x-JTecj>&@uV$^3ssgtKM?w^R!(N-PtLT zf2CpJ^rOba`QC20&F+_Av1ey4ztySCMe9J|w;tuW6(3steKlD~ofnlcN7y7DqC6My z5C@e-M={yiEI9SYZwXt1SPT3wvp6!3mR9!frkhkU`npC)i8;^MWX;Jc>M9fR-cB>c zxURymoCVw=y$`hPNhXgAmoe9O`GV0{Vj>$X6`rELs*II%Fe^ax&UIV1m-L5{=44(6 zLPyW?B76 zRRK?mBAl$O%hfL$8`qh_>0?gx8b)a4!&a#SVY3oUyj2^U14uvoMrCogYG(SH zG8qEa29*rGsABr-=+LwdO_SP@#){D?4dwdnF&~_8ToF-%-a1=TJD7&q;*e0REvB~w z+pDv}Vca-~OQ!zi_A7LlB-w7qtW(z6Kgv!mP`>HGl{+-Z4}-sR4X5isRYznu_xd?- zm8tg5b@!;x>D^HIzWV*zD0nhqO~EXgfNnKp!G%*m`O4v1|96x1by@`bfH8OThH#RA z)D*@c2zANTYdvo5!F;w#O3ZGz%H1`HcQkk7%@K)fZ#xVcGYw)^xDiG^scsrrejMoG zt`8nlSTlD~^rP)=!>K3eZecO!UcQf%8O+b*f=s%N8`vwe*CloFI$LOUTKT2dooZ|R zYqV?s07cPFXYBPAX$R?Yr_x-5MaLnU+Pwlki(@G5wEHJivmLc&iJ7r29Q>YY zj}F|<(&HZ{c8#Wt(-uI(iJPd1##RyGgT+y22Aue&p#5g`M$B&SgQ9^gM*oN{jhS-B z%%YVA^HF#mOpaR51M|xoO2+Bt5yb0O)w)(;u zJ12}G4a?%3+k*qBf=)LY!I@RAS$Xlkl21wx2+AQl6LJ2897QAXkYC{aT&8k_5~~)g z`QLRvPqQKuwXSxEE$Q>bX`GtV{lg>i^^Yp|v)xlH3;?k45de_+zk4M9fBaAVe(3*) z|EQw`cN5ed>>l7t0E03N$&k8`rJs@>`H9Fxbw^G34piA<){2iJ=! z?oB(5*;o}ua+X%t4?HKU;Vxf(=lejn z>189SifpUZE!2whqSbh3+um&`rHamR65tKiLOwuvuFY!71@zS0&<}Z5lR!I7 zredu+=I44u>{jzmYa95GCqt0>h5AC@RyE7oV1G{tjn}Rv@o@y}_lDyc=3$_#d$wR4 z%3=QimSeNhM5u#10!6Y=J=#eX@zsCh2{I@Jb5*VBSN2Z<24qK`ZQ1TyebhHaq~s~g z!mOfUTG7_*^rB`Di}=Vj@nR5n77e?dFyP3kQx+&F*6ii5+tF-M1B)>>lvq0ssbc8K zQ0SPMkZw-(hCf!Qao{lrv3D{>Kj;xq`if3FdYrvsgPOXRgNLw|yDYKNt*s2SiGa;h zuI8bQROpN5vJ`!b<7?X*t>;zXproQif>uyohrTu?=gt;nDn-jo)#Zwf+eC?O5(AE8 z@or>8=4by=is>pM?db6sMz4Q#^ZQ>-l%9@p)IT?ZAR0K~;GP8DFF?n`F;b7@<{VqE zfAngrT=Q|=E&t%SBRs$rlQ%?z&r=Bg+*DGrbiEI5Z*GMeaEhFy0>h0a?lY(Vt^ky8!;#gu(z=13-T%r3Vl$fTQ#&8W)7{QFdoE_#&AjhXg!iAug1P;Hsn`!zGv zMr(1U!c5_x*?n13+=g7=iOglF8fpM9|8^`g&A<@Jj$$FMi<`T)Hz*Q^Gt9(lZx1a# z;YrQT5Rz&9Tt7z=UK2PQfKW3oP`_Z7)l^&W>VWMtGM@Qi9m(m(%9=pD#N5CyhReqV z|Ceu6_UIq4S5Xf%Ikm-$=*IU$gt+hxig*vL{kV@zhY>0FZ#t$fr8Qck8)SG#*+#yU zz_Q@7eYP43Q)-J$fU&VRz|A5m9~8N{%y?^*h&9yFvCpC>{E77PLi7NJB>Vv?Qvkn4 z8Mnr9CRJ#l@?93X7@_*F!rkYy7U(p5=Zzr(?+60!D=%dzUcST5wwa#KuH?9I-*^}6 zGKMhgPSm8r$t)1f=BF>XOKu!Tt|1mvukiQXMv4wxgf6F3c5YaoMlg*K#sY_ox_;d( zHX4Z?Lvu(^#wxL(F(MJi9}4WcL?4+4>&;uO
    BJ!38CwZM6sSO;u6twi5+2G(145hsG(#ILMF*DA8Od{i$kb4lXTrHbyqYlU28Gu zMp}-&{aK`hGV^o3uq_S$;^z3_(02j)o9xJJ>tNz4uR-h0R6`4|ExOo8B~y71X#d;W z3(iU$sIeXUr!x)H)5&q53gvC9&0pH3TAr(g?P{A}Ht`J2XX<0;SG|MZESR>kt;kac zYF5jGxgAgXhqKAVGuo%GG*=#Gvnbv09|n6>5pDjSfc96}AzACwC(AVYGWLrl2hLEc?A-O+?QQerI!+@s{KnKa2(40sRX+AN@V_=1i83|(w@bsD-KCAIH;$# z<%_Ym`}w@=37)&iWE|b^nOmMZ*TR=^5L}0SG#pFKr4UU>$={lAS;Ox}wSWao#iai- z_h`5`S2$wne}}r(K?r3tP2>ueDXlQP6c~UUddSVJH;TIywb6=bHe2IO9oIQlP}1*w zRokaZm;Bg1O^Md*+*7`|d)Y~YrR2X#AU$3t^X{{hTXq^siO({Jj&dzIW*-Xe7dhsA z8rJW+FQFo4MN|Eo7op7i#at_2z8lQmZR-`9D4Q^_d>mN+d`#rju{E4mejT2KMgYu??x%0acx<}EsJd}V*RnzGN4*xkJ zIpu}HUfrpteV@q5X!n^&=qtoe+#|}~Q$iv|7;zaoYQV{i`VQg`bo+fXm#bng@s&_& za|_*>jGQ~CgVn%9ZyS>&Nr@3VxcRL;y6qOI9T(iW~ zpCrM``KGk%f%2u$9Y0zrG*i!5bu~n$YWfWV_pyu^cD1(i(_vqr>T? z0A~Wy6&-CPu1}Sv+Qz!7S(g+YnwV;J1Ua`(3MAsr@PB%O1>;6+6hYHG*AaLVdeA## zJ1P{9JxFEN((s1;1}19M-)0lLZf1YCC}7?4(JWtXTR-De`~%iskX<(5yt7d%DhMU(z1q z8;4{rS>R_3?>8Y_vQqKL?HijqRPdYQ(hwU2s-p48y&I4AM~ROpPbc$xjO&ic?wZvI zUh;wAzQQORZ!8S+c3^hV%Z0K+^M@&D0t<*2F|6VdlXsOrJ45zo_(r{!v+imHj5qE2RvhXubz?RK!^ zpxt-I8@>!0Qv->%3I9>91;2`VA#!uvWE>r{ZGTNa!QV+gVaXoJ{&JDu!4}to)BG3=-o^?bPLHw{2dC=^Mg|^g|mpU&gZ~WYN%Qlvawe z!hW5ewGXmog>anm+Zd?X20^I!KB>P+en?fUPFE|CmQ)x{DT<*&_o6?(e)HjbRbH-;`6v!0n&d`$F02G5LJ5Xj z%FS%gSEFPZ=`d-P{use=^NUzWlKsdvpRKi4eb=0fiVm4n-H5M0CbMilyI`3qN=OX( zH6*(5bAGvUDa8Vv%sF8o!T(?ZC0dX|VL%cRy@%4*+M~HX7QC!l;1{YxV?g z&%KWJD_jcA3iKn(&MIqBqC>5p@KtJdYk~>5H%b-JG2g>uAf}y<1o}NGy4{bX^4=2o>iOeU8<4!P4 zpBF@$@1`X#zG9}>F1f~y$v&PGQ#83N9&DQK%W|*arkrly69p#{v^ot{VV2xeAl+}P zKaJIs1W{Gz4P?({OUIGtkxiDmmWmuPHft2PXN@z7Nx)YbQ z!7FB4ki^ha#0|IgKEW!B`oqVqu%ln9m%q+6m|ZPz@>;oPGbbjSS<}D!*P6uy9Xo z0*}mGC0bl;rkdFAkBH0{n;t_V{ox64uH}dB*%PyJIOCEp4!y#N+;tZ=t!|ZvfzUL0OA3}Jj^;zNwz`GK2MF+_CXdBeGhIcr1sXTQ zoxpkg7q_+h8+lgTMAw;L=kddHjlZs}wzhfQ4>?^YuA6ulFE-NN);E`*1#3YQl+@j| z(k?m;w5FU-J^bP=e8@j5b|f*EYI#XEPh5fH+H*8q$+PGBUVmNOV`fRSeA_-o5PtWx z)-ZF@+T*yeT&A8uVcz@@&b{H+!uj~d)~WOmHMVm5>x*qr56$!>6Rz1?SNSA^AJ?lL zI7}U^X+{d+K|(QQleTj2jqbLe4Cg~2;|nixhe`@lC+fLIzSp1foehh6X9}hiCdaYu z$xv|^^(@ZKbE7*IEEPD+wHi58^s^K1UIp-%Xq1INkW1K)uPrH#9`C#qSK7x683nJe zj=~FLkOXIrqF&OPQqdk;J3NySSQpFZ@YgF$m*?ig(-tq@Up+b!%uYetFPSE^rmtaM zpYF%ARmr9@2A}R^5ol=Q+h62792U7%?C*0T9oK}f+**!}6tT?G-HlWqq~2HG9xWt(_RrR?RVGy!*ev_vC5oGfC2lf!etJE*Yz_J%Txe>&i5amqgYhYl4T3HFI0@ z1szA%OeeQThhIh4jqY`3{H*PR%Jn+iR|CAUL8fyy2lv3z5Zc_%jKFr6#*3BkriW3^ zYjtNEUR(~_GS_{Lp2}cjjmjQ3PUFq|Z?qf0qqO9Q&bexTAc)$E!qPe2!0F|){tMLU z>65Ves)n?cW~(g&$CA5_L06=%e zr{lq@c9>k$stks$w2o&~z|<87e%-HS<_~DJJeIqbL1~H7kE$ajoR`b{iDG6ATFV6; z6)QvT$CJm`tYHnIh{Cnn=UIn^&Zkgp_=m;C!mm zyp?@9`Z3`U`xWnjDOZ;C;TeN#=g5GC)`WYD>qw$@2VS+`?Y^>RN~$y&xAwXZVlSKt zA@YT@=L3vAZqx}Zl6-$~c)kGwB4jfH|5(W+Y`<2H)s61P;VkQg1ha>8d{oM_Rj9Alu6m# zri{Ho&+&2+sk_XLsBHO=ETeK8rny>pn;aaqh4q}m^et?(0*Zl!7TD2!@_yOz{f-=@B4Ogb=M@3qqFW`=k0m@f%g=0Q8F!y zTW@{T;=7TjhyZ+7PA2c>DVu=y9zDNG_T5*C0%035>z(go8-|7U_bxfUGAT4~!|}!( zi$}1FQFF?(Vo-BbKIqM~$TdkIhVP;ZWC}~8qsOBWux|}|p=|wJ4|*f$W;vcyDwiK5 zSF}}Vzh!_id+#vQR7c%pN4YiDB2bsZd*Q)_5++UP8(oztxPGS{u`gL4Gd+Efae-)dVRZM)vA~Rm-GDYhfGd|sua&gO6gRrLru~bYPsbn`G*BT*KU`jSxEx}my zVs9v19!Leu?xeXZjBA(qjN_cly}TjLk|!KXj^3368MIz!s;mq3B9wd2o4~X@9gNpc z4EZ?n<1~{R9WIEIB`or>wO`g$UL@wnYZM+|@F zSwRcw!}-P%l1p0upbL?|q9NayF5Q@pSYhynks=nP?|spo$dFPIf1IomkniI;_wM%?qO8%6J1)wa!%u3&|Fu?l5Qx$XW;&rAe(u5N3% z&99HGE0nS&aiRq#OExDP7xyin?@@-RL~CiHmTY3BzUm*0w}%gWxr*SICq{di%vw+( zNwGGfPdB!r!#GIUul3i+f;DGimSq}S4$BosH9OBaF>M4J_fRR@y7iByHjiC&7F*^zqSwHnhnX_+J{P*c?)ok&pi+0w{~!F34|#n>-gUOc;;OTo)f zz`hvm2tiHwqg49b7HTvx+--Ple#soWS(MJscnh+rl#N6l?fduAdc!7AzYA6PdqCV!!dvg(dP2FEh!eHI!K~RQ! z-#oVBu3<{3qD*)=61~H=w%{OKGAGKE%ME?g@g3zyaZtP)?mc28F`uhdkj+4*R3lU6 zIu}nqR|igFlHlToas64MLijwOoz@g&Wf5NdF67lL-%rV%m|T#@>5I`RfXTfZ8g;8W9wTeO%yg)<}Kcc-9{@hxAp^_%M0Mh4BQIAL8d!y zg_M2J&ug+0R0mZ_b{SDLda_sODFR{YV4>%(EeOK3ZxkHdW3a0T z>;c>w9pDkt!wdos05o(ZFueY{l*HRDj2!zyqRYx&=N)TU*i!lB2%y359ANya*kx z%MqUxIe00m(C*!zQ;2y0I}xdH5cJ!fw~r8d#k&=ZhtRnw9bo-E+^ zLVdBwceC^Mtct`^dxW~Skyo5mF5#Y@q zC=6x>020FOVA=QT%x}c@tlzQGgC_MzO<#cl%&pZ4Qvh_iOPBc|F!Kp}eoBGX)5GBs zNoSTFq5^V(8Ne;D5I)%BNDJtdH#)sn2DOI7x;anmVIo(&tCau%7y&!S70?4He3WO> z!XO0F5jb}N75HTQV1%ka1))JT_Xw<1fVRGkVjPd+&gl2AT%YW%X<-}rJ0k!dxIMpK zxpt4*OauA>(R%pywfraX(Vc3F1K0x&#Qv0JNN6;1`_~>Gm))Mae=Yp^9o12D zfYB{%kbA`epi8KM2o7iqT*IzuvmBBH4YbH)(+L4o z%fg7Y1fusfrJtN`LIAM8tNdO^zBKKdSA#Sti2%u*9>Xpg03IzJ^~C8&r}(&k8vb_G z^*F323#3{D1N71?Hh}|Rbbz3}LufF1^}2UuvR~!A*A^G`Zv*~%L&&W+;vM>m((2~x z#>Ms#RtE45uaADFlvn}1|}1(hz!Pu zv3AQ85w>fVF=Bz_2GpMV0?b5(B0SLBAfjaB1|bzeD60`3QNzaPk=$u%3>xbBtMrLh zc_&zQ1q2=|l)>Y<`woXNRB!CA-K6Qu@2A)66t78yQ_bZqAmG?oe{}>*O6RudN-hLoye3#}sDyph&xGZI3loE`RiSNvY2()=)YDSDW+-5q9 zWhsjG_tgusYYai8!PDPqSq&x7V@XT8)Q!gzsX|GE4U269Gc}}NjJ-e2&QkkTUA3Qy zb!kSkt`Y!Csb&uPd!($Gon2GUB!j19jSH$(LZRZ7iniuv9s}%Ot zGY4ed?9L)(ze22SzQ7ExY{Fj@V#K`g{@sNI?|<{VVg2KGqXJS^MgC8P)!%`^RW@XV~qtRw$rKe$H(q*O5rq|P< z(bd;xVP{~}*3r{r1G(M&aj#+dCl3!Mwoml*wEybifeE7i-#Fg<)x(26hO2Af!Cf}w zxr}5&io`)5Lhw_`{D?aBBjt?>qGW44jeSv*wW7hK<`T_lKxG|-KYabzc& za$faQeO-sZiPPtN@|f3`31XjVN_(rrd%yuJSvj^4207MI+RvnA{yjn7V$ThYVjuM_ zLYooES+xNJ`!&sJ-GYx8ZK^3HJWuo4MJ$35UO0QZU{TiMO(YuW=4yC<;XuC8a!ZeF z@M)j(IRR?Ja$} zbH;0!4IA{4%Dyk+&SDEys^)DCZfzjJ$ncdIuKJbZ!LQS9;BxP(FF|BDQvyfI>~ikp zguP>wqR?Q+eBUaCg7PjzzAz%^?FFq6YL*U0Pa>{VcZm0*qfmGVf5X?ITE`S^ah{@Y_zU!vvn1 z)zt9w0=cF4{NBLJ&p^&yf?p!2vIMZ3nzD@^XNr)t7h0x>kRv5Y)=N<&kz3D#u5HS! zqwfh`eqNvQ&AE2B=Rlh$yPW#pghoA2J#NZ36C?eDDIXcMVfRNVelmlePBhj3qN!%3 zW7A{S)A|Gwczp&fRvIRK7J3?XEk=DB7A-v;Jr+~1@m+9{T4pQa;xjY5Xag!dtTY{QhrQs78;c;XhsC4C zIVQ$G_Y?1(*x&!CU$Jj}d0Bo~a&F}e=Pa6^ne-dz2=1U#^b`A~moduuQu;B5nX|>t zw2D?`MhT-R%})BJ${s28y-L$@Yn-2Vg#|g+r^O0lHRM56{L&w)%v9>-EKju*^D+b_ z#%9#?1&mb|;rqT;6T8PMjdn+}&Bg-)n4PID1s`!9J)=(jY%Zlqon#HQ*vcY@4Gmo7 z+xy1e91$=)U7S$4ugvgrmz5cq7Mq(kz4T5FPWqCquwTrJ=}+O#;w8Q$sIs_I<2NWX^rX%R*09~AR=2Z79&Osq^jjXl|Rldkr-j3qVeoj`YsAy{C8{l;UAQQl(+o?f!~JzpxAzQMgt zIRP?W&iWPQ4k@`zC9H7g9NnRY*FSPgDQIxrTpmc?^#y%8xx4wK;k33$Rm!td?d;~f zyVHSV#-VlZK9l0AFf<;{_3f&sEHA!_nv2y%TcgY&lWKzMJ-$iUb8h{b$EZN%fdQ^c zzVKjv@H7iPP6=;K4f~DdD74+Tb0^hiY8Q3UTU8QAY+rY~o)4HpRJkj_i}Mdfv<*j@ zNUX`XhyJwg&kH9GhjZp`NBhPtRZbI9FF)5>8Yv}PZaTJ}`i2+6S{$#D=vtoAM_THx zwvq6kM=RrbZUQlRm93_>sRk4=R6?RvP(lNhT$|X(WpRwdm@b(FiAu^FQblB`$5IU8 z;F54qBhzICzaJ1)52ZNbmFf$atR{yBW}F#QDEXq&7tO4=vI2hcNJa^cFtllJO zPFv6yGl#Q{107-&s_7q_At`Aabor;^zkbdb6K;P`PH$@h#ecYkck@!iw6GaBep*B_ zmr|phc2ZT@zbwz-dX?95yI6Zd4Z9+5)ps3MD4`0x>qcmGyuEAp+*MbU09wyK#pB%$ zZ}zyI6$qO<94}wE+a7@#B)1+GkEPf?5T&VG>souJT$vlUt{|s9_c9dC%K{f24y*lX zxT~Ajx&!Mghmq2%7;nwRd6ojJjj31=ajoX&_iSg}wU6V?>F(wXK1Tkk#E$hh7OHjn z>6zjm(vjN7K>!rdwWf zXQO9{gdcaZJ)Ks=N1MS(k4=|v3AC*+H~ z?XnpX_dD$;U-$Wj%i~tWr&K&A$wCFe)e<6`TA3^~c zd{3*+1$y|3(|0F+_oj_+9B$8(OWkSMX1(F=6WppSF+i2^%f8?lc{Din>mtGSH*rg~ zn=8}S>zQEc%2r02&FpMXrAf3oTr38J|P41n|5b1d@~MG=gOmF{V=xK=i-mAHswAf2Nk|)EXXNp*)kJv?=@#%o3BQHr@s;7+&B127CD!4j`7Dy53 zy9$KNFqvDhqr?$Uo=AI7%jHr>Z%EDj?u3iea@peBmc=*vZS1dzB(hKKSsAv!x+?vG zouga>ho6;LH0Qjer#$y|y#>mzaK@RvO;^|>=3^GlGkjIVM=+`7c19g>P=3kg8b}KL z8tJbz@VT78-rH9XT-J3r0Gb#cUdXv$-+=`_-x?ZSD>qOlt!wVw>jHsR7VQgGc{WJd zLbA_I)8-d4~M=o5*C%9?^ z2M4P3M7tHA#&bUP#9enmIid3Jt*zbhm%$|B%u)8W4?ASiF~kJ;Q&MdYrc^!!WKnJV zA*D_zeN*7cTU9`cmsgO}kX5X~IzB0HX zN*ezlMrlrs@V&O#6WPF1{;4fw?B^XPkaRunykBv^JIJ(j^ZNE{X*^(PH=i%2OD_D-ay3J$uc{e+OwdxYLqk5iB;hZqQ4cw`UTA2c zb%-)8YWqnuu^1TZ%3ocR0(HwCDz{u$$7q;d_We4nQbyu%(Nk_n^QNKmUTI1PIt7@* zW2iu&n1XjCM^_-3cJ95yNek`;%iZ?{=<@Tv4k?XSO+nKFYjS|)a&7-P)YY|pqCMUB zd3isawjeC8!8^fA^z4h2iO!1jmE|5KCiR@WbHdnC9TOXe4^I4ccj|a|PG97=>7?(b zy5tPwUu`XV4@!m+>U8MoRdj?O*ln-1@9biGPmLVxwHV0;mgIX=ckL8)HX&G5C{pKV zr{c+_DW+N*zj8`)mZ|F9l!V}Z5vEMx)fb?KeLj$Y>S>>-Kow?DB6gN8ZqeN7W|Zy8 zC{IB=Z#*XC#LKl;x4h&_N&{B7lhZYe4=u*nag=V?Mp58$qiylvExtP#X6KsPtY!Z3 zn3Vq6RRbgNSXY`TKO&Wm9qaOZf8)LPK7VX_Q>lel0{9IZ3EQFqOM4vm*Pk_d#J{HD z;c$6`q@c4Fil}1u#X9ec=x%o*GYXZHprQl^{$poJ%i+le5|FRxpk+?&kjJCl{E_saPDlm zS5Q?y{H>4lfGwzLT>k}is55$5dEzV@%X5s@Y=(Jd#m)zYYn}TqrF>{Wggk@ zX3)iQ zZKw1bJOGfJFZyB&{8O!IFc6#qXv5i@K>!12x?t

    dM;W*3o(Yv|p!$X@lnf3B9Q3 z?>cP0^9Ddq+q4b}czPg-e*<{9E`?bhcpdWhy>fjernu4ZCM-Z)xm53WrAJ>6=O4z3%lyGkGaj@I zp5>%UAQfBNWF=_c-yP|ZFEK!?^E>@`>w@$bwzP5Py;ep0&shC#@+i=BEPqu(&ZZU9 z{+58>)gXd^i|oj+^_WheB>_mkUA6=9VSOK~M@9TQRS>A^XLy|>L&QKY!7L=f(wiU2 zYUf~WJR7z3U>@8X&990)&DPBazgLmEz5dOupC@-uXaHa$_%&qh8Hjl_1Zv=#U(ad2 z0X!)BM&bd7>;&i#q;$gczfqVib1VXZ*tYfp0L%l-r``Z$!*Oh$Z1w*H-0%W$tE z1Ncdx{0$_o9;|JwJbxO=J5N3Ux>jo!{?eC}ER7i?R8n=SuNunN)Yh_Qjok*ZlQ;fD zdhzB}E<9j6O+is}nv1q*zj>#5e!+up9x;;(_#u~EM~VV;`MsW+A1_K@H_K}h=Wm() zi#-5DYGe}$ftKo%0t9GZh-$xbHW$Eq4YE#xw#4u58zV>4ga^$Tg{KX2l&Ru2rBc`S zh{O%D;Ag{?s5|mQ;dWD`iTnFFL{~J`fg%FxuEyC^)9=E+H0p4d`=xICZ9CWxFP4Xg zCvYw=O`w>*EhJOH$&B=Ka1|q-9+x8sB+DG)zA7odkNZM@12hdlNUmgHKvU+WKUwT- z(`20DEvw08QIX{|*P7|p1-fVy=P)l%j8Dv~0B5FRc8$IlmW2x^WlEF;b{pCoD_R>{ zS5b|jmvQS4UlVQneKvoVBDb@4G?afxoPUA-ySDH(7y-ef?42}-#Mk^!QFE`_LS}7d zRy}rY78+(oW_=naJso-)EmjsS8f`W)X7n`CoFjEwY{tbjqcDJ`(+3M$Lf; z{QqUt+^ZtEGf(7j2qbxHAO9sp1cdZ|saX1l5Rtzql>RZ4k@dG)=~bWnn*Sw?@kklL z0<{OdC3-}NrH0`LCvhg$&d+>HV(SVw>M^kod;#SW+Fo}?gt3wh(d^U*KEwiTmWHTb zVczBX#xd&7pV^RC!D^+0zZD?-u~Ik6q?taN&76@DxN+u##^H&`d%T?DcI;x{9q?(w z?o{de*x9`GSBS}n2_Z`9X!Cs3VfS_ornNMQS5VT)>vrJ;PywcIt=?Np; zSla3!=UDkSh~5RtAY?$m;jgwSQE2^8{iqti*3wDpdv~=3)BC=bzK|f1UhLD_^V?ne z#yWR+I@unmB{0fFrp$UFbB}Kf>t0XT4E*;RY+j--OvSI^N%WoxO4< zNfOUIo7^KamLj3PiWF;*dok9di#A5S+ln@k%K2Ih)A$xk30;Gp2SW(}?uLs4rW_Vt z3vR9X=2CAT5h~vzrJsqQx1B9+H~pdow~7rFzade+28m@oHvxc(zN<6r$WGfP&EItA zt8cmE?-z%xX9xdc`A)L2d2kzT%WsKUipPK-xeB7BREU}U4S&|Op^o>tQMmsqG)M(# zdODbpbpb7KBGs@Q_683w!!dsXLg=F*R3Yq@yq zpH8Pb!PKq8-0t2Lv2tcuTC#BQpwg+CF>f^bWV;GOKTism$64j+AjN4V@R$EFm--k_ ztEJ=1qZ+A|gR$*)n(Q5b(53xgF57kF<6>GiY@Fdk1tXkvG zI!7k>i2MkbN$c!nFA)=84I=}ISVC~@x*M-2w~hs~i%-BBMX`_LLX_Gm1 zXCp7k#u_vptBvV;*hlY-n0~^NfnkE|qWMH~bS_Kn^PH_?b?@|(U&4J&Yy!S9TEg3l zf!p`ifTF95MiQMg4nlhJ97aFKlLNZira6~KvxBRx5%cA-Nd!)=fm8*uUNhRQAaUTE z{3Lz#A?zW8H=Z(-M7NjrJ5odJl~tt%=eUokkHyunUl$%uOdSi2Cs}k~O2NC$09T8Z z-4}E;E#&V&@V(k+wsBqDw%4=lBUACrnWi)ven4^C<+uuaX=TQM9bdk9pt5A9j&xy+ zq{sAwKa5M216^jP@8!dd1)*YGZ{lSi05AKUl)9=MNY?Tuu&ZzdxQ}**4@DsdXIb2= z=R%)>kH*HfZ<1rYUUt2)gLC0|Xv3==EOO+aG^BVKYcJ$fjfWyu+M-t|^2PYvrQlp| z+ZqNM4bJiLgQ^`2LEWB~To+0;mUk1H&n1UM0uU?}&(|LkCA>l(VR0ikHp5B08lzHi zjTg?QrMoF-&(5jfwR^tSpg|=Aj7IAj@opy$cIS7vF*p$U7c?fP^z%C*-cD{GKKH6u z8^Y!W*ZK`yT$r!*fL@o_-H!r9jS2Iw2|k6C2O5r|^nKBUdGw-ydD-IO8^uv%K59an z+BH9*gz5@1t3nh1l|Qysnjc$gFsA_*{CWqU~WI^1wi^a7GTt_-aYp(^S)rojh2@t4DwRrMTv!Q7bn zvSCr+(~8thBH*M>9xJo_9s)w7GD=ZUhs9^x3l;OnbwdSqC0}u+gK4T7)ndGd9hM42 z#hW$+*<~lfw_`Y$wZb7|??+9VxdnoE<+H**VtJWPYNi=>8}@;Q1zZrAFdf6t4bRiYtuwzJS|r6}zszHJNa3x_%T0S*ka&h>+2zy!UFavb)dcKKd8Z^vZR)c z%6$LumBzPgS%`%XQ$`Z=$o^iP);M9N{g!{qJO>tf&4R(MW9K}su^*OhBZo$({b&)u zzw(ILz=-B`dlM_Y4*fFV{x1g0s(z$&JkXi)-x=%w%NNT(#`^!oV)?iC{a35yzj?j< ztFc~l%*kP#?WSe^q~m?rl3q{RP?OuEuyI$s-f}oCRe?lHp}bDEy2bX7Bd%=)Vc))C zG43!Fx4SF$c1HMXb2C-7bEAX=2Q+smehLk(~F zM}v@$;i0+CRO@U*KVMuNTzF^Csy^ku$t}CLwx;RfWRlnAdUG_-zKF^4x#+ z0Vm^$DW6Rt^;|`&o#zHAS#_UADVFq@6Wkmr39ye{hj;HCzWeh zADEY2InxL)UL^O)lq9*X-ROiuo|y4zWJkNErc zWJpf2eb`E|Ary3vmm-W(9%#*At$x^=;7E{^T#o=hb~Uq~g83YUiocTU#1WZwf;+R$ z_4)h!{MD2Q!-XC{^t2UntCgJTH6pu|6JS9JYcgGbcJcFxih!_zy4W071K-J-|T(M`S#<1P?bb=W? z&^AQ{(T+U0rI0hO*zKbiyFvRiN1EbkpCKJLD(q^*#?F)Paa@_;lND@}20(i)lvc{- zCPW;tAm)r<9pPJ!WFy7gE5ClL;!C5X33-dN_$fPs6HB(n6d&fogvqjmNZN8aq-6#_G)Z9VPKdKsIRc(H08LEmL|XW5ON{BfqfQ~ixQk`Vr!GZEDCFkGT4<<4mHTohI% z55W)&U`&joV+ZUXw(F}ePf(K{<~T?Z^6$3me|tE8{dx-Hzx%Y!zxvj^hQm?(&(q*v zw`(s?&Dw;cnCFwr3#&1id;Sk<+-A+p;iN*cq*(4SMe66&bDfI4SYBVV8}@@wC!cQd z2AdM=xE{!U$vTrp0w1`mKC$M)_PJ*;FF!2SgyUap{%w6J=z5(M0Nt1BgKC607_2^SxmI_?|QjGm1|5=;%;zmG15sQb4+!5fBg% zknR?c?(XjHZieoj0lpdaef7Ow@4bI~`+3gHIs5EbJI`8cul?H?45w37f^s&F8hK~d zhPb@_)K_#K>*fXtEh*g>FU6|F`%82jz>O^LqO6@Q7wMd;Vu>790oK0mc6aXkA9CDp znCM3>9+jSUQG2aIjP9DH$^J2Bt!r1^368%Vk&Jaw4s@_}mh{fL#|Q`F(-T=vX)CxU z_B_|+Ye*u5yN^?L@n#Ya?MsA+ zcL+7tmEYlKGn{pm8~xJ9U>cyr;w$Wke+Nea)iwODee4ggNk0@o{}UVqS0Ll_FX&c2 zDCr-l58mKc8*r6{ZYwK?3w{YpA0YNqRlTXIUh&>RF2n44LdDlXYm-X7i5^>&C&xls z)>K86qR@8l4Kr0(VE`j%0BWgfxZM!z)Rm0PN3H(Zi+1NaF~6s=ZDC|~b7zi;;F|d} z*F@(%M_cw%U@;u=17I1Jq#2(v5{TB zga6pU-M{d8)`@TLMVVGUWzlV%Pksj#pHb7{3s~1)NlE)&D?iA3{)EiwD+T^@#ZOHF zS~?s&GsI$Yxgp(J%ltK1ZuAN>)aOIn0x;E@XaUWuCmx< zk#ArWuJB0Mn2V}VFp3HpmLFggmf#?GP{P2 zPV2Ssr(sJB%uo(X#M5r}qV5$K7zK_F<5LBTsT|dhg6HF;{LO_zSi0Jcah6YGu;fOx zo{N`B-l8G*(3Q@X!h8~ynq?k9%GU5~=rwEHTFK=HepOPb)0St2>R3zHMu8~lOWGU- zqLfy;meftznROOe(+TRTilXP_><%nc6<^_q7M^HOe0kE9sYqPmiCp4*4Byt&NffU= z_qpMv*ux1IjoY)m@Lsf@oYOv^k&4tmiFMC{S_JqR+q^JG7D7F{`hwP+41@YZ<`h0APz8eUV~}6+@6R3ekTy%XXYG`0HX7Bq_GFU(M=GO0YC#%Xo~PeH2$w zbk$?x7be8ENr^znG8pF#)BSV21NQU}0}CFzNzS*wfl)}Qq^V9CMEAc~uk*~T^~bi| z`j~r9d`{N>{pm%D%-Dt<*wFy9A0h5x{q)W;Ybmp}rhNnEg6lX4ektQ~I#Y?zgK8$- zHIvg8UwzW(2lSuV?`oTT3GLSt6eM)+xJr#K3}k%j-^WbIvmh%t{p#g=l(H_qYS*)Y zIp>*MH4mC734w(t9e;ds>E8C7d{e@rb=Sne&g3RR(v567*b8wE*z~eU_tQM% zkVuG%8xUdsDa|yaj^Iu{U?U~mvj1sJxVG9CZM8D9{PJr!s}dc=FI&nAv-x`(N42%) z_10J)YAUz%{cpQ{bilhakLP}A!H$&@yO1@C#&hlrkQ>h8J?B(%oLAd9V2wYy05aAL zfy|K{s%pXw{g0|^QEpD2d;Lvn#<%eIZ?69x$O(;h!4&5a|xWp3p!8@e`Rg4NnFvrsIGE`17orPV5q(*W%8EL&ZNR#5a=Ie$<&wwUY#<6J72&7n$n*FgFNwLmwa zlbfy-Ikxev8>y`6TqDVW1lr{usxsBR++&` z<2cV+BE~A=Q$R?AYBrL~D&Dfq)q1Nr?!ejjF~sTie(NAe984usNao2hhOs}oi@Qfo z%#=ziRt6YA;5~|E!B^~tf>FS7Z{Rgno4D$%9_l+CT;2g4HwvNdU0R%Li>TF+xtA(p zpF@IOad>AiZ?{_KnA7yT)7}kmm&n+?t#QZ(R~X||sOgG7Qd?40)cEC9&j(~r`K00! z2>i!4DujaO0_c~W^OB6%RjIMu5YR#v3cj)bGb0<->`TonvBO>DdKcGtMB7nQ)nNSi z;0okB_~?*ZqSyJ4J`C8?;q?nEHW}iV=fwqg>mlNhydA~QJ+hI!IT1F>9_8u;;TxA^ z9J*j;0-O5S`bXm;(jLii7xmG^xspjK??M>afJUW6GF5ay>0ULe)mat*6ZLeGd~V1L zGDyN!L!9yiw^EKwGjs~e_o7(ImDVg6RmHJ;=UE1GuEL=={4HaJ1Be1I^21Sds|8gn z)U13GasKcTPN7*U;x|d9-pWWaw<_>0B>hkaKxgFp0?V?&$+UzP1$9+54ehH9BNy_>GYlXo+~xoh~o1!#6&Z=WOL79bO^r z>s|X4Gq{L1Ph?410J@f)nqMU_y=Sw&BZ4zG- zd1EssVez@N;>wnLk3osv&R<*oPs=#2y%kAgF@SJB0r~nfg=r~ zf7{iuV0qa14BpC!%HsfyPPkSFNKp1zc?i0Y{DSXfX94t1x8 z46&AK{A)SpC`Z~ej_^-gF5-{W4<>JmP(yxz!vC9t?fW@&}9 zqxWDYD#Qi~g3})HSy0coq|dX~o{u-u&I=(Vg&KQjbxC{6YdRf(CRtAu=f=%`jVz;-X!A~qle5Xr!=A2 zUeCkU$i7>+!&YOgWYM68!fINb*_k=? zIhb@=wDs6vD`ejpA@+Y*`LeLQU}pb^l`l+I{%=;khO`SGGUaL9-!w{c8j%}$$cXa= zXH0vbg^k*TREDCGMYD=H3RjaLgrC@|-__%;C@@g4P_cJbw-M0bS5#j~HnAPOLlXfX zzMQu;sF6*i>1;qABw`?YpFh1}&wlE-@2U=bbF&AomLM;#1NTH2Y8+RrZ)6A=r`iU0 z81u}aKj$Q?wba$I)v;C6$g-WgWe{(>@)Uvp+IHt=_mj8jpsTb?h54$3)>QstJR=ZlRG&U|$bNY$g z?OW(3sfC~Td?@)RF8uyNNy z)TZr#8$0c8r#{e)^6^YOwmr1-yBxcoXGGw27O6Pq{pEP6`NQFWjaJ$J!1@B_0Spm!&E+19Ga{WbDSK>l9F7} zlDu~+a-4*_y|}6gMmM7Z`I(62Z&d^affs8{Vi+jF(=qhUfMvf%jerx7N%h8j0%Vj!81*q4m8)4BC)Bit;zRZBW8H*wWNlFNBc(PCrr@HSmm#(c_Z z7yA+k+`$Cg%-YHN#4CDM845*~0ab=qVx%|(iTSt9K0ODZm{rw!K(^`mf7TcXW9 zhzOi3;>&Qfey7(xre(5A;-zLmoU<8yrdgrgeLCw1GOO5(;o7aC#o%yhmp}_lOt&2j zZ{;5NJbwozQnT#kX=aSD-aD$9+t_Q8`A>Rk^FVdP0hqOx5HgeV<%AE&6p~G7 zHg5~3G2*k{vwWvs*PG+%c5~jnq3p4` z-K}{pmu`+;5agm141|;)J({t;!5`oAr_dsEl0RNC!neFu8_(l9+Ma-vX4s#_vAP~; zEw_2+k|Oh#SYB@$m(yMy^NzcKJP*zrTSuJ~Y)miSd7TAd+sC_!x)`XD$Y82e-qg_3 z=Gd@r^M*(uK+VQUR9@ib$+ccE5o0i!PlpE88m}5HD?oOgg;?pbzQ(DEMxn=cg$*6Yqo*$=*A9O4%?lx^XILI7t6ISyCt6i;NxlFQ8hw> z6L?o@=CB$tcQk9tm;ce^-OX0)BPl)hO6KOvsiVIB(-mD59vASWz1gHfhcu79s&_Sg zl|?Hs;e5ZQoiq`=N1t$gM7$F?11U~~I9awToSW{C_VgtSHA`35xvT}+1ArP&Uti1^ zA=1q~oF6$_NCQsHI*-98x{hm1gEb#sG&!4_<;5+E+2`loggWw~=w7WgVf*vkbnfB2 z2r!$|yl zX1Ok|PM4r)dk~URif^XN^Hu9JI|jwRtC30YEcapAWk(q)owK3$O#S8Qsx4bjkhlc4 zRb+3x>29nf`}TJGZXY0$*cIffRZ zB=_#5c7~AJYpx*pHbpWttQ;S=)(_cFuW^wChjXDiWw|Xa0Qoi=>-X2d&QjA>jno#mfdxtg{oYTB)59+7bE)h@n2z|3ZKQWx2d5k9B*q5$XNo8I`iNN`yf`}i?$-&$r%fFmNKK1izMGQaXW@}LP>1QG1{P( z(|!YDLNOH6zOx5GALoK+;M0O}6#Sw9bn2ONh5oDvnAU4JyX2VpoBoldxl>)@G{IHd<`mvWx9q1@e| zHD@0#vbgFe?7WYOwj?szJXR3|Q~aA`UQMtN9{Z?)nx4#Zafaa$sMkH28Am?M_0K3r zO&7f9QE;-bF0&}}&^va9dU9~ihT-ui_zhv**^o4`1oq=~@3g@?ktL&SBea?TXKj^o&=w%Jnc{yYw%w-IBOAUC;cE$`4T(%r5s0psJ|wbcpD76n7; zb3iK7i%z8u2Q}Aa%oFW)x_-(2%gM~92sfL9A;Z~_)e97ETUu(91$<*XAON$>zISQx z>l+UQw7$Kr)r&ihhfB&IZ;GedZZc6~Z=!tmGzK%54F^B@>}U*faGC%u*H@|}H5QP4 z`%EFq5)(5DrwnqcJA1mVt<5_A_MS~CfLtSpwCThcOSeG?(us!Cgr3I%zWCGT@^^Uo zIVkv0Dvt!cXq4)mjBFau6{Bj*u`yh;$a}D1glweR+?fSBsI1KF^RG-BH=X9GytEAy zth1ZcTk&-^5@{W$+lpyz7BrgC5ZZodpEbv@n@3vp_TZxo-WiinL|hrB{Z7d6DU*<0 z?~-qFlVpwE1rg%mo%bPk$LaQBT6JH0-BluDiq`5M6cUM{4%&!m#eC((Ow3WH6T>~W zM%r%It5X1Y+N+!EX{93%kNveM&TY}vR8_Q!TxQ@S$% zj4uZhSMktm@={xi zoQ-pr$Q^0}Kc&aVy&?^n!LxB6V%;)gLqCUl=7X?Jtv)c~raiAAL1hjnCm<#jKY1Jk z7&3UY#U}R@(;ift_IyU{Y%JM1?DHPNjJ)R66Mdid zt}g0h8d{^EZZpTQH_IbwUyoPU*W`hntC^KA#|%%owSC5ls64JsCP=qG1L(N$h*l-q z%e`is>9oO{UGR_S)~w|h_;YmJ3H!4eA_6;+xXP6(T*PNl(@&KEbZn2mc*mC28#|NP z4ZxF57HWNQq{KNj?xQEn?w-FzQm7QU}t#T%fi%fpb0GH zm&A0-aBw{>H&9c>Z*D(j!RM(vPIBq@WMO{qskZ)+&MW<`EDfkX(?K20w>WT`BraRX zaBmxf&i=AKY&c?*-ud>R73k29D}I*3NdYN?(zn**{k^DQI`b#rWB>epyBzjcJwS;v zG=-HAX6bpcAow#7+yeZv9O3%h_w*O^`+f@JSaBqz?z9#_p-qlx%1ZHTO zghAX6M{gym1HGp@WQCek!khzL4L$L~s=@dCv=TwIv)^C*>35$dC;LauHi#vf3;!t` z)c;kq+_%d5N04o?d%t`B3?cpOTLm}xhoX7jg-g2pBzo&TPA$sM7Woi6f3F&(b<7`;d!olrhEJnV& zU4XXlFXU)+X!_j~PAjmM2gBd2YjtQc$)1vHh702uBsf$Hg2!q7r`(ZXf>?k2>;>(C z{6NCGR5`gvU!Y#+P$qH)ZTx0W$ukdeP9id}hsB1TaShYt9PXYWBF09U4avj1(Zbs! zy_vzZYMzDX106+2p234AW)3Q1)?z7@`jdWjxs#2@Me;%hoWsEz+a?;b<3xH>(`stN zzNh@o1_Kf6d)sQmmE?ms$7yPT8iy^(Vx+;K8w#sMs2p}OYfuLHmksH!Ja6Uv(K_^h zTrd7FhKPT!7XQH*@h|JpOr)Eczs^)ZxUwCcj)IFhgyc{X4`hC=SZ4*JjqeORdlHhI zZ7D>mz4esbXuE`0wnps<_Xs4brRV8nc@RY_N32fhKSg_VMHP(RKP<5Qf64a$t%QFs zu;~WA1@}h0oFCOq*A?B1G=F~Y@ZK%RnSHbiH@A@rAIMB8fl0=|KCGrveRlRhuX1P; z0eH4~vFCIlwkl-pD%es(FbaQTTJ{KXtz?xr`Pf}IWmI~OSWK2|ZZD&Sz z4K%hSN;AXmo!aB`XONyONJP+}$*#{2A}Ed!P-+;yERw2zB-`xD@>cxr6*tthC-h0x zsqu-VP37sr6!!UX8+0{mkVM_LwR&icQzqqRF0V`D!$_0};iIH9;v8;`1b|(jQvh2W z_vY?WNVYiUX}UpFzA<}I0@jJQH{Ruk5f_JoXK?&DU;4Hh)5s^bx^!h{r_{%pAGV$5 zJ9qnvX+*x&W#w+RU|5tOre_RI(VR(J?0Z!m9ia?evwinu#~xfmZJgBnKIZ(UNX=)U zHT@%f8)bjj2U+PUr+EW_D86Ai8d=@*7`a!0b*K4rXOyLC9qwX8{Awz+x6Ad%5*_N@ zr>>b;T&O=vT^|j8g45}FZSH&uU4%@GGK?3EsMhOiX_=Hay8X-A8f^U*m_n@Y0M#75 z|6OzRzb>u)x8>WvkxU=`zI3}Yi(;d$@NIv1#K7(8+nD6CB`CDF?V7~vJhd57R;4N9 z9-(VL92QxV;0$>y8gi%|i5Bj;>!Yv0AE7WdYuo6#L}E1(&iy`bUAC1pPGcgx6oJMd z65HzpS-QH)oE#9UH-v#Go?Jg~G|aX)XMb~ZHs^XaYadgS1Kp#Co9jnJtWi#<3ccZ< zJ2n^%e(00NW+u)|h8L498*tkS}e5Y*3e6^O&*(j8g#V%#OuwUxB9v~*tcq}7_Ja-*CA;B+crURi(BfB?MCo0H?6 zJ6qn{-9BHfsm`z3Nj^H74{d+{JHLgv`r2d$1=hWqCF()ea=;|zFGi_^n@bIcLdx{PqcDwef(gUFw z%tga#$UuVzLgXeY0_xl0My;2@!dSH4V0o*n9u1u}UFRuFF2#7pnAT$dK2OqxgO z2*r_s-qfmn#M491oHygTQ=sO`qsk{KV&>PadBi-9RykEp_R=?jazs2F^Ya<`@gpmF zIJ{2ahy75avu^IIAUBBGc>MwRh~&r~a8d6sCdhu2<;gQ}v7c9vOaC(7X+KhFhl)R{ z&QA72J0J(VAs6I&qSilOUAjy$9po^)d__ddeCcb#vNdleyX$mrK-wJ|j;k-or1LK1CRByG-DR{# z33PX1`Mqy;vmEz%VcHJ_f-pgLX1V7VDW<`Ga&>m?OVK_V^N>7J&6{yzH*6~ApQa`}&MoF)xN6mXoorOz{M1k8pxoBecL>+IAS+Pbhhdp!$tqMVw z^j5Z_pbo-XDjG@#t9j@6184p-t=*!f2(;&;I&Re#8C5>{6BwUzqX8AspIlWLUZj9{ z>3Wp*5HE}F%wAXNV6v0o`J1y4eipp zHuHM-AZB=c&De*6Oud?+g8A{|UC9h@&u|2R6rT^F?+e~>uA>C?t-h{uHnh&F2d*~l ztJC*H(j}-*CE{%(Hx2QnQa5QnWMy+xEaIEpNaB5rIusm}HI-Ic9mJmQWOSS~wYHfB zU;3=ysU#~sYpty-hJq?6b2n-^qh>0&yU=Q7sRy9b{!&9lm_n8?U9Kn<`_Qp2WlacM zdp1yo_hOVT;1OMzOO8+$xlZ_cxtB0+cFu*ZzH+f{ph0x2qYLQ0uN*_OfNaOAGd4?N z)&paE2vKn(hR>L$M{#(=tcNvR0z#>P1qgc@Bq4ZRVX zJ)I0`z zL8NtjPwwM>fL{zgRdZ9|{IX>#e^_jgDxLv<0pjmVYrzLH+iYoOED-4^8Xfbj4rChrqLys82-En^_?+>+8nJ1KuRMZtDM-Mm7k zf{IX2Rt@a4CE24?X*DXogH7&gJ+#o$VjKpzk+)Mo&nhTAz-@D&wq5)b+BWqvO1i zaVlpseV0#!`6Z@9zRK9w!;vUHoz_B%fr$m@9Z@Rwt0Qel#-OCa=V$MuxjMpRT-N+B zm-mk$MR%Hqhp0zrB#pWXxN5`SEVf9?Lo5x&vdRq|vJ&UUq@kK5uQnro|2?jn0$5Ikca8x!_0rGMV2H|^lPqZIL&hujPS6O>I6xrHWs z$5u(xMxU%~rKoY+f&-E)JpiEZU%}Ids7r z*d|n#WrJ(@$?WvZ3i5fdcPG=6+=j<&%IXRRJ6H?x&4A5_j*+%@c6ny1vyE@5^yx#@-3G@<&9!@j4iX?)og|08e#Pz{>)1Q%}(Tz-0<6$M;_ z`gC{s2JXdAJ=Twq?>P#}W)NQ)RguFyUOjiC$7(Q+o@Z~a6UzUk2&as$-$;WyrO0pU z$9JA&{vJPH_nPa0rpR0}OubdpmP2QM`!gsF_V7;!W~5KPmGp5w>sAAeL-}`T(flG7 zXqppJw!c)(DHDEI{Hu)e&(P@G-HiT_Pp-jOOsVoWz{LgIb}m{t zt$A3D{fwEf1SJ^`j{Y^^mlzA9B~{QsHC8SLFpb=HXpPg)7(edbUj>(Y+=1)qsYbe` z1t?Abkto(VuK+Z}dyX+BX!m$OAGvq~f4$(Wj#EUO&+vuRMZkUWIv@b+& zip;Ahtt-W!Cfka?%Srjs%e>oalf@NVBdH+6dSjx;Sel+<{PUz8**c>w>q?qyiwB+1Yd9=i{PRSL{hzSI;geHCw=>X}bhw&oLa zrBT$9VYV5%HHy5NF*sZ5c6F)Oe_1rpH-UU3n}B>BZK20U?nH@S``kuW?+qzoVPKV? z2Naa0-eOI`AUxi7S>N9|bPHU~={(-gd)Qn~A6;x@E5Lr7<~(S@5|KSWP7#NN9>RK8o{D*{MJCK;cT> z<4M#kzmkKlEmLD%PSbDKxSrpq3!v#gdF7an(8YVcn1(&>&6iv#)cA<9z{Uj%f5xQb zcD5Lz9@_6siiS!Yx0-+2=y)R&+0tb&XWJtM1UHDVeQ+z&yC#-|azJc*dkNMzfk89& z#>N|tKxK$$7grf@y-f~d&vglmX6k(AfpEQiJ{Q7uV>x`7AaPT-A6acLaAUnuLs)+` zMQf1bbXCiTOgw9|>d$1Ze6qn+Cf=KRxZYuedAK2&1E@c^kl?HCdUMbY4&+FYZtqhs zOm|)7Xy)~}+r~ZbY!D$B7?2ZHam#~gNaQS6MdM0Kmqroq62zNs*GG$W{Lx+hC8IQ< z|L7X`0}mYax_q+JF=PY!YV!+@W)&01;mwtrH!3}WXkX|aqeh;w4q z7gKiyftN}%>d&v*=hS70V(ce5+Ur;Q*bGo3eZ|^YE;@y_cIQv36J7Pkx83)J>a3Se z%1*n2(7;c#?b6qd^4C|lc; z1d@baQRMNMwV!Sk1(VYe;uvkoQMl|1W#&q$&tI%C+Sgp=O7TLx?cMg2GEJ+^xw)Bb zHk>tcI(8WSNeyxk-9GV3(p~c(6W3YqY4q=!U0J9Y_I2Obj6u{@^@C#|j0_O}74Y!n zpxN1QrBHMUG>iJgme9q4-Dmc^?7GB=H6;fTXwOuoDxJ^v)`;;hSCQHBCdF>|&sjTe zNH8%aCv6$Y7PNGTh9ru&54xDH)duGc_EaT zZc9T~A8w8i7w2uj68(^sn3KJV?I7bIlr+}#L@I)tEx^^eRNeVl4|1*bxeWPrOB(nS zFYgcoJo`fYn^F>jxxwU90Ab;@M9FXq!|X zz|*Mtxw(^o0ly+?Ylm(iA)^(09HUw5VI{!j%W|9q6#r0=h>$7IH<5?cqH7sYc~VQP zQNfLeYTIAsa`ya*n`tRE)siWRfy4$g+RtBzr|z8_&0#m;tu#H+Xir zteaM85M8A1=Sk?AIX5@5Fr<4J2?aJ2iX@nf-QC^Zd7R5(X_Rg^@U(Gg(ia)yijVN1 zFS%uiBe|nSJVTKHSIP!K%|MJ~j&Wjqa=O~auAsiJFRYIkFg49F5PDxUD3Av6)Gw)W zP*ViOt_8E!H<|w>7Ok3m<$9VIMTC;z8qGTzT6#74f^pKc!J>7RXM@C%amsV7osr@G z5e9)h!Xmrs@f^ho=l&rkoXO&x@b)DpJY1qF{obBcrU4na;!f+jBNHAxBp@c%jk6k{KZ)XE&3*~Cke;(cS5k++tzPK6K>4r`PDmt>ts7+X9{`%V>8{r2m6z+l)O@hbqbyfvOC#|NKc$1vvb@vnVP91jO zvgXf%g3P%-QU^oq_BgbI?AlkI`w^kBW`pN?ZdYcllPTAr1CyMz@9o~Or{@LI${YAcC6$!=zp$MHd#984h1%=z;(MO zIvmOL8ta$rD%t|zp?YV*@Y2#RFHCp*;H9igv7=h_W7V*CP_jyQ7-##9$(%Zi3{?ze z%v(Jct>q;^@OL(18h25A>-zlK0p+DsOdk~E6~kk}^0fI~;&2|IW*qoN4?U_ueBvC= z9d0$DE+lbmPp3-nIl0mL%5pXb&c`JK^KvJ3s|p7;_64#9lnKIiX1j!0oV0TK(}%UC zk?~s+qMo{#RU$@a5JWTC7=n1~E0Np)Ng6@5OeF7`ge=JWW;xG7yyhLyD96X~@^XGb z$u2NbD7VUP_M2V)@_f8hii$$fgX6WlIevH1jcZp0Jf5BH@~%2HTta>!YE#1ThuxmU z+?iSgE^20rhCT|Qd#zcf&a4p{oQMHCj2(Nf2$Q-|bIz@W?shtEnb*mP8~Vm;871u% zoQ2^gixbjeH#Qc!2KICN@=S%fjUxaA>`e;2;B`rO77wK86IstjO-)->U1{#G7-8es zk_=oHQnDVTrmGkx97%VL)dZPLW)-#0A9E3YI#E*MoQ+-I&YwE1GfnnOr6UkVd3NvJ z<QR$r%F&5TlW%{@d`sNvxr zkQ>IWDMIm$L5W??jM)~gw57_Mv)sZ}XF~#RaISrqO@!v$=GMD~m(1_v-L`N`D3Jwk{6u|mTRMlju!}u;? zE+n0N2Z{O~;=QXC-!JmxcTYGtRJ3duq8W8b5flIl==Td%&=65PkifrX_XcXqx8IPs zpf5Ca>*}puT+}~8Rk73jk^Y?#IT+pVjoRATVUg1F{+Sq@mS3!9J#f+OZ)q47k)TWw z2GaJIW7m_&$G3yY-_FZyPzq9dPISfkchMqE>WvC+$$cOG@g}Nxw89zVQeStvCNvJ-ffhhePlstz0 ztyBx8AShBC)TNUXcOXFjPy4xGM>zcLhCk+u!#mhkJY<1mkf?-e6&jM?`2mNyU;%v6 zm|$&lydswadt4|_g8lh*8naLHD|1Igie;V8mO@Fa4 z`R32i6A6Sia)WWJ0y|v3-!LC`Fv^C_%kcGeu@n0TWL3j=4g!xbaTI8bw#CUYJKJ_Q z^<3qOs*>v}ldGOZt@Fez7a&8f~# z#hB#f8JlhUjk8gxs;cTH`<0t)6HJtFFv`!e6`XrO0mrCj>nosuV~TRs=U?)AGw05i z`Xa_0ik?k=2OJY-Wf>iB*Oe$tALG}}sV>c~EaeyrU-uiQQ#&4N>XKfUy1^8kSAV2B z?WVbN^X+g?DApg;{p%SA_o0V-{s7_ndBDNHK@5QD6Mm*JbG-qggr)|&sD4R-?pI5b z@4GAX^i6EEbUo!iv^hgb5J{yWEGe0wjnK5Xl-baRM z&S2M{YMc#q8;hgttjcppwbCsC2S?& zD-%dnDqdhyh0%ir&HX+ctky{!%U2nogPjsOSm=JKjQ?Egf7Qh|n*LA^UZG;fw;8d# z&aVBQZh1^0c+r*0p{stzP4gcTh^Z`Vtf{aTDp>-WW7v-1O^-HQ^trBB7|eFQSYr<% zGwN-rgEDr$a=k@9COerR=n01!<@Uon^9To~6wVBxNP-IJxYbksfx}9l#Z%>}{X|PN zq~7x>Kgb;m(TQz9%@t2wep8rZPs8is5yoSSO;I7u{DNh{&SzSYLQK8Yqj5859nPBH zzy7%)n`CVn3bTmrumM=*eSvZsxzqVdYOX6oAhHM5&hu3^i2kIv%+4BQ*d!?@DkB0{ zwWV!vF2n*$F%yk$E%VVLe^UZ!=tm{7Q2K@e{&(#DB1fzG9|NA|C(g+0#ct06*%~`q z4*U(&N740_F%EH_><%v;^w_&?!8thApFUFvlh#-^^XUkIV~bKbhp6T>o#K zl4aJzZ&N6Aqq2P`8Vb)e|6GOqoIL+c2by{(X#-U3xn8%U4QJPzi0keCxvMb zDgWF_htUj@O!OP^x5>R}qZ!^SzU4Lx`8h~n`{;4&c%J;%X^DZIfrtBd~W%PX_+J%<*I`9x}13D4) zLnk8KU+Vcguq2&@mAQq!m5q_U^|vH%Xs--m*S{obD<@%|%79)AQhIXo3Ereb=oxA- z-pRnM1gk{pkY-LJQA2Kk%Ji3|`Fun~)s!`lh9k(^wc7UPx%GPin1*+dDOTL`Jjq0d z7y_HfxXA}~OI*Yw$Rt8}i+z+V8UrhzhrMmV4cW^I+(9$1C|LHnJe-p@&xQz|O*+>0h1Gjy*Lfa=m`MumA$ZH6%$5w{^y@sh zRO5Z?4`AMNeKA8S;p?pos%7l{=-@HQE;^rfB1{v( z5-R?eWszRd0Fx7_4C4;v+MoT>cYgVZ{XSQI>qmc>s&W-I%wrj_T<7yYI-oxPjAlqY zZ7Z85krgBFY)GZI9XsL&J)T^n`r}QVvCm!Mju<27j~{orLFeaFXH9Ez`JfekqrA$V zw};hBcmqXVPc-f;AP0hxw0CBmzT*T09l}XN0+kz5pCJ63bdCPoUAhdHxhpKi~=fI_*wUmtT zD>t1u3*e#9x4#~7uTT)px}$+&{Ui~N}uc~K8IBl$Q?lmVmn(W690}7EJD%0CQ8g^M@&-={) zPluTjnq&mz4(&KI$D-TLUw4nxrHAQpEzQYj5Lh!wrki7F?bsRd`enHoCF<|tCZ;{* zC{Y=Qe+WOpP-7%e=T7c$@5|r-ysP`jy|~_)!99+i`0UEm?Weu**E5~MRRKn=g-Tyh z=-BT=MRKEI{-pXhq0Hq`wNalZt#-q%$e9%n4Qf2jSc@810t>;;1hF!MdM|rT>!$_+ z!Jlq$X*yI5`J-&qdQC{@4tX_IE$?j-JduN(ywtlbmeA_4&KlKeOR{iR`7P-EQ(e%e z>bhfA)kitHyYw*&y@Lm*y@}5<_6{(*vS zLg_$uWF{9)3u~FYHgE4&o=B20IQ+Cyk5&?U#=F8V*6Dk|GR;@3?%;C)(YB{+0zKPm z@ugC{!Lo3#qYo={=3*;858TBkkmg6R_&B6MchG>qK3oQPz1Ttw->8RiPkVj4y}0M~ zL$JHo=A&~5-dfivj>N8Cc51^Ikd3>SmGu9aG5mc@(f#!S`Zl=!zf!YYX_~G0uArnRY5nT0hHsW4?Lfl=HuxR#%<(dJawC&V6W)v`G>VzNL_8OU#>AhNJ(5J7kuFexDt6lPV2&GR zd@AZknfii{3N394$Oz=JM-XuQ<^3&4wEGK1TC&qRBelMU;nC>~x!4lw=yq%wuwhTK zI-kv(GQ6<7T;fn-;m1|{0EL(U+c>;Ds-5S2MRem?>Q6dpBD*piVq5lY8^QO_BuwV( z8w%~PhBnJ3R2TCne*O3F@jsdcwlDtQ?Twe`Hg94;pS{4Ac}o&EKvqH;CMul!;au7$ ztC^*;Et*aN#?cc`!B#1$9>ixG%@4&R#L zFm9#ddEz!fvJ*jUyswWyP15}wRTibn%o@$=Gcg_&C~QnCguHYUu@RomDTmU^NRQe6 zGZn~7rK2QR_nheY)c$j+*GpdMuT_#KcF{*F^f0~}t>^W{py~nclh+k1?oKZt%ZMgZAMZg%NG|S%Ss%TQMpsC*X*3b0PTy)>i!7r~ zSl#Ov#Ukq;*SQhWTIWdU!N@eLWc2$z>9->yMHuPRnxtOnS4a%w^`m;<-(fy|L9DY! z?*Gvq`SqH>dqhFMPkhH6;co~&KRG_}7tWCIV6S%pWES{!g6l?{e|qyx!lO z?jNM^zbN^c>%S^7P=EP$)7j{oTKrUHWd2^VpXtCD)&Gh5-$%yp)c=87{gwLf!x0{u z>q`xm)-TZE=mFJ;{%JV=E*$)0IKu2K{++FbqJ-2I1A5Jw5)m=aN4Q|9z z0RnWNfMQ4bCSTfd0hfWpTN}LYU~rtjDCRN_5~SvRkV2`LB%zBpNxSN0_JHm| zlmY>vAX-G}%Ty7kS3(FPYYp8fCcae<*|P~Aw>Q&%ZGhNImp{@$IP}|-pGOy6?kzN# zZgaqGuNAi48Uu8c50396B0X3HW!?59X6~)Repl>+Z}jTiJGI(gf__EbIP*-!WK?Cn zl#951lnlsU~$X zEuwr+8v3*StZifM4+G{%g`;Aa_Ag?lp9&nt`$}H&-xTo+DK*z}$DKJ> zNbR6WnL40l=*KL7=@IT4iun{QciZva7aBiizKFsq4>zTT7a|icvP?YT(fJtm8vdQ;un(O4;^jzG^qWU5a77h?$J@9*9fJv=&!MmH@Q0WB- zHPHD37s~!yU-=sVapeME76RC3kc*mVY2=$@p zh^$S_$bP-Ouvo;3aDe|>6d$_wTJQ3(#oFXt+$TZF*I{ch-4Mm=j0$gj37d^iJOST1 z(PAcZt2G~T2PH4YV5?h^J$DH|%Ce$_W{QPAvB*His3ME%pXo^T7g>OJXH%(2uvU$H z#Lqj?ZLXk(z;v+41wilA`Whkk%Z$nr7X-l;E zmNs8-KGQU2_hNZ7(EUBtZt3zok|6T~ERD5il&;cW(#V@0xw!pbYhM8uwoF+*^3}5SDpNA) z1BA)yzB!UPs+yFW$!z23>CJp?)m*gWQVC{}x`LQP@+dx78(jjbh#lVx8aCO(=C_8B zjcx1cE!&%4b6`@mvvm@1%Z2f`qhGaB%I*A|StWWmES$fU?3*Ioih+uXX3pkkqn+K; z(nCrX6d1$1G1|!ay)R$9eU!4q%*4DhZA@+Lk2yZI%ShbH)sE57L>;;+KiO*gdAc8K z*=+Nb>xS_UrtaO|{DN+$^^akVzTJd)Lf@O59{vz_w5ADisSt~Lg2#u2NO`$9J7c|j zjm5WgHHB?QRm;pBX<85XNn$S3-JK?)dB8bIC-m|R znGvcdPS1tqRmO!kAVKIXA+gaaC~|BO{(i2pCi$kuzSC;*MoV})&nb~gh0(t1_naxH zkb0B+F_1IJdtGNjPolR)u6B0*a)GJO)$sWXpHx=5Sad%PVMunNT};%Soe$y_?%=yz z7V}AOjB!Y5yH|j!iH8V5!;PEBMf6eSq8wKLY5F|zQ)mvFc4!2~<~LHEiK`RqS%&%M z($NPB@(1oOejIx1+RK(kg2X)hK%Dq{-Suqbpp*wv^h4!Qhr(N7OP)pV{^=#|~ zIh)N=*OVF^A0aY`uhW0bZ4hK^;|#@X>#sL>Uoky9HJR&`SAmIgN^Ii`+^B2Dt^3U$ z%W97w*I_-mSNhJ9_lwR)GW#n-9ABmvJM_{0enLoll9Nz_uBx8#PILTtW(%hbW=yAX-(UZG52qM7j^A<=>oiO|SAxX67dQM0XP0NUZ$=iZWD*w>a z+-@kGI@D*VT|c$mo*--F>U~sHQ-u3r?Q6&#rL3wI#d(~-AZDxcW8P-eLavc`ti~Vc zw$)hq+;D(P!68IcIAnKHc* zHw5WD3$sxe>JUFuY}3NU=@Wly_3@(VDLbux?RL#@@LT&*LtlM@^_g()h7G<9E4?7c z>}-bIfIB$Y)D`TSi-FFmu3t@JF3k(4yhObhXoRdqE0kX#+9MTNF2RPIE4Cn$D&LFz zVR}IRneKG4uKui!d~Z?KijKl$lL!7e^R1!txTjAM_&=GTCpDXBnTze3$bu6ok5Jbv z^V47xcX{*r3lr3`3Jmhr5URWv|D38g8sqJa)Cuw32N_Ajo?BRUUsHLk$Qh8Gd48rU zP}+%wanEn^QHQdah<0L(7SHOP^W~8X%0_zoeP)x(o@aK_rys;lK`6*|7o>oOHj#~8gFNVsW&Z!udBn^ukucLBHtrW zrz3QnWZiZPKKmvEgjXM5Os?w04w^Ecvt#853z@x3BCDOLzmRPq zLo~K+Jh31XQoTW*!6#sj~4v(ODl{t!<6uM-^afCEovWfiq29F0i zZzfRM94)>1ZXhdJ2nFiu*bKzR>x9iMt>8*=jw2IFGmsg^ztMT4^O_`6bXu2RNdD}q z=Z~Sl5F~?Kt1OCdxrSSQO1irUJ%sgJzl)fAkqgp^`D5WR$c-b_8dWGqm z9=@kt^P@pUYC=s+?ZG8{abaI2Q7X?uKK)x`q;j5v@%q?~JH}$Z4QeZDd~r$6_m@Ii z1ZyK&FAVh75O%Uu&CD8U@ZWtipZRfH=Qi%@`LE|IG%hn-63zecIQ#7M8p+KMEC^;8 z-jqAm#Ky=};rx4fibkmtvPlp8H0(pw0iXFCG zsEL`Yl84Dceg?;AWnaU&J8vXD-pViRX2xl3ljFH}6ZYo2I81GYjo8%Ko*7DBj&Dd$lq&KJ6 zDHgMgwe1%0$F_yax0gz6mo8l$#+v6)TS;o_i7$Q&<|&5qt`)=NDa^?C@)T1H%jIeI z=T8m4B07lS6usNp7FaHrbz`Ki1aM z)YkZ4L7%V_d(5!*uuKl!c)3br`Gx$7lGCjIx`?Y*Jm`qQ| zX#AQ?uNxT!M3-HtW-m1lIV9lsLC1m?qKWzwO?}mW91d`s=s*R57bnQ&0vb}A80-%#Rp3>E-=R6^Tcrm5&g^B{7;w;2KoeSzXlqV z)c$%5jE)*_)$ut3>#w6c|h_q#-i;VBp^}F#x=)0>|wQKzewTIQX9cWe_)dX5Ct}&DgTo|wk6b(fY zsl>q^4Ot>B;-MN?{FW2syy>R5 zxoZ_)t9|vLeF3*XiF|Ivx+#jB~0dJiHp;2_PKl~B&)o@u2(f)V#h_UA9B{0*o3+E z>$0V8e=G|2X;T7|`W@$+yyjt%jcl{twj)j)=&$myq*AEb`4F*~ofc1%OWekJ9C9H{ zc2q7ERYsZXcEXGtoFSY1w}AN=#Q@=IuNx*UW+G3!@n&#|g4NBdo^0?r%_xO<(tjDr zS;-6I5MAAQJ>@WLWY3o$qgrrADV?VVot{wF@TyOU^%d02H{SRf>N|QhTJc6BMax<% zCL8qIG&_3LtMw7(g*!r9L9aLI3- zi?PPms{)b{i6!doR|E4}u0_q;-JY`n+%C_yxtHVZ48(kH=i4f&DL+RqclTYBxF^>P zv9QDH965_}@hOvuK%vYni>7hU%-g~2pY!eyso~%E)lS8_s7UuMvtw{m3_o(Y9R=U7 za;i4>nyiTy`!t^kNDYHONptVdQ z@BF9W%cVGiMH)AG7V}FQ{fWP2dKcZ?jFkUWJl>f>HYOk8cmW430ilecsZ5kJDX9C} zxDVxLp67hgL#+NutuJl-)(}iCb8TY|ZD&Svu}ej8qwt;|Np4CgY}dBmKqX^&B$i>@ z#k3ly>`Il4juCt(eB$<74rxBjEzt<=6^7+EU6?)*){+D-nszMla0T^t-?dEPtxasK zEQ~d+f80ic^@RWARgNB|Iq+WxUgdZn%%=PZ4s*;X&WtFQvSNKWu7EJvoSJ+E>? zSDtCveXlYuOTZ@J$)&ID6%xnefQRm@2PS#&z$DKDN;WZ_wc=TCe+?mjW@WvP-y>Zh zRShB4)q@Qg8LT0q$-aVsSE(!ny^4j~`?_G8V7*)JR?SA`z(wXy$Ju}+KCKTeM&|>h z4`=vDxx#vbM8?d$8wuITl^-{siMdri!f~oMttq6<_dUh^T#DkraDHD)PyGB1!?8O> z!;R&X_(=&NEZLL2*U~w!#A-haP-|S@F-5)^aH%;ckrD08Do!OEGbb&fGV>Pd)Az&T z%sa6$cS?PtCZpgJ5bp|KD-$an&&j{$R%m&}{%WjuxlT=eNPft+2vf7QNtbSt#7>A@ zKi~B$ME76Ts?0}H>B$L6oi-&jsorrmc|NX|2 zslKv}_gUc_GoK1CMx_}PV-0YKe~!yg%HX#}eVoA(s+GWrN}7%!toH0YONFNsff_eD za@ds7SdH(fEaTCkLP7T{;T8GKgR5xG3ccf{h}sjm*EcCHU5ut&%r>EroZQ>m)j%6~EV z_^}CJhIc_pROsa=-{4184xdrVF+*tJm!;iCVoNhM6mgQAvp8q@uqk(h!UsQiCC|OS zt;b&dD%;JVvb=0637Ij=Yb}V6zeACXFMd;R<3er9&UW8N7FtS8n4P z*c+a#cYu7e5-}4MYQLJ zCIdYk zt%f>1Gab-PeE$Y6m~P@fW5S&TRJG`eKyV z#bT_x<+UUQZA*3L7GaA%(*sr^;n||B@kT@4+*{4hL~^!OdpK3La>lbT!$0X&Ra&sf zVw|_$sSi-P%w_XteifBwyCY{Vk$pKQbgiz}`G?OqLm$i1wAr)N9sR=ba%`LhH7U2s zp_tqcx&pSAlXS)Ut;M>uA~`cGW8!hj*S<&W0=WPp#>+o_|*96c9%q>`qU0!_PObZ=&^~D4ti7 z-p2<&LaR`or<73!iSa-0)l4TaTZl`b4;=9*JDbz(-)MvwGA{g~wq8itNt>X(v@^Wm z(Qrrq^8kt6(M0Z|d|oWiXB;g(GD6oX+iEV{bMA2Xkr;b5_|ph_`tX~AQ|Hj82^|&9 z^&3NG$M2-uh*H`GV0RBn3)r}j`0L&qH6SpmjwT+=_lW4ovm>x?+d3VvyY<0v)pmE} z0|m}arL5IP6P5glk>W<1wO23Qx!1F;#^QSMA8d0NmPKu7RZDCbn@HywjgN12 z|1jHcIT7>8kY>kpR*TVH*>a?cws}O}j-`Az!?0=$!(hGGq%#PE<5{a_dmjD3#v-bE zvE}f_V4D^0ILVs%7?Hxvh4iNWTw!+Ix!WG}vRgCDwv!eHg~Pu1jk35AvL=Iii$8qb z66}=?RdPhr?z{-iAsIOL7e*J{$=6=1pxm2Zz4N`D zeKwmM<8j|2MdAIAizaz;s^+c!p%gRdNZ8yAT!GT;U*tKukt1v9pe^_}E-aebM)oI|ws?V!Bjf`nWiN6`RJDHIr zGK0DgQFawG&t%nIq8BMuYRBxQS-kCZ7JByM<~4-6l<#o5Pv)a<}ON0pgqbFAZ2#+}biHSzbp&ENhsRj%puaWoq#wRxJ-0Rf+QIsRMn zmRL28=ef*hnIfN%zZr)3Q<2o5;x0>%+-GsfjYL8&!NV_kzA{YHw;V6aT#%42k;bx` zkzP@$vbeD{W>RKiZPpuqZ+`wZrqtGXqe?^DLB~`s`G?H9-DaQpkx|2g?Iea4n7J$_ z1th{rf?M%>xSZ>}NP5)QougP&wj%mBtI${MUrx|0TJg7ww~c-A&g>rsdwpjS_p}* z7sj@CGf=#hQeO9b^vF)H>GUgOG%+s!1HkG??W;d6aqNVa>RC@VR7u0;A zNIO|FubfJvCs%96@T+sQu(ff^9wy7aH>~ri?%uFO^LyvuN0E)+($Vrl#PVUJGTBWJ zY<-ufHgDg(&7#0{e||}~D0&+Of61B8R@M_F+A228T+}a&4yn&#U9eOj!5}!7 zT!9;_gmT90lp2bhq|RhH9B-6k1@nJFmuWM|=$I%6oXJ*#kho zV;iORy3#n6QBhQsn=%eXPBnA6#Y4h-nD~C099Fk+zIl3CH&fzxt+rm+#&ViMjmy}b z&n}--2Pae>yqB$OFX-+NTP?bu8l$*#u9gb8OeM7cN|uE9c)`7T|)++@V$Vy%k82;mlbkBD&AjDtwHAn}MlM^j#r%=C+UzLl~4qxduZSFe_P z(kM!$Pxi%Jy;{Gb+@U(!G2r*4`7V{L6?#t>d@maU5oOPUm(XXI^+_7H7K%vo}q<=_)e&^`5o0f}M`sxiU)i+drgItH#d}cT3_{ z+UW;fUgcp=j89)HkS~y}Ll!lX6r@QFMnfd%5|RSS)65cmB@)+o@4LSZQGIHxoRgmX^J$`-koRS{;u!mq$vi+s`T)QjtAWr z_%p%7z3J~)Ps58vS__Kf4VaDUbWv*~isgUrOEhUpG-_x=4IYLseMuM3q2(dg&(2Y`XoIZ;_59IM`xhiv#sRpv@c;>``SdU zCp^T#vX2&Qszf?zL=3{Pt+4*+N*0`B+AE*UQE?ozXQh^T*0&y!igwwNr0;{=( zsE`)#ACRY(Qr6hy77p+TXh<&F?i)BP45951$P)X;7YL#!T4Chn&tCB9TQ;`-&0G+O zpuMRq0LNSYymb%Q|Ndbjc{?1bo6C9NbqmcprYPYcyFLAOl^XE=UGdx!17wM`kS0D> zmZ1bvjp0`)x5{JwNClLRgQHsj2OLkwx0imRyu%)%0{Xy7T<=D@CB(OAVbl-=2!*eA ztmXJ60O#eUI^b}IcfX|=s1%0T4M=%jP&_~i|M^bIxgl5cnBS~>}T4NIYIaqPZC@NNDM;HWk~m{&anN# zB&2D&x&P|tA5oS(V|Q5E(;t8o$M2fJ*#lE5ec*cjn)+b^ma z_FnPO&F2D|!qYLlfbB4nAZ`ZyRs$%*07n9NL|H!zY-%XcPwdUXv2F`=7N{6{d%Ogmck>gLwO~$k$v)2N~EPLMO8J%2Cr~qY>Cwu_er= zasXD$SvD=9Ql&A7chUQ3pX^+#v$X=V#1L z<;}f!d;gV?t>g1vWmI(*DJ`$W9~Rp-O?z>YiepqJ{vGQxPx>wnS)F#q(*n(JoTvvJ>@y=9-rb~s z&6Q=B(m75%omq!1$LZ%sZgr{lzN9K$-ZJinn{a~SY+`cG1No(8slwi;@~t~Ql}=CB z>`#pc+o;O7qm}QMe;@8|%A)1;B?1NbmVlni{g!R}uhL%!xj8WXbxaEESHpcvTWZQ- zgRQc%j!!u$G24_V%J;4pJc=ucFL>aoX)Hp%X*R3E>Zhu+{VD#lnLq!{kRq|((Keq4 zUk$1iEHJEz3ZIsKG2Rs`<^;R$d#8z8@IMbqR%3H*+yt%}cP$#G*(OjTaw870NxAmY zqTaibu)`{UN@%ABVOYTWdQo{?j=an@c%>Wm^!LeJ4eXwrMZx|YmT7lAKi@1qeh01< zH|K@SHxrUGIpiB3ou0(yE08=GC3viMU!7z9Et>0qZCx4?xjArU^qDFyDtaTL{P3hU zB$_n0W?ICwW}+e-ulk)={MdPEuOfPvQW1kxmRC9-6Flqw(RPL4+2~Y#C29UhRwlQr zomOkl^TvYLp9YL``>+Vv4bL;pm#3Z+ZK#}K@sl7=1a&P;MmP+1rxT=(}q#5+6 z^&~0#UH3w147E+*D(%(49^&m95tA z@Q^P2`ga|tF@Lp+Nj4J4UQ)Iad*OP+00RNj^=7x)iLSn^%k+cTa=R&c&fkskg6j#- zR9-$KL1Vjt7Z92(dYy{)amieIGUs&77Aq=~1rb-SXzRA3Dy>_d3TugKr^UT9T7l(r zLt1eIdB5vGA2IMHO2KRMmk=-d=#W~|<5BmP|@?je?Qekl`mpL}yBp}SCL zhTfD`-@OT&bmPT>?~}l;oOAVlYYmcpBlW`@5|pPNHW;%5Sp!yO(!KITgBBb}@-Xz09l0ZC50YPA+@d zZ&=YIRo_G%m6u^uJH5Vg$De?n?u+_If25H|oAgqb)<$l=v=c>aE$t*0zk_GLaSKV; zMo`1Kyq8H{VpoH|;o3HmjEV_{uEJ>&J3Q$ZZJtT;>E^Vr`*V4HNtA3&yZt(qbWr?A zv*iMAeN9GY+I@ePUwkw#i6^t${5-dDqukPatxHVv5nn4+O1oaQ^)mLr+gqIy2{cG$ zH@D_YXB%#&m&1_lL~`nK!M`doWM%VAMQx&kpKs-*POPl}es9{w95ul+zVB&7=cdWs zNVC60kJH_n?>~z=+PZA}a4?JLPG^OmV{iX`Wlh^tt*8_Y;=D4<{ny9gUhA7|T3NLF zXSm(knW|jYtkI#y<3{XUPg#vyzU&^rKb2^}X_18bP_&7j(y*N|i1F6W*LCD?Ov4q- zG}Uv?rFF4W%#ZX^nTO)^QdYB6w|4E?TEG85?@;6vyQYYFNBS<;y5tvO1M9%C%f(ye z8S0I{-6lcow1@`8s*p#(6Wqmx-l^EHI9540m8GJuX{xHBYO1=o7jVKR#eEvV8`o=Z z)HXEVuMwjb5*4~3G$Mr|enWCjf=2Qdo%Ah+47yu6bmi9XgGLc1eCpr>-T-0(~!O9y1-{xkZE4< zD*zMVZp~>3!ZEuje*$#|1{4^z?Sb0cK>9m0u=W8&|LcJY1I-cm^)G09ds+ZMu-@Uf zy_Da{gWXE{#VHVMI99;|{OiWjzcFy_fnDl705bTLJzxOD#{`5DPz=z}XtF%kU|>>X zpjKC-qoZbERb>LUmY5l-S#GJb(gPz)cZ;4C*jj=vKHfitpl7H>L;GvXX@6$@lQ0fv zdRT4U7X}#M#5m#u;XE{Pf6(1O$>9(cLnIuN<3UgXE%%1sQ2nDA_V4RuJw_Zqci0XD zaZGrzuNC{$foc}VXTI$R9CjZDmRC|Rqx2Psf}sWRkJSBTkoJt}L~T8kno6imfF7_B zNxZia2`x1U&<6^Nu74!!uR_}+>_ma>JB;6}>tOA2{W26`%=>QR*nUB<9_VDd4!QKGJdZ~Ttz|;IcS|~p<0xL7> zEN}xFnz}>^7-{_D1z-Ey(ag}s^0OM?U^^NxW8RYp+aGt(P|zJxv5t3VdMq%6(a`+s z?Mb)j7-_*L)2*L7Y)E?H{b=WOpXU3QOGompcRXzan06suAc4F?hsbz_Y=k z`o|H1au`2H2HLAL@7d*vW^|}HnRS@&h=8@~QMFdM^FPt`SIs#XpcA#{(4DyzXyUg4 z6UrUpY84b6XJ1qY+~7fS%8277HFG;zA>u*CgW zV*6L>zNH`f`>zb)&lCH;5gfe<0GhC7Qy9Yjt7&nMuoI`mLuCWP(t`)= zx}yIDP^Gx<(f=vj0JQ_QK_N7oB+iG>V0v*KIeh!~@!qHYXcIzcd`VFFj!pv54H*s& z81ISf*mi`6F%^guwjm)jQk<73FnP2M18h+Y5G8Yot3*U)R+`jT0ZHxzvkI(IR z{*J+SU@HfbJ*Gt-v}8fGg>8-pjXC!O%*Uwhks+ex#binc6 zxE|XU5A+WEp8jZKynUqmMw1AIblynG2Ne!AuXlz&Kd{ zg89fOXv{xb`2dsBi2>n0H%A-!Ky&j08sC;V%yRXzL1S!Agc6qJmM&lleQ4>tA>T{WWZx3TPe& zpb$gJ^$+fPkJ%Gl^}gL7?Xv)l(hs@U(E0KB4TtHBN&c6L`Ab-jtz1Og(?XbXcH}e+5;v}G`GV*4z}(Vnn|g|L(EW29u4I7 znEXow0%|2}(I50k@}wN1I;jNk?21Ib>dXFKPJC}g3Y*$ z{{r=oJGs9@gWWNrO#MmNeudq?s0;}32.7.3 2.1.1 1.1.8.3 + true + true @@ -963,6 +965,12 @@ orc-core ${orc.spark.version} compile + + + ${hive.groupid} + hive-exec + + @@ -994,6 +1002,12 @@ spark-sql_${scala.binary.version} ${spark.version} provided + + + org.apache.orc + orc-core + + org.apache.spark @@ -1013,6 +1027,16 @@ tests ${spark.version} test + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + + org.apache.spark @@ -1037,6 +1061,16 @@ tests ${spark.version} test + + + org.codehaus.janino + janino + + + org.codehaus.janino + commons-compiler + + org.apache.spark @@ -1516,6 +1550,18 @@ org.apache.hbase * + + org.apache.hive + hive-storage-api + + + org.codehaus.janino + commons-compiler + + + org.codehaus.janino + janino + @@ -1572,6 +1618,18 @@ org.apache.hbase * + + org.apache.hive + hive-storage-api + + + org.codehaus.janino + commons-compiler + + + org.codehaus.janino + janino + @@ -1736,6 +1794,75 @@ + + + + io.confluent + kafka-avro-serializer + ${confluent.version} + + + io.confluent + common-config + ${confluent.version} + + + io.confluent + common-utils + ${confluent.version} + + + io.confluent + kafka-schema-registry-client + ${confluent.version} + + + io.confluent + kafka-protobuf-serializer + ${confluent.version} + + + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + + org.eclipse.jetty + jetty-http + ${jetty.version} + + + org.eclipse.jetty + jetty-io + ${jetty.version} + + + org.eclipse.jetty + jetty-util + ${jetty.version} + + + org.eclipse.jetty + jetty-webapp + ${jetty.version} + + + org.eclipse.jetty + jetty-xml + ${jetty.version} + + + org.eclipse.jetty + jetty-security + ${jetty.version} + @@ -2267,6 +2394,8 @@ hudi-spark3.2plus-common ${scalatest.spark3.version} ${kafka.spark3.version} + 2.3.10 + 2.8.1 diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala index 15769c4c8f411..d905939144f7d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala @@ -161,7 +161,7 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi sparkAdapter.resolveOutputColumns( sparkSession, catalogTable.catalogTableName, - sparkAdapter.getSchemaUtils.toAttributes(expectedSchema), + expectedSchema.toAttributes, query, byName = true, conf) @@ -173,7 +173,7 @@ object InsertIntoHoodieTableCommand extends Logging with ProvidesHoodieConfig wi sparkAdapter.resolveOutputColumns( sparkSession, catalogTable.catalogTableName, - sparkAdapter.getSchemaUtils.toAttributes(expectedSchema), + expectedSchema.toAttributes, query, byName = false, conf) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 8c121b1eafe6e..6751aaca33305 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -1418,69 +1418,6 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin roDf.where(col(recordKeyField) === 0).select(dataField).collect()(0).getLong(0)) } - @ParameterizedTest - @CsvSource(value = Array("true,6", "false,6")) - def testSnapshotQueryAfterInflightDeltaCommit(enableFileIndex: Boolean, tableVersion: Int): Unit = { - if (HoodieSparkUtils.gteqSpark3_4) { - val (tableName, tablePath) = ("hoodie_mor_snapshot_read_test_table", s"${basePath}_mor_test_table") - val orderingFields = "col3" - val recordKeyField = "key" - val dataField = "age" - - val options = Map[String, String]( - DataSourceWriteOptions.TABLE_TYPE.key -> HoodieTableType.MERGE_ON_READ.name, - DataSourceWriteOptions.OPERATION.key -> UPSERT_OPERATION_OPT_VAL, - DataSourceWriteOptions.PRECOMBINE_FIELD.key -> orderingFields, - DataSourceWriteOptions.RECORDKEY_FIELD.key -> recordKeyField, - DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "", - DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator", - HoodieWriteConfig.TBL_NAME.key -> tableName, - "hoodie.insert.shuffle.parallelism" -> "1", - "hoodie.upsert.shuffle.parallelism" -> "1") - val pathForQuery = getPathForROQuery(tablePath, !enableFileIndex, 0) - - var (writeOpts, readOpts) = getWriterReaderOpts(HoodieRecordType.AVRO, options, enableFileIndex) - writeOpts = writeOpts ++ Map( - HoodieTableConfig.VERSION.key() -> tableVersion.toString) - - val firstDf = spark.range(0, 10).toDF(recordKeyField) - .withColumn(orderingFields, expr(recordKeyField)) - .withColumn(dataField, expr(recordKeyField + " + 1000")) - firstDf.write.format("hudi") - .options(writeOpts) - .mode(SaveMode.Overwrite) - .save(tablePath) - - val secondDf = spark.range(0, 10).toDF(recordKeyField) - .withColumn(orderingFields, expr(recordKeyField)) - .withColumn(dataField, expr(recordKeyField + " + 2000")) - secondDf.write.format("hudi") - .options(writeOpts) - .mode(SaveMode.Append).save(tablePath) - - // Snapshot query on MOR - val snapshotDf = spark.read.format("org.apache.hudi") - .options(readOpts) - .load(pathForQuery) - - // Delete last completed instant - metaClient = createMetaClient(spark, tablePath) - val files = storage.listDirectEntries(new StoragePath(s"$tablePath/.hoodie")).stream() - .filter(JavaConversions.getPredicate((f: StoragePathInfo) => f.getPath.getName.contains(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) - && !f.getPath.getName.contains("inflight") - && !f.getPath.getName.contains("requested"))) - .collect(Collectors.toList[StoragePathInfo]).asScala - assertEquals(1, files.size) - storage.deleteFile(files.head.getPath) - - // verify snapshot query returns data written using firstDf - assertEquals(10, snapshotDf.count()) - assertEquals( - 1000L, - snapshotDf.where(col(recordKeyField) === 0).select(dataField).collect()(0).getLong(0)) - } - } - @ParameterizedTest @CsvSource(Array( "AVRO, AVRO, END_MAP", diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala index ee1edbcccb296..5eb75476cab2e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala @@ -17,8 +17,6 @@ package org.apache.hudi.functional -import org.apache.avro.Schema -import org.apache.calcite.runtime.SqlFunctions.abs import org.apache.hudi.HoodieBaseRelation.projectSchema import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig} import org.apache.hudi.common.model.{HoodieRecord, OverwriteNonDefaultsWithLatestAvroPayload} @@ -28,11 +26,14 @@ import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig} import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.hudi.testutils.SparkClientFunctionalTestHarness.getSparkSqlConf import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, DefaultSource, HoodieBaseRelation, HoodieSparkUtils, HoodieUnsafeRDD} + +import org.apache.avro.Schema import org.apache.parquet.hadoop.util.counters.BenchmarkCounter import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.{Dataset, HoodieUnsafeUtils, Row, SaveMode} + import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail} import org.junit.jupiter.api.{Disabled, Tag, Test} @@ -398,7 +399,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with assertEquals(expectedRecordCount, rows.length) // verify within 10% of margin. - assertTrue((abs(expectedBytesRead - bytesRead) / expectedBytesRead) < 0.1) + assertTrue((Math.abs(expectedBytesRead - bytesRead) / expectedBytesRead) < 0.1) val readColumns = targetColumns ++ relation.mandatoryFields val (_, projectedStructType, _) = projectSchema(Left(tableState.schema), readColumns) diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala index 6393ef524edbe..17de9e00fea45 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.adapter -import org.apache.avro.Schema -import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.util.JsonUtils import org.apache.hudi.spark3.internal.ReflectUtil import org.apache.hudi.{AvroConversionUtils, DefaultSource, HoodieSparkUtils, Spark3RowSerDe} + +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.parquet.schema.MessageType import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters} @@ -43,6 +45,7 @@ import org.apache.spark.storage.StorageLevel import java.time.ZoneId import java.util.TimeZone import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConverters.mapAsScalaMapConverter import scala.collection.convert.Wrappers.JConcurrentMapWrapper /** @@ -62,10 +65,8 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging { val encoder = RowEncoder(schema).resolveAndBind() new Spark3RowSerDe(encoder) } -import java.time.ZoneId -import org.apache.spark.api.java -override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters + override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters override def getSparkParsePartitionUtil: SparkParsePartitionUtil = Spark3ParsePartitionUtil @@ -117,7 +118,7 @@ override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAv new ParquetReadSupport() } - override def getReaderSchemas(storage: HoodieStorage, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): + override def getReaderSchemas(conf: Configuration, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): org.apache.hudi.common.util.collection.Pair[StructType, StructType] = { org.apache.hudi.common.util.collection.Pair.of( HoodieInternalRowUtils.getCachedSchema(readerSchema), diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index 8a406f431d086..b7f1e69d60888 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.adapter import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.{Spark34HoodieFileScanRDD, SparkAdapterSupport$} import org.apache.hudi.io.storage.HoodieSparkParquetReader -import org.apache.hudi.storage.HoodieStorage import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index 2b2ffd9f37a33..014fdccc357bc 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -33,7 +33,8 @@ import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.io.storage.HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.hudi.common.table.ParquetTableSchemaResolver +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter + import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -53,6 +54,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{AtomicType, DataType, StructType} import org.apache.spark.util.SerializableConfiguration + /** * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior * that's not possible to customize in any other way @@ -70,7 +72,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu HOption.empty() } else { HOption.ofNullable( - ParquetTableSchemaResolver.convertAvroSchemaToParquet(avroTableSchema, new Configuration()) + getAvroSchemaConverter(new Configuration()).convert(avroTableSchema) ) } } @@ -83,7 +85,10 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { val conf = sparkSession.sessionState.conf - conf.parquetVectorizedReaderEnabled && schema.forall(_.dataType.isInstanceOf[AtomicType]) + conf.parquetVectorizedReaderEnabled && + schema.forall(_.dataType.isInstanceOf[AtomicType]) && + ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && + supportBatchWithTableSchema } def supportsColumnar(sparkSession: SparkSession, schema: StructType): Boolean = { @@ -95,14 +100,6 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu supportBatch(sparkSession, schema) } - /** - * Returns whether the reader can return the rows as batch or not. - */ - override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - val conf = sparkSession.sessionState.conf - ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && supportBatchWithTableSchema - } - override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index bc2aaf9bcb7a0..cb28e3a3ec264 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -66,6 +66,28 @@ com.github.os72 protoc-jar-maven-plugin + 3.11.4 + + + proto-test-compile + generate-test-sources + + run + + + test + ${project.build.directory}/generated-test-sources/protobuf + + src/test/resources + + + + + + com.google.protobuf:protoc:${proto.version} + ${protoc.version} + true + @@ -132,6 +154,12 @@ + + org.apache.hudi + hudi-spark-common_${scala.binary.version} + ${project.version} + + @@ -514,5 +542,11 @@ ${thrift.version} test + + org.apache.hudi + hudi-spark-common_2.12 + 0.14.2-SNAPSHOT + test + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index 856b5266c97cb..79cfa0b1e9c2a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -174,6 +174,8 @@ public class HoodieMetadataTableValidator implements Serializable { private final String taskLabels; + private final List throwables = new ArrayList<>(); + public HoodieMetadataTableValidator(JavaSparkContext jsc, Config cfg) { this.jsc = jsc; this.cfg = cfg; @@ -191,6 +193,27 @@ public HoodieMetadataTableValidator(JavaSparkContext jsc, Config cfg) { this.taskLabels = generateValidationTaskLabels(); } + /** + * Returns list of Throwable which were encountered during validation. This method is useful + * when ignoreFailed parameter is set to true. + */ + public List getThrowables() { + return throwables; + } + + /** + * Returns true if there is a validation failure encountered during validation. + * This method is useful when ignoreFailed parameter is set to true. + */ + public boolean hasValidationFailure() { + for (Throwable throwable : throwables) { + if (throwable instanceof HoodieValidationException) { + return true; + } + } + return false; + } + private String generateValidationTaskLabels() { List labelList = new ArrayList<>(); if (cfg.validateLatestBaseFiles) { @@ -430,6 +453,7 @@ private boolean doHoodieMetadataTableValidationOnce() { if (!cfg.ignoreFailed) { throw e; } + throwables.add(e); return false; } } @@ -494,12 +518,12 @@ public boolean doMetadataTableValidation() { HoodieMetadataValidationContext fsBasedContext = new HoodieMetadataValidationContext(engineContext, cfg, metaClient, false)) { Set finalBaseFilesForCleaning = baseFilesForCleaning; - List> result = new ArrayList<>( + List> result = new ArrayList<>( engineContext.parallelize(allPartitions, allPartitions.size()).map(partitionPath -> { try { validateFilesInPartition(metadataTableBasedContext, fsBasedContext, partitionPath, finalBaseFilesForCleaning); LOG.info(String.format("Metadata table validation succeeded for partition %s (partition %s)", partitionPath, taskLabels)); - return Pair.of(true, ""); + return Pair.of(true, null); } catch (HoodieValidationException e) { LOG.error( String.format("Metadata table validation failed for partition %s due to HoodieValidationException (partition %s)", @@ -507,26 +531,25 @@ public boolean doMetadataTableValidation() { if (!cfg.ignoreFailed) { throw e; } - return Pair.of(false, e.getMessage() + " for partition: " + partitionPath); + return Pair.of(false, new HoodieValidationException("Validation failed for partition: " + partitionPath, e)); } }).collectAsList()); try { validateRecordIndex(engineContext, metaClient, metadataTableBasedContext.getTableMetadata()); - result.add(Pair.of(true, "")); + result.add(Pair.of(true, null)); } catch (HoodieValidationException e) { - LOG.error( - "Metadata table validation failed due to HoodieValidationException in record index validation", e); - if (!cfg.ignoreFailed) { - throw e; - } - result.add(Pair.of(false, e.getMessage())); + handleValidationException( + e, result, "Metadata table validation failed due to HoodieValidationException in partition stats validation"); } - for (Pair res : result) { + for (Pair res : result) { finalResult &= res.getKey(); if (res.getKey().equals(false)) { - LOG.error("Metadata Validation failed for table: " + cfg.basePath + " with error: " + res.getValue()); + LOG.error("Metadata Validation failed for table: {}", cfg.basePath, res.getValue()); + if (res.getRight() != null) { + throwables.add(res.getRight()); + } } } @@ -544,6 +567,14 @@ public boolean doMetadataTableValidation() { } } + private void handleValidationException(HoodieValidationException e, List> result, String errorMsg) { + LOG.error("{} for table: {} ", errorMsg, cfg.basePath, e); + if (!cfg.ignoreFailed) { + throw e; + } + result.add(Pair.of(false, e)); + } + /** * Check metadata is initialized and available to ready. * If not we will log.warn and skip current validation. @@ -577,7 +608,7 @@ private boolean checkMetadataTableIsAvailable() { /** * Compare the listing partitions result between metadata table and fileSystem. */ - private List validatePartitions(HoodieSparkEngineContext engineContext, String basePath) { + List validatePartitions(HoodieSparkEngineContext engineContext, String basePath) { // compare partitions List allPartitionPathsFromFS = FSUtils.getAllPartitionPaths(engineContext, basePath, false, cfg.assumeDatePartitioning); HoodieTimeline completedTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java index 246be5f8ec614..4673eceed1577 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java @@ -60,7 +60,6 @@ public void configure(Map configs, boolean isKey) { /** * We need to inject sourceSchema instead of reader schema during deserialization or later stages of the pipeline. * - * @param includeSchemaAndVersion * @param topic * @param isKey * @param payload @@ -70,13 +69,12 @@ public void configure(Map configs, boolean isKey) { */ @Override protected Object deserialize( - boolean includeSchemaAndVersion, String topic, Boolean isKey, byte[] payload, Schema readerSchema) throws SerializationException { - return super.deserialize(includeSchemaAndVersion, topic, isKey, payload, sourceSchema); + return super.deserialize(topic, isKey, payload, sourceSchema); } protected TypedProperties getConvertToTypedProperties(Map configs) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java index 11998f2cfacdc..8ed11bd323e85 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java @@ -19,8 +19,7 @@ package org.apache.hudi.utilities.streamer; -import org.apache.hudi.DataSourceWriteOptions; -import org.apache.hudi.HoodieWriterUtils; +import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.async.AsyncClusteringService; import org.apache.hudi.async.AsyncCompactService; import org.apache.hudi.async.SparkAsyncClusteringService; @@ -51,6 +50,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.HoodieWriterUtils; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringUpdateException; import org.apache.hudi.exception.HoodieException; @@ -176,7 +176,7 @@ private static TypedProperties combineProperties(Config cfg, Option { - validator.validatePartitions(engineContext, baseStoragePath, metaClient); + validator.validatePartitions(engineContext, basePath); }); } else { // 3rd partition creation time is > last completed instant @@ -168,7 +161,7 @@ public void testAdditionalPartitionsinMDT(boolean testFailureCase) throws Interr validator.setPartitionCreationTime(Option.of(HoodieActiveTimeline.createNewInstantTime())); // validate that all 3 partitions are returned - assertEquals(mdtPartitions, validator.validatePartitions(engineContext, baseStoragePath, metaClient)); + assertEquals(mdtPartitions, validator.validatePartitions(engineContext, basePath)); } } @@ -193,21 +186,6 @@ void setFsPartitionsToReturn(List fsPartitionsToReturn) { void setPartitionCreationTime(Option partitionCreationTime) { this.partitionCreationTime = partitionCreationTime; } - - @Override - List getPartitionsFromFileSystem(HoodieEngineContext engineContext, StoragePath basePath, HoodieStorage storage, HoodieTimeline completedTimeline) { - return fsPartitionsToReturn; - } - - @Override - List getPartitionsFromMDT(HoodieEngineContext engineContext, StoragePath basePath, HoodieStorage storage) { - return metadataPartitionsToReturn; - } - - @Override - Option getPartitionCreationInstant(HoodieStorage storage, StoragePath basePath, String partition) { - return this.partitionCreationTime; - } } @Test @@ -253,7 +231,8 @@ public void testRliValidationFalsePositiveCase() throws IOException { config.validateAllFileGroups = true; // lets ensure we have a pending commit when FS based polling is done. and the commit completes when MDT is polled. - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration())).build(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf( + new Configuration(jsc.hadoopConfiguration())).build(); // moving out the completed commit meta file to a temp location HoodieInstant lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get(); String latestCompletedCommitMetaFile = basePath + "/.hoodie/" + lastInstant.getFileName(); @@ -281,15 +260,6 @@ public MockHoodieMetadataTableValidatorForRli(JavaSparkContext jsc, Config cfg) super(jsc, cfg); } - @Override - JavaPairRDD> getRecordLocationsFromRLI(HoodieSparkEngineContext sparkEngineContext, - String basePath, - String latestCompletedCommit) { - // move the completed file back to ".hoodie" to simuate the false positive case. - new File(destFilePath).renameTo(new File(originalFilePath)); - return super.getRecordLocationsFromRLI(sparkEngineContext, basePath, latestCompletedCommit); - } - public void setDestFilePath(String destFilePath) { this.destFilePath = destFilePath; } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index 2b258bcb54256..a4b96798f51f0 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -661,9 +661,9 @@ public void testTimestampMillis() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1000, tableBasePath, sqlContext); - TestHelpers.assertCommitMetadata("00000", tableBasePath, 1); + TestHelpers.assertCommitMetadata("00000", tableBasePath, fs, 1); TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.createMetaClient(storage, tableBasePath)); + HoodieTestUtils.init(hadoopConf, tableBasePath)); Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); assertEquals(1000, sparkSession.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts > '1980-01-01'").count()); @@ -678,9 +678,9 @@ public void testTimestampMillis() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1450, tableBasePath, sqlContext); - TestHelpers.assertCommitMetadata("00001", tableBasePath, 2); + TestHelpers.assertCommitMetadata("00001", tableBasePath, fs, 2); tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.createMetaClient(storage, tableBasePath)); + HoodieTestUtils.init(hadoopConf, tableBasePath)); tableSchema = tableSchemaResolver.getTableAvroSchema(false); assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); sqlContext.clearCache(); @@ -716,9 +716,9 @@ public void testLogicalTypes() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1000, tableBasePath, sqlContext); - TestHelpers.assertCommitMetadata("00000", tableBasePath, 1); + TestHelpers.assertCommitMetadata("00000", tableBasePath, fs, 1); TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.createMetaClient(storage, tableBasePath)); + HoodieTestUtils.init(hadoopConf, tableBasePath)); Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); Map hudiOpts = new HashMap<>(); hudiOpts.put("hoodie.datasource.write.recordkey.field", "id"); @@ -732,9 +732,9 @@ public void testLogicalTypes() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1450, tableBasePath, sqlContext); - TestHelpers.assertCommitMetadata("00001", tableBasePath, 2); + TestHelpers.assertCommitMetadata("00001", tableBasePath, fs, 2); tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.createMetaClient(storage, tableBasePath)); + HoodieTestUtils.init(hadoopConf, tableBasePath)); tableSchema = tableSchemaResolver.getTableAvroSchema(false); logicalAssertions(tableSchema, tableBasePath, hudiOpts, HoodieTableVersion.current().versionCode()); } finally { @@ -796,7 +796,7 @@ private void assertBoundaryCounts(Dataset df, String exprZero, String exprT @ParameterizedTest @CsvSource(value = {"SIX,AVRO,CLUSTER", "CURRENT,AVRO,NONE", "CURRENT,AVRO,CLUSTER", "CURRENT,SPARK,NONE", "CURRENT,SPARK,CLUSTER"}) - void testCOWLogicalRepair(String tableVersion, String recordType, String operation) throws Exception { + void testCOWLogicalRepair(String tableVersion, String recordType, String operation) throws Throwable { timestampNTZCompatibility(() -> { try { String dirName = "trips_logical_types_json_cow_write"; @@ -844,7 +844,7 @@ void testCOWLogicalRepair(String tableVersion, String recordType, String operati // Validate raw parquet files HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(storage.getConf()) + .setConf(hadoopConf) .setBasePath(tableBasePath) .build(); @@ -881,7 +881,7 @@ void testCOWLogicalRepair(String tableVersion, String recordType, String operati "CURRENT,SPARK,NONE,PARQUET", "CURRENT,SPARK,CLUSTER,PARQUET", "CURRENT,SPARK,COMPACT,PARQUET"}) - void testMORLogicalRepair(String tableVersion, String recordType, String operation, String logBlockType) throws Exception { + void testMORLogicalRepair(String tableVersion, String recordType, String operation, String logBlockType) throws Throwable { timestampNTZCompatibility(() -> { try { String tableSuffix; @@ -901,7 +901,7 @@ void testMORLogicalRepair(String tableVersion, String recordType, String operati String tableBasePath = zipOutput.toString(); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(storage.getConf()) + .setConf(hadoopConf) .setBasePath(tableBasePath) .build(); @@ -953,16 +953,14 @@ void testMORLogicalRepair(String tableVersion, String recordType, String operati String prevTimezone = sparkSession.conf().get("spark.sql.session.timeZone"); try { - if (!HoodieSparkUtils.gteqSpark3_5()) { - sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "false"); - } + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "false"); sparkSession.conf().set("spark.sql.session.timeZone", "UTC"); Dataset df = sparkSession.read().format("hudi").load(tableBasePath); assertDataframe(df, 12, 14); metaClient = HoodieTableMetaClient.builder() - .setConf(storage.getConf()) + .setConf(hadoopConf) .setBasePath(tableBasePath) .build(); @@ -999,9 +997,7 @@ void testMORLogicalRepair(String tableVersion, String recordType, String operati } } finally { sparkSession.conf().set("spark.sql.session.timeZone", prevTimezone); - if (!HoodieSparkUtils.gteqSpark3_5()) { - sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "true"); - } + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "true"); } } catch (Exception e) { throw new RuntimeException(e); @@ -1055,7 +1051,7 @@ private List collectLatestBaseFilePaths(HoodieTableMetaClient metaClient HoodieTableFileSystemView fsView = null; try { fsView = FileSystemViewManager.createInMemoryFileSystemView( - new HoodieLocalEngineContext(metaClient.getStorageConf()), + new HoodieLocalEngineContext(metaClient.getHadoopConf()), metaClient, HoodieMetadataConfig.newBuilder().enable(false).build()); fsView.loadAllPartitions(); final HoodieTableFileSystemView fileSystemView = fsView; @@ -3440,4 +3436,26 @@ private static Stream testORCDFSSource() { arguments(true, Collections.singletonList(TripsWithDistanceTransformer.class.getName())) ); } + + public static void timestampNTZCompatibility(Runnable r) throws Throwable { + // TODO: Remove this when we get rid of spark3.3. TimestampNTZ needs this config + // to be set to true to work. + boolean isSpark33 = HoodieSparkUtils.isSpark3_3(); + String propertyValue = null; + if (isSpark33) { + propertyValue = System.getProperty("spark.testing"); + System.setProperty("spark.testing", "true"); + } + try { + r.run(); + } finally { + if (isSpark33) { + if (propertyValue == null) { + System.clearProperty("spark.testing"); + } else { + System.setProperty("spark.testing", propertyValue); + } + } + } + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java index 16d190ac45d15..ab2a89c62c6a2 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java @@ -105,7 +105,7 @@ public void testKafkaAvroSchemaDeserializer() { avroDeserializer.configure(new HashMap(config), false); bytesOrigRecord = avroSerializer.serialize(topic, avroRecord); // record is serialized in orig schema and deserialized using same schema. - assertEquals(avroRecord, avroDeserializer.deserialize(false, topic, false, bytesOrigRecord, origSchema)); + assertEquals(avroRecord, avroDeserializer.deserialize(topic, false, bytesOrigRecord, origSchema)); IndexedRecord avroRecordWithAllField = createExtendUserRecord(); byte[] bytesExtendedRecord = avroSerializer.serialize(topic, avroRecordWithAllField); @@ -115,12 +115,12 @@ public void testKafkaAvroSchemaDeserializer() { avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(config)); avroDeserializer.configure(new HashMap(config), false); // record is serialized w/ evolved schema, and deserialized w/ evolved schema - IndexedRecord avroRecordWithAllFieldActual = (IndexedRecord) avroDeserializer.deserialize(false, topic, false, bytesExtendedRecord, evolSchema); + IndexedRecord avroRecordWithAllFieldActual = (IndexedRecord) avroDeserializer.deserialize(topic, false, bytesExtendedRecord, evolSchema); assertEquals(avroRecordWithAllField, avroRecordWithAllFieldActual); assertEquals(avroRecordWithAllFieldActual.getSchema(), evolSchema); // read old record w/ evolved schema. - IndexedRecord actualRec = (IndexedRecord) avroDeserializer.deserialize(false, topic, false, bytesOrigRecord, origSchema); + IndexedRecord actualRec = (IndexedRecord) avroDeserializer.deserialize(topic, false, bytesOrigRecord, origSchema); // record won't be equal to original record as we read w/ evolved schema. "age" will be added w/ default value of null assertNotEquals(avroRecord, actualRec); GenericRecord genericRecord = (GenericRecord) actualRec; diff --git a/pom.xml b/pom.xml index 3b421b9795d08..c55abe27f4f58 100644 --- a/pom.xml +++ b/pom.xml @@ -104,7 +104,7 @@ ${pulsar.spark.scala12.version} 2.4.5 3.1.1.4 - 5.3.4 + 5.5.0 2.17 3.0.1-b12 1.10.1 From b132c9bc7ff5d9d29024f7d108bf56e46bebf284 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Wed, 21 Jan 2026 14:52:27 -0800 Subject: [PATCH 03/32] Fix some bugs --- .../io/storage/HoodieSparkParquetReader.java | 6 +++--- .../org/apache/hudi/AvroConversionUtils.scala | 9 ++++++--- .../org/apache/hudi/SparkConversionUtils.scala | 4 +++- .../org/apache/parquet/schema/SchemaRepair.java | 2 +- .../spark/sql/adapter/Spark2Adapter.scala | 3 +-- .../Spark34LegacyHoodieParquetFileFormat.scala | 5 ++--- .../deltastreamer/TestHoodieDeltaStreamer.java | 17 +++++++++-------- pom.xml | 3 ++- 8 files changed, 27 insertions(+), 22 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index c929f1dac0f67..45b0093b8ffd7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -145,7 +145,7 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema } private MessageType getFileSchema() { - if (fileSchemaOption.isPresent()) { + if (!fileSchemaOption.isPresent()) { MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path); fileSchemaOption = Option.of(messageType); } @@ -154,7 +154,7 @@ private MessageType getFileSchema() { @Override public Schema getSchema() { - if (schemaOption.isPresent()) { + if (!schemaOption.isPresent()) { // Some types in avro are not compatible with parquet. // Avro only supports representing Decimals as fixed byte array // and therefore if we convert to Avro directly we'll lose logical type-info. @@ -168,7 +168,7 @@ public Schema getSchema() { } protected StructType getStructSchema() { - if (structTypeOption.isPresent()) { + if (!structTypeOption.isPresent()) { MessageType messageType = getFileSchema(); structTypeOption = Option.of(convertToStruct(messageType)); } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index a107d00466e62..7948b1a4baa05 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -97,8 +97,11 @@ object AvroConversionUtils { * TODO convert directly from GenericRecord into InternalRow instead */ def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = { - if (rdd.isEmpty()) { - ss.emptyDataFrame + // Avoid calling isEmpty() which can cause serialization issues with Ordering$Reverse + // Check partition count instead, which doesn't require task serialization + val structType = convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr)) + if (rdd.getNumPartitions == 0) { + ss.createDataFrame(ss.sparkContext.emptyRDD[Row], structType) } else { ss.createDataFrame(rdd.mapPartitions { records => if (records.isEmpty) Iterator.empty @@ -108,7 +111,7 @@ object AvroConversionUtils { val converter = createConverterToRow(schema, dataType) records.map { r => converter(r) } } - }, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr))) + }, structType) } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala index 799bda2f8e5ba..736f69fd17157 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala @@ -27,7 +27,9 @@ import org.apache.spark.sql.{DataFrameUtil, Dataset, Row, SparkSession} object SparkConversionUtils { def createDataFrame[T](rdd: RDD[HoodieRecord[T]], ss: SparkSession, structType: StructType): Dataset[Row] = { - if (rdd.isEmpty()) { + // Avoid calling isEmpty() which can cause serialization issues with Ordering$Reverse + // Check partition count instead, which doesn't require task serialization + if (rdd.getNumPartitions == 0) { ss.emptyDataFrame } else { DataFrameUtil.createFromInternalRows(ss, structType, rdd.map(_.getData.asInstanceOf[InternalRow])) diff --git a/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java b/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java index ad38ff806ce09..578c42fa4b3a1 100644 --- a/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java +++ b/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java @@ -27,7 +27,7 @@ public class SchemaRepair { public static MessageType repairLogicalTypes(MessageType fileSchema, Option tableSchema) { - if (tableSchema.isPresent()) { + if (!tableSchema.isPresent()) { return fileSchema; } return repairLogicalTypes(fileSchema, tableSchema.get()); diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index d28ddf22c218c..47f2ce46c7bf9 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -25,7 +25,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark2HoodieFileScanRDD, Spark2RowSerDe} import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.storage.{HoodieStorage, StoragePath} import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark2HoodieFileScanRDD, Spark2RowSerDe} import org.apache.avro.Schema @@ -235,7 +234,7 @@ class Spark2Adapter extends SparkAdapter { batch } - override def getReaderSchemas(storage: HoodieStorage, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): + override def getReaderSchemas(conf: Configuration, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType): org.apache.hudi.common.util.collection.Pair[StructType, StructType] = { org.apache.hudi.common.util.collection.Pair.of( HoodieInternalRowUtils.getCachedSchema(readerSchema), diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index 014fdccc357bc..fd8e52ea1e92c 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -33,8 +33,7 @@ import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.io.storage.HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter - +import org.apache.hudi.common.table.ParquetTableSchemaResolver import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -72,7 +71,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu HOption.empty() } else { HOption.ofNullable( - getAvroSchemaConverter(new Configuration()).convert(avroTableSchema) + ParquetTableSchemaResolver.convertAvroSchemaToParquet(avroTableSchema, new Configuration()) ) } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index a4b96798f51f0..7b21ef0880239 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -662,8 +662,10 @@ public void testTimestampMillis() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1000, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00000", tableBasePath, fs, 1); - TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.init(hadoopConf, tableBasePath)); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setBasePath(cfg.targetBasePath) + .setConf(new Configuration()).build(); + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); assertEquals(1000, sparkSession.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts > '1980-01-01'").count()); @@ -679,8 +681,6 @@ public void testTimestampMillis() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1450, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00001", tableBasePath, fs, 2); - tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.init(hadoopConf, tableBasePath)); tableSchema = tableSchemaResolver.getTableAvroSchema(false); assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); sqlContext.clearCache(); @@ -717,8 +717,10 @@ public void testLogicalTypes() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1000, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00000", tableBasePath, fs, 1); - TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.init(hadoopConf, tableBasePath)); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setBasePath(cfg.targetBasePath) + .setConf(new Configuration()).build(); + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); Map hudiOpts = new HashMap<>(); hudiOpts.put("hoodie.datasource.write.recordkey.field", "id"); @@ -733,8 +735,7 @@ public void testLogicalTypes() throws Exception { new HoodieDeltaStreamer(cfg, jsc).sync(); assertRecordCount(1450, tableBasePath, sqlContext); TestHelpers.assertCommitMetadata("00001", tableBasePath, fs, 2); - tableSchemaResolver = new TableSchemaResolver( - HoodieTestUtils.init(hadoopConf, tableBasePath)); + tableSchemaResolver = new TableSchemaResolver(metaClient); tableSchema = tableSchemaResolver.getTableAvroSchema(false); logicalAssertions(tableSchema, tableBasePath, hudiOpts, HoodieTableVersion.current().versionCode()); } finally { diff --git a/pom.xml b/pom.xml index c55abe27f4f58..9f6e1eef7bb8a 100644 --- a/pom.xml +++ b/pom.xml @@ -100,6 +100,7 @@ ${fasterxml.spark3.version} 2.0.0 2.8.0 + 3.0.0 2.8.1 ${pulsar.spark.scala12.version} 2.4.5 @@ -2627,7 +2628,7 @@ hudi-spark3-common hudi-spark3.2plus-common ${scalatest.spark3.version} - ${kafka.spark3.version} + ${kafka.spark34.version} 2.3.10 2.8.1 + + org.apache.hive + hive-storage-api + ${hive.storage.version} + test + + ${hive.groupid} hive-jdbc From ac14313ca63c54456e3a49c7f99a77df7a446a3a Mon Sep 17 00:00:00 2001 From: Lokesh Jain Date: Thu, 5 Feb 2026 00:20:57 +0530 Subject: [PATCH 16/32] Fix test and address review comments --- .../apache/hudi/io/storage/HoodieSparkParquetReader.java | 6 +++--- .../common/table/log/AbstractHoodieLogRecordReader.java | 4 +++- .../java/org/apache/hudi/io/storage/HoodieFileReader.java | 2 ++ .../src/main/scala/org/apache/hudi/HoodieBaseRelation.scala | 4 ++-- .../main/scala/org/apache/hudi/IncrementalRelation.scala | 4 ++-- pom.xml | 2 ++ 6 files changed, 14 insertions(+), 8 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index c312327397684..c836fff4138fe 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -64,7 +64,6 @@ public class HoodieSparkParquetReader implements HoodieSparkFileReader { private final Configuration conf; private final BaseFileUtils parquetUtils; private List readerIterators = new ArrayList<>(); - public static final String ENABLE_LOGICAL_TIMESTAMP_REPAIR = "spark.hudi.logicalTimestampField.repair.enable"; private Option fileSchemaOption = Option.empty(); private Option structTypeOption = Option.empty(); private Option schemaOption = Option.empty(); @@ -127,8 +126,9 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema requestedSchema = readerSchema; } // Set configuration for timestamp_millis type repair. - conf.set(ENABLE_LOGICAL_TIMESTAMP_REPAIR, Boolean.toString(AvroSchemaUtils.hasTimestampMillisField(readerSchema))); - + if (!storage.getConf().contains(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR)) { + storage.getConf().set(ENABLE_LOGICAL_TIMESTAMP_REPAIR, Boolean.toString(AvroSchemaUtils.hasTimestampMillisField(readerSchema))); + } MessageType fileSchema = getFileSchema(); Schema nonNullSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema); Option messageSchema = Option.of(getAvroSchemaConverter(conf).convert(nonNullSchema)); 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 7222f334e55b8..ecca93fa487b4 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 @@ -44,6 +44,7 @@ import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.action.InternalSchemaMerger; import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; +import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; @@ -187,7 +188,8 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List readerSchema != null && AvroSchemaUtils.hasTimestampMillisField(readerSchema)); if (keyFieldOverride.isPresent()) { // NOTE: This branch specifically is leveraged handling Metadata Table diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java index 00fff9a220c64..b6a2b8d60a074 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java @@ -41,6 +41,8 @@ */ public interface HoodieFileReader extends AutoCloseable { + String ENABLE_LOGICAL_TIMESTAMP_REPAIR = "spark.hudi.logicalTimestampField.repair.enable"; + String[] readMinMaxRecordKeys(); BloomFilter readBloomFilter(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 32f6c96851f6a..f09d3ae3c9602 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -47,7 +47,7 @@ import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} import org.apache.hudi.io.storage.HoodieAvroHFileReader -import org.apache.hudi.io.storage.HoodieSparkParquetReader +import org.apache.hudi.io.storage.HoodieFileReader import org.apache.hudi.metadata.HoodieTableMetadata import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging @@ -111,7 +111,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, protected lazy val conf: Configuration = { val c = new Configuration(sqlContext.sparkContext.hadoopConfiguration) - c.set(HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, + c.set(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, AvroSchemaUtils.hasTimestampMillisField(tableAvroSchema).toString) c } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala index 1a2dc403d2aac..3790839e88755 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala @@ -36,7 +36,7 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.{HoodieException, HoodieIncrementalPathNotFoundException} import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.utils.SerDeHelper -import org.apache.hudi.io.storage.HoodieSparkParquetReader +import org.apache.hudi.io.storage.HoodieFileReader import org.apache.hudi.table.HoodieSparkTable import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD @@ -210,7 +210,7 @@ class IncrementalRelation(val sqlContext: SQLContext, LegacyHoodieParquetFileFormat.setTableAvroSchemaInConf( sqlContext.sparkContext.hadoopConfiguration, tableAvroSchema) sqlContext.sparkContext.hadoopConfiguration.set( - HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, + HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, AvroSchemaUtils.hasTimestampMillisField(tableAvroSchema).toString) } val formatClassName = metaClient.getTableConfig.getBaseFileFormat match { diff --git a/pom.xml b/pom.xml index 9f6e1eef7bb8a..c83d55f1b8dd9 100644 --- a/pom.xml +++ b/pom.xml @@ -2348,6 +2348,8 @@ hudi-spark2-common 2.0.0 + + 2.6.0 1.10.1 1.6.0 1.8.2 From bd20879e299130081dd0a41db76ab19c42737d34 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Thu, 5 Feb 2026 14:19:11 -0800 Subject: [PATCH 17/32] Cherry pick bug fixes --- .../apache/hudi/io/storage/HoodieSparkParquetReader.java | 6 +++--- .../common/table/log/AbstractHoodieLogRecordReader.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index c836fff4138fe..0747ad321fdc4 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -125,9 +125,9 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema if (requestedSchema == null) { requestedSchema = readerSchema; } - // Set configuration for timestamp_millis type repair. - if (!storage.getConf().contains(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR)) { - storage.getConf().set(ENABLE_LOGICAL_TIMESTAMP_REPAIR, Boolean.toString(AvroSchemaUtils.hasTimestampMillisField(readerSchema))); + // Set configuration for timestamp_millis type repair (only when not already set). + if (conf.get(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR) == null) { + conf.set(ENABLE_LOGICAL_TIMESTAMP_REPAIR, Boolean.toString(AvroSchemaUtils.hasTimestampMillisField(readerSchema))); } MessageType fileSchema = getFileSchema(); Schema nonNullSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema); 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 ecca93fa487b4..10fe69984abad 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 @@ -188,8 +188,8 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List readerSchema != null && AvroSchemaUtils.hasTimestampMillisField(readerSchema)); + this.enableLogicalTimestampFieldRepair = fs.getConf().getBoolean(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, + readerSchema != null && AvroSchemaUtils.hasTimestampMillisField(readerSchema)); if (keyFieldOverride.isPresent()) { // NOTE: This branch specifically is leveraged handling Metadata Table From 94dff70dd32b658b50347b310c8bae134a80f782 Mon Sep 17 00:00:00 2001 From: Lokesh Jain Date: Thu, 5 Feb 2026 02:19:47 +0530 Subject: [PATCH 18/32] Fix build --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index c83d55f1b8dd9..684aa62b967e9 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,9 @@ 2.9.9 2.10.2 org.apache.hive - 2.3.1 + 2.3.4 + + 2.6.0 1.10.1 1.8.2 0.273 @@ -2348,8 +2350,6 @@ hudi-spark2-common 2.0.0 - - 2.6.0 1.10.1 1.6.0 1.8.2 From d17b0e86dfd0d79a6b8b0762841cae7884f3b1d5 Mon Sep 17 00:00:00 2001 From: Lokesh Jain Date: Thu, 5 Feb 2026 11:34:05 +0530 Subject: [PATCH 19/32] Fix build --- .../org/apache/spark/sql/adapter/Spark3_4Adapter.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index df1273cc96a12..76ed34e25d236 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.adapter import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.{Spark34HoodieFileScanRDD, SparkAdapterSupport$} -import org.apache.hudi.io.storage.HoodieSparkParquetReader +import org.apache.hudi.io.storage.HoodieFileReader import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration @@ -84,7 +84,7 @@ class Spark3_4Adapter extends BaseSpark3Adapter { override def getParquetReadSupport(conf: Configuration, messageSchema: org.apache.hudi.common.util.Option[MessageType]): ParquetReadSupport = { val enableTimestampFieldRepair = conf.getBoolean( - HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, true) + HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, true) new HoodieParquetReadSupport( Option.empty[ZoneId], enableVectorizedReader = true, @@ -169,7 +169,7 @@ class Spark3_4Adapter extends BaseSpark3Adapter { val nonNullRequestedSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema) val cachedRequestedSchema = HoodieInternalRowUtils.getCachedSchema(nonNullRequestedSchema) val requestedSchemaInMessageType = org.apache.hudi.common.util.Option.of(getAvroSchemaConverter(conf).convert(nonNullRequestedSchema)) - val enableTimestampFieldRepair = conf.getBoolean(HoodieSparkParquetReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, true) + val enableTimestampFieldRepair = conf.getBoolean(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, true) val repairedRequestedSchema = repairSchemaIfSpecified(enableTimestampFieldRepair, fileSchema, requestedSchemaInMessageType) val repairedRequestedStructType = new ParquetToSparkSchemaConverter(conf).convert(repairedRequestedSchema) val evolution = new SparkBasicSchemaEvolution(repairedRequestedStructType, cachedRequestedSchema, SQLConf.get.sessionLocalTimeZone) From 808d4a42adf95c20e339d5cc2b990bde816c5a62 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Thu, 5 Feb 2026 23:08:35 -0800 Subject: [PATCH 20/32] Revert "Fix tests" This reverts commit d3b78978af6a019a806572d5a0358825780702ba. --- hudi-utilities/pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index b9e40ef001a2e..cb28e3a3ec264 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -439,13 +439,6 @@ - - org.apache.hive - hive-storage-api - ${hive.storage.version} - test - - ${hive.groupid} hive-jdbc From 4f3563a296ed1b6bbc79260e75acffba983dccac Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Thu, 5 Feb 2026 23:09:05 -0800 Subject: [PATCH 21/32] Revert "Fix build" This reverts commit 94dff70dd32b658b50347b310c8bae134a80f782. --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 684aa62b967e9..c83d55f1b8dd9 100644 --- a/pom.xml +++ b/pom.xml @@ -118,9 +118,7 @@ 2.9.9 2.10.2 org.apache.hive - 2.3.4 - - 2.6.0 + 2.3.1 1.10.1 1.8.2 0.273 @@ -2350,6 +2348,8 @@ hudi-spark2-common 2.0.0 + + 2.6.0 1.10.1 1.6.0 1.8.2 From 4e5ed98ec1b874f26b5f684b7f5dad290bb7b000 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Fri, 6 Feb 2026 11:56:08 -0800 Subject: [PATCH 22/32] Fix build issues - 2.11 depending on 2.12 version --- hudi-utilities/pom.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index cb28e3a3ec264..6375b0c985fcd 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -542,11 +542,5 @@ ${thrift.version} test - - org.apache.hudi - hudi-spark-common_2.12 - 0.14.2-SNAPSHOT - test - From cf421c8f05905adc90de730badd57881c49f291d Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Fri, 6 Feb 2026 13:16:57 -0800 Subject: [PATCH 23/32] Fix build issues - Column stream explicty collection.2 --- .../apache/hudi/common/util/ParquetUtils.java | 18 +++++++++++------- .../hudi/metadata/HoodieTableMetadataUtil.java | 7 ++++--- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index e848b166d0ecf..c15ea5fbc34ec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -312,10 +312,12 @@ public List> readRangeFromParquetMetadata( Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName); // Collect stats from all individual Parquet blocks - Map>> columnToStatsListMap = - (Map>>) metadata.getBlocks().stream().sequential() - .flatMap(blockMetaData -> - blockMetaData.getColumns().stream() + // NOTE: Explicit cast on inner stream helps Java with type inference + @SuppressWarnings("unchecked") + Stream> blockStream = metadata.getBlocks().stream().sequential() + .flatMap(blockMetaData -> { + Stream> columnStream = + (Stream>) (Stream) blockMetaData.getColumns().stream() .filter(f -> cols.contains(f.getPath().toDotString())) .map(columnChunkMetaData -> { Statistics stats = columnChunkMetaData.getStatistics(); @@ -335,9 +337,11 @@ public List> readRangeFromParquetMetadata( columnChunkMetaData.getValueCount(), columnChunkMetaData.getTotalSize(), columnChunkMetaData.getTotalUncompressedSize()); - }) - ) - .collect(groupingByCollector); + }); + return columnStream; + }); + + Map>> columnToStatsListMap = blockStream.collect(groupingByCollector); // Combine those into file-level statistics // NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index c34f0309db5d7..4ad0d3126f9aa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -241,7 +241,7 @@ class ColumnStats { Collector, ?, Map>> collector = Collectors.toMap(colRangeMetadata -> colRangeMetadata.getColumnName(), Function.identity()); - return (Map>) targetFields.stream() + Stream> stream = targetFields.stream() .map(field -> { ColumnStats colStats = allColumnStats.get(field.name()); return HoodieColumnRangeMetadata.create( @@ -257,8 +257,9 @@ class ColumnStats { 0, 0 ); - }) - .collect(collector); + }); + + return stream.collect(collector); } /** From 2f0ae3bc61268985bc55f9bbddd55f095fd1bfa0 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Fri, 6 Feb 2026 13:54:08 -0800 Subject: [PATCH 24/32] Fix build issues - Use projection schema instead of repairFileSchema --- .../org/apache/hudi/io/storage/HoodieAvroParquetReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index 275ca3ea738de..6619c7f771b89 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -165,7 +165,7 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema Option promotedSchema = Option.empty(); if (!renamedColumns.isPresent() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema)) { AvroReadSupport.setAvroReadSchema(conf, repairedFileSchema); - AvroReadSupport.setRequestedProjection(conf, repairedFileSchema); + AvroReadSupport.setRequestedProjection(conf, schema); promotedSchema = Option.of(schema); } else { AvroReadSupport.setAvroReadSchema(conf, schema); From a10b077e3bb4ffe228809da6cb0878c0e4f92237 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Fri, 6 Feb 2026 16:06:36 -0800 Subject: [PATCH 25/32] Fix build issues - remove spark3.5 profiles from github workflow bot --- .github/workflows/bot.yml | 104 -------------------------------------- 1 file changed, 104 deletions(-) diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 5c851b8041c34..13104d0a55fec 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -264,110 +264,6 @@ jobs: export PATH="$JAVA_HOME/bin:$PATH" mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - test-spark-java11-17-java-tests: - runs-on: ubuntu-latest - strategy: - matrix: - include: - - scalaProfile: "scala-2.12" - sparkProfile: "spark3.5" - sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" - - scalaProfile: "scala-2.13" - sparkProfile: "spark3.5" - sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" - - steps: - - uses: actions/checkout@v3 - - name: Set up JDK 11 - uses: actions/setup-java@v3 - with: - java-version: '11' - distribution: 'temurin' - architecture: x64 - cache: maven - - name: Build Project - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - run: - mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" - - name: Set up JDK 17 - uses: actions/setup-java@v3 - with: - java-version: '17' - distribution: 'temurin' - architecture: x64 - cache: maven - - name: Quickstart Test - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - run: - mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl hudi-examples/hudi-examples-spark $MVN_ARGS - - name: Java UT - Common & Spark - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - SPARK_MODULES: ${{ matrix.sparkModules }} - run: - mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - - name: Java FT - Spark - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - SPARK_MODULES: ${{ matrix.sparkModules }} - run: - mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - - test-spark-java11-17-scala-tests: - runs-on: ubuntu-latest - strategy: - matrix: - include: - - scalaProfile: "scala-2.12" - sparkProfile: "spark3.5" - sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" - - scalaProfile: "scala-2.13" - sparkProfile: "spark3.5" - sparkModules: "hudi-spark-datasource/hudi-spark3.5.x" - - steps: - - uses: actions/checkout@v3 - - name: Set up JDK 11 - uses: actions/setup-java@v3 - with: - java-version: '11' - distribution: 'temurin' - architecture: x64 - cache: maven - - name: Build Project - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - run: - mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" - - name: Set up JDK 17 - uses: actions/setup-java@v3 - with: - java-version: '17' - distribution: 'temurin' - architecture: x64 - cache: maven - - name: Scala UT - Common & Spark - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - SPARK_MODULES: ${{ matrix.sparkModules }} - run: - mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - - name: Scala FT - Spark - env: - SCALA_PROFILE: ${{ matrix.scalaProfile }} - SPARK_PROFILE: ${{ matrix.sparkProfile }} - SPARK_MODULES: ${{ matrix.sparkModules }} - run: - mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS - test-flink: runs-on: ubuntu-latest strategy: From ac619b60537721e3f95c498481b40d0e6f7c83ba Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Fri, 6 Feb 2026 16:33:52 -0800 Subject: [PATCH 26/32] Fix build issues - Fix IT test pom.xml update --- hudi-cli/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index fcc9a63a87795..de463b42d8315 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -143,6 +143,13 @@ + + + javax.validation + validation-api + 2.0.1.Final + + org.scala-lang From 0f39b5357ca7e1260cbc5faf2a0b209d480e67a3 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Sat, 7 Feb 2026 23:51:42 -0800 Subject: [PATCH 27/32] Fix build issues - Use projection schema fields from repaired schema --- .../io/storage/HoodieAvroParquetReader.java | 40 ++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index 6619c7f771b89..ade684aa2d04c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -44,7 +44,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import static org.apache.hudi.avro.AvroSchemaUtils.getRepairedSchema; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; @@ -165,7 +167,8 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema Option promotedSchema = Option.empty(); if (!renamedColumns.isPresent() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema)) { AvroReadSupport.setAvroReadSchema(conf, repairedFileSchema); - AvroReadSupport.setRequestedProjection(conf, schema); + Schema projectionSchema = computeSafeProjection(repairedFileSchema, schema); + AvroReadSupport.setRequestedProjection(conf, projectionSchema); promotedSchema = Option.of(schema); } else { AvroReadSupport.setAvroReadSchema(conf, schema); @@ -185,6 +188,41 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema return parquetReaderIterator; } + /** + * Computes a safe projection schema by intersecting the requested schema with the file schema. + * This ensures we only request fields that actually exist in the file, enabling column pruning + * while avoiding "field not found" errors. + * + * @param fileSchema The schema from the file (with repaired types) + * @param requestedSchema The schema we'd like to read + * @return A projection schema containing only fields that exist in both schemas + */ + private Schema computeSafeProjection(Schema fileSchema, Schema requestedSchema) { + Map fileFields = fileSchema.getFields().stream() + .collect(Collectors.toMap(Schema.Field::name, f -> f)); + + List projectedFields = requestedSchema.getFields().stream() + .filter(field -> fileFields.containsKey(field.name())) + .map(field -> { + Schema.Field fileField = fileFields.get(field.name()); + return new Schema.Field(fileField.name(), fileField.schema(), fileField.doc(), fileField.defaultVal()); + }) + .collect(Collectors.toList()); + + if (projectedFields.isEmpty()) { + return fileSchema; + } + + Schema projectedSchema = Schema.createRecord( + fileSchema.getName(), + fileSchema.getDoc(), + fileSchema.getNamespace(), + fileSchema.isError() + ); + projectedSchema.setFields(projectedFields); + return projectedSchema; + } + @Override public ClosableIterator getRecordKeyIterator() throws IOException { ClosableIterator recordKeyIterator = getIndexedRecordIterator(HoodieAvroUtils.getRecordKeySchema()); From e2af120956fb636f6d9ca2e1921c71046bdc6f7d Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Sun, 8 Feb 2026 01:09:02 -0800 Subject: [PATCH 28/32] Fix build issues - throw exception only for spark 3.4 gt --- .../scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index d0cff9650eedb..1e7ea964017d9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -854,7 +854,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df2, tempRecordPath, tableType.equals("COPY_ON_WRITE")) // after implicit type change, read the table with vectorized read enabled - if (HoodieSparkUtils.gteqSpark3_3) { + if (HoodieSparkUtils.gteqSpark3_4) { assertThrows(classOf[SparkException]) { withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { readTable(tempRecordPath) From 7508f661a52015bf30167d69bac0ba83a2364ff2 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Sun, 8 Feb 2026 02:56:41 -0800 Subject: [PATCH 29/32] Fix build issues - throw exception only for spark 3.3 --- .../org/apache/hudi/TestAvroSchemaResolutionSupport.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index 1e7ea964017d9..4b90eef3f1bc3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -854,7 +854,9 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df2, tempRecordPath, tableType.equals("COPY_ON_WRITE")) // after implicit type change, read the table with vectorized read enabled - if (HoodieSparkUtils.gteqSpark3_4) { + // Spark 3.3 has a ClassCastException bug with vectorized nested column reader after type changes + // Spark 3.4+ with computeSafeProjection fix handles type promotions correctly + if (HoodieSparkUtils.gteqSpark3_3 && !HoodieSparkUtils.gteqSpark3_4) { assertThrows(classOf[SparkException]) { withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { readTable(tempRecordPath) From 6a7a5fbc159ad90a225b05df33501c50ddfd487d Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Sun, 8 Feb 2026 16:16:05 -0800 Subject: [PATCH 30/32] Fix build issues - remove exceptions --- .../org/apache/hudi/TestAvroSchemaResolutionSupport.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index 4b90eef3f1bc3..18205971bd8ae 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -854,9 +854,8 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df2, tempRecordPath, tableType.equals("COPY_ON_WRITE")) // after implicit type change, read the table with vectorized read enabled - // Spark 3.3 has a ClassCastException bug with vectorized nested column reader after type changes - // Spark 3.4+ with computeSafeProjection fix handles type promotions correctly - if (HoodieSparkUtils.gteqSpark3_3 && !HoodieSparkUtils.gteqSpark3_4) { + val isCow = tableType.equals("COPY_ON_WRITE") + if (HoodieSparkUtils.gteqSpark3_3 && !HoodieSparkUtils.gteqSpark3_4 && isCow) { assertThrows(classOf[SparkException]) { withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { readTable(tempRecordPath) From 6bea797695bbd068d955f9030cd3569843147212 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Mon, 9 Feb 2026 11:34:40 -0800 Subject: [PATCH 31/32] Fix build issues - remove exceptions for spark33 --- .../hudi/TestAvroSchemaResolutionSupport.scala | 13 ++----------- .../Spark33LegacyHoodieParquetFileFormat.scala | 3 +-- 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index 18205971bd8ae..9f9afd1c337bb 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -854,17 +854,8 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df2, tempRecordPath, tableType.equals("COPY_ON_WRITE")) // after implicit type change, read the table with vectorized read enabled - val isCow = tableType.equals("COPY_ON_WRITE") - if (HoodieSparkUtils.gteqSpark3_3 && !HoodieSparkUtils.gteqSpark3_4 && isCow) { - assertThrows(classOf[SparkException]) { - withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { - readTable(tempRecordPath) - } - } - } else { - withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { - readTable(tempRecordPath) - } + withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { + readTable(tempRecordPath) } withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala index add53e12e2493..5c3b8cd297561 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala @@ -121,8 +121,7 @@ class Spark33LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val enableVectorizedReader: Boolean = - sqlConf.parquetVectorizedReaderEnabled && - resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) + ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize From 281228a612219482e0770ecd63224ac876a5b129 Mon Sep 17 00:00:00 2001 From: Pavithran Ravichandiran Date: Mon, 9 Feb 2026 13:41:09 -0800 Subject: [PATCH 32/32] Fix build issues - remove exceptions for spark33 --- .../apache/hudi/TestAvroSchemaResolutionSupport.scala | 2 ++ .../Spark33LegacyHoodieParquetFileFormat.scala | 11 ++++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index 9f9afd1c337bb..e597d552337df 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -854,6 +854,8 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df2, tempRecordPath, tableType.equals("COPY_ON_WRITE")) // after implicit type change, read the table with vectorized read enabled + // The supportBatch override in Spark33/34LegacyHoodieParquetFileFormat ensures that + // nested types use row-based reading instead of columnar batches, preventing ClassCastException withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "true") { readTable(tempRecordPath) } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala index 5c3b8cd297561..e42536735ce93 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala @@ -64,6 +64,13 @@ import java.net.URI */ class Spark33LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat { + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + conf.parquetVectorizedReaderEnabled && + schema.forall(_.dataType.isInstanceOf[AtomicType]) && + ParquetUtils.isBatchReadSupportedForSchema(conf, schema) + } + override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -121,7 +128,9 @@ class Spark33LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val enableVectorizedReader: Boolean = - ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) + sqlConf.parquetVectorizedReaderEnabled && + resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) && + ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize